+clu #3627 Cluster router group with multiple paths per node

* Use the ordinary routees.paths config property instead of
  cluster.routees-path
* Backwards compatible in deprecation phase
This commit is contained in:
Patrik Nordwall 2013-10-16 11:06:38 +02:00
parent c2faa48351
commit 402674ce10
19 changed files with 168 additions and 103 deletions

View file

@ -151,8 +151,8 @@ akka {
# In case of routing, the actors to be routed to can be specified # In case of routing, the actors to be routed to can be specified
# in several ways: # in several ways:
# - nr-of-instances: will create that many children # - nr-of-instances: will create that many children
# - routees.paths: will look the paths up using actorFor and route to # - routees.paths: will route messages to these paths using ActorSelection,
# them, i.e. will not create children # i.e. will not create children
# - resizer: dynamically resizable number of routees as specified in # - resizer: dynamically resizable number of routees as specified in
# resizer below # resizer below
router = "from-code" router = "from-code"

View file

@ -216,11 +216,7 @@ akka {
# Useful for master-worker scenario where all routees are remote. # Useful for master-worker scenario where all routees are remote.
allow-local-routees = on allow-local-routees = on
# Actor path of the routees to lookup with actorFor on the member # Deprecated in 2.3, use routees.paths instead
# nodes in the cluster. E.g. "/user/myservice". If this isn't defined
# the routees will be deployed instead of looked up.
# max-nr-of-instances-per-node should not be configured (default value is 1)
# when routees-path is defined.
routees-path = "" routees-path = ""
# Use members with specified role, or all members if undefined or empty. # Use members with specified role, or all members if undefined or empty.

View file

@ -90,8 +90,7 @@ private[akka] class ClusterActorRefProvider(
private[akka] class ClusterDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extends RemoteDeployer(_settings, _pm) { private[akka] class ClusterDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extends RemoteDeployer(_settings, _pm) {
override def parseConfig(path: String, config: Config): Option[Deploy] = { override def parseConfig(path: String, config: Config): Option[Deploy] = {
// For backwards compatibility we must add this fake routees.paths so that the deployer creates a Group // For backwards compatibility we must transform 'cluster.routees-path' to 'routees.paths'
// even though routees.paths is not defined. This will be cleaned up by ticket #3627
val config2 = val config2 =
if (config.hasPath("cluster.routees-path")) if (config.hasPath("cluster.routees-path"))
config.withFallback(ConfigFactory.parseString(s"""routees.paths=["${config.getString("cluster.routees-path")}"]""")) config.withFallback(ConfigFactory.parseString(s"""routees.paths=["${config.getString("cluster.routees-path")}"]"""))

View file

@ -219,11 +219,11 @@ final case class AdaptiveLoadBalancingGroup(
* Java API * Java API
* @param metricsSelector decides what probability to use for selecting a routee, based * @param metricsSelector decides what probability to use for selecting a routee, based
* on remaining capacity as indicated by the node metrics * on remaining capacity as indicated by the node metrics
* @param routeePaths string representation of the actor paths of the routees, messages are * @param routeesPaths string representation of the actor paths of the routees, messages are
* sent with [[akka.actor.ActorSelection]] to these paths * sent with [[akka.actor.ActorSelection]] to these paths
*/ */
def this(metricsSelector: MetricsSelector, def this(metricsSelector: MetricsSelector,
routeePaths: java.lang.Iterable[String]) = this(paths = immutableSeq(routeePaths)) routeesPaths: java.lang.Iterable[String]) = this(paths = immutableSeq(routeesPaths))
override def createRouter(system: ActorSystem): Router = override def createRouter(system: ActorSystem): Router =
new Router(AdaptiveLoadBalancingRoutingLogic(system, metricsSelector)) new Router(AdaptiveLoadBalancingRoutingLogic(system, metricsSelector))
@ -566,11 +566,11 @@ case class AdaptiveLoadBalancingRouter(
* Java API: Constructor that sets the routees to be used. * Java API: Constructor that sets the routees to be used.
* *
* @param selector the selector is responsible for producing weighted mix of routees from the node metrics * @param selector the selector is responsible for producing weighted mix of routees from the node metrics
* @param routeePaths string representation of the actor paths of the routees that will be looked up * @param routeesPaths string representation of the actor paths of the routees that will be looked up
* using `actorFor` in [[akka.actor.ActorRefProvider]] * using `actorFor` in [[akka.actor.ActorRefProvider]]
*/ */
def this(selector: MetricsSelector, routeePaths: java.lang.Iterable[String]) = def this(selector: MetricsSelector, routeesPaths: java.lang.Iterable[String]) =
this(metricsSelector = selector, routees = immutableSeq(routeePaths)) this(metricsSelector = selector, routees = immutableSeq(routeesPaths))
/** /**
* Java API: Constructor that sets the resizer to be used. * Java API: Constructor that sets the resizer to be used.

View file

@ -40,12 +40,13 @@ import akka.routing.RoutingLogic
import akka.actor.RelativeActorPath import akka.actor.RelativeActorPath
import com.typesafe.config.Config import com.typesafe.config.Config
import akka.routing.DeprecatedRouterConfig import akka.routing.DeprecatedRouterConfig
import akka.japi.Util.immutableSeq
object ClusterRouterGroupSettings { object ClusterRouterGroupSettings {
def fromConfig(config: Config): ClusterRouterGroupSettings = def fromConfig(config: Config): ClusterRouterGroupSettings =
ClusterRouterGroupSettings( ClusterRouterGroupSettings(
totalInstances = config.getInt("nr-of-instances"), totalInstances = config.getInt("nr-of-instances"),
routeesPath = config.getString("cluster.routees-path"), routeesPaths = immutableSeq(config.getStringList("routees.paths")),
allowLocalRoutees = config.getBoolean("cluster.allow-local-routees"), allowLocalRoutees = config.getBoolean("cluster.allow-local-routees"),
useRole = ClusterRouterSettingsBase.useRoleOption(config.getString("cluster.use-role"))) useRole = ClusterRouterSettingsBase.useRoleOption(config.getString("cluster.use-role")))
} }
@ -56,26 +57,40 @@ object ClusterRouterGroupSettings {
@SerialVersionUID(1L) @SerialVersionUID(1L)
case class ClusterRouterGroupSettings( case class ClusterRouterGroupSettings(
totalInstances: Int, totalInstances: Int,
routeesPath: String, routeesPaths: immutable.Seq[String],
allowLocalRoutees: Boolean, allowLocalRoutees: Boolean,
useRole: Option[String]) extends ClusterRouterSettingsBase { useRole: Option[String]) extends ClusterRouterSettingsBase {
@deprecated("Use constructor with routeesPaths Seq", "2.3")
def this(
totalInstances: Int,
routeesPath: String,
allowLocalRoutees: Boolean,
useRole: Option[String]) =
this(totalInstances, List(routeesPath), allowLocalRoutees, useRole)
/** /**
* Java API * Java API
*/ */
def this(totalInstances: Int, routeesPaths: java.lang.Iterable[String], allowLocalRoutees: Boolean, useRole: String) =
this(totalInstances, immutableSeq(routeesPaths), allowLocalRoutees, ClusterRouterSettingsBase.useRoleOption(useRole))
/**
* Java API
*/
@deprecated("Use constructor with routeesPaths Iterable", "2.3")
def this(totalInstances: Int, routeesPath: String, allowLocalRoutees: Boolean, useRole: String) = def this(totalInstances: Int, routeesPath: String, allowLocalRoutees: Boolean, useRole: String) =
this(totalInstances, routeesPath, allowLocalRoutees, ClusterRouterSettingsBase.useRoleOption(useRole)) this(totalInstances, routeesPath, allowLocalRoutees, ClusterRouterSettingsBase.useRoleOption(useRole))
if (totalInstances <= 0) throw new IllegalArgumentException("totalInstances of cluster router must be > 0") if (totalInstances <= 0) throw new IllegalArgumentException("totalInstances of cluster router must be > 0")
if (!isRouteesPathDefined) throw new IllegalArgumentException("routeesPath must be defined") if ((routeesPaths eq null) || routeesPaths.isEmpty || routeesPaths.head == "")
throw new IllegalArgumentException("routeesPaths must be defined")
routeesPath match { routeesPaths.foreach(p p match {
case RelativeActorPath(elements) // good case RelativeActorPath(elements) // good
case _ case _
throw new IllegalArgumentException("routeesPath [%s] is not a valid relative actor path" format routeesPath) throw new IllegalArgumentException(s"routeesPaths [$p] is not a valid relative actor path")
} })
def isRouteesPathDefined: Boolean = (routeesPath ne null) && routeesPath != ""
} }
@ -140,9 +155,7 @@ private[akka] trait ClusterRouterSettingsBase {
@SerialVersionUID(1L) @SerialVersionUID(1L)
final case class ClusterRouterGroup(local: Group, settings: ClusterRouterGroupSettings) extends Group with ClusterRouterConfigBase { final case class ClusterRouterGroup(local: Group, settings: ClusterRouterGroupSettings) extends Group with ClusterRouterConfigBase {
require(settings.routeesPath.nonEmpty, "routeesPath must be defined") override def paths: immutable.Iterable[String] = if (settings.allowLocalRoutees) settings.routeesPaths else Nil
override def paths: immutable.Iterable[String] = if (settings.allowLocalRoutees) List(settings.routeesPath) else Nil
/** /**
* INTERNAL API * INTERNAL API
@ -259,7 +272,23 @@ private[akka] class ClusterRouterPoolActor(
doAddRoutees() doAddRoutees()
} }
override def maxInstancesPerNode: Int = settings.maxInstancesPerNode def selectDeploymentTarget: Option[Address] = {
val currentRoutees = cell.router.routees
val currentNodes = availableNodes
if (currentNodes.isEmpty || currentRoutees.size >= settings.totalInstances) {
None
} else {
// find the node with least routees
val numberOfRouteesPerNode: Map[Address, Int] =
currentRoutees.foldLeft(currentNodes.map(_ -> 0).toMap.withDefaultValue(0)) { (acc, x)
val address = fullAddress(x)
acc + (address -> (acc(address) + 1))
}
val (address, count) = numberOfRouteesPerNode.minBy(_._2)
if (count < settings.maxInstancesPerNode) Some(address) else None
}
}
} }
@ -277,6 +306,12 @@ private[akka] class ClusterRouterGroupActor(val settings: ClusterRouterGroupSett
override def receive = clusterReceive orElse super.receive override def receive = clusterReceive orElse super.receive
var usedRouteePaths: Map[Address, Set[String]] =
if (settings.allowLocalRoutees)
Map(cluster.selfAddress -> settings.routeesPaths.toSet)
else
Map.empty
/** /**
* Adds routees based on totalInstances and maxInstancesPerNode settings * Adds routees based on totalInstances and maxInstancesPerNode settings
*/ */
@ -284,8 +319,9 @@ private[akka] class ClusterRouterGroupActor(val settings: ClusterRouterGroupSett
@tailrec @tailrec
def doAddRoutees(): Unit = selectDeploymentTarget match { def doAddRoutees(): Unit = selectDeploymentTarget match {
case None // done case None // done
case Some(target) case Some((address, path))
val routee = group.routeeFor(target + settings.routeesPath, context) val routee = group.routeeFor(address + path, context)
usedRouteePaths = usedRouteePaths.updated(address, usedRouteePaths.getOrElse(address, Set.empty) + path)
// must register each one, since registered routees are used in selectDeploymentTarget // must register each one, since registered routees are used in selectDeploymentTarget
cell.addRoutee(routee) cell.addRoutee(routee)
@ -296,8 +332,28 @@ private[akka] class ClusterRouterGroupActor(val settings: ClusterRouterGroupSett
doAddRoutees() doAddRoutees()
} }
override def maxInstancesPerNode: Int = 1 def selectDeploymentTarget: Option[(Address, String)] = {
val currentRoutees = cell.router.routees
val currentNodes = availableNodes
if (currentNodes.isEmpty || currentRoutees.size >= settings.totalInstances) {
None
} else {
// find the node with least routees
val unusedNodes = currentNodes filterNot usedRouteePaths.contains
if (unusedNodes.nonEmpty) {
Some((unusedNodes.head, settings.routeesPaths.head))
} else {
val (address, used) = usedRouteePaths.minBy { case (address, used) used.size }
// pick next of the unused paths
settings.routeesPaths.collectFirst { case p if !used.contains(p) (address, p) }
}
}
}
override def removeMember(member: Member): Unit = {
usedRouteePaths -= member.address
super.removeMember(member)
}
} }
/** /**
@ -364,36 +420,16 @@ private[akka] trait ClusterRouterActor { this: RouterActor ⇒
} }
/** /**
* Adds routees based on totalInstances and maxInstancesPerNode settings * Adds routees based on settings
*/ */
def addRoutees(): Unit def addRoutees(): Unit
def maxInstancesPerNode: Int def addMember(member: Member): Unit = {
def selectDeploymentTarget: Option[Address] = {
val currentRoutees = cell.router.routees
val currentNodes = availableNodes
if (currentNodes.isEmpty || currentRoutees.size >= settings.totalInstances) {
None
} else {
// find the node with least routees
val numberOfRouteesPerNode: Map[Address, Int] =
currentRoutees.foldLeft(currentNodes.map(_ -> 0).toMap.withDefaultValue(0)) { (acc, x)
val address = fullAddress(x)
acc + (address -> (acc(address) + 1))
}
val (address, count) = numberOfRouteesPerNode.minBy(_._2)
if (count < maxInstancesPerNode) Some(address) else None
}
}
def addMember(member: Member) = {
nodes += member.address nodes += member.address
addRoutees() addRoutees()
} }
def removeMember(member: Member) = { def removeMember(member: Member): Unit = {
val address = member.address val address = member.address
nodes -= address nodes -= address

View file

@ -149,7 +149,7 @@ final case class ClusterRouterConfig(local: DeprecatedRouterConfig, settings: Cl
new ClusterRouterPoolActor(local.supervisorStrategy, ClusterRouterPoolSettings(settings.totalInstances, new ClusterRouterPoolActor(local.supervisorStrategy, ClusterRouterPoolSettings(settings.totalInstances,
settings.maxInstancesPerNode, settings.allowLocalRoutees, settings.useRole)) settings.maxInstancesPerNode, settings.allowLocalRoutees, settings.useRole))
else else
new ClusterRouterGroupActor(ClusterRouterGroupSettings(settings.totalInstances, settings.routeesPath, new ClusterRouterGroupActor(ClusterRouterGroupSettings(settings.totalInstances, List(settings.routeesPath),
settings.allowLocalRoutees, settings.useRole)) settings.allowLocalRoutees, settings.useRole))
override def supervisorStrategy: SupervisorStrategy = local.supervisorStrategy override def supervisorStrategy: SupervisorStrategy = local.supervisorStrategy

View file

@ -147,9 +147,9 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
/master-node-2/workers { /master-node-2/workers {
router = round-robin-group router = round-robin-group
nr-of-instances = 100 nr-of-instances = 100
routees.paths = ["/user/worker"]
cluster { cluster {
enabled = on enabled = on
routees-path = "/user/worker"
allow-local-routees = on allow-local-routees = on
} }
} }

View file

@ -25,7 +25,7 @@ import akka.routing.Routees
object AdaptiveLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig { object AdaptiveLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig {
class Routee extends Actor { class Echo extends Actor {
def receive = { def receive = {
case _ sender ! Reply(Cluster(context.system).selfAddress) case _ sender ! Reply(Cluster(context.system).selfAddress)
} }
@ -117,7 +117,7 @@ abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoa
val router = system.actorOf(ClusterRouterPool( val router = system.actorOf(ClusterRouterPool(
local = AdaptiveLoadBalancingPool(HeapMetricsSelector), local = AdaptiveLoadBalancingPool(HeapMetricsSelector),
settings = ClusterRouterPoolSettings(totalInstances = 10, maxInstancesPerNode = 1, allowLocalRoutees = true, useRole = None)). settings = ClusterRouterPoolSettings(totalInstances = 10, maxInstancesPerNode = 1, allowLocalRoutees = true, useRole = None)).
props(Props[Routee]), props(Props[Echo]),
name) name)
// it may take some time until router receives cluster member events // it may take some time until router receives cluster member events
awaitAssert { currentRoutees(router).size must be(roles.size) } awaitAssert { currentRoutees(router).size must be(roles.size) }

View file

@ -30,7 +30,7 @@ import akka.routing.Routees
object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig { object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
class SomeActor(routeeType: RouteeType) extends Actor { class SomeActor(routeeType: RouteeType) extends Actor {
def this() = this(DeployRoutee) def this() = this(PoolRoutee)
def receive = { def receive = {
case "hit" sender ! Reply(routeeType, self) case "hit" sender ! Reply(routeeType, self)
@ -40,8 +40,8 @@ object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
case class Reply(routeeType: RouteeType, ref: ActorRef) case class Reply(routeeType: RouteeType, ref: ActorRef)
sealed trait RouteeType extends Serializable sealed trait RouteeType extends Serializable
object DeployRoutee extends RouteeType object PoolRoutee extends RouteeType
object LookupRoutee extends RouteeType object GroupRoutee extends RouteeType
val first = role("first") val first = role("first")
val second = role("second") val second = role("second")
@ -71,10 +71,8 @@ object ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
/router4 { /router4 {
router = round-robin router = round-robin
nr-of-instances = 10 nr-of-instances = 10
cluster { routees.paths = ["/user/myserviceA", "/user/myserviceB"]
enabled = on cluster.enabled = on
routees-path = "/user/myservice"
}
} }
/router5 { /router5 {
router = round-robin router = round-robin
@ -153,7 +151,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
router1 ! "hit" router1 ! "hit"
} }
val replies = receiveReplies(DeployRoutee, iterationCount) val replies = receiveReplies(PoolRoutee, iterationCount)
replies(first) must be > (0) replies(first) must be > (0)
replies(second) must be > (0) replies(second) must be > (0)
@ -169,19 +167,20 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
// cluster consists of first and second // cluster consists of first and second
system.actorOf(Props(classOf[SomeActor], LookupRoutee), "myservice") system.actorOf(Props(classOf[SomeActor], GroupRoutee), "myserviceA")
system.actorOf(Props(classOf[SomeActor], GroupRoutee), "myserviceB")
enterBarrier("myservice-started") enterBarrier("myservice-started")
runOn(first) { runOn(first) {
// 2 nodes, 1 routee on each node // 2 nodes, 2 routees on each node
awaitAssert(currentRoutees(router4).size must be(2)) awaitAssert(currentRoutees(router4).size must be(4))
val iterationCount = 10 val iterationCount = 10
for (i 0 until iterationCount) { for (i 0 until iterationCount) {
router4 ! "hit" router4 ! "hit"
} }
val replies = receiveReplies(LookupRoutee, iterationCount) val replies = receiveReplies(GroupRoutee, iterationCount)
replies(first) must be > (0) replies(first) must be > (0)
replies(second) must be > (0) replies(second) must be > (0)
@ -207,7 +206,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
router1 ! "hit" router1 ! "hit"
} }
val replies = receiveReplies(DeployRoutee, iterationCount) val replies = receiveReplies(PoolRoutee, iterationCount)
replies.values.foreach { _ must be > (0) } replies.values.foreach { _ must be > (0) }
replies.values.sum must be(iterationCount) replies.values.sum must be(iterationCount)
@ -221,15 +220,15 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
// cluster consists of first, second, third and fourth // cluster consists of first, second, third and fourth
runOn(first) { runOn(first) {
// 4 nodes, 1 routee on each node // 4 nodes, 2 routee on each node
awaitAssert(currentRoutees(router4).size must be(4)) awaitAssert(currentRoutees(router4).size must be(8))
val iterationCount = 10 val iterationCount = 10
for (i 0 until iterationCount) { for (i 0 until iterationCount) {
router4 ! "hit" router4 ! "hit"
} }
val replies = receiveReplies(LookupRoutee, iterationCount) val replies = receiveReplies(GroupRoutee, iterationCount)
replies.values.foreach { _ must be > (0) } replies.values.foreach { _ must be > (0) }
replies.values.sum must be(iterationCount) replies.values.sum must be(iterationCount)
@ -249,7 +248,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
router3 ! "hit" router3 ! "hit"
} }
val replies = receiveReplies(DeployRoutee, iterationCount) val replies = receiveReplies(PoolRoutee, iterationCount)
replies(first) must be(0) replies(first) must be(0)
replies(second) must be > (0) replies(second) must be > (0)
@ -271,7 +270,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
router5 ! "hit" router5 ! "hit"
} }
val replies = receiveReplies(DeployRoutee, iterationCount) val replies = receiveReplies(PoolRoutee, iterationCount)
replies(first) must be > (0) replies(first) must be > (0)
replies(second) must be > (0) replies(second) must be > (0)
@ -296,7 +295,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
router2 ! "hit" router2 ! "hit"
} }
val replies = receiveReplies(DeployRoutee, iterationCount) val replies = receiveReplies(PoolRoutee, iterationCount)
// note that router2 has totalInstances = 3, maxInstancesPerNode = 1 // note that router2 has totalInstances = 3, maxInstancesPerNode = 1
val routees = currentRoutees(router2) val routees = currentRoutees(router2)
@ -317,16 +316,16 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
def routeeAddresses = (routees map { case ActorSelectionRoutee(sel) fullAddress(sel.anchor) }).toSet def routeeAddresses = (routees map { case ActorSelectionRoutee(sel) fullAddress(sel.anchor) }).toSet
runOn(first) { runOn(first) {
// 4 nodes, 1 routee on each node // 4 nodes, 2 routees on each node
awaitAssert(currentRoutees(router4).size must be(4)) awaitAssert(currentRoutees(router4).size must be(8))
testConductor.blackhole(first, second, Direction.Both).await testConductor.blackhole(first, second, Direction.Both).await
awaitAssert(routees.size must be(3)) awaitAssert(routees.size must be(6))
routeeAddresses must not contain (address(second)) routeeAddresses must not contain (address(second))
testConductor.passThrough(first, second, Direction.Both).await testConductor.passThrough(first, second, Direction.Both).await
awaitAssert(routees.size must be(4)) awaitAssert(routees.size must be(8))
routeeAddresses must contain(address(second)) routeeAddresses must contain(address(second))
} }
@ -360,7 +359,7 @@ abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMult
router2 ! "hit" router2 ! "hit"
} }
val replies = receiveReplies(DeployRoutee, iterationCount) val replies = receiveReplies(PoolRoutee, iterationCount)
routeeAddresses.size must be(3) routeeAddresses.size must be(3)
replies.values.sum must be(iterationCount) replies.values.sum must be(iterationCount)

View file

@ -24,6 +24,16 @@ object ClusterDeployerSpec {
cluster.allow-local-routees = off cluster.allow-local-routees = off
} }
/user/service2 { /user/service2 {
dispatcher = mydispatcher
mailbox = mymailbox
router = round-robin
nr-of-instances = 20
routees.paths = ["/user/myservice"]
cluster.enabled = on
cluster.allow-local-routees = off
}
# deprecated cluster.routees-path
/user/service3 {
dispatcher = mydispatcher dispatcher = mydispatcher
mailbox = mymailbox mailbox = mymailbox
router = round-robin router = round-robin
@ -73,7 +83,23 @@ class ClusterDeployerSpec extends AkkaSpec(ClusterDeployerSpec.deployerConf) {
service, service,
deployment.get.config, deployment.get.config,
ClusterRouterGroup(RoundRobinGroup(List("/user/myservice")), ClusterRouterGroupSettings( ClusterRouterGroup(RoundRobinGroup(List("/user/myservice")), ClusterRouterGroupSettings(
totalInstances = 20, routeesPath = "/user/myservice", allowLocalRoutees = false, useRole = None)), totalInstances = 20, routeesPaths = List("/user/myservice"), allowLocalRoutees = false, useRole = None)),
ClusterScope,
"mydispatcher",
"mymailbox")))
}
"be able to parse 'akka.actor.deployment._' with deprecated 'cluster.routees-path'" in {
val service = "/user/service3"
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1))
deployment must not be (None)
deployment must be(Some(
Deploy(
service,
deployment.get.config,
ClusterRouterGroup(RoundRobinGroup(List("/user/myservice")), ClusterRouterGroupSettings(
totalInstances = 20, routeesPaths = List("/user/myservice"), allowLocalRoutees = false, useRole = None)),
ClusterScope, ClusterScope,
"mydispatcher", "mydispatcher",
"mymailbox"))) "mymailbox")))

View file

@ -457,14 +457,12 @@ That is not done by the router. The configuration for a group looks like this:
available at that point it will be removed from the router and it will only re-try when the available at that point it will be removed from the router and it will only re-try when the
cluster members are changed. cluster members are changed.
It is the relative actor path defined in ``routees-path`` that identify what actor to lookup. It is the relative actor paths defined in ``routees.paths`` that identify what actor to lookup.
It is possible to limit the lookup of routees to member nodes tagged with a certain role by It is possible to limit the lookup of routees to member nodes tagged with a certain role by
specifying ``use-role``. specifying ``use-role``.
``nr-of-instances`` defines total number of routees in the cluster, but there will not be ``nr-of-instances`` defines total number of routees in the cluster. Setting ``nr-of-instances``
more than one per node. That routee actor could easily fan out to local children if more parallelism to a high value will result in new routees added to the router when nodes join the cluster.
is needed. Setting ``nr-of-instances`` to a high value will result in new routees
added to the router when nodes join the cluster.
The same type of router could also have been defined in code: The same type of router could also have been defined in code:
@ -506,7 +504,7 @@ The service that receives text from users and splits it up into words, delegates
Note, nothing cluster specific so far, just plain actors. Note, nothing cluster specific so far, just plain actors.
All nodes start ``StatsService`` and ``StatsWorker`` actors. Remember, routees are the workers in this case. All nodes start ``StatsService`` and ``StatsWorker`` actors. Remember, routees are the workers in this case.
The router is configured with ``routees-path``: The router is configured with ``routees.paths``:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#config-router-lookup .. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#config-router-lookup

View file

@ -73,10 +73,22 @@ Example in Java::
getContext().actorOf(new RoundRobinPool(5).props(Props.create(Worker.class)), getContext().actorOf(new RoundRobinPool(5).props(Props.create(Worker.class)),
"router2"); "router2");
To support multiple routee paths for a cluster aware router sending to paths the deployment configuration
property ``cluster.routees-path`` has been changed to string list ``routees.paths`` property.
The old ``cluster.routees-path`` is deprecated, but still working during the deprecation phase.
Example::
/router4 {
router = round-robin
nr-of-instances = 10
routees.paths = ["/user/myserviceA", "/user/myserviceB"]
cluster.enabled = on
}
The API for creating custom routers and resizers have changed without keeping the old API as deprecated. The API for creating custom routers and resizers have changed without keeping the old API as deprecated.
That should be a an API used by only a few users and they should be able to migrate to the new API That should be a an API used by only a few users and they should be able to migrate to the new API
without much trouble. without much trouble.
Read more about the new routers in the :ref:`documentation for Scala <routing-scala>` and Read more about the new routers in the :ref:`documentation for Scala <routing-scala>` and
:ref:`documentation for Java <routing-java>`. :ref:`documentation for Java <routing-java>`.

View file

@ -448,14 +448,12 @@ That is not done by the router. The configuration for a group looks like this:
available at that point it will be removed from the router and it will only re-try when the available at that point it will be removed from the router and it will only re-try when the
cluster members are changed. cluster members are changed.
It is the relative actor path defined in ``routees-path`` that identify what actor to lookup. It is the relative actor paths defined in ``routees.paths`` that identify what actor to lookup.
It is possible to limit the lookup of routees to member nodes tagged with a certain role by It is possible to limit the lookup of routees to member nodes tagged with a certain role by
specifying ``use-role``. specifying ``use-role``.
``nr-of-instances`` defines total number of routees in the cluster, but there will not be ``nr-of-instances`` defines total number of routees in the cluster. Setting ``nr-of-instances``
more than one per node. That routee actor could easily fan out to local children if more parallelism to a high value will result in new routees added to the router when nodes join the cluster.
is needed. Setting ``nr-of-instances`` to a high value will result in new routees
added to the router when nodes join the cluster.
The same type of router could also have been defined in code: The same type of router could also have been defined in code:
@ -495,7 +493,7 @@ The service that receives text from users and splits it up into words, delegates
Note, nothing cluster specific so far, just plain actors. Note, nothing cluster specific so far, just plain actors.
All nodes start ``StatsService`` and ``StatsWorker`` actors. Remember, routees are the workers in this case. All nodes start ``StatsService`` and ``StatsWorker`` actors. Remember, routees are the workers in this case.
The router is configured with ``routees-path``: The router is configured with ``routees.paths``:
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#config-router-lookup .. includecode:: ../../../akka-samples/akka-sample-cluster/src/main/resources/application.conf#config-router-lookup

View file

@ -3,6 +3,7 @@ package sample.cluster.stats.japi;
import java.util.Collections; import java.util.Collections;
import sample.cluster.stats.japi.StatsMessages.StatsJob; import sample.cluster.stats.japi.StatsMessages.StatsJob;
//#imports //#imports
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Props; import akka.actor.Props;
@ -60,13 +61,13 @@ public class StatsService extends UntypedActor {
abstract class StatsService2 extends UntypedActor { abstract class StatsService2 extends UntypedActor {
//#router-lookup-in-code //#router-lookup-in-code
int totalInstances = 100; int totalInstances = 100;
String routeesPath = "/user/statsWorker"; Iterable<String> routeesPaths = Collections.singletonList("/user/statsWorker");
boolean allowLocalRoutees = true; boolean allowLocalRoutees = true;
String useRole = "compute"; String useRole = "compute";
ActorRef workerRouter = getContext().actorOf( ActorRef workerRouter = getContext().actorOf(
new ClusterRouterGroup( new ClusterRouterGroup(
new ConsistentHashingGroup(Collections.<String>emptyList()), new ClusterRouterGroupSettings( new ConsistentHashingGroup(routeesPaths), new ClusterRouterGroupSettings(
totalInstances, routeesPath, allowLocalRoutees, useRole)).props(), totalInstances, routeesPaths, allowLocalRoutees, useRole)).props(),
"workerRouter2"); "workerRouter2");
//#router-lookup-in-code //#router-lookup-in-code
} }

View file

@ -26,9 +26,9 @@ akka.actor.deployment {
/statsService/workerRouter { /statsService/workerRouter {
router = consistent-hashing-group router = consistent-hashing-group
nr-of-instances = 100 nr-of-instances = 100
routees.paths = ["/user/statsWorker"]
cluster { cluster {
enabled = on enabled = on
routees-path = "/user/statsWorker"
allow-local-routees = on allow-local-routees = on
use-role = compute use-role = compute
} }
@ -60,9 +60,9 @@ akka.actor.deployment {
# metrics-selector = cpu # metrics-selector = cpu
metrics-selector = mix metrics-selector = mix
nr-of-instances = 100 nr-of-instances = 100
routees.paths = ["/user/factorialBackend"]
cluster { cluster {
enabled = on enabled = on
routees-path = "/user/factorialBackend"
use-role = backend use-role = backend
allow-local-routees = off allow-local-routees = off
} }

View file

@ -149,7 +149,7 @@ abstract class FactorialFrontend2 extends Actor {
val backend = context.actorOf( val backend = context.actorOf(
ClusterRouterGroup(AdaptiveLoadBalancingGroup(HeapMetricsSelector), ClusterRouterGroup(AdaptiveLoadBalancingGroup(HeapMetricsSelector),
ClusterRouterGroupSettings( ClusterRouterGroupSettings(
totalInstances = 100, routeesPath = "/user/factorialBackend", totalInstances = 100, routeesPaths = List("/user/factorialBackend"),
allowLocalRoutees = true, useRole = Some("backend"))).props(), allowLocalRoutees = true, useRole = Some("backend"))).props(),
name = "factorialBackendRouter2") name = "factorialBackendRouter2")
//#router-lookup-in-code //#router-lookup-in-code

View file

@ -230,7 +230,7 @@ abstract class StatsService2 extends Actor {
val workerRouter = context.actorOf( val workerRouter = context.actorOf(
ClusterRouterGroup(ConsistentHashingGroup(Nil), ClusterRouterGroupSettings( ClusterRouterGroup(ConsistentHashingGroup(Nil), ClusterRouterGroupSettings(
totalInstances = 100, routeesPath = "/user/statsWorker", totalInstances = 100, routeesPaths = List("/user/statsWorker"),
allowLocalRoutees = true, useRole = Some("compute"))).props(), allowLocalRoutees = true, useRole = Some("compute"))).props(),
name = "workerRouter2") name = "workerRouter2")
//#router-lookup-in-code //#router-lookup-in-code

View file

@ -34,9 +34,9 @@ object StatsSampleSpecConfig extends MultiNodeConfig {
/statsService/workerRouter { /statsService/workerRouter {
router = consistent-hashing-group router = consistent-hashing-group
nr-of-instances = 100 nr-of-instances = 100
routees.paths = ["/user/statsWorker"]
cluster { cluster {
enabled = on enabled = on
routees-path = "/user/statsWorker"
allow-local-routees = on allow-local-routees = on
use-role = compute use-role = compute
} }

View file

@ -37,9 +37,9 @@ object StatsSampleJapiSpecConfig extends MultiNodeConfig {
/statsService/workerRouter { /statsService/workerRouter {
router = consistent-hashing-group router = consistent-hashing-group
nr-of-instances = 100 nr-of-instances = 100
routees.paths = ["/user/statsWorker"]
cluster { cluster {
enabled = on enabled = on
routees-path = "/user/statsWorker"
allow-local-routees = on allow-local-routees = on
use-role = compute use-role = compute
} }