More tests working on artery

* non-multi-jvm tests from akka-cluster
* akka-cluster-metrics
* akka-cluster-tools
* akka-cluster-sharding
This commit is contained in:
Johan Andrén 2016-09-14 11:40:42 +02:00
parent 3b57947b1f
commit 848d56cc2f
11 changed files with 127 additions and 75 deletions

View file

@ -6,11 +6,13 @@ package akka.cluster.metrics
import scala.language.postfixOps import scala.language.postfixOps
import java.util.logging.LogManager import java.util.logging.LogManager
import org.slf4j.bridge.SLF4JBridgeHandler import org.slf4j.bridge.SLF4JBridgeHandler
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.actor.ExtendedActorSystem import akka.actor.ExtendedActorSystem
import akka.actor.Address import akka.actor.Address
import java.io.Closeable import java.io.Closeable
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.actor.Props import akka.actor.Props
import akka.actor.Actor import akka.actor.Actor
@ -22,6 +24,7 @@ import akka.actor.ActorLogging
import org.scalatest.mock.MockitoSugar import org.scalatest.mock.MockitoSugar
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
import akka.remote.RARP
/** /**
* Redirect different logging sources to SLF4J. * Redirect different logging sources to SLF4J.
@ -132,7 +135,7 @@ trait MetricsCollectorFactory { this: AkkaSpec ⇒
*/ */
class MockitoSigarMetricsCollector(system: ActorSystem) class MockitoSigarMetricsCollector(system: ActorSystem)
extends SigarMetricsCollector( extends SigarMetricsCollector(
Address("akka.tcp", system.name), Address(if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka" else "akka.tcp", system.name),
MetricsConfig.defaultDecayFactor, MetricsConfig.defaultDecayFactor,
MockitoSigarProvider().createSigarInstance) { MockitoSigarProvider().createSigarInstance) {
} }

View file

@ -16,6 +16,7 @@ import akka.remote.testkit.MultiNodeSpec
import akka.remote.testkit.STMultiNodeSpec import akka.remote.testkit.STMultiNodeSpec
import akka.testkit._ import akka.testkit._
import akka.cluster.pubsub._ import akka.cluster.pubsub._
import akka.remote.RARP
import akka.remote.transport.ThrottlerTransportAdapter.Direction import akka.remote.transport.ThrottlerTransportAdapter.Direction
import akka.util.Timeout import akka.util.Timeout

View file

@ -21,10 +21,12 @@ import akka.actor.ActorLogging
import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Status import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Status
import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Delta import akka.cluster.pubsub.DistributedPubSubMediator.Internal.Delta
import akka.actor.ActorSystem import akka.actor.ActorSystem
import scala.concurrent.Await import scala.concurrent.Await
import akka.actor.Identify import akka.actor.Identify
import akka.actor.RootActorPath import akka.actor.RootActorPath
import akka.actor.ActorIdentity import akka.actor.ActorIdentity
import akka.remote.RARP
object DistributedPubSubRestartSpec extends MultiNodeConfig { object DistributedPubSubRestartSpec extends MultiNodeConfig {
val first = role("first") val first = role("first")
@ -136,10 +138,16 @@ class DistributedPubSubRestartSpec extends MultiNodeSpec(DistributedPubSubRestar
runOn(third) { runOn(third) {
Await.result(system.whenTerminated, 10.seconds) Await.result(system.whenTerminated, 10.seconds)
val newSystem = ActorSystem( val newSystem = {
system.name, val port = Cluster(system).selfAddress.port.get
ConfigFactory.parseString(s"akka.remote.netty.tcp.port=${Cluster(system).selfAddress.port.get}").withFallback( val config = ConfigFactory.parseString(
system.settings.config)) if (RARP(system).provider.remoteSettings.Artery.Enabled) s"akka.remote.artery.canonical.port=$port"
else s"akka.remote.netty.tcp.port=$port"
).withFallback(system.settings.config)
ActorSystem(system.name, config)
}
try { try {
// don't join the old cluster // don't join the old cluster
Cluster(newSystem).join(Cluster(newSystem).selfAddress) Cluster(newSystem).join(Cluster(newSystem).selfAddress)

View file

@ -58,27 +58,27 @@ object ClusterSingletonProxySpec {
} }
} }
val cfg = """akka { val cfg = """
akka {
loglevel = INFO loglevel = INFO
cluster {
cluster { auto-down-unreachable-after = 10s
auto-down-unreachable-after = 10s min-nr-of-members = 2
}
min-nr-of-members = 2 actor.provider = "cluster"
} remote {
log-remote-lifecycle-events = off
actor.provider = "cluster" netty.tcp {
hostname = "127.0.0.1"
remote { port = 0
log-remote-lifecycle-events = off }
netty.tcp { artery.canonical {
hostname = "127.0.0.1" hostname = "127.0.0.1"
port = 0 port = 0
} }
} }
} }
""" """
class Singleton extends Actor with ActorLogging { class Singleton extends Actor with ActorLogging {

View file

@ -4,11 +4,11 @@
package akka.cluster.singleton package akka.cluster.singleton
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.ActorSystem import akka.actor.ActorSystem
import akka.actor.PoisonPill import akka.actor.PoisonPill
import akka.cluster.Cluster import akka.cluster.Cluster
import akka.cluster.MemberStatus import akka.cluster.MemberStatus
import akka.remote.RARP
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.TestActors import akka.testkit.TestActors
import akka.testkit.TestProbe import akka.testkit.TestProbe
@ -22,6 +22,10 @@ class ClusterSingletonRestartSpec extends AkkaSpec("""
hostname = "127.0.0.1" hostname = "127.0.0.1"
port = 0 port = 0
} }
artery.canonical {
hostname = "127.0.0.1"
port = 0
}
} }
""") { """) {
@ -64,10 +68,17 @@ class ClusterSingletonRestartSpec extends AkkaSpec("""
shutdown(sys1) shutdown(sys1)
// it will be downed by the join attempts of the new incarnation // it will be downed by the join attempts of the new incarnation
sys3 = ActorSystem( sys3 = {
system.name, val sys1port = Cluster(sys1).selfAddress.port.get
ConfigFactory.parseString(s"akka.remote.netty.tcp.port=${Cluster(sys1).selfAddress.port.get}").withFallback(
system.settings.config)) val sys3Config =
ConfigFactory.parseString(
if (RARP(sys1).provider.remoteSettings.Artery.Enabled) s"akka.remote.artery.canonical.port=$sys1port"
else s"akka.remote.netty.tcp.port=$sys1port"
).withFallback(system.settings.config)
ActorSystem(system.name, sys3Config)
}
join(sys3, sys2) join(sys3, sys2)
within(5.seconds) { within(5.seconds) {

View file

@ -11,16 +11,14 @@ import akka.actor.ActorRef
import akka.actor.Props import akka.actor.Props
import akka.cluster.MemberStatus._ import akka.cluster.MemberStatus._
import akka.cluster.ClusterEvent._ import akka.cluster.ClusterEvent._
import akka.remote.RARP
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
object AutoDownSpec { object AutoDownSpec {
final case class DownCalled(address: Address) final case class DownCalled(address: Address)
val memberA = TestMember(Address("akka.tcp", "sys", "a", 2552), Up)
val memberB = TestMember(Address("akka.tcp", "sys", "b", 2552), Up)
val memberC = TestMember(Address("akka.tcp", "sys", "c", 2552), Up)
class AutoDownTestActor( class AutoDownTestActor(
memberA: Member,
autoDownUnreachableAfter: FiniteDuration, autoDownUnreachableAfter: FiniteDuration,
probe: ActorRef) probe: ActorRef)
extends AutoDownBase(autoDownUnreachableAfter) { extends AutoDownBase(autoDownUnreachableAfter) {
@ -36,13 +34,22 @@ object AutoDownSpec {
} }
} }
} }
class AutoDownSpec extends AkkaSpec { class AutoDownSpec extends AkkaSpec("akka.actor.provider=remote") {
import AutoDownSpec._ import AutoDownSpec._
val protocol =
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
else "akka.tcp"
val memberA = TestMember(Address(protocol, "sys", "a", 2552), Up)
val memberB = TestMember(Address(protocol, "sys", "b", 2552), Up)
val memberC = TestMember(Address(protocol, "sys", "c", 2552), Up)
def autoDownActor(autoDownUnreachableAfter: FiniteDuration): ActorRef = def autoDownActor(autoDownUnreachableAfter: FiniteDuration): ActorRef =
system.actorOf(Props(classOf[AutoDownTestActor], autoDownUnreachableAfter, testActor)) system.actorOf(Props(classOf[AutoDownTestActor], memberA, autoDownUnreachableAfter, testActor))
"AutoDown" must { "AutoDown" must {

View file

@ -17,6 +17,7 @@ import akka.cluster.ClusterEvent._
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.testkit.ImplicitSender import akka.testkit.ImplicitSender
import akka.actor.ActorRef import akka.actor.ActorRef
import akka.remote.RARP
import akka.testkit.TestProbe import akka.testkit.TestProbe
object ClusterDomainEventPublisherSpec { object ClusterDomainEventPublisherSpec {
@ -29,18 +30,22 @@ object ClusterDomainEventPublisherSpec {
class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublisherSpec.config) class ClusterDomainEventPublisherSpec extends AkkaSpec(ClusterDomainEventPublisherSpec.config)
with BeforeAndAfterEach with ImplicitSender { with BeforeAndAfterEach with ImplicitSender {
val protocol =
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
else "akka.tcp"
var publisher: ActorRef = _ var publisher: ActorRef = _
val aUp = TestMember(Address("akka.tcp", "sys", "a", 2552), Up) val aUp = TestMember(Address(protocol, "sys", "a", 2552), Up)
val aLeaving = aUp.copy(status = Leaving) val aLeaving = aUp.copy(status = Leaving)
val aExiting = aLeaving.copy(status = Exiting) val aExiting = aLeaving.copy(status = Exiting)
val aRemoved = aExiting.copy(status = Removed) val aRemoved = aExiting.copy(status = Removed)
val bExiting = TestMember(Address("akka.tcp", "sys", "b", 2552), Exiting) val bExiting = TestMember(Address(protocol, "sys", "b", 2552), Exiting)
val bRemoved = bExiting.copy(status = Removed) val bRemoved = bExiting.copy(status = Removed)
val cJoining = TestMember(Address("akka.tcp", "sys", "c", 2552), Joining, Set("GRP")) val cJoining = TestMember(Address(protocol, "sys", "c", 2552), Joining, Set("GRP"))
val cUp = cJoining.copy(status = Up) val cUp = cJoining.copy(status = Up)
val cRemoved = cUp.copy(status = Removed) val cRemoved = cUp.copy(status = Removed)
val a51Up = TestMember(Address("akka.tcp", "sys", "a", 2551), Up) val a51Up = TestMember(Address(protocol, "sys", "a", 2551), Up)
val dUp = TestMember(Address("akka.tcp", "sys", "d", 2552), Up, Set("GRP")) val dUp = TestMember(Address(protocol, "sys", "d", 2552), Up, Set("GRP"))
val g0 = Gossip(members = SortedSet(aUp)).seen(aUp.uniqueAddress) val g0 = Gossip(members = SortedSet(aUp)).seen(aUp.uniqueAddress)
val g1 = Gossip(members = SortedSet(aUp, cJoining)).seen(aUp.uniqueAddress).seen(cJoining.uniqueAddress) val g1 = Gossip(members = SortedSet(aUp, cJoining)).seen(aUp.uniqueAddress).seen(cJoining.uniqueAddress)

View file

@ -7,6 +7,7 @@ package akka.cluster.routing
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import akka.actor.Address import akka.actor.Address
import akka.actor.RootActorPath import akka.actor.RootActorPath
import akka.remote.RARP
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import akka.routing.ActorSelectionRoutee import akka.routing.ActorSelectionRoutee
import akka.routing.ActorRefRoutee import akka.routing.ActorRefRoutee
@ -16,10 +17,14 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
akka.remote.netty.tcp.port = 0 akka.remote.netty.tcp.port = 0
""")) { """)) {
val a1 = Address("akka.tcp", "sys", "a1", 2551) val protocol =
val b1 = Address("akka.tcp", "sys", "b1", 2551) if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
val c1 = Address("akka.tcp", "sys", "c1", 2551) else "akka.tcp"
val d1 = Address("akka.tcp", "sys", "d1", 2551)
val a1 = Address(protocol, "sys", "a1", 2551)
val b1 = Address(protocol, "sys", "b1", 2551)
val c1 = Address(protocol, "sys", "c1", 2551)
val d1 = Address(protocol, "sys", "d1", 2551)
val routeeA = ActorSelectionRoutee(system.actorSelection(RootActorPath(a1) / "user" / "a")) val routeeA = ActorSelectionRoutee(system.actorSelection(RootActorPath(a1) / "user" / "a"))
val routeeB = ActorSelectionRoutee(system.actorSelection(RootActorPath(b1) / "user" / "b")) val routeeB = ActorSelectionRoutee(system.actorSelection(RootActorPath(b1) / "user" / "b"))

View file

@ -13,6 +13,7 @@ import akka.actor.ActorRef
import akka.cluster.ddata.Replicator.Internal._ import akka.cluster.ddata.Replicator.Internal._
import akka.cluster.ddata.Replicator._ import akka.cluster.ddata.Replicator._
import akka.actor.ActorSelection import akka.actor.ActorSelection
import akka.remote.RARP
object WriteAggregatorSpec { object WriteAggregatorSpec {
@ -55,7 +56,11 @@ class WriteAggregatorSpec extends AkkaSpec("""
""") """)
with ImplicitSender { with ImplicitSender {
val nodeA = Address("akka.tcp", "Sys", "a", 2552) val protocol =
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
else "akka.tcp"
val nodeA = Address(protocol, "Sys", "a", 2552)
val nodeB = nodeA.copy(host = Some("b")) val nodeB = nodeA.copy(host = Some("b"))
val nodeC = nodeA.copy(host = Some("c")) val nodeC = nodeA.copy(host = Some("c"))
val nodeD = nodeA.copy(host = Some("d")) val nodeD = nodeA.copy(host = Some("d"))

View file

@ -28,6 +28,10 @@ akka {
} }
""")) with ImplicitSender with DefaultTimeout with DeathWatchSpec { """)) with ImplicitSender with DefaultTimeout with DeathWatchSpec {
val protocol =
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
else "akka.tcp"
val other = ActorSystem("other", ConfigFactory.parseString("akka.remote.netty.tcp.port=2666") val other = ActorSystem("other", ConfigFactory.parseString("akka.remote.netty.tcp.port=2666")
.withFallback(system.settings.config)) .withFallback(system.settings.config))
@ -49,7 +53,7 @@ akka {
// pick an unused port // pick an unused port
val port = SocketUtil.temporaryServerAddress().getPort val port = SocketUtil.temporaryServerAddress().getPort
// simulate de-serialized ActorRef // simulate de-serialized ActorRef
val ref = rarp.resolveActorRef(s"akka.tcp://OtherSystem@localhost:$port/user/foo/bar#1752527294") val ref = rarp.resolveActorRef(s"$protocol://OtherSystem@localhost:$port/user/foo/bar#1752527294")
system.actorOf(Props(new Actor { system.actorOf(Props(new Actor {
context.watch(ref) context.watch(ref)
def receive = { def receive = {
@ -67,7 +71,7 @@ akka {
} }
"receive Terminated when watched node is unknown host" in { "receive Terminated when watched node is unknown host" in {
val path = RootActorPath(Address("akka.tcp", system.name, "unknownhost", 2552)) / "user" / "subject" val path = RootActorPath(Address(protocol, system.name, "unknownhost", 2552)) / "user" / "subject"
system.actorOf(Props(new Actor { system.actorOf(Props(new Actor {
context.watch(context.actorFor(path)) context.watch(context.actorFor(path))
def receive = { def receive = {
@ -79,7 +83,7 @@ akka {
} }
"receive ActorIdentity(None) when identified node is unknown host" in { "receive ActorIdentity(None) when identified node is unknown host" in {
val path = RootActorPath(Address("akka.tcp", system.name, "unknownhost2", 2552)) / "user" / "subject" val path = RootActorPath(Address(protocol, system.name, "unknownhost2", 2552)) / "user" / "subject"
system.actorSelection(path) ! Identify(path) system.actorSelection(path) ! Identify(path)
expectMsg(60.seconds, ActorIdentity(path, None)) expectMsg(60.seconds, ActorIdentity(path, None))
} }
@ -87,7 +91,7 @@ akka {
"quarantine systems after unsuccessful system message delivery if have not communicated before" in { "quarantine systems after unsuccessful system message delivery if have not communicated before" in {
// Synthesize an ActorRef to a remote system this one has never talked to before. // Synthesize an ActorRef to a remote system this one has never talked to before.
// This forces ReliableDeliverySupervisor to start with unknown remote system UID. // This forces ReliableDeliverySupervisor to start with unknown remote system UID.
val extinctPath = RootActorPath(Address("akka.tcp", "extinct-system", "localhost", SocketUtil.temporaryServerAddress().getPort)) / "user" / "noone" val extinctPath = RootActorPath(Address(protocol, "extinct-system", "localhost", SocketUtil.temporaryServerAddress().getPort)) / "user" / "noone"
val transport = RARP(system).provider.transport val transport = RARP(system).provider.transport
val extinctRef = new RemoteActorRef(transport, transport.localAddressForRemote(extinctPath.address), val extinctRef = new RemoteActorRef(transport, transport.localAddressForRemote(extinctPath.address),
extinctPath, Nobody, props = None, deploy = None) extinctPath, Nobody, props = None, deploy = None)

View file

@ -45,6 +45,9 @@ class RemoteRouterSpec extends AkkaSpec("""
val port = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get val port = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get
val sysName = system.name val sysName = system.name
val protocol =
if (RARP(system).provider.remoteSettings.Artery.Enabled) "akka"
else "akka.tcp"
val conf = ConfigFactory.parseString( val conf = ConfigFactory.parseString(
s""" s"""
akka { akka {
@ -52,7 +55,7 @@ class RemoteRouterSpec extends AkkaSpec("""
/blub { /blub {
router = round-robin-pool router = round-robin-pool
nr-of-instances = 2 nr-of-instances = 2
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"] target.nodes = ["$protocol://${sysName}@localhost:${port}"]
} }
/elastic-blub { /elastic-blub {
router = round-robin-pool router = round-robin-pool
@ -60,10 +63,10 @@ class RemoteRouterSpec extends AkkaSpec("""
lower-bound = 2 lower-bound = 2
upper-bound = 3 upper-bound = 3
} }
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"] target.nodes = ["$protocol://${sysName}@localhost:${port}"]
} }
/remote-blub { /remote-blub {
remote = "akka.tcp://${sysName}@localhost:${port}" remote = "$protocol://${sysName}@localhost:${port}"
router = round-robin-pool router = round-robin-pool
nr-of-instances = 2 nr-of-instances = 2
} }
@ -71,12 +74,12 @@ class RemoteRouterSpec extends AkkaSpec("""
remote = "akka://MasterRemoteRouterSpec" remote = "akka://MasterRemoteRouterSpec"
router = round-robin-pool router = round-robin-pool
nr-of-instances = 2 nr-of-instances = 2
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"] target.nodes = ["$protocol://${sysName}@localhost:${port}"]
} }
/local-blub2 { /local-blub2 {
router = round-robin-pool router = round-robin-pool
nr-of-instances = 4 nr-of-instances = 4
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"] target.nodes = ["$protocol://${sysName}@localhost:${port}"]
} }
} }
}""").withFallback(system.settings.config) }""").withFallback(system.settings.config)
@ -104,7 +107,7 @@ class RemoteRouterSpec extends AkkaSpec("""
val children = replies.toSet val children = replies.toSet
children should have size 2 children should have size 2
children.map(_.parent) should have size 1 children.map(_.parent) should have size 1
children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}"))
masterSystem.stop(router) masterSystem.stop(router)
} }
@ -112,12 +115,12 @@ class RemoteRouterSpec extends AkkaSpec("""
val probe = TestProbe()(masterSystem) val probe = TestProbe()(masterSystem)
val router = masterSystem.actorOf(new RemoteRouterConfig( val router = masterSystem.actorOf(new RemoteRouterConfig(
RoundRobinPool(2), RoundRobinPool(2),
Seq(Address("akka.tcp", sysName, "localhost", port))).props(echoActorProps), "blub2") Seq(Address(protocol, sysName, "localhost", port))).props(echoActorProps), "blub2")
val replies = collectRouteePaths(probe, router, 5) val replies = collectRouteePaths(probe, router, 5)
val children = replies.toSet val children = replies.toSet
children should have size 2 children should have size 2
children.map(_.parent) should have size 1 children.map(_.parent) should have size 1
children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}"))
masterSystem.stop(router) masterSystem.stop(router)
} }
@ -128,81 +131,81 @@ class RemoteRouterSpec extends AkkaSpec("""
val children = replies.toSet val children = replies.toSet
children.size should be >= 2 children.size should be >= 2
children.map(_.parent) should have size 1 children.map(_.parent) should have size 1
children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}"))
masterSystem.stop(router) masterSystem.stop(router)
} }
"deploy remote routers based on configuration" in { "deploy remote routers based on configuration" in {
val probe = TestProbe()(masterSystem) val probe = TestProbe()(masterSystem)
val router = masterSystem.actorOf(FromConfig.props(echoActorProps), "remote-blub") val router = masterSystem.actorOf(FromConfig.props(echoActorProps), "remote-blub")
router.path.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}") router.path.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")
val replies = collectRouteePaths(probe, router, 5) val replies = collectRouteePaths(probe, router, 5)
val children = replies.toSet val children = replies.toSet
children should have size 2 children should have size 2
val parents = children.map(_.parent) val parents = children.map(_.parent)
parents should have size 1 parents should have size 1
parents.head should ===(router.path) parents.head should ===(router.path)
children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}"))
masterSystem.stop(router) masterSystem.stop(router)
} }
"deploy remote routers based on explicit deployment" in { "deploy remote routers based on explicit deployment" in {
val probe = TestProbe()(masterSystem) val probe = TestProbe()(masterSystem)
val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps)
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "remote-blub2") .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "remote-blub2")
router.path.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}") router.path.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")
val replies = collectRouteePaths(probe, router, 5) val replies = collectRouteePaths(probe, router, 5)
val children = replies.toSet val children = replies.toSet
children should have size 2 children should have size 2
val parents = children.map(_.parent) val parents = children.map(_.parent)
parents should have size 1 parents should have size 1
parents.head should ===(router.path) parents.head should ===(router.path)
children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}"))
masterSystem.stop(router) masterSystem.stop(router)
} }
"let remote deployment be overridden by local configuration" in { "let remote deployment be overridden by local configuration" in {
val probe = TestProbe()(masterSystem) val probe = TestProbe()(masterSystem)
val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps)
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "local-blub") .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "local-blub")
router.path.address.toString should ===("akka://MasterRemoteRouterSpec") router.path.address.toString should ===("akka://MasterRemoteRouterSpec")
val replies = collectRouteePaths(probe, router, 5) val replies = collectRouteePaths(probe, router, 5)
val children = replies.toSet val children = replies.toSet
children should have size 2 children should have size 2
val parents = children.map(_.parent) val parents = children.map(_.parent)
parents should have size 1 parents should have size 1
parents.head.address should ===(Address("akka.tcp", sysName, "localhost", port)) parents.head.address should ===(Address(protocol, sysName, "localhost", port))
children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}"))
masterSystem.stop(router) masterSystem.stop(router)
} }
"let remote deployment router be overridden by local configuration" in { "let remote deployment router be overridden by local configuration" in {
val probe = TestProbe()(masterSystem) val probe = TestProbe()(masterSystem)
val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps)
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "local-blub2") .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "local-blub2")
router.path.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}") router.path.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")
val replies = collectRouteePaths(probe, router, 5) val replies = collectRouteePaths(probe, router, 5)
val children = replies.toSet val children = replies.toSet
children should have size 4 children should have size 4
val parents = children.map(_.parent) val parents = children.map(_.parent)
parents should have size 1 parents should have size 1
parents.head should ===(router.path) parents.head should ===(router.path)
children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}"))
masterSystem.stop(router) masterSystem.stop(router)
} }
"let remote deployment be overridden by remote configuration" in { "let remote deployment be overridden by remote configuration" in {
val probe = TestProbe()(masterSystem) val probe = TestProbe()(masterSystem)
val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps) val router = masterSystem.actorOf(RoundRobinPool(2).props(echoActorProps)
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "remote-override") .withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"$protocol://${sysName}@localhost:${port}")))), "remote-override")
router.path.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}") router.path.address.toString should ===(s"$protocol://${sysName}@localhost:${port}")
val replies = collectRouteePaths(probe, router, 5) val replies = collectRouteePaths(probe, router, 5)
val children = replies.toSet val children = replies.toSet
children should have size 4 children should have size 4
val parents = children.map(_.parent) val parents = children.map(_.parent)
parents should have size 1 parents should have size 1
parents.head should ===(router.path) parents.head should ===(router.path)
children foreach (_.address.toString should ===(s"akka.tcp://${sysName}@localhost:${port}")) children foreach (_.address.toString should ===(s"$protocol://${sysName}@localhost:${port}"))
masterSystem.stop(router) masterSystem.stop(router)
} }
@ -213,7 +216,7 @@ class RemoteRouterSpec extends AkkaSpec("""
} }
val router = masterSystem.actorOf(new RemoteRouterConfig( val router = masterSystem.actorOf(new RemoteRouterConfig(
RoundRobinPool(1, supervisorStrategy = escalator), RoundRobinPool(1, supervisorStrategy = escalator),
Seq(Address("akka.tcp", sysName, "localhost", port))).props(Props.empty), "blub3") Seq(Address(protocol, sysName, "localhost", port))).props(Props.empty), "blub3")
router.tell(GetRoutees, probe.ref) router.tell(GetRoutees, probe.ref)
EventFilter[ActorKilledException](occurrences = 1).intercept { EventFilter[ActorKilledException](occurrences = 1).intercept {