!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

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

@ -16,14 +16,18 @@ import akka.cluster.MultiNodeClusterSpec
import akka.pattern.ask
import akka.remote.testkit.MultiNodeConfig
import akka.remote.testkit.MultiNodeSpec
import akka.routing.CurrentRoutees
import akka.routing.RoundRobinRouter
import akka.routing.RoutedActorRef
import akka.routing.RouterRoutees
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 ClusterRoundRobinRoutedActorMultiJvmSpec extends MultiNodeConfig {
object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
class SomeActor(routeeType: RouteeType) extends Actor {
def this() = this(DeployRoutee)
@ -91,22 +95,24 @@ object ClusterRoundRobinRoutedActorMultiJvmSpec extends MultiNodeConfig {
}
class ClusterRoundRobinRoutedActorMultiJvmNode1 extends ClusterRoundRobinRoutedActorSpec
class ClusterRoundRobinRoutedActorMultiJvmNode2 extends ClusterRoundRobinRoutedActorSpec
class ClusterRoundRobinRoutedActorMultiJvmNode3 extends ClusterRoundRobinRoutedActorSpec
class ClusterRoundRobinRoutedActorMultiJvmNode4 extends ClusterRoundRobinRoutedActorSpec
class ClusterRoundRobinMultiJvmNode1 extends ClusterRoundRobinSpec
class ClusterRoundRobinMultiJvmNode2 extends ClusterRoundRobinSpec
class ClusterRoundRobinMultiJvmNode3 extends ClusterRoundRobinSpec
class ClusterRoundRobinMultiJvmNode4 extends ClusterRoundRobinSpec
abstract class ClusterRoundRobinRoutedActorSpec extends MultiNodeSpec(ClusterRoundRobinRoutedActorMultiJvmSpec)
abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMultiJvmSpec)
with MultiNodeClusterSpec
with ImplicitSender with DefaultTimeout {
import ClusterRoundRobinRoutedActorMultiJvmSpec._
import ClusterRoundRobinMultiJvmSpec._
lazy val router1 = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "router1")
lazy val router2 = system.actorOf(Props[SomeActor].withRouter(ClusterRouterConfig(RoundRobinRouter(),
ClusterRouterSettings(totalInstances = 3, maxInstancesPerNode = 1, useRole = None))), "router2")
lazy val router3 = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "router3")
lazy val router4 = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "router4")
lazy val router5 = system.actorOf(Props[SomeActor].withRouter(RoundRobinRouter()), "router5")
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)
@ -126,7 +132,7 @@ abstract class ClusterRoundRobinRoutedActorSpec extends MultiNodeSpec(ClusterRou
}
def currentRoutees(router: ActorRef) =
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees
Await.result(router ? GetRoutees, remaining).asInstanceOf[Routees].routees
"A cluster router with a RoundRobin router" must {
"start cluster with 2 nodes" taggedAs LongRunningTest in {
@ -294,7 +300,7 @@ abstract class ClusterRoundRobinRoutedActorSpec extends MultiNodeSpec(ClusterRou
// note that router2 has totalInstances = 3, maxInstancesPerNode = 1
val routees = currentRoutees(router2)
val routeeAddresses = routees map fullAddress
val routeeAddresses = routees map { case ActorRefRoutee(ref) fullAddress(ref) }
routeeAddresses.size must be(3)
replies.values.sum must be(iterationCount)
@ -308,7 +314,7 @@ abstract class ClusterRoundRobinRoutedActorSpec extends MultiNodeSpec(ClusterRou
// myservice is already running
def routees = currentRoutees(router4)
def routeeAddresses = (routees map fullAddress).toSet
def routeeAddresses = (routees map { case ActorSelectionRoutee(sel) fullAddress(sel.anchor) }).toSet
runOn(first) {
// 4 nodes, 1 routee on each node
@ -333,15 +339,17 @@ abstract class ClusterRoundRobinRoutedActorSpec extends MultiNodeSpec(ClusterRou
runOn(first) {
def routees = currentRoutees(router2)
def routeeAddresses = (routees map fullAddress).toSet
def routeeAddresses = (routees map { case ActorRefRoutee(ref) fullAddress(ref) }).toSet
routees foreach watch
routees foreach { case ActorRefRoutee(ref) watch(ref) }
val notUsedAddress = ((roles map address).toSet -- routeeAddresses).head
val downAddress = routeeAddresses.find(_ != address(first)).get
val downRoutee = routees.find(_.path.address == downAddress).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(downRoutee)
expectMsgType[Terminated](15.seconds).actor must be(downRouteeRef)
awaitAssert {
routeeAddresses must contain(notUsedAddress)
routeeAddresses must not contain (downAddress)