pekko/akka-cluster/src/multi-jvm/scala/akka/cluster/RestartNodeSpec.scala

160 lines
5.2 KiB
Scala
Raw Normal View History

/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.cluster
import scala.collection.immutable
import scala.concurrent.duration._
import akka.Done
import akka.actor.Actor
import akka.actor.ActorIdentity
import akka.actor.ActorRef
import akka.actor.ActorSystem
import akka.actor.Address
import akka.actor.Deploy
import akka.actor.Identify
import akka.actor.Props
import akka.actor.RootActorPath
import akka.actor.Terminated
import akka.cluster.MemberStatus._
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.testkit._
import com.typesafe.config.ConfigFactory
import akka.util.ccompat._
@ccompatUsedUntil213
object RestartNodeMultiJvmSpec extends MultiNodeConfig {
val first = role("first")
val second = role("second")
val third = role("third")
2019-03-11 10:38:24 +01:00
commonConfig(
debugConfig(on = false).withFallback(ConfigFactory.parseString("""
akka.cluster.auto-down-unreachable-after = 5s
2017-01-25 07:20:24 +01:00
akka.cluster.allow-weakly-up-members = off
#akka.remote.use-passive-connections = off
2019-03-11 10:38:24 +01:00
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
/**
* This was used together with sleep in EndpointReader before deliverAndAck
* to reproduce issue with misaligned ACKs when restarting system,
* issue #19780
*/
class Watcher(a: Address, replyTo: ActorRef) extends Actor {
context.actorSelection(RootActorPath(a) / "user" / "address-receiver") ! Identify(None)
def receive = {
case ActorIdentity(None, Some(ref)) =>
context.watch(ref)
replyTo ! Done
case _: Terminated =>
}
}
}
class RestartNodeMultiJvmNode1 extends RestartNodeSpec
class RestartNodeMultiJvmNode2 extends RestartNodeSpec
class RestartNodeMultiJvmNode3 extends RestartNodeSpec
abstract class RestartNodeSpec
2019-03-11 10:38:24 +01:00
extends MultiNodeSpec(RestartNodeMultiJvmSpec)
with MultiNodeClusterSpec
with ImplicitSender {
import RestartNodeMultiJvmSpec._
@volatile var secondUniqueAddress: UniqueAddress = _
// use a separate ActorSystem, to be able to simulate restart
lazy val secondSystem = ActorSystem(system.name, system.settings.config)
def seedNodes: immutable.IndexedSeq[Address] = Vector(first, secondUniqueAddress.address, third)
2019-03-13 10:56:20 +01:00
lazy val restartedSecondSystem = ActorSystem(
system.name,
ConfigFactory.parseString(s"""
akka.remote.netty.tcp.port = ${secondUniqueAddress.address.port.get}
akka.remote.artery.canonical.port = ${secondUniqueAddress.address.port.get}
""").withFallback(system.settings.config))
override def afterAll(): Unit = {
runOn(second) {
if (secondSystem.whenTerminated.isCompleted)
shutdown(restartedSecondSystem)
else
shutdown(secondSystem)
}
super.afterAll()
}
"Cluster nodes" must {
"be able to restart and join again" taggedAs LongRunningTest in within(60.seconds) {
// secondSystem is a separate ActorSystem, to be able to simulate restart
// we must transfer its address to first
runOn(first, third) {
system.actorOf(Props(new Actor {
def receive = {
case a: UniqueAddress =>
secondUniqueAddress = a
sender() ! "ok"
}
}).withDeploy(Deploy.local), name = "address-receiver")
enterBarrier("second-address-receiver-ready")
}
runOn(second) {
enterBarrier("second-address-receiver-ready")
secondUniqueAddress = Cluster(secondSystem).selfUniqueAddress
2019-03-11 10:38:24 +01:00
List(first, third).foreach { r =>
system.actorSelection(RootActorPath(r) / "user" / "address-receiver") ! secondUniqueAddress
expectMsg(5.seconds, "ok")
}
}
enterBarrier("second-address-transferred")
// now we can join first, secondSystem, third together
runOn(first, third) {
cluster.joinSeedNodes(seedNodes)
awaitMembersUp(3)
}
runOn(second) {
Cluster(secondSystem).joinSeedNodes(seedNodes)
2015-01-16 11:09:59 +01:00
awaitAssert(Cluster(secondSystem).readView.members.size should ===(3))
Various scala-2.13.0-M5 fixes fix akka-actor-tests compile errors some tests still fail though Fix test failures in akka-actor-test Manually work arround missing implicit Factory[Nothing, Seq[Nothing]] see https://github.com/scala/scala-collection-compat/issues/137 akka-remote scalafix changes Fix shutdownAll compile error test:akka-remote scalafix changes akka-multi-node-testkit scalafix Fix akka-remote-tests multi-jvm compile errors akka-stream-tests/test:scalafix Fix test:akka-stream-tests Crude implementation of ByteString.map scalafix akka-actor-typed, akka-actor-typed-tests akka-actor-typed-tests compile and succeed scalafix akka-camel scalafix akka-cluster akka-cluster compile & test scalafix akka-cluster-metrics Fix akka-cluster-metrics scalafix akka-cluster-tools akka-cluster-tools compile and test scalafix akka-distributed-data akka-distributed-data fixes scalafix akka-persistence scalafix akka-cluster-sharding fix akka-cluster-sharding scalafix akka-contrib Fix akka-cluster-sharding-typed test scalafix akka-docs Use scala-stm 0.9 (released for M5) akka-docs Remove dependency on collections-compat Cherry-pick the relevant constructs to our own private utils Shorten 'scala.collections.immutable' by importing it Duplicate 'immutable' imports Use 'foreach' on futures Replace MapLike with regular Map Internal API markers Simplify ccompat by moving PackageShared into object Since we don't currently need to differentiate between 2.11 and Avoid relying on 'union' (and ++) being left-biased Fix akka-actor/doc by removing -Ywarn-unused Make more things more private Copyright headers Use 'unsorted' to go from SortedSet to Set Duplicate import Use onComplete rather than failed.foreach Clarify why we partly duplicate scala-collection-compat
2018-11-22 16:18:10 +01:00
awaitAssert(Cluster(secondSystem).readView.members.unsorted.map(_.status) should ===(Set(Up)))
}
enterBarrier("started")
// shutdown secondSystem
runOn(second) {
// send system message just before shutdown, reproducer for issue #19780
secondSystem.actorOf(Props(classOf[Watcher], address(first), testActor), "testwatcher")
expectMsg(Done)
shutdown(secondSystem, remaining)
}
enterBarrier("second-shutdown")
// then immediately start restartedSecondSystem, which has the same address as secondSystem
runOn(second) {
Cluster(restartedSecondSystem).joinSeedNodes(seedNodes)
2015-01-16 11:09:59 +01:00
awaitAssert(Cluster(restartedSecondSystem).readView.members.size should ===(3))
Various scala-2.13.0-M5 fixes fix akka-actor-tests compile errors some tests still fail though Fix test failures in akka-actor-test Manually work arround missing implicit Factory[Nothing, Seq[Nothing]] see https://github.com/scala/scala-collection-compat/issues/137 akka-remote scalafix changes Fix shutdownAll compile error test:akka-remote scalafix changes akka-multi-node-testkit scalafix Fix akka-remote-tests multi-jvm compile errors akka-stream-tests/test:scalafix Fix test:akka-stream-tests Crude implementation of ByteString.map scalafix akka-actor-typed, akka-actor-typed-tests akka-actor-typed-tests compile and succeed scalafix akka-camel scalafix akka-cluster akka-cluster compile & test scalafix akka-cluster-metrics Fix akka-cluster-metrics scalafix akka-cluster-tools akka-cluster-tools compile and test scalafix akka-distributed-data akka-distributed-data fixes scalafix akka-persistence scalafix akka-cluster-sharding fix akka-cluster-sharding scalafix akka-contrib Fix akka-cluster-sharding-typed test scalafix akka-docs Use scala-stm 0.9 (released for M5) akka-docs Remove dependency on collections-compat Cherry-pick the relevant constructs to our own private utils Shorten 'scala.collections.immutable' by importing it Duplicate 'immutable' imports Use 'foreach' on futures Replace MapLike with regular Map Internal API markers Simplify ccompat by moving PackageShared into object Since we don't currently need to differentiate between 2.11 and Avoid relying on 'union' (and ++) being left-biased Fix akka-actor/doc by removing -Ywarn-unused Make more things more private Copyright headers Use 'unsorted' to go from SortedSet to Set Duplicate import Use onComplete rather than failed.foreach Clarify why we partly duplicate scala-collection-compat
2018-11-22 16:18:10 +01:00
awaitAssert(Cluster(restartedSecondSystem).readView.members.unsorted.map(_.status) should ===(Set(Up)))
}
runOn(first, third) {
awaitAssert {
2015-01-16 11:09:59 +01:00
Cluster(system).readView.members.size should ===(3)
Cluster(system).readView.members.exists { m =>
m.address == secondUniqueAddress.address && m.uniqueAddress.longUid != secondUniqueAddress.longUid
}
}
}
enterBarrier("second-restarted")
}
}
}