!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:
Patrik Nordwall 2013-09-19 08:00:05 +02:00
parent 81ca6fe8c8
commit ebadd567b2
104 changed files with 9671 additions and 5006 deletions

View file

@ -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)
}
}
}

View file

@ -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
"""))
}

View file

@ -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")
}
}
}

View file

@ -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)
}
}
}

View file

@ -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

View file

@ -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")

View file

@ -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]

View file

@ -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")
}
}
}