2013-01-14 14:09:53 +01:00
|
|
|
/**
|
2017-01-04 17:37:10 +01:00
|
|
|
* Copyright (C) 2009-2017 Lightbend Inc. <http://www.lightbend.com>
|
2013-01-14 14:09:53 +01:00
|
|
|
*/
|
|
|
|
|
|
2015-04-27 14:25:10 +02:00
|
|
|
package akka.cluster.singleton
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
import language.postfixOps
|
|
|
|
|
import scala.collection.immutable
|
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
import com.typesafe.config.ConfigFactory
|
|
|
|
|
import akka.actor.Actor
|
|
|
|
|
import akka.actor.ActorLogging
|
|
|
|
|
import akka.actor.ActorRef
|
2013-01-28 08:47:52 +01:00
|
|
|
import akka.actor.Address
|
2013-01-14 14:09:53 +01:00
|
|
|
import akka.actor.Props
|
|
|
|
|
import akka.actor.RootActorPath
|
|
|
|
|
import akka.cluster.Cluster
|
|
|
|
|
import akka.cluster.ClusterEvent._
|
|
|
|
|
import akka.cluster.Member
|
|
|
|
|
import akka.remote.testconductor.RoleName
|
|
|
|
|
import akka.remote.testkit.MultiNodeConfig
|
|
|
|
|
import akka.remote.testkit.MultiNodeSpec
|
|
|
|
|
import akka.remote.testkit.STMultiNodeSpec
|
|
|
|
|
import akka.testkit._
|
|
|
|
|
import akka.testkit.TestEvent._
|
|
|
|
|
import akka.actor.Terminated
|
2013-03-26 18:17:50 +01:00
|
|
|
import akka.actor.Identify
|
|
|
|
|
import akka.actor.ActorIdentity
|
|
|
|
|
import akka.actor.ActorSelection
|
2013-11-19 15:53:40 +01:00
|
|
|
import akka.cluster.MemberStatus
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
object ClusterSingletonManagerSpec extends MultiNodeConfig {
|
|
|
|
|
val controller = role("controller")
|
2013-03-14 20:32:43 +01:00
|
|
|
val observer = role("observer")
|
2013-01-14 14:09:53 +01:00
|
|
|
val first = role("first")
|
|
|
|
|
val second = role("second")
|
|
|
|
|
val third = role("third")
|
|
|
|
|
val fourth = role("fourth")
|
|
|
|
|
val fifth = role("fifth")
|
|
|
|
|
val sixth = role("sixth")
|
|
|
|
|
|
|
|
|
|
commonConfig(ConfigFactory.parseString("""
|
|
|
|
|
akka.loglevel = INFO
|
2016-06-10 15:04:13 +02:00
|
|
|
akka.actor.provider = "cluster"
|
2013-01-14 14:09:53 +01:00
|
|
|
akka.remote.log-remote-lifecycle-events = off
|
2013-09-11 16:09:51 +02:00
|
|
|
akka.cluster.auto-down-unreachable-after = 0s
|
2014-01-31 15:10:43 -05:00
|
|
|
"""))
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-03-14 20:32:43 +01:00
|
|
|
nodeConfig(first, second, third, fourth, fifth, sixth)(
|
|
|
|
|
ConfigFactory.parseString("akka.cluster.roles =[worker]"))
|
|
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
object PointToPointChannel {
|
|
|
|
|
case object RegisterConsumer
|
|
|
|
|
case object UnregisterConsumer
|
|
|
|
|
case object RegistrationOk
|
|
|
|
|
case object UnexpectedRegistration
|
|
|
|
|
case object UnregistrationOk
|
|
|
|
|
case object UnexpectedUnregistration
|
|
|
|
|
case object Reset
|
|
|
|
|
case object ResetOk
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2015-06-02 21:01:00 -07:00
|
|
|
* This channel is extremely strict with regards to
|
2013-01-14 14:09:53 +01:00
|
|
|
* registration and unregistration of consumer to
|
|
|
|
|
* be able to detect misbehaviour (e.g. two active
|
|
|
|
|
* singleton instances).
|
|
|
|
|
*/
|
|
|
|
|
class PointToPointChannel extends Actor with ActorLogging {
|
2014-01-31 15:10:43 -05:00
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
import PointToPointChannel._
|
|
|
|
|
|
|
|
|
|
def receive = idle
|
|
|
|
|
|
|
|
|
|
def idle: Receive = {
|
|
|
|
|
case RegisterConsumer ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
log.info("RegisterConsumer: [{}]", sender().path)
|
|
|
|
|
sender() ! RegistrationOk
|
|
|
|
|
context.become(active(sender()))
|
2013-01-14 14:09:53 +01:00
|
|
|
case UnregisterConsumer ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
log.info("UnexpectedUnregistration: [{}]", sender().path)
|
|
|
|
|
sender() ! UnexpectedUnregistration
|
2013-01-14 14:09:53 +01:00
|
|
|
context stop self
|
2014-01-16 15:16:35 +01:00
|
|
|
case Reset ⇒ sender() ! ResetOk
|
2013-01-14 14:09:53 +01:00
|
|
|
case msg ⇒ // no consumer, drop
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def active(consumer: ActorRef): Receive = {
|
2014-01-16 15:16:35 +01:00
|
|
|
case UnregisterConsumer if sender() == consumer ⇒
|
|
|
|
|
log.info("UnregistrationOk: [{}]", sender().path)
|
|
|
|
|
sender() ! UnregistrationOk
|
2013-01-14 14:09:53 +01:00
|
|
|
context.become(idle)
|
|
|
|
|
case UnregisterConsumer ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
log.info("UnexpectedUnregistration: [{}], expected [{}]", sender().path, consumer.path)
|
|
|
|
|
sender() ! UnexpectedUnregistration
|
2013-01-14 14:09:53 +01:00
|
|
|
context stop self
|
|
|
|
|
case RegisterConsumer ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
log.info("Unexpected RegisterConsumer [{}], active consumer [{}]", sender().path, consumer.path)
|
|
|
|
|
sender() ! UnexpectedRegistration
|
2013-01-14 14:09:53 +01:00
|
|
|
context stop self
|
|
|
|
|
case Reset ⇒
|
|
|
|
|
context.become(idle)
|
2014-01-16 15:16:35 +01:00
|
|
|
sender() ! ResetOk
|
2013-01-14 14:09:53 +01:00
|
|
|
case msg ⇒ consumer ! msg
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
object Consumer {
|
|
|
|
|
case object End
|
|
|
|
|
case object GetCurrent
|
2014-01-31 15:10:43 -05:00
|
|
|
case object Ping
|
|
|
|
|
case object Pong
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* The Singleton actor
|
|
|
|
|
*/
|
2015-09-16 14:51:00 +02:00
|
|
|
class Consumer(queue: ActorRef, delegateTo: ActorRef) extends Actor with ActorLogging {
|
2014-01-31 15:10:43 -05:00
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
import Consumer._
|
|
|
|
|
import PointToPointChannel._
|
|
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
var current = 0
|
2015-09-16 14:51:00 +02:00
|
|
|
var stoppedBeforeUnregistration = true
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
override def preStart(): Unit = queue ! RegisterConsumer
|
|
|
|
|
|
2015-09-16 14:51:00 +02:00
|
|
|
override def postStop(): Unit = {
|
|
|
|
|
if (stoppedBeforeUnregistration)
|
|
|
|
|
log.warning("Stopped before unregistration")
|
|
|
|
|
}
|
|
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
def receive = {
|
|
|
|
|
case n: Int if n <= current ⇒
|
|
|
|
|
context.stop(self)
|
|
|
|
|
case n: Int ⇒
|
|
|
|
|
current = n
|
|
|
|
|
delegateTo ! n
|
|
|
|
|
case x @ (RegistrationOk | UnexpectedRegistration) ⇒
|
|
|
|
|
delegateTo ! x
|
|
|
|
|
case GetCurrent ⇒
|
2014-01-16 15:16:35 +01:00
|
|
|
sender() ! current
|
2013-01-14 14:09:53 +01:00
|
|
|
//#consumer-end
|
|
|
|
|
case End ⇒
|
|
|
|
|
queue ! UnregisterConsumer
|
|
|
|
|
case UnregistrationOk ⇒
|
2015-09-16 14:51:00 +02:00
|
|
|
stoppedBeforeUnregistration = false
|
2013-01-14 14:09:53 +01:00
|
|
|
context stop self
|
2014-01-31 15:10:43 -05:00
|
|
|
case Ping ⇒
|
2014-06-20 23:05:51 +02:00
|
|
|
sender() ! Pong
|
2013-01-14 14:09:53 +01:00
|
|
|
//#consumer-end
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class ClusterSingletonManagerMultiJvmNode1 extends ClusterSingletonManagerSpec
|
|
|
|
|
class ClusterSingletonManagerMultiJvmNode2 extends ClusterSingletonManagerSpec
|
|
|
|
|
class ClusterSingletonManagerMultiJvmNode3 extends ClusterSingletonManagerSpec
|
|
|
|
|
class ClusterSingletonManagerMultiJvmNode4 extends ClusterSingletonManagerSpec
|
|
|
|
|
class ClusterSingletonManagerMultiJvmNode5 extends ClusterSingletonManagerSpec
|
|
|
|
|
class ClusterSingletonManagerMultiJvmNode6 extends ClusterSingletonManagerSpec
|
|
|
|
|
class ClusterSingletonManagerMultiJvmNode7 extends ClusterSingletonManagerSpec
|
2013-03-14 20:32:43 +01:00
|
|
|
class ClusterSingletonManagerMultiJvmNode8 extends ClusterSingletonManagerSpec
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
class ClusterSingletonManagerSpec extends MultiNodeSpec(ClusterSingletonManagerSpec) with STMultiNodeSpec with ImplicitSender {
|
2014-01-31 15:10:43 -05:00
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
import ClusterSingletonManagerSpec._
|
|
|
|
|
import ClusterSingletonManagerSpec.PointToPointChannel._
|
|
|
|
|
import ClusterSingletonManagerSpec.Consumer._
|
|
|
|
|
|
|
|
|
|
override def initialParticipants = roles.size
|
|
|
|
|
|
2013-03-26 18:17:50 +01:00
|
|
|
val identifyProbe = TestProbe()
|
|
|
|
|
|
2013-05-24 14:43:01 +02:00
|
|
|
val controllerRootActorPath = node(controller)
|
|
|
|
|
|
2014-01-31 15:10:43 -05:00
|
|
|
var _msg = 0
|
|
|
|
|
|
|
|
|
|
def msg(): Int = {
|
|
|
|
|
_msg += 1
|
|
|
|
|
_msg
|
|
|
|
|
}
|
|
|
|
|
|
2013-03-26 18:17:50 +01:00
|
|
|
def queue: ActorRef = {
|
2013-05-24 14:43:01 +02:00
|
|
|
// this is used from inside actor construction, i.e. other thread, and must therefore not call `node(controller`
|
|
|
|
|
system.actorSelection(controllerRootActorPath / "user" / "queue").tell(Identify("queue"), identifyProbe.ref)
|
2013-03-26 18:17:50 +01:00
|
|
|
identifyProbe.expectMsgType[ActorIdentity].ref.get
|
|
|
|
|
}
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-03-05 10:47:11 +01:00
|
|
|
def join(from: RoleName, to: RoleName): Unit = {
|
|
|
|
|
runOn(from) {
|
|
|
|
|
Cluster(system) join node(to).address
|
2014-01-31 15:10:43 -05:00
|
|
|
if (Cluster(system).selfRoles.contains("worker")) {
|
|
|
|
|
createSingleton()
|
|
|
|
|
createSingletonProxy()
|
|
|
|
|
}
|
2013-03-05 10:47:11 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
def awaitMemberUp(memberProbe: TestProbe, nodes: RoleName*): Unit = {
|
|
|
|
|
runOn(nodes.filterNot(_ == nodes.head): _*) {
|
2015-01-16 11:09:59 +01:00
|
|
|
memberProbe.expectMsgType[MemberUp](15.seconds).member.address should ===(node(nodes.head).address)
|
2013-04-28 22:05:40 +02:00
|
|
|
}
|
|
|
|
|
runOn(nodes.head) {
|
2015-01-16 11:09:59 +01:00
|
|
|
memberProbe.receiveN(nodes.size, 15.seconds).collect { case MemberUp(m) ⇒ m.address }.toSet should ===(
|
2013-04-28 22:05:40 +02:00
|
|
|
nodes.map(node(_).address).toSet)
|
|
|
|
|
}
|
|
|
|
|
enterBarrier(nodes.head.name + "-up")
|
|
|
|
|
}
|
|
|
|
|
|
2013-01-14 14:09:53 +01:00
|
|
|
def createSingleton(): ActorRef = {
|
|
|
|
|
//#create-singleton-manager
|
2016-06-02 14:06:57 +02:00
|
|
|
system.actorOf(
|
|
|
|
|
ClusterSingletonManager.props(
|
|
|
|
|
singletonProps = Props(classOf[Consumer], queue, testActor),
|
|
|
|
|
terminationMessage = End,
|
|
|
|
|
settings = ClusterSingletonManagerSettings(system).withRole("worker")),
|
2015-06-04 21:21:37 +02:00
|
|
|
name = "consumer")
|
2013-01-14 14:09:53 +01:00
|
|
|
//#create-singleton-manager
|
|
|
|
|
}
|
|
|
|
|
|
2014-01-31 15:10:43 -05:00
|
|
|
def createSingletonProxy(): ActorRef = {
|
|
|
|
|
//#create-singleton-proxy
|
2016-06-02 14:06:57 +02:00
|
|
|
system.actorOf(
|
|
|
|
|
ClusterSingletonProxy.props(
|
|
|
|
|
singletonManagerPath = "/user/consumer",
|
|
|
|
|
settings = ClusterSingletonProxySettings(system).withRole("worker")),
|
2014-01-31 15:10:43 -05:00
|
|
|
name = "consumerProxy")
|
|
|
|
|
//#create-singleton-proxy
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def verifyProxyMsg(oldest: RoleName, proxyNode: RoleName, msg: Int): Unit = {
|
|
|
|
|
enterBarrier("before-" + msg + "-proxy-verified")
|
|
|
|
|
|
|
|
|
|
// send a message to the proxy
|
|
|
|
|
runOn(proxyNode) {
|
|
|
|
|
// make sure that the proxy has received membership changes
|
|
|
|
|
// and points to the current singleton
|
|
|
|
|
val p = TestProbe()
|
2016-12-01 18:49:38 +01:00
|
|
|
val oldestAddress = node(oldest).address
|
|
|
|
|
within(10.seconds) {
|
2014-01-31 15:10:43 -05:00
|
|
|
awaitAssert {
|
|
|
|
|
system.actorSelection("/user/consumerProxy").tell(Ping, p.ref)
|
|
|
|
|
p.expectMsg(1.second, Pong)
|
2016-12-01 18:49:38 +01:00
|
|
|
val replyFromAddress = p.lastSender.path.address
|
|
|
|
|
if (oldest == proxyNode)
|
|
|
|
|
replyFromAddress.hasLocalScope should ===(true)
|
|
|
|
|
else
|
|
|
|
|
replyFromAddress should ===(oldestAddress)
|
2014-01-31 15:10:43 -05:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
// then send the real message
|
|
|
|
|
system.actorSelection("/user/consumerProxy") ! msg
|
|
|
|
|
}
|
|
|
|
|
|
2016-12-01 18:49:38 +01:00
|
|
|
enterBarrier(s"sent-msg-$msg")
|
|
|
|
|
|
2014-01-31 15:10:43 -05:00
|
|
|
// expect a message on the oldest node
|
|
|
|
|
runOn(oldest) {
|
2016-12-01 18:49:38 +01:00
|
|
|
expectMsg(msg)
|
2014-01-31 15:10:43 -05:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-" + msg + "-proxy-verified")
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
def consumer(oldest: RoleName): ActorSelection =
|
2015-06-04 21:21:37 +02:00
|
|
|
system.actorSelection(RootActorPath(node(oldest).address) / "user" / "consumer" / "singleton")
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
def verifyRegistration(oldest: RoleName): Unit = {
|
2013-04-28 22:05:40 +02:00
|
|
|
enterBarrier("before-" + oldest.name + "-registration-verified")
|
|
|
|
|
runOn(oldest) {
|
2013-01-14 14:09:53 +01:00
|
|
|
expectMsg(RegistrationOk)
|
2013-04-28 22:05:40 +02:00
|
|
|
consumer(oldest) ! GetCurrent
|
2013-09-10 13:35:51 +02:00
|
|
|
expectMsg(0)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
2013-04-28 22:05:40 +02:00
|
|
|
enterBarrier("after-" + oldest.name + "-registration-verified")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def verifyMsg(oldest: RoleName, msg: Int): Unit = {
|
|
|
|
|
enterBarrier("before-" + msg + "-verified")
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
runOn(controller) {
|
|
|
|
|
queue ! msg
|
|
|
|
|
// make sure it's not terminated, which would be wrong
|
|
|
|
|
expectNoMsg(1 second)
|
|
|
|
|
}
|
2013-04-28 22:05:40 +02:00
|
|
|
runOn(oldest) {
|
|
|
|
|
expectMsg(5.seconds, msg)
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
2013-04-28 22:05:40 +02:00
|
|
|
runOn(roles.filterNot(r ⇒ r == oldest || r == controller || r == observer): _*) {
|
2013-01-14 14:09:53 +01:00
|
|
|
expectNoMsg(1 second)
|
|
|
|
|
}
|
2013-04-28 22:05:40 +02:00
|
|
|
enterBarrier("after-" + msg + "-verified")
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def crash(roles: RoleName*): Unit = {
|
|
|
|
|
runOn(controller) {
|
|
|
|
|
queue ! Reset
|
|
|
|
|
expectMsg(ResetOk)
|
|
|
|
|
roles foreach { r ⇒
|
|
|
|
|
log.info("Shutdown [{}]", node(r).address)
|
2013-04-23 16:44:14 +02:00
|
|
|
testConductor.exit(r, 0).await
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"A ClusterSingletonManager" must {
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
"startup 6 node cluster" in within(60 seconds) {
|
|
|
|
|
|
|
|
|
|
val memberProbe = TestProbe()
|
|
|
|
|
Cluster(system).subscribe(memberProbe.ref, classOf[MemberUp])
|
|
|
|
|
memberProbe.expectMsgClass(classOf[CurrentClusterState])
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
runOn(controller) {
|
|
|
|
|
// watch that it is not terminated, which would indicate misbehaviour
|
|
|
|
|
watch(system.actorOf(Props[PointToPointChannel], "queue"))
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("queue-started")
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
join(first, first)
|
|
|
|
|
awaitMemberUp(memberProbe, first)
|
2013-09-10 13:35:51 +02:00
|
|
|
verifyRegistration(first)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(first, msg = msg())
|
2013-03-14 20:32:43 +01:00
|
|
|
|
|
|
|
|
// join the observer node as well, which should not influence since it doesn't have the "worker" role
|
2013-04-28 22:05:40 +02:00
|
|
|
join(observer, first)
|
|
|
|
|
awaitMemberUp(memberProbe, observer, first)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyProxyMsg(first, first, msg = msg())
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
join(second, first)
|
|
|
|
|
awaitMemberUp(memberProbe, second, observer, first)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(first, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, second, msg = msg())
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
join(third, first)
|
|
|
|
|
awaitMemberUp(memberProbe, third, second, observer, first)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(first, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, third, msg = msg())
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
join(fourth, first)
|
|
|
|
|
awaitMemberUp(memberProbe, fourth, third, second, observer, first)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(first, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, fourth, msg = msg())
|
2013-01-14 14:09:53 +01:00
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
join(fifth, first)
|
|
|
|
|
awaitMemberUp(memberProbe, fifth, fourth, third, second, observer, first)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(first, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, fifth, msg = msg())
|
2013-03-05 10:47:11 +01:00
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
join(sixth, first)
|
|
|
|
|
awaitMemberUp(memberProbe, sixth, fifth, fourth, third, second, observer, first)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(first, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, sixth, msg = msg())
|
2013-04-28 22:05:40 +02:00
|
|
|
|
|
|
|
|
enterBarrier("after-1")
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
2014-01-31 15:10:43 -05:00
|
|
|
"let the proxy route messages to the singleton in a 6 node cluster" in within(60 seconds) {
|
|
|
|
|
verifyProxyMsg(first, first, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, second, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, third, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, fourth, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, fifth, msg = msg())
|
|
|
|
|
verifyProxyMsg(first, sixth, msg = msg())
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
"hand over when oldest leaves in 6 nodes cluster " in within(30 seconds) {
|
|
|
|
|
val leaveRole = first
|
|
|
|
|
val newOldestRole = second
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
runOn(leaveRole) {
|
|
|
|
|
Cluster(system) leave node(leaveRole).address
|
|
|
|
|
}
|
|
|
|
|
|
2013-09-10 13:35:51 +02:00
|
|
|
verifyRegistration(second)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(second, msg = msg())
|
|
|
|
|
verifyProxyMsg(second, second, msg = msg())
|
|
|
|
|
verifyProxyMsg(second, third, msg = msg())
|
|
|
|
|
verifyProxyMsg(second, fourth, msg = msg())
|
|
|
|
|
verifyProxyMsg(second, fifth, msg = msg())
|
|
|
|
|
verifyProxyMsg(second, sixth, msg = msg())
|
2013-01-14 14:09:53 +01:00
|
|
|
|
|
|
|
|
runOn(leaveRole) {
|
2015-06-04 21:21:37 +02:00
|
|
|
system.actorSelection("/user/consumer").tell(Identify("singleton"), identifyProbe.ref)
|
2013-03-26 18:17:50 +01:00
|
|
|
identifyProbe.expectMsgPF() {
|
|
|
|
|
case ActorIdentity("singleton", None) ⇒ // already terminated
|
|
|
|
|
case ActorIdentity("singleton", Some(singleton)) ⇒
|
|
|
|
|
watch(singleton)
|
2013-04-09 14:48:17 +02:00
|
|
|
expectTerminated(singleton)
|
2013-03-26 18:17:50 +01:00
|
|
|
}
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-leave")
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
"take over when oldest crashes in 5 nodes cluster" in within(60 seconds) {
|
2013-05-28 09:02:03 +02:00
|
|
|
// mute logging of deadLetters during shutdown of systems
|
|
|
|
|
if (!log.isDebugEnabled)
|
|
|
|
|
system.eventStream.publish(Mute(DeadLettersFilter[Any]))
|
2013-01-14 14:09:53 +01:00
|
|
|
enterBarrier("logs-muted")
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
crash(second)
|
2013-09-10 13:35:51 +02:00
|
|
|
verifyRegistration(third)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(third, msg = msg())
|
|
|
|
|
verifyProxyMsg(third, third, msg = msg())
|
|
|
|
|
verifyProxyMsg(third, fourth, msg = msg())
|
|
|
|
|
verifyProxyMsg(third, fifth, msg = msg())
|
|
|
|
|
verifyProxyMsg(third, sixth, msg = msg())
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
"take over when two oldest crash in 3 nodes cluster" in within(60 seconds) {
|
|
|
|
|
crash(third, fourth)
|
2013-09-10 13:35:51 +02:00
|
|
|
verifyRegistration(fifth)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(fifth, msg = msg())
|
|
|
|
|
verifyProxyMsg(fifth, fifth, msg = msg())
|
|
|
|
|
verifyProxyMsg(fifth, sixth, msg = msg())
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
2013-04-28 22:05:40 +02:00
|
|
|
"take over when oldest crashes in 2 nodes cluster" in within(60 seconds) {
|
|
|
|
|
crash(fifth)
|
2013-09-10 13:35:51 +02:00
|
|
|
verifyRegistration(sixth)
|
2014-01-31 15:10:43 -05:00
|
|
|
verifyMsg(sixth, msg = msg())
|
|
|
|
|
verifyProxyMsg(sixth, sixth, msg = msg())
|
2013-01-14 14:09:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
}
|