!act,rem,clu #3549 Simplify and enhance routers
* Separate routing logic, to be usable stand alone, e.g. in actors * Simplify RouterConfig, only a factory * Move reading of config from Deployer to the RouterConfig * Distiction between Pool and Group router types * Remove usage of actorFor, use ActorSelection * Management messages to add and remove routees * Simplify the internals of RoutedActorCell & co * Move resize specific code to separate RoutedActorCell subclass * Change resizer api to only return capacity change * Resizer only allowed together with Pool * Re-implement all routers, and keep old api during deprecation phase * Replace ClusterRouterConfig, deprecation * Rewrite documentation * Migration guide * Also includes related ticket: +act #3087 Create nicer Props factories for RouterConfig
This commit is contained in:
parent
81ca6fe8c8
commit
ebadd567b2
104 changed files with 9671 additions and 5006 deletions
|
|
@ -136,30 +136,30 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
akka.actor.deployment {
|
||||
/master-node-1/workers {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 100
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 1
|
||||
allow-local-routees = off
|
||||
allow-local-routees = on
|
||||
}
|
||||
}
|
||||
/master-node-2/workers {
|
||||
router = round-robin
|
||||
router = round-robin-group
|
||||
nr-of-instances = 100
|
||||
cluster {
|
||||
enabled = on
|
||||
routees-path = "/user/worker"
|
||||
allow-local-routees = off
|
||||
allow-local-routees = on
|
||||
}
|
||||
}
|
||||
/master-node-3/workers = {
|
||||
router = adaptive
|
||||
router = adaptive-pool
|
||||
nr-of-instances = 100
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 1
|
||||
allow-local-routees = off
|
||||
allow-local-routees = on
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -486,7 +486,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
* itself.
|
||||
*/
|
||||
class Master(settings: StressMultiJvmSpec.Settings, batchInterval: FiniteDuration, tree: Boolean) extends Actor {
|
||||
val workers = context.actorOf(Props[Worker].withRouter(FromConfig), "workers")
|
||||
val workers = context.actorOf(FromConfig.props(Props[Worker]), "workers")
|
||||
val payload = Array.fill(settings.payloadSize)(ThreadLocalRandom.current.nextInt(127).toByte)
|
||||
val retryTimeout = 5.seconds.dilated(context.system)
|
||||
val idCounter = Iterator from 0
|
||||
|
|
@ -1314,6 +1314,6 @@ abstract class StressSpec
|
|||
}
|
||||
enterBarrier("after-" + step)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -28,6 +28,7 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
|
|||
akka.loggers = ["akka.testkit.TestEventListener"]
|
||||
akka.loglevel = INFO
|
||||
akka.remote.log-remote-lifecycle-events = off
|
||||
akka.cluster.failure-detector.monitored-by-nr-of-members = 3
|
||||
"""))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,212 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
import java.lang.management.ManagementFactory
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
import akka.actor._
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.MultiNodeClusterSpec
|
||||
import akka.cluster.NodeMetrics
|
||||
import akka.pattern.ask
|
||||
import akka.remote.testkit.{ MultiNodeSpec, MultiNodeConfig }
|
||||
import akka.routing.CurrentRoutees
|
||||
import akka.routing.FromConfig
|
||||
import akka.routing.RouterRoutees
|
||||
import akka.testkit.{ LongRunningTest, DefaultTimeout, ImplicitSender }
|
||||
import akka.cluster.routing._
|
||||
|
||||
object AdaptiveLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class Routee extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! Reply(Cluster(context.system).selfAddress)
|
||||
}
|
||||
}
|
||||
|
||||
class Memory extends Actor with ActorLogging {
|
||||
var usedMemory: Array[Array[Int]] = _
|
||||
def receive = {
|
||||
case AllocateMemory ⇒
|
||||
val heap = ManagementFactory.getMemoryMXBean.getHeapMemoryUsage
|
||||
// getMax can be undefined (-1)
|
||||
val max = math.max(heap.getMax, heap.getCommitted)
|
||||
val used = heap.getUsed
|
||||
log.debug("used heap before: [{}] bytes, of max [{}]", used, heap.getMax)
|
||||
// allocate 70% of free space
|
||||
val allocateBytes = (0.7 * (max - used)).toInt
|
||||
val numberOfArrays = allocateBytes / 1024
|
||||
usedMemory = Array.ofDim(numberOfArrays, 248) // each 248 element Int array will use ~ 1 kB
|
||||
log.debug("used heap after: [{}] bytes", ManagementFactory.getMemoryMXBean.getHeapMemoryUsage.getUsed)
|
||||
sender ! "done"
|
||||
}
|
||||
}
|
||||
|
||||
case object AllocateMemory
|
||||
case class Reply(address: Address)
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster.metrics.collect-interval = 1s
|
||||
akka.cluster.metrics.gossip-interval = 1s
|
||||
akka.cluster.metrics.moving-average-half-life = 2s
|
||||
akka.actor.deployment {
|
||||
/router3 = {
|
||||
router = adaptive
|
||||
metrics-selector = cpu
|
||||
nr-of-instances = 9
|
||||
}
|
||||
/router4 = {
|
||||
router = adaptive
|
||||
metrics-selector = "akka.cluster.routing.TestCustomMetricsSelector"
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 2
|
||||
}
|
||||
}
|
||||
}
|
||||
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
}
|
||||
|
||||
class TestCustomMetricsSelector(config: Config) extends MetricsSelector {
|
||||
override def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] = Map.empty
|
||||
}
|
||||
|
||||
class AdaptiveLoadBalancingRouterMultiJvmNode1 extends AdaptiveLoadBalancingRouterSpec
|
||||
class AdaptiveLoadBalancingRouterMultiJvmNode2 extends AdaptiveLoadBalancingRouterSpec
|
||||
class AdaptiveLoadBalancingRouterMultiJvmNode3 extends AdaptiveLoadBalancingRouterSpec
|
||||
|
||||
abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoadBalancingRouterMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with DefaultTimeout {
|
||||
import AdaptiveLoadBalancingRouterMultiJvmSpec._
|
||||
|
||||
def currentRoutees(router: ActorRef) =
|
||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees
|
||||
|
||||
def receiveReplies(expectedReplies: Int): Map[Address, Int] = {
|
||||
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
|
||||
(receiveWhile(5 seconds, messages = expectedReplies) {
|
||||
case Reply(address) ⇒ address
|
||||
}).foldLeft(zero) {
|
||||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fills in self address for local ActorRef
|
||||
*/
|
||||
def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
|
||||
case Address(_, _, None, None) ⇒ cluster.selfAddress
|
||||
case a ⇒ a
|
||||
}
|
||||
|
||||
def startRouter(name: String): ActorRef = {
|
||||
val router = system.actorOf(Props[Routee].withRouter(ClusterRouterConfig(
|
||||
local = AdaptiveLoadBalancingRouter(HeapMetricsSelector),
|
||||
settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 1, useRole = None))), name)
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router).size must be(roles.size) }
|
||||
currentRoutees(router).map(fullAddress).toSet must be(roles.map(address).toSet)
|
||||
router
|
||||
}
|
||||
|
||||
"A cluster with a AdaptiveLoadBalancingRouter" must {
|
||||
"start cluster nodes" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(roles: _*)
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"use all nodes in the cluster when not overloaded" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router1 = startRouter("router1")
|
||||
|
||||
// collect some metrics before we start
|
||||
Thread.sleep(cluster.settings.MetricsInterval.toMillis * 10)
|
||||
|
||||
val iterationCount = 100
|
||||
1 to iterationCount foreach { _ ⇒
|
||||
router1 ! "hit"
|
||||
// wait a while between each message, since metrics is collected periodically
|
||||
Thread.sleep(10)
|
||||
}
|
||||
|
||||
val replies = receiveReplies(iterationCount)
|
||||
|
||||
replies(first) must be > (0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be > (0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
|
||||
}
|
||||
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"prefer node with more free heap capacity" taggedAs LongRunningTest in {
|
||||
System.gc()
|
||||
enterBarrier("gc")
|
||||
|
||||
runOn(second) {
|
||||
within(20.seconds) {
|
||||
system.actorOf(Props[Memory], "memory") ! AllocateMemory
|
||||
expectMsg("done")
|
||||
}
|
||||
}
|
||||
enterBarrier("heap-allocated")
|
||||
|
||||
runOn(first) {
|
||||
val router2 = startRouter("router2")
|
||||
|
||||
// collect some metrics before we start
|
||||
Thread.sleep(cluster.settings.MetricsInterval.toMillis * 10)
|
||||
|
||||
val iterationCount = 3000
|
||||
1 to iterationCount foreach { _ ⇒
|
||||
router2 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(iterationCount)
|
||||
|
||||
replies(third) must be > (replies(second))
|
||||
replies.values.sum must be(iterationCount)
|
||||
|
||||
}
|
||||
|
||||
enterBarrier("after-3")
|
||||
}
|
||||
|
||||
"create routees from configuration" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router3 = system.actorOf(Props[Memory].withRouter(FromConfig()), "router3")
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router3).size must be(9) }
|
||||
currentRoutees(router3).map(fullAddress).toSet must be(Set(address(first)))
|
||||
}
|
||||
enterBarrier("after-4")
|
||||
}
|
||||
|
||||
"create routees from cluster.enabled configuration" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router4 = system.actorOf(Props[Memory].withRouter(FromConfig()), "router4")
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router4).size must be(6) }
|
||||
currentRoutees(router4).map(fullAddress).toSet must be(Set(
|
||||
address(first), address(second), address(third)))
|
||||
}
|
||||
enterBarrier("after-5")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,175 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster.oldrouting
|
||||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Address
|
||||
import akka.actor.Props
|
||||
import akka.cluster.MultiNodeClusterSpec
|
||||
import akka.pattern.ask
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.routing.ConsistentHashingRouter
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
|
||||
import akka.routing.CurrentRoutees
|
||||
import akka.routing.FromConfig
|
||||
import akka.routing.RouterRoutees
|
||||
import akka.testkit._
|
||||
import akka.cluster.routing._
|
||||
|
||||
object ClusterConsistentHashingRouterMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("""
|
||||
common-router-settings = {
|
||||
router = consistent-hashing
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 2
|
||||
}
|
||||
}
|
||||
|
||||
akka.actor.deployment {
|
||||
/router1 = ${common-router-settings}
|
||||
/router3 = ${common-router-settings}
|
||||
/router4 = ${common-router-settings}
|
||||
}
|
||||
""")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
}
|
||||
|
||||
class ClusterConsistentHashingRouterMultiJvmNode1 extends ClusterConsistentHashingRouterSpec
|
||||
class ClusterConsistentHashingRouterMultiJvmNode2 extends ClusterConsistentHashingRouterSpec
|
||||
class ClusterConsistentHashingRouterMultiJvmNode3 extends ClusterConsistentHashingRouterSpec
|
||||
|
||||
abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterConsistentHashingRouterMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with DefaultTimeout {
|
||||
import ClusterConsistentHashingRouterMultiJvmSpec._
|
||||
|
||||
lazy val router1 = system.actorOf(Props[Echo].withRouter(FromConfig()), "router1")
|
||||
|
||||
def currentRoutees(router: ActorRef) =
|
||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees
|
||||
|
||||
/**
|
||||
* Fills in self address for local ActorRef
|
||||
*/
|
||||
private def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
|
||||
case Address(_, _, None, None) ⇒ cluster.selfAddress
|
||||
case a ⇒ a
|
||||
}
|
||||
|
||||
"A cluster router with a consistent hashing router" must {
|
||||
"start cluster with 2 nodes" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(first, second)
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"create routees from configuration" in {
|
||||
runOn(first) {
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router1).size must be(4) }
|
||||
currentRoutees(router1).map(fullAddress).toSet must be(Set(address(first), address(second)))
|
||||
}
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"select destination based on hashKey" in {
|
||||
runOn(first) {
|
||||
router1 ! ConsistentHashableEnvelope(message = "A", hashKey = "a")
|
||||
val destinationA = expectMsgType[ActorRef]
|
||||
router1 ! ConsistentHashableEnvelope(message = "AA", hashKey = "a")
|
||||
expectMsg(destinationA)
|
||||
}
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"deploy routees to new member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
runOn(first) {
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router1).size must be(6) }
|
||||
currentRoutees(router1).map(fullAddress).toSet must be(roles.map(address).toSet)
|
||||
}
|
||||
|
||||
enterBarrier("after-3")
|
||||
}
|
||||
|
||||
"deploy programatically defined routees to the member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router2 = system.actorOf(Props[Echo].withRouter(ClusterRouterConfig(local = ConsistentHashingRouter(),
|
||||
settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 2, useRole = None))), "router2")
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router2).size must be(6) }
|
||||
currentRoutees(router2).map(fullAddress).toSet must be(roles.map(address).toSet)
|
||||
}
|
||||
|
||||
enterBarrier("after-4")
|
||||
}
|
||||
|
||||
"handle combination of configured router and programatically defined hashMapping" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
def hashMapping: ConsistentHashMapping = {
|
||||
case s: String ⇒ s
|
||||
}
|
||||
|
||||
val router3 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter(hashMapping = hashMapping)), "router3")
|
||||
|
||||
assertHashMapping(router3)
|
||||
}
|
||||
|
||||
enterBarrier("after-5")
|
||||
}
|
||||
|
||||
"handle combination of configured router and programatically defined hashMapping and ClusterRouterConfig" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
def hashMapping: ConsistentHashMapping = {
|
||||
case s: String ⇒ s
|
||||
}
|
||||
|
||||
val router4 = system.actorOf(Props[Echo].withRouter(ClusterRouterConfig(
|
||||
local = ConsistentHashingRouter(hashMapping = hashMapping),
|
||||
settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 1, useRole = None))), "router4")
|
||||
|
||||
assertHashMapping(router4)
|
||||
}
|
||||
|
||||
enterBarrier("after-6")
|
||||
}
|
||||
|
||||
def assertHashMapping(router: ActorRef): Unit = {
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router).size must be(6) }
|
||||
currentRoutees(router).map(fullAddress).toSet must be(roles.map(address).toSet)
|
||||
|
||||
router ! "a"
|
||||
val destinationA = expectMsgType[ActorRef]
|
||||
router ! "a"
|
||||
expectMsg(destinationA)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -1,7 +1,7 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster.routing
|
||||
package akka.cluster.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -22,6 +22,7 @@ import akka.routing.RoutedActorRef
|
|||
import akka.routing.RouterRoutees
|
||||
import akka.testkit._
|
||||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import akka.cluster.routing._
|
||||
|
||||
object ClusterRoundRobinRoutedActorMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
|
|
@ -232,7 +233,7 @@ abstract class ClusterRoundRobinRoutedActorSpec extends MultiNodeSpec(ClusterRou
|
|||
enterBarrier("after-5")
|
||||
}
|
||||
|
||||
"deploy routees to only remote nodes when allow-local-routees = off" taggedAs LongRunningTest in {
|
||||
"deploy routees to only remote nodes when allow-local-routees = off" taggedAs LongRunningTest in within(15.seconds) {
|
||||
|
||||
runOn(first) {
|
||||
// max-nr-of-instances-per-node=1 times 3 nodes
|
||||
|
|
@ -10,17 +10,18 @@ import scala.concurrent.Await
|
|||
import scala.concurrent.duration._
|
||||
import com.typesafe.config.Config
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
import akka.actor._
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.MultiNodeClusterSpec
|
||||
import akka.cluster.NodeMetrics
|
||||
import akka.pattern.ask
|
||||
import akka.remote.testkit.{ MultiNodeSpec, MultiNodeConfig }
|
||||
import akka.routing.CurrentRoutees
|
||||
import akka.routing.GetRoutees
|
||||
import akka.routing.FromConfig
|
||||
import akka.routing.RouterRoutees
|
||||
import akka.testkit.{ LongRunningTest, DefaultTimeout, ImplicitSender }
|
||||
import akka.routing.ActorRefRoutee
|
||||
import akka.routing.Routees
|
||||
|
||||
object AdaptiveLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
|
|
@ -93,7 +94,7 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
|
|||
import AdaptiveLoadBalancingRouterMultiJvmSpec._
|
||||
|
||||
def currentRoutees(router: ActorRef) =
|
||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees
|
||||
Await.result(router ? GetRoutees, remaining).asInstanceOf[Routees].routees
|
||||
|
||||
def receiveReplies(expectedReplies: Int): Map[Address, Int] = {
|
||||
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
|
||||
|
|
@ -113,12 +114,15 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
|
|||
}
|
||||
|
||||
def startRouter(name: String): ActorRef = {
|
||||
val router = system.actorOf(Props[Routee].withRouter(ClusterRouterConfig(
|
||||
local = AdaptiveLoadBalancingRouter(HeapMetricsSelector),
|
||||
settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 1, useRole = None))), name)
|
||||
val router = system.actorOf(ClusterRouterPool(
|
||||
local = AdaptiveLoadBalancingPool(HeapMetricsSelector),
|
||||
settings = ClusterRouterPoolSettings(totalInstances = 10, maxInstancesPerNode = 1, allowLocalRoutees = true, useRole = None)).
|
||||
props(Props[Routee]),
|
||||
name)
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router).size must be(roles.size) }
|
||||
currentRoutees(router).map(fullAddress).toSet must be(roles.map(address).toSet)
|
||||
val routees = currentRoutees(router)
|
||||
routees.map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }.toSet must be(roles.map(address).toSet)
|
||||
router
|
||||
}
|
||||
|
||||
|
|
@ -189,20 +193,22 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
|
|||
|
||||
"create routees from configuration" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router3 = system.actorOf(Props[Memory].withRouter(FromConfig()), "router3")
|
||||
val router3 = system.actorOf(FromConfig.props(Props[Memory]), "router3")
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router3).size must be(9) }
|
||||
currentRoutees(router3).map(fullAddress).toSet must be(Set(address(first)))
|
||||
val routees = currentRoutees(router3)
|
||||
routees.map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }.toSet must be(Set(address(first)))
|
||||
}
|
||||
enterBarrier("after-4")
|
||||
}
|
||||
|
||||
"create routees from cluster.enabled configuration" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router4 = system.actorOf(Props[Memory].withRouter(FromConfig()), "router4")
|
||||
val router4 = system.actorOf(FromConfig.props(Props[Memory]), "router4")
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router4).size must be(6) }
|
||||
currentRoutees(router4).map(fullAddress).toSet must be(Set(
|
||||
val routees = currentRoutees(router4)
|
||||
routees.map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }.toSet must be(Set(
|
||||
address(first), address(second), address(third)))
|
||||
}
|
||||
enterBarrier("after-5")
|
||||
|
|
|
|||
|
|
@ -5,9 +5,7 @@ package akka.cluster.routing
|
|||
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Address
|
||||
|
|
@ -19,10 +17,13 @@ import akka.remote.testkit.MultiNodeSpec
|
|||
import akka.routing.ConsistentHashingRouter
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
|
||||
import akka.routing.CurrentRoutees
|
||||
import akka.routing.GetRoutees
|
||||
import akka.routing.FromConfig
|
||||
import akka.routing.RouterRoutees
|
||||
import akka.testkit._
|
||||
import akka.routing.ActorRefRoutee
|
||||
import akka.routing.ConsistentHashingPool
|
||||
import akka.routing.Routees
|
||||
|
||||
object ClusterConsistentHashingRouterMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
|
|
@ -66,10 +67,10 @@ abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterC
|
|||
with ImplicitSender with DefaultTimeout {
|
||||
import ClusterConsistentHashingRouterMultiJvmSpec._
|
||||
|
||||
lazy val router1 = system.actorOf(Props[Echo].withRouter(FromConfig()), "router1")
|
||||
lazy val router1 = system.actorOf(FromConfig.props(Props[Echo]), "router1")
|
||||
|
||||
def currentRoutees(router: ActorRef) =
|
||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees
|
||||
Await.result(router ? GetRoutees, remaining).asInstanceOf[Routees].routees
|
||||
|
||||
/**
|
||||
* Fills in self address for local ActorRef
|
||||
|
|
@ -79,7 +80,7 @@ abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterC
|
|||
case a ⇒ a
|
||||
}
|
||||
|
||||
"A cluster router with a consistent hashing router" must {
|
||||
"A cluster router with a consistent hashing pool" must {
|
||||
"start cluster with 2 nodes" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(first, second)
|
||||
enterBarrier("after-1")
|
||||
|
|
@ -89,7 +90,8 @@ abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterC
|
|||
runOn(first) {
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router1).size must be(4) }
|
||||
currentRoutees(router1).map(fullAddress).toSet must be(Set(address(first), address(second)))
|
||||
val routees = currentRoutees(router1)
|
||||
routees.map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }.toSet must be(Set(address(first), address(second)))
|
||||
}
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
|
@ -111,7 +113,8 @@ abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterC
|
|||
runOn(first) {
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router1).size must be(6) }
|
||||
currentRoutees(router1).map(fullAddress).toSet must be(roles.map(address).toSet)
|
||||
val routees = currentRoutees(router1)
|
||||
routees.map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }.toSet must be(roles.map(address).toSet)
|
||||
}
|
||||
|
||||
enterBarrier("after-3")
|
||||
|
|
@ -119,11 +122,14 @@ abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterC
|
|||
|
||||
"deploy programatically defined routees to the member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router2 = system.actorOf(Props[Echo].withRouter(ClusterRouterConfig(local = ConsistentHashingRouter(),
|
||||
settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 2, useRole = None))), "router2")
|
||||
val router2 = system.actorOf(ClusterRouterPool(local = ConsistentHashingPool(nrOfInstances = 0),
|
||||
settings = ClusterRouterPoolSettings(totalInstances = 10, maxInstancesPerNode = 2, allowLocalRoutees = true, useRole = None)).
|
||||
props(Props[Echo]),
|
||||
"router2")
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router2).size must be(6) }
|
||||
currentRoutees(router2).map(fullAddress).toSet must be(roles.map(address).toSet)
|
||||
val routees = currentRoutees(router2)
|
||||
routees.map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }.toSet must be(roles.map(address).toSet)
|
||||
}
|
||||
|
||||
enterBarrier("after-4")
|
||||
|
|
@ -135,7 +141,7 @@ abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterC
|
|||
case s: String ⇒ s
|
||||
}
|
||||
|
||||
val router3 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter(hashMapping = hashMapping)), "router3")
|
||||
val router3 = system.actorOf(ConsistentHashingPool(nrOfInstances = 0, hashMapping = hashMapping).props(Props[Echo]), "router3")
|
||||
|
||||
assertHashMapping(router3)
|
||||
}
|
||||
|
|
@ -149,9 +155,11 @@ abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterC
|
|||
case s: String ⇒ s
|
||||
}
|
||||
|
||||
val router4 = system.actorOf(Props[Echo].withRouter(ClusterRouterConfig(
|
||||
local = ConsistentHashingRouter(hashMapping = hashMapping),
|
||||
settings = ClusterRouterSettings(totalInstances = 10, maxInstancesPerNode = 1, useRole = None))), "router4")
|
||||
val router4 = system.actorOf(ClusterRouterPool(
|
||||
local = ConsistentHashingPool(nrOfInstances = 0, hashMapping = hashMapping),
|
||||
settings = ClusterRouterPoolSettings(totalInstances = 10, maxInstancesPerNode = 1, allowLocalRoutees = true, useRole = None)).
|
||||
props(Props[Echo]),
|
||||
"router4")
|
||||
|
||||
assertHashMapping(router4)
|
||||
}
|
||||
|
|
@ -162,7 +170,8 @@ abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterC
|
|||
def assertHashMapping(router: ActorRef): Unit = {
|
||||
// it may take some time until router receives cluster member events
|
||||
awaitAssert { currentRoutees(router).size must be(6) }
|
||||
currentRoutees(router).map(fullAddress).toSet must be(roles.map(address).toSet)
|
||||
val routees = currentRoutees(router)
|
||||
routees.map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }.toSet must be(roles.map(address).toSet)
|
||||
|
||||
router ! "a"
|
||||
val destinationA = expectMsgType[ActorRef]
|
||||
|
|
|
|||
|
|
@ -0,0 +1,373 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster.routing
|
||||
|
||||
import language.postfixOps
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Address
|
||||
import akka.actor.Props
|
||||
import akka.actor.Terminated
|
||||
import akka.cluster.MultiNodeClusterSpec
|
||||
import akka.pattern.ask
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
import akka.testkit._
|
||||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import akka.routing.FromConfig
|
||||
import akka.routing.RoundRobinPool
|
||||
import akka.routing.RouterRoutees
|
||||
import akka.routing.ActorRefRoutee
|
||||
import akka.routing.ActorSelectionRoutee
|
||||
import akka.routing.RoutedActorRef
|
||||
import akka.routing.GetRoutees
|
||||
import akka.routing.Routees
|
||||
|
||||
object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class SomeActor(routeeType: RouteeType) extends Actor {
|
||||
def this() = this(DeployRoutee)
|
||||
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! Reply(routeeType, self)
|
||||
}
|
||||
}
|
||||
|
||||
case class Reply(routeeType: RouteeType, ref: ActorRef)
|
||||
|
||||
sealed trait RouteeType extends Serializable
|
||||
object DeployRoutee extends RouteeType
|
||||
object LookupRoutee extends RouteeType
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("""
|
||||
akka.actor.deployment {
|
||||
/router1 {
|
||||
router = round-robin
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 2
|
||||
}
|
||||
}
|
||||
/router3 {
|
||||
router = round-robin
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 1
|
||||
allow-local-routees = off
|
||||
}
|
||||
}
|
||||
/router4 {
|
||||
router = round-robin
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
routees-path = "/user/myservice"
|
||||
}
|
||||
}
|
||||
/router5 {
|
||||
router = round-robin
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
use-role = a
|
||||
}
|
||||
}
|
||||
}
|
||||
""")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
nodeConfig(first, second)(ConfigFactory.parseString("""akka.cluster.roles =["a", "c"]"""))
|
||||
nodeConfig(third)(ConfigFactory.parseString("""akka.cluster.roles =["b", "c"]"""))
|
||||
|
||||
testTransport(on = true)
|
||||
|
||||
}
|
||||
|
||||
class ClusterRoundRobinMultiJvmNode1 extends ClusterRoundRobinSpec
|
||||
class ClusterRoundRobinMultiJvmNode2 extends ClusterRoundRobinSpec
|
||||
class ClusterRoundRobinMultiJvmNode3 extends ClusterRoundRobinSpec
|
||||
class ClusterRoundRobinMultiJvmNode4 extends ClusterRoundRobinSpec
|
||||
|
||||
abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with DefaultTimeout {
|
||||
import ClusterRoundRobinMultiJvmSpec._
|
||||
|
||||
lazy val router1 = system.actorOf(FromConfig.props(Props[SomeActor]), "router1")
|
||||
lazy val router2 = system.actorOf(ClusterRouterPool(RoundRobinPool(nrOfInstances = 0),
|
||||
ClusterRouterPoolSettings(totalInstances = 3, maxInstancesPerNode = 1, allowLocalRoutees = true, useRole = None)).
|
||||
props(Props[SomeActor]),
|
||||
"router2")
|
||||
lazy val router3 = system.actorOf(FromConfig.props(Props[SomeActor]), "router3")
|
||||
lazy val router4 = system.actorOf(FromConfig.props(), "router4")
|
||||
lazy val router5 = system.actorOf(RoundRobinPool(nrOfInstances = 0).props(Props[SomeActor]), "router5")
|
||||
|
||||
def receiveReplies(routeeType: RouteeType, expectedReplies: Int): Map[Address, Int] = {
|
||||
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
|
||||
(receiveWhile(5 seconds, messages = expectedReplies) {
|
||||
case Reply(`routeeType`, ref) ⇒ fullAddress(ref)
|
||||
}).foldLeft(zero) {
|
||||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fills in self address for local ActorRef
|
||||
*/
|
||||
private def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
|
||||
case Address(_, _, None, None) ⇒ cluster.selfAddress
|
||||
case a ⇒ a
|
||||
}
|
||||
|
||||
def currentRoutees(router: ActorRef) =
|
||||
Await.result(router ? GetRoutees, remaining).asInstanceOf[Routees].routees
|
||||
|
||||
"A cluster router with a RoundRobin router" must {
|
||||
"start cluster with 2 nodes" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(first, second)
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"deploy routees to the member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
router1.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
// max-nr-of-instances-per-node=2 times 2 nodes
|
||||
awaitAssert(currentRoutees(router1).size must be(4))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router1 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
replies(first) must be > (0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be(0)
|
||||
replies(fourth) must be(0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"lookup routees on the member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
// cluster consists of first and second
|
||||
|
||||
system.actorOf(Props(classOf[SomeActor], LookupRoutee), "myservice")
|
||||
enterBarrier("myservice-started")
|
||||
|
||||
runOn(first) {
|
||||
// 2 nodes, 1 routee on each node
|
||||
awaitAssert(currentRoutees(router4).size must be(2))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router4 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(LookupRoutee, iterationCount)
|
||||
|
||||
replies(first) must be > (0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be(0)
|
||||
replies(fourth) must be(0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-3")
|
||||
}
|
||||
|
||||
"deploy routees to new nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
// add third and fourth
|
||||
awaitClusterUp(first, second, third, fourth)
|
||||
|
||||
runOn(first) {
|
||||
// max-nr-of-instances-per-node=2 times 4 nodes
|
||||
awaitAssert(currentRoutees(router1).size must be(8))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router1 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
replies.values.foreach { _ must be > (0) }
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-4")
|
||||
}
|
||||
|
||||
"lookup routees on new nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
// cluster consists of first, second, third and fourth
|
||||
|
||||
runOn(first) {
|
||||
// 4 nodes, 1 routee on each node
|
||||
awaitAssert(currentRoutees(router4).size must be(4))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router4 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(LookupRoutee, iterationCount)
|
||||
|
||||
replies.values.foreach { _ must be > (0) }
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-5")
|
||||
}
|
||||
|
||||
"deploy routees to only remote nodes when allow-local-routees = off" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
// max-nr-of-instances-per-node=1 times 3 nodes
|
||||
awaitAssert(currentRoutees(router3).size must be(3))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router3 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
replies(first) must be(0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be > (0)
|
||||
replies(fourth) must be > (0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-6")
|
||||
}
|
||||
|
||||
"deploy routees to specified node role" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
awaitAssert(currentRoutees(router5).size must be(2))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router5 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
replies(first) must be > (0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be(0)
|
||||
replies(fourth) must be(0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-7")
|
||||
}
|
||||
|
||||
"deploy programatically defined routees to the member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
router2.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
// totalInstances = 3, maxInstancesPerNode = 1
|
||||
awaitAssert(currentRoutees(router2).size must be(3))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router2 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
// note that router2 has totalInstances = 3, maxInstancesPerNode = 1
|
||||
val routees = currentRoutees(router2)
|
||||
val routeeAddresses = routees map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }
|
||||
|
||||
routeeAddresses.size must be(3)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-8")
|
||||
}
|
||||
|
||||
"remove routees for unreachable nodes, and add when reachable again" taggedAs LongRunningTest in within(30.seconds) {
|
||||
|
||||
// myservice is already running
|
||||
|
||||
def routees = currentRoutees(router4)
|
||||
def routeeAddresses = (routees map { case ActorSelectionRoutee(sel) ⇒ fullAddress(sel.anchor) }).toSet
|
||||
|
||||
runOn(first) {
|
||||
// 4 nodes, 1 routee on each node
|
||||
awaitAssert(currentRoutees(router4).size must be(4))
|
||||
|
||||
testConductor.blackhole(first, second, Direction.Both).await
|
||||
|
||||
awaitAssert(routees.size must be(3))
|
||||
routeeAddresses must not contain (address(second))
|
||||
|
||||
testConductor.passThrough(first, second, Direction.Both).await
|
||||
awaitAssert(routees.size must be(4))
|
||||
routeeAddresses must contain(address(second))
|
||||
|
||||
}
|
||||
|
||||
enterBarrier("after-9")
|
||||
}
|
||||
|
||||
"deploy programatically defined routees to other node when a node becomes down" taggedAs LongRunningTest in {
|
||||
muteMarkingAsUnreachable()
|
||||
|
||||
runOn(first) {
|
||||
def routees = currentRoutees(router2)
|
||||
def routeeAddresses = (routees map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }).toSet
|
||||
|
||||
routees foreach { case ActorRefRoutee(ref) ⇒ watch(ref) }
|
||||
val notUsedAddress = ((roles map address).toSet -- routeeAddresses).head
|
||||
val downAddress = routeeAddresses.find(_ != address(first)).get
|
||||
val downRouteeRef = routees.collectFirst {
|
||||
case ActorRefRoutee(ref) if ref.path.address == downAddress ⇒ ref
|
||||
}.get
|
||||
|
||||
cluster.down(downAddress)
|
||||
expectMsgType[Terminated](15.seconds).actor must be(downRouteeRef)
|
||||
awaitAssert {
|
||||
routeeAddresses must contain(notUsedAddress)
|
||||
routeeAddresses must not contain (downAddress)
|
||||
}
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router2 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
routeeAddresses.size must be(3)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-10")
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
Loading…
Add table
Add a link
Reference in a new issue