2018-10-29 17:19:37 +08:00
|
|
|
/*
|
2019-01-02 18:55:26 +08:00
|
|
|
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
|
2013-04-14 22:30:09 +02:00
|
|
|
*/
|
2018-04-24 16:03:55 +01:00
|
|
|
|
2015-04-27 14:25:10 +02:00
|
|
|
package akka.cluster.client
|
2013-04-14 22:30:09 +02:00
|
|
|
|
|
|
|
|
import language.postfixOps
|
|
|
|
|
import scala.concurrent.duration._
|
|
|
|
|
import com.typesafe.config.ConfigFactory
|
2019-03-11 10:38:24 +01:00
|
|
|
import akka.actor.{
|
|
|
|
|
Actor,
|
|
|
|
|
ActorPath,
|
|
|
|
|
ActorRef,
|
|
|
|
|
ActorSystem,
|
|
|
|
|
Address,
|
|
|
|
|
ExtendedActorSystem,
|
|
|
|
|
NoSerializationVerificationNeeded,
|
|
|
|
|
Props
|
|
|
|
|
}
|
2013-04-14 22:30:09 +02:00
|
|
|
import akka.cluster.Cluster
|
2016-05-04 04:50:16 -07:00
|
|
|
import akka.cluster.client.ClusterClientSpec.TestClientListener.LatestContactPoints
|
|
|
|
|
import akka.cluster.client.ClusterClientSpec.TestReceptionistListener.LatestClusterClients
|
2013-04-14 22:30:09 +02:00
|
|
|
import akka.remote.testconductor.RoleName
|
|
|
|
|
import akka.remote.testkit.MultiNodeConfig
|
|
|
|
|
import akka.remote.testkit.MultiNodeSpec
|
|
|
|
|
import akka.remote.testkit.STMultiNodeSpec
|
|
|
|
|
import akka.testkit._
|
2015-04-27 14:25:10 +02:00
|
|
|
import akka.cluster.pubsub._
|
2015-02-06 08:43:28 +01:00
|
|
|
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
2016-05-04 04:50:16 -07:00
|
|
|
import akka.util.Timeout
|
2019-04-04 15:35:18 +02:00
|
|
|
import akka.util.unused
|
2016-05-04 04:50:16 -07:00
|
|
|
|
2015-09-16 10:47:19 +02:00
|
|
|
import scala.concurrent.Await
|
2013-04-14 22:30:09 +02:00
|
|
|
|
|
|
|
|
object ClusterClientSpec extends MultiNodeConfig {
|
|
|
|
|
val client = role("client")
|
|
|
|
|
val first = role("first")
|
|
|
|
|
val second = role("second")
|
|
|
|
|
val third = role("third")
|
|
|
|
|
val fourth = role("fourth")
|
|
|
|
|
|
|
|
|
|
commonConfig(ConfigFactory.parseString("""
|
|
|
|
|
akka.loglevel = INFO
|
2016-06-10 15:04:13 +02:00
|
|
|
akka.actor.provider = "cluster"
|
2013-04-14 22:30:09 +02:00
|
|
|
akka.remote.log-remote-lifecycle-events = off
|
2013-09-11 16:09:51 +02:00
|
|
|
akka.cluster.auto-down-unreachable-after = 0s
|
2015-02-06 08:43:28 +01:00
|
|
|
akka.cluster.client.heartbeat-interval = 1s
|
|
|
|
|
akka.cluster.client.acceptable-heartbeat-pause = 3s
|
2016-05-04 04:50:16 -07:00
|
|
|
akka.cluster.client.refresh-contacts-interval = 1s
|
2015-07-01 12:06:08 +02:00
|
|
|
# number-of-contacts must be >= 4 because we shutdown all but one in the end
|
2015-09-18 09:27:19 +02:00
|
|
|
akka.cluster.client.receptionist.number-of-contacts = 4
|
2016-05-04 04:50:16 -07:00
|
|
|
akka.cluster.client.receptionist.heartbeat-interval = 10s
|
|
|
|
|
akka.cluster.client.receptionist.acceptable-heartbeat-pause = 10s
|
|
|
|
|
akka.cluster.client.receptionist.failure-detection-interval = 1s
|
2015-09-16 10:47:19 +02:00
|
|
|
akka.test.filter-leeway = 10s
|
2013-04-14 22:30:09 +02:00
|
|
|
"""))
|
|
|
|
|
|
2015-02-06 08:43:28 +01:00
|
|
|
testTransport(on = true)
|
|
|
|
|
|
2015-08-18 17:27:42 +02:00
|
|
|
case class Reply(msg: Any, node: Address)
|
|
|
|
|
|
2013-04-14 22:30:09 +02:00
|
|
|
class TestService(testActor: ActorRef) extends Actor {
|
|
|
|
|
def receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case "shutdown" =>
|
2015-09-16 10:47:19 +02:00
|
|
|
context.system.terminate()
|
2019-02-09 15:25:39 +01:00
|
|
|
case msg =>
|
2019-03-11 10:38:24 +01:00
|
|
|
testActor.forward(msg)
|
2019-04-04 15:35:18 +02:00
|
|
|
sender() ! Reply(s"$msg-ack", Cluster(context.system).selfAddress)
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class Service extends Actor {
|
|
|
|
|
def receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case msg => sender() ! msg
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2016-05-04 04:50:16 -07:00
|
|
|
//#clientEventsListener
|
|
|
|
|
class ClientListener(targetClient: ActorRef) extends Actor {
|
|
|
|
|
override def preStart(): Unit =
|
|
|
|
|
targetClient ! SubscribeContactPoints
|
|
|
|
|
|
|
|
|
|
def receive: Receive =
|
|
|
|
|
receiveWithContactPoints(Set.empty)
|
|
|
|
|
|
|
|
|
|
def receiveWithContactPoints(contactPoints: Set[ActorPath]): Receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case ContactPoints(cps) =>
|
2016-05-04 04:50:16 -07:00
|
|
|
context.become(receiveWithContactPoints(cps))
|
|
|
|
|
// Now do something with the up-to-date "cps"
|
2019-02-09 15:25:39 +01:00
|
|
|
case ContactPointAdded(cp) =>
|
2016-05-04 04:50:16 -07:00
|
|
|
context.become(receiveWithContactPoints(contactPoints + cp))
|
|
|
|
|
// Now do something with an up-to-date "contactPoints + cp"
|
2019-02-09 15:25:39 +01:00
|
|
|
case ContactPointRemoved(cp) =>
|
2016-05-04 04:50:16 -07:00
|
|
|
context.become(receiveWithContactPoints(contactPoints - cp))
|
|
|
|
|
// Now do something with an up-to-date "contactPoints - cp"
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
//#clientEventsListener
|
|
|
|
|
|
|
|
|
|
object TestClientListener {
|
|
|
|
|
case object GetLatestContactPoints
|
|
|
|
|
case class LatestContactPoints(contactPoints: Set[ActorPath]) extends NoSerializationVerificationNeeded
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class TestClientListener(targetClient: ActorRef) extends ClientListener(targetClient) {
|
|
|
|
|
|
|
|
|
|
import TestClientListener._
|
|
|
|
|
|
|
|
|
|
override def receiveWithContactPoints(contactPoints: Set[ActorPath]): Receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case GetLatestContactPoints =>
|
2016-05-04 04:50:16 -07:00
|
|
|
sender() ! LatestContactPoints(contactPoints)
|
2019-02-09 15:25:39 +01:00
|
|
|
case msg: Any =>
|
2016-05-04 04:50:16 -07:00
|
|
|
super.receiveWithContactPoints(contactPoints)(msg)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
//#receptionistEventsListener
|
|
|
|
|
class ReceptionistListener(targetReceptionist: ActorRef) extends Actor {
|
|
|
|
|
override def preStart(): Unit =
|
|
|
|
|
targetReceptionist ! SubscribeClusterClients
|
|
|
|
|
|
|
|
|
|
def receive: Receive =
|
|
|
|
|
receiveWithClusterClients(Set.empty)
|
|
|
|
|
|
|
|
|
|
def receiveWithClusterClients(clusterClients: Set[ActorRef]): Receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case ClusterClients(cs) =>
|
2016-05-04 04:50:16 -07:00
|
|
|
context.become(receiveWithClusterClients(cs))
|
|
|
|
|
// Now do something with the up-to-date "c"
|
2019-02-09 15:25:39 +01:00
|
|
|
case ClusterClientUp(c) =>
|
2016-05-04 04:50:16 -07:00
|
|
|
context.become(receiveWithClusterClients(clusterClients + c))
|
|
|
|
|
// Now do something with an up-to-date "clusterClients + c"
|
2019-02-09 15:25:39 +01:00
|
|
|
case ClusterClientUnreachable(c) =>
|
2016-05-04 04:50:16 -07:00
|
|
|
context.become(receiveWithClusterClients(clusterClients - c))
|
|
|
|
|
// Now do something with an up-to-date "clusterClients - c"
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
//#receptionistEventsListener
|
|
|
|
|
|
|
|
|
|
object TestReceptionistListener {
|
|
|
|
|
case object GetLatestClusterClients
|
|
|
|
|
case class LatestClusterClients(clusterClients: Set[ActorRef]) extends NoSerializationVerificationNeeded
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class TestReceptionistListener(targetReceptionist: ActorRef) extends ReceptionistListener(targetReceptionist) {
|
|
|
|
|
|
|
|
|
|
import TestReceptionistListener._
|
|
|
|
|
|
|
|
|
|
override def receiveWithClusterClients(clusterClients: Set[ActorRef]): Receive = {
|
2019-02-09 15:25:39 +01:00
|
|
|
case GetLatestClusterClients =>
|
2016-05-04 04:50:16 -07:00
|
|
|
sender() ! LatestClusterClients(clusterClients)
|
2019-02-09 15:25:39 +01:00
|
|
|
case msg: Any =>
|
2016-05-04 04:50:16 -07:00
|
|
|
super.receiveWithClusterClients(clusterClients)(msg)
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class ClusterClientMultiJvmNode1 extends ClusterClientSpec
|
|
|
|
|
class ClusterClientMultiJvmNode2 extends ClusterClientSpec
|
|
|
|
|
class ClusterClientMultiJvmNode3 extends ClusterClientSpec
|
|
|
|
|
class ClusterClientMultiJvmNode4 extends ClusterClientSpec
|
|
|
|
|
class ClusterClientMultiJvmNode5 extends ClusterClientSpec
|
|
|
|
|
|
|
|
|
|
class ClusterClientSpec extends MultiNodeSpec(ClusterClientSpec) with STMultiNodeSpec with ImplicitSender {
|
|
|
|
|
import ClusterClientSpec._
|
|
|
|
|
|
|
|
|
|
override def initialParticipants = roles.size
|
|
|
|
|
|
|
|
|
|
def join(from: RoleName, to: RoleName): Unit = {
|
|
|
|
|
runOn(from) {
|
2019-03-11 10:38:24 +01:00
|
|
|
Cluster(system).join(node(to).address)
|
2013-04-14 22:30:09 +02:00
|
|
|
createReceptionist()
|
|
|
|
|
}
|
|
|
|
|
enterBarrier(from.name + "-joined")
|
|
|
|
|
}
|
|
|
|
|
|
2015-05-08 08:49:08 +02:00
|
|
|
def createReceptionist(): Unit = ClusterClientReceptionist(system)
|
2013-04-14 22:30:09 +02:00
|
|
|
|
|
|
|
|
def awaitCount(expected: Int): Unit = {
|
|
|
|
|
awaitAssert {
|
2015-05-07 11:01:59 +02:00
|
|
|
DistributedPubSub(system).mediator ! DistributedPubSubMediator.Count
|
2015-01-16 11:09:59 +01:00
|
|
|
expectMsgType[Int] should ===(expected)
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2015-02-06 08:43:28 +01:00
|
|
|
var remainingServerRoleNames = Set(first, second, third, fourth)
|
|
|
|
|
|
|
|
|
|
def roleName(addr: Address): Option[RoleName] = remainingServerRoleNames.find(node(_).address == addr)
|
2013-04-14 22:30:09 +02:00
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
def initialContacts = (remainingServerRoleNames - first - fourth).map { r =>
|
2015-06-30 09:28:30 +02:00
|
|
|
node(r) / "system" / "receptionist"
|
2015-02-06 08:43:28 +01:00
|
|
|
}
|
2013-04-14 22:30:09 +02:00
|
|
|
|
2019-04-04 15:35:18 +02:00
|
|
|
@unused
|
|
|
|
|
def docOnly = { //not used, only demo
|
|
|
|
|
//#initialContacts
|
|
|
|
|
val initialContacts = Set(
|
2019-05-15 18:01:34 +02:00
|
|
|
ActorPath.fromString("akka://OtherSys@host1:2552/system/receptionist"),
|
|
|
|
|
ActorPath.fromString("akka://OtherSys@host2:2552/system/receptionist"))
|
2019-04-04 15:35:18 +02:00
|
|
|
val settings = ClusterClientSettings(system).withInitialContacts(initialContacts)
|
|
|
|
|
//#initialContacts
|
|
|
|
|
|
|
|
|
|
// make the compiler happy and thinking we use it
|
|
|
|
|
settings.acceptableHeartbeatPause
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-14 22:30:09 +02:00
|
|
|
"A ClusterClient" must {
|
|
|
|
|
|
|
|
|
|
"startup cluster" in within(30 seconds) {
|
|
|
|
|
join(first, first)
|
|
|
|
|
join(second, first)
|
|
|
|
|
join(third, first)
|
|
|
|
|
join(fourth, first)
|
|
|
|
|
runOn(fourth) {
|
2013-04-17 22:14:19 +02:00
|
|
|
val service = system.actorOf(Props(classOf[TestService], testActor), "testService")
|
2015-05-08 08:49:08 +02:00
|
|
|
ClusterClientReceptionist(system).registerService(service)
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
runOn(first, second, third, fourth) {
|
|
|
|
|
awaitCount(1)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-1")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
"communicate to actor on any node in cluster" in within(10 seconds) {
|
|
|
|
|
runOn(client) {
|
2019-03-13 10:56:20 +01:00
|
|
|
val c = system.actorOf(
|
|
|
|
|
ClusterClient.props(ClusterClientSettings(system).withInitialContacts(initialContacts)),
|
|
|
|
|
"client1")
|
2013-09-09 14:56:16 +02:00
|
|
|
c ! ClusterClient.Send("/user/testService", "hello", localAffinity = true)
|
2015-08-18 17:27:42 +02:00
|
|
|
expectMsgType[Reply].msg should be("hello-ack")
|
2015-02-06 08:43:28 +01:00
|
|
|
system.stop(c)
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
runOn(fourth) {
|
|
|
|
|
expectMsg("hello")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-2")
|
|
|
|
|
}
|
|
|
|
|
|
2017-01-03 16:42:31 +01:00
|
|
|
"work with ask" in within(10 seconds) {
|
|
|
|
|
runOn(client) {
|
|
|
|
|
import akka.pattern.ask
|
2019-03-13 10:56:20 +01:00
|
|
|
val c = system.actorOf(
|
|
|
|
|
ClusterClient.props(ClusterClientSettings(system).withInitialContacts(initialContacts)),
|
|
|
|
|
"ask-client")
|
2017-01-03 16:42:31 +01:00
|
|
|
implicit val timeout = Timeout(remaining)
|
|
|
|
|
val reply = c ? ClusterClient.Send("/user/testService", "hello-request", localAffinity = true)
|
|
|
|
|
Await.result(reply.mapTo[Reply], remaining).msg should be("hello-request-ack")
|
|
|
|
|
system.stop(c)
|
|
|
|
|
}
|
|
|
|
|
runOn(fourth) {
|
|
|
|
|
expectMsg("hello-request")
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("after-3")
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-14 22:30:09 +02:00
|
|
|
"demonstrate usage" in within(15 seconds) {
|
|
|
|
|
def host1 = first
|
|
|
|
|
def host2 = second
|
|
|
|
|
def host3 = third
|
|
|
|
|
|
|
|
|
|
//#server
|
|
|
|
|
runOn(host1) {
|
|
|
|
|
val serviceA = system.actorOf(Props[Service], "serviceA")
|
2015-05-08 08:49:08 +02:00
|
|
|
ClusterClientReceptionist(system).registerService(serviceA)
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
runOn(host2, host3) {
|
|
|
|
|
val serviceB = system.actorOf(Props[Service], "serviceB")
|
2015-05-08 08:49:08 +02:00
|
|
|
ClusterClientReceptionist(system).registerService(serviceB)
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
//#server
|
|
|
|
|
|
2013-09-17 14:52:31 +02:00
|
|
|
runOn(host1, host2, host3, fourth) {
|
2013-09-09 14:56:16 +02:00
|
|
|
awaitCount(4)
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("services-replicated")
|
|
|
|
|
|
2013-04-14 22:30:09 +02:00
|
|
|
//#client
|
|
|
|
|
runOn(client) {
|
2019-03-13 10:56:20 +01:00
|
|
|
val c = system.actorOf(
|
|
|
|
|
ClusterClient.props(ClusterClientSettings(system).withInitialContacts(initialContacts)),
|
|
|
|
|
"client")
|
2013-04-22 13:03:29 +02:00
|
|
|
c ! ClusterClient.Send("/user/serviceA", "hello", localAffinity = true)
|
|
|
|
|
c ! ClusterClient.SendToAll("/user/serviceB", "hi")
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
//#client
|
|
|
|
|
|
2013-09-09 14:56:16 +02:00
|
|
|
runOn(client) {
|
|
|
|
|
// note that "hi" was sent to 2 "serviceB"
|
2015-01-16 11:09:59 +01:00
|
|
|
receiveN(3).toSet should ===(Set("hello", "hi"))
|
2013-09-09 14:56:16 +02:00
|
|
|
}
|
|
|
|
|
|
2013-04-14 22:30:09 +02:00
|
|
|
// strange, barriers fail without this sleep
|
|
|
|
|
Thread.sleep(1000)
|
2017-01-03 16:42:31 +01:00
|
|
|
enterBarrier("after-4")
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
|
2016-05-04 04:50:16 -07:00
|
|
|
"report events" in within(15 seconds) {
|
|
|
|
|
runOn(client) {
|
|
|
|
|
implicit val timeout = Timeout(1.second.dilated)
|
2017-06-07 05:54:47 -07:00
|
|
|
val client = Await.result(system.actorSelection("/user/client").resolveOne(), timeout.duration)
|
|
|
|
|
val listener = system.actorOf(Props(classOf[TestClientListener], client), "reporter-client-listener")
|
2016-05-04 04:50:16 -07:00
|
|
|
|
|
|
|
|
val expectedContacts = Set(first, second, third, fourth).map(node(_) / "system" / "receptionist")
|
2017-06-07 05:54:47 -07:00
|
|
|
awaitAssert({
|
|
|
|
|
listener ! TestClientListener.GetLatestContactPoints
|
|
|
|
|
expectMsgType[LatestContactPoints].contactPoints should ===(expectedContacts)
|
|
|
|
|
}, max = 10.seconds)
|
2016-05-04 04:50:16 -07:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
enterBarrier("reporter-client-listener-tested")
|
|
|
|
|
|
|
|
|
|
runOn(first, second, third) {
|
|
|
|
|
// Only run this test on a node that knows about our client. It could be that no node knows
|
|
|
|
|
// but there isn't a means of expressing that at least one of the nodes needs to pass the test.
|
|
|
|
|
implicit val timeout = Timeout(2.seconds.dilated)
|
|
|
|
|
val r = ClusterClientReceptionist(system).underlying
|
|
|
|
|
r ! GetClusterClients
|
|
|
|
|
val cps = expectMsgType[ClusterClients]
|
|
|
|
|
if (cps.clusterClients.exists(_.path.name == "client")) {
|
|
|
|
|
log.info("Testing that the receptionist has just one client")
|
|
|
|
|
val l = system.actorOf(Props(classOf[TestReceptionistListener], r), "reporter-receptionist-listener")
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
val expectedClient =
|
|
|
|
|
Await.result(system.actorSelection(node(client) / "user" / "client").resolveOne(), timeout.duration)
|
2017-06-07 05:54:47 -07:00
|
|
|
awaitAssert({
|
|
|
|
|
val probe = TestProbe()
|
|
|
|
|
l.tell(TestReceptionistListener.GetLatestClusterClients, probe.ref)
|
2018-01-10 10:48:53 +01:00
|
|
|
// "ask-client" might still be around, filter
|
|
|
|
|
probe.expectMsgType[LatestClusterClients].clusterClients should contain(expectedClient)
|
2017-06-07 05:54:47 -07:00
|
|
|
}, max = 10.seconds)
|
2016-05-04 04:50:16 -07:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2017-01-03 16:42:31 +01:00
|
|
|
enterBarrier("after-5")
|
2016-05-04 04:50:16 -07:00
|
|
|
}
|
|
|
|
|
|
2017-06-04 11:40:17 +02:00
|
|
|
"report a removal of a receptionist" in within(10 seconds) {
|
|
|
|
|
runOn(client) {
|
|
|
|
|
val unreachableContact = node(client) / "system" / "receptionist"
|
|
|
|
|
val expectedRoles = Set(first, second, third, fourth)
|
|
|
|
|
val expectedContacts = expectedRoles.map(node(_) / "system" / "receptionist")
|
|
|
|
|
|
|
|
|
|
// We need to slow down things otherwise our receptionists can sometimes tell us
|
|
|
|
|
// that our unreachableContact is unreachable before we get a chance to
|
|
|
|
|
// subscribe to events.
|
2019-02-09 15:25:39 +01:00
|
|
|
expectedRoles.foreach { role =>
|
2017-06-04 11:40:17 +02:00
|
|
|
testConductor.blackhole(client, role, Direction.Both).await
|
|
|
|
|
}
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
val c = system.actorOf(
|
|
|
|
|
ClusterClient.props(ClusterClientSettings(system).withInitialContacts(expectedContacts + unreachableContact)),
|
|
|
|
|
"client5")
|
2017-06-04 11:40:17 +02:00
|
|
|
|
|
|
|
|
val probe = TestProbe()
|
|
|
|
|
c.tell(SubscribeContactPoints, probe.ref)
|
|
|
|
|
|
2019-02-09 15:25:39 +01:00
|
|
|
expectedRoles.foreach { role =>
|
2017-06-04 11:40:17 +02:00
|
|
|
testConductor.passThrough(client, role, Direction.Both).await
|
|
|
|
|
}
|
|
|
|
|
|
2017-06-07 05:54:47 -07:00
|
|
|
probe.fishForMessage(10.seconds, "removal") {
|
2019-02-09 15:25:39 +01:00
|
|
|
case ContactPointRemoved(`unreachableContact`) => true
|
|
|
|
|
case _ => false
|
2017-06-04 11:40:17 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("after-7")
|
|
|
|
|
}
|
|
|
|
|
|
2015-02-06 08:43:28 +01:00
|
|
|
"re-establish connection to another receptionist when server is shutdown" in within(30 seconds) {
|
2013-04-14 22:30:09 +02:00
|
|
|
runOn(first, second, third, fourth) {
|
2013-04-17 22:14:19 +02:00
|
|
|
val service2 = system.actorOf(Props(classOf[TestService], testActor), "service2")
|
2015-05-08 08:49:08 +02:00
|
|
|
ClusterClientReceptionist(system).registerService(service2)
|
2013-04-14 22:30:09 +02:00
|
|
|
awaitCount(8)
|
|
|
|
|
}
|
|
|
|
|
enterBarrier("service2-replicated")
|
|
|
|
|
|
|
|
|
|
runOn(client) {
|
2019-03-11 10:38:24 +01:00
|
|
|
val client =
|
2019-03-13 10:56:20 +01:00
|
|
|
system.actorOf(
|
|
|
|
|
ClusterClient.props(ClusterClientSettings(system).withInitialContacts(initialContacts)),
|
|
|
|
|
"client2")
|
2013-04-14 22:30:09 +02:00
|
|
|
|
2017-06-07 05:54:47 -07:00
|
|
|
client ! ClusterClient.Send("/user/service2", "bonjour", localAffinity = true)
|
2015-08-18 17:27:42 +02:00
|
|
|
val reply = expectMsgType[Reply]
|
|
|
|
|
reply.msg should be("bonjour-ack")
|
|
|
|
|
val receptionistRoleName = roleName(reply.node) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Some(r) => r
|
|
|
|
|
case None => fail("unexpected missing roleName: " + reply.node)
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
2013-04-23 16:44:14 +02:00
|
|
|
testConductor.exit(receptionistRoleName, 0).await
|
2015-02-06 08:43:28 +01:00
|
|
|
remainingServerRoleNames -= receptionistRoleName
|
2017-06-07 05:54:47 -07:00
|
|
|
awaitAssert({
|
2017-06-07 18:00:02 +02:00
|
|
|
client ! ClusterClient.Send("/user/service2", "hi again", localAffinity = true)
|
2017-06-07 05:54:47 -07:00
|
|
|
expectMsgType[Reply](1 second).msg should be("hi again-ack")
|
|
|
|
|
}, max = remaining - 3.seconds)
|
2017-06-07 18:00:02 +02:00
|
|
|
system.stop(client)
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
enterBarrier("verifed-3")
|
|
|
|
|
receiveWhile(2 seconds) {
|
2019-02-09 15:25:39 +01:00
|
|
|
case "hi again" =>
|
|
|
|
|
case other => fail("unexpected message: " + other)
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
2016-05-04 04:50:16 -07:00
|
|
|
enterBarrier("verifed-4")
|
|
|
|
|
runOn(client) {
|
|
|
|
|
// Locate the test listener from a previous test and see that it agrees
|
|
|
|
|
// with what the client is telling it about what receptionists are alive
|
2017-06-07 05:54:47 -07:00
|
|
|
val listener = system.actorSelection("/user/reporter-client-listener")
|
2016-05-04 04:50:16 -07:00
|
|
|
val expectedContacts = remainingServerRoleNames.map(node(_) / "system" / "receptionist")
|
2017-06-07 05:54:47 -07:00
|
|
|
awaitAssert({
|
|
|
|
|
listener ! TestClientListener.GetLatestContactPoints
|
|
|
|
|
expectMsgType[LatestContactPoints].contactPoints should ===(expectedContacts)
|
|
|
|
|
}, max = 10.seconds)
|
2016-05-04 04:50:16 -07:00
|
|
|
}
|
2017-01-03 16:42:31 +01:00
|
|
|
enterBarrier("after-6")
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
|
2015-02-06 08:43:28 +01:00
|
|
|
"re-establish connection to receptionist after partition" in within(30 seconds) {
|
|
|
|
|
runOn(client) {
|
2019-03-13 10:56:20 +01:00
|
|
|
val c = system.actorOf(
|
|
|
|
|
ClusterClient.props(ClusterClientSettings(system).withInitialContacts(initialContacts)),
|
|
|
|
|
"client3")
|
2015-02-06 08:43:28 +01:00
|
|
|
|
|
|
|
|
c ! ClusterClient.Send("/user/service2", "bonjour2", localAffinity = true)
|
2015-08-18 17:27:42 +02:00
|
|
|
val reply = expectMsgType[Reply]
|
|
|
|
|
reply.msg should be("bonjour2-ack")
|
|
|
|
|
val receptionistRoleName = roleName(reply.node) match {
|
2019-02-09 15:25:39 +01:00
|
|
|
case Some(r) => r
|
|
|
|
|
case None => fail("unexpected missing roleName: " + reply.node)
|
2015-02-06 08:43:28 +01:00
|
|
|
}
|
|
|
|
|
// shutdown all but the one that the client is connected to
|
2019-02-09 15:25:39 +01:00
|
|
|
remainingServerRoleNames.foreach { r =>
|
2015-02-06 08:43:28 +01:00
|
|
|
if (r != receptionistRoleName)
|
|
|
|
|
testConductor.exit(r, 0).await
|
|
|
|
|
}
|
|
|
|
|
remainingServerRoleNames = Set(receptionistRoleName)
|
|
|
|
|
// network partition between client and server
|
|
|
|
|
testConductor.blackhole(client, receptionistRoleName, Direction.Both).await
|
|
|
|
|
c ! ClusterClient.Send("/user/service2", "ping", localAffinity = true)
|
|
|
|
|
// if we would use remote watch the failure detector would trigger and
|
|
|
|
|
// connection quarantined
|
2019-04-04 15:35:18 +02:00
|
|
|
expectNoMessage(5 seconds)
|
2015-02-06 08:43:28 +01:00
|
|
|
|
|
|
|
|
testConductor.passThrough(client, receptionistRoleName, Direction.Both).await
|
|
|
|
|
|
|
|
|
|
val expectedAddress = node(receptionistRoleName).address
|
|
|
|
|
awaitAssert {
|
2015-09-16 10:47:19 +02:00
|
|
|
val probe = TestProbe()
|
|
|
|
|
c.tell(ClusterClient.Send("/user/service2", "bonjour3", localAffinity = true), probe.ref)
|
|
|
|
|
val reply = probe.expectMsgType[Reply](1 second)
|
2015-08-18 17:27:42 +02:00
|
|
|
reply.msg should be("bonjour3-ack")
|
|
|
|
|
reply.node should be(expectedAddress)
|
2015-02-06 08:43:28 +01:00
|
|
|
}
|
|
|
|
|
system.stop(c)
|
|
|
|
|
}
|
|
|
|
|
|
2017-06-04 11:40:17 +02:00
|
|
|
enterBarrier("after-8")
|
2015-02-06 08:43:28 +01:00
|
|
|
}
|
|
|
|
|
|
2015-09-16 10:47:19 +02:00
|
|
|
"re-establish connection to receptionist after server restart" in within(30 seconds) {
|
|
|
|
|
runOn(client) {
|
|
|
|
|
remainingServerRoleNames.size should ===(1)
|
2019-02-09 15:25:39 +01:00
|
|
|
val remainingContacts = remainingServerRoleNames.map { r =>
|
2015-09-16 10:47:19 +02:00
|
|
|
node(r) / "system" / "receptionist"
|
|
|
|
|
}
|
2019-03-11 10:38:24 +01:00
|
|
|
val c =
|
2019-03-13 10:56:20 +01:00
|
|
|
system.actorOf(
|
|
|
|
|
ClusterClient.props(ClusterClientSettings(system).withInitialContacts(remainingContacts)),
|
|
|
|
|
"client4")
|
2015-09-16 10:47:19 +02:00
|
|
|
|
|
|
|
|
c ! ClusterClient.Send("/user/service2", "bonjour4", localAffinity = true)
|
|
|
|
|
expectMsg(10.seconds, Reply("bonjour4-ack", remainingContacts.head.address))
|
|
|
|
|
|
|
|
|
|
val logSource = s"${system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress}/user/client4"
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
EventFilter.info(start = "Connected to", source = logSource, occurrences = 1).intercept {
|
|
|
|
|
EventFilter.info(start = "Lost contact", source = logSource, occurrences = 1).intercept {
|
2015-09-16 10:47:19 +02:00
|
|
|
// shutdown server
|
|
|
|
|
testConductor.shutdown(remainingServerRoleNames.head).await
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
c ! ClusterClient.Send("/user/service2", "shutdown", localAffinity = true)
|
|
|
|
|
Thread.sleep(2000) // to ensure that it is sent out before shutting down system
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// There is only one client JVM and one server JVM left. The other JVMs have been exited
|
|
|
|
|
// by previous test steps. However, on the we don't know which server JVM that is left here
|
|
|
|
|
// so we let the following run on all server JVMs, but there is actually only one alive.
|
|
|
|
|
runOn(remainingServerRoleNames.toSeq: _*) {
|
|
|
|
|
Await.ready(system.whenTerminated, 20.seconds)
|
|
|
|
|
// start new system on same port
|
2016-09-16 12:29:49 +02:00
|
|
|
val port = Cluster(system).selfAddress.port.get
|
2019-03-13 10:56:20 +01:00
|
|
|
val sys2 = ActorSystem(
|
|
|
|
|
system.name,
|
|
|
|
|
ConfigFactory.parseString(s"""
|
2016-12-01 18:49:38 +01:00
|
|
|
akka.remote.artery.canonical.port=$port
|
2019-05-01 08:12:09 +01:00
|
|
|
akka.remote.classic.netty.tcp.port=$port
|
2016-12-01 18:49:38 +01:00
|
|
|
""").withFallback(system.settings.config))
|
2015-09-16 10:47:19 +02:00
|
|
|
Cluster(sys2).join(Cluster(sys2).selfAddress)
|
|
|
|
|
val service2 = sys2.actorOf(Props(classOf[TestService], testActor), "service2")
|
|
|
|
|
ClusterClientReceptionist(sys2).registerService(service2)
|
|
|
|
|
Await.ready(sys2.whenTerminated, 20.seconds)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
2013-04-14 22:30:09 +02:00
|
|
|
}
|
|
|
|
|
}
|