2012-08-28 08:36:14 +02:00
|
|
|
/**
|
2014-02-02 19:05:45 -06:00
|
|
|
* Copyright (C) 2009-2014 Typesafe Inc. <http://www.typesafe.com>
|
2012-08-28 08:36:14 +02:00
|
|
|
*/
|
|
|
|
|
package akka.cluster.routing
|
|
|
|
|
|
2012-11-07 16:35:14 +01:00
|
|
|
import scala.collection.immutable
|
2013-09-19 08:00:05 +02:00
|
|
|
import akka.routing.RouterConfig
|
|
|
|
|
import akka.routing.Router
|
|
|
|
|
import akka.actor.Props
|
2012-09-07 10:02:05 +02:00
|
|
|
import akka.actor.ActorContext
|
2013-09-19 08:00:05 +02:00
|
|
|
import akka.routing.Routee
|
|
|
|
|
import java.util.concurrent.atomic.AtomicInteger
|
2012-08-28 08:36:14 +02:00
|
|
|
import akka.actor.Address
|
2013-09-19 08:00:05 +02:00
|
|
|
import akka.actor.ActorCell
|
2012-08-28 08:36:14 +02:00
|
|
|
import akka.actor.Deploy
|
2013-09-19 08:00:05 +02:00
|
|
|
import com.typesafe.config.ConfigFactory
|
|
|
|
|
import akka.routing.ActorRefRoutee
|
2012-08-28 08:36:14 +02:00
|
|
|
import akka.remote.RemoteScope
|
2013-09-19 08:00:05 +02:00
|
|
|
import akka.actor.Actor
|
|
|
|
|
import akka.actor.SupervisorStrategy
|
2012-08-28 08:36:14 +02:00
|
|
|
import akka.routing.Resizer
|
|
|
|
|
import akka.routing.RouterConfig
|
2013-09-19 08:00:05 +02:00
|
|
|
import akka.routing.Pool
|
|
|
|
|
import akka.routing.Group
|
2012-09-17 12:54:08 +02:00
|
|
|
import akka.remote.routing.RemoteRouterConfig
|
2013-09-19 08:00:05 +02:00
|
|
|
import akka.routing.RouterActor
|
|
|
|
|
import akka.cluster.Cluster
|
|
|
|
|
import akka.cluster.ClusterEvent._
|
|
|
|
|
import akka.actor.ActorRef
|
|
|
|
|
import akka.cluster.Member
|
|
|
|
|
import scala.annotation.tailrec
|
2012-09-07 14:54:53 +02:00
|
|
|
import akka.actor.RootActorPath
|
2013-09-19 08:00:05 +02:00
|
|
|
import akka.cluster.MemberStatus
|
|
|
|
|
import akka.routing.ActorSelectionRoutee
|
|
|
|
|
import akka.actor.ActorInitializationException
|
|
|
|
|
import akka.routing.RouterPoolActor
|
|
|
|
|
import akka.actor.ActorSystem
|
|
|
|
|
import akka.actor.ActorSystem
|
|
|
|
|
import akka.routing.RoutingLogic
|
2012-09-10 14:41:51 +02:00
|
|
|
import akka.actor.RelativeActorPath
|
2013-09-19 08:00:05 +02:00
|
|
|
import com.typesafe.config.Config
|
|
|
|
|
import akka.routing.DeprecatedRouterConfig
|
2013-10-16 11:06:38 +02:00
|
|
|
import akka.japi.Util.immutableSeq
|
2013-09-19 08:00:05 +02:00
|
|
|
|
|
|
|
|
object ClusterRouterGroupSettings {
|
|
|
|
|
def fromConfig(config: Config): ClusterRouterGroupSettings =
|
|
|
|
|
ClusterRouterGroupSettings(
|
|
|
|
|
totalInstances = config.getInt("nr-of-instances"),
|
2013-10-16 11:06:38 +02:00
|
|
|
routeesPaths = immutableSeq(config.getStringList("routees.paths")),
|
2013-09-19 08:00:05 +02:00
|
|
|
allowLocalRoutees = config.getBoolean("cluster.allow-local-routees"),
|
|
|
|
|
useRole = ClusterRouterSettingsBase.useRoleOption(config.getString("cluster.use-role")))
|
|
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
|
|
|
|
/**
|
2013-09-19 08:00:05 +02:00
|
|
|
* `totalInstances` of cluster router must be > 0
|
2012-08-28 08:36:14 +02:00
|
|
|
*/
|
2012-09-11 19:11:20 +02:00
|
|
|
@SerialVersionUID(1L)
|
2013-09-19 08:00:05 +02:00
|
|
|
case class ClusterRouterGroupSettings(
|
|
|
|
|
totalInstances: Int,
|
2013-10-16 11:06:38 +02:00
|
|
|
routeesPaths: immutable.Seq[String],
|
2013-09-19 08:00:05 +02:00
|
|
|
allowLocalRoutees: Boolean,
|
|
|
|
|
useRole: Option[String]) extends ClusterRouterSettingsBase {
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-10-16 11:06:38 +02:00
|
|
|
@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)
|
|
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
/**
|
|
|
|
|
* Java API
|
|
|
|
|
*/
|
2013-10-16 11:06:38 +02:00
|
|
|
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")
|
2013-09-19 08:00:05 +02:00
|
|
|
def this(totalInstances: Int, routeesPath: String, allowLocalRoutees: Boolean, useRole: String) =
|
|
|
|
|
this(totalInstances, routeesPath, allowLocalRoutees, ClusterRouterSettingsBase.useRoleOption(useRole))
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
if (totalInstances <= 0) throw new IllegalArgumentException("totalInstances of cluster router must be > 0")
|
2013-10-16 11:06:38 +02:00
|
|
|
if ((routeesPaths eq null) || routeesPaths.isEmpty || routeesPaths.head == "")
|
|
|
|
|
throw new IllegalArgumentException("routeesPaths must be defined")
|
2012-08-30 12:13:50 +02:00
|
|
|
|
2013-10-16 11:06:38 +02:00
|
|
|
routeesPaths.foreach(p ⇒ p match {
|
2013-09-19 08:00:05 +02:00
|
|
|
case RelativeActorPath(elements) ⇒ // good
|
|
|
|
|
case _ ⇒
|
2013-10-16 11:06:38 +02:00
|
|
|
throw new IllegalArgumentException(s"routeesPaths [$p] is not a valid relative actor path")
|
|
|
|
|
})
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
object ClusterRouterPoolSettings {
|
|
|
|
|
def fromConfig(config: Config): ClusterRouterPoolSettings =
|
|
|
|
|
ClusterRouterPoolSettings(
|
|
|
|
|
totalInstances = config.getInt("nr-of-instances"),
|
|
|
|
|
maxInstancesPerNode = config.getInt("cluster.max-nr-of-instances-per-node"),
|
|
|
|
|
allowLocalRoutees = config.getBoolean("cluster.allow-local-routees"),
|
|
|
|
|
useRole = ClusterRouterSettingsBase.useRoleOption(config.getString("cluster.use-role")))
|
|
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
/**
|
|
|
|
|
* `totalInstances` of cluster router must be > 0
|
|
|
|
|
* `maxInstancesPerNode` of cluster router must be > 0
|
|
|
|
|
* `maxInstancesPerNode` of cluster router must be 1 when routeesPath is defined
|
|
|
|
|
*/
|
|
|
|
|
@SerialVersionUID(1L)
|
|
|
|
|
case class ClusterRouterPoolSettings(
|
|
|
|
|
totalInstances: Int,
|
|
|
|
|
maxInstancesPerNode: Int,
|
|
|
|
|
allowLocalRoutees: Boolean,
|
|
|
|
|
useRole: Option[String]) extends ClusterRouterSettingsBase {
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
/**
|
|
|
|
|
* Java API
|
|
|
|
|
*/
|
|
|
|
|
def this(totalInstances: Int, maxInstancesPerNode: Int, allowLocalRoutees: Boolean, useRole: String) =
|
|
|
|
|
this(totalInstances, maxInstancesPerNode, allowLocalRoutees, ClusterRouterSettingsBase.useRoleOption(useRole))
|
2013-04-08 13:19:44 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
if (maxInstancesPerNode <= 0) throw new IllegalArgumentException("maxInstancesPerNode of cluster pool router must be > 0")
|
|
|
|
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] object ClusterRouterSettingsBase {
|
|
|
|
|
def useRoleOption(role: String): Option[String] = role match {
|
|
|
|
|
case null | "" ⇒ None
|
|
|
|
|
case _ ⇒ Some(role)
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] trait ClusterRouterSettingsBase {
|
|
|
|
|
def totalInstances: Int
|
|
|
|
|
def allowLocalRoutees: Boolean
|
|
|
|
|
def useRole: Option[String]
|
2012-09-10 14:11:03 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
if (totalInstances <= 0) throw new IllegalArgumentException("totalInstances of cluster router must be > 0")
|
|
|
|
|
}
|
2012-09-10 14:11:03 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
/**
|
|
|
|
|
* [[akka.routing.RouterConfig]] implementation for deployment on cluster nodes.
|
|
|
|
|
* Delegates other duties to the local [[akka.routing.RouterConfig]],
|
|
|
|
|
* which makes it possible to mix this with the built-in routers such as
|
|
|
|
|
* [[akka.routing.RoundRobinRouter]] or custom routers.
|
|
|
|
|
*/
|
|
|
|
|
@SerialVersionUID(1L)
|
|
|
|
|
final case class ClusterRouterGroup(local: Group, settings: ClusterRouterGroupSettings) extends Group with ClusterRouterConfigBase {
|
|
|
|
|
|
2013-10-16 11:06:38 +02:00
|
|
|
override def paths: immutable.Iterable[String] = if (settings.allowLocalRoutees) settings.routeesPaths else Nil
|
2012-09-10 14:11:03 +02:00
|
|
|
|
|
|
|
|
/**
|
2013-09-19 08:00:05 +02:00
|
|
|
* INTERNAL API
|
2012-09-10 14:11:03 +02:00
|
|
|
*/
|
2013-09-19 08:00:05 +02:00
|
|
|
override private[akka] def createRouterActor(): RouterActor = new ClusterRouterGroupActor(settings)
|
2013-03-14 20:32:43 +01:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
|
|
|
|
case ClusterRouterGroup(_: ClusterRouterGroup, _) ⇒ throw new IllegalStateException(
|
|
|
|
|
"ClusterRouterGroup is not allowed to wrap a ClusterRouterGroup")
|
|
|
|
|
case ClusterRouterGroup(local, _) ⇒
|
|
|
|
|
copy(local = this.local.withFallback(local).asInstanceOf[Group])
|
|
|
|
|
case ClusterRouterConfig(local, _) ⇒
|
|
|
|
|
copy(local = this.local.withFallback(local).asInstanceOf[Group])
|
|
|
|
|
case _ ⇒
|
|
|
|
|
copy(local = this.local.withFallback(other).asInstanceOf[Group])
|
2013-03-14 20:32:43 +01:00
|
|
|
}
|
2013-09-19 08:00:05 +02:00
|
|
|
|
2012-09-10 14:11:03 +02:00
|
|
|
}
|
|
|
|
|
|
2012-09-11 19:11:20 +02:00
|
|
|
/**
|
2013-09-19 08:00:05 +02:00
|
|
|
* [[akka.routing.RouterConfig]] implementation for deployment on cluster nodes.
|
|
|
|
|
* Delegates other duties to the local [[akka.routing.RouterConfig]],
|
|
|
|
|
* which makes it possible to mix this with the built-in routers such as
|
|
|
|
|
* [[akka.routing.RoundRobinRouter]] or custom routers.
|
2012-09-11 19:11:20 +02:00
|
|
|
*/
|
|
|
|
|
@SerialVersionUID(1L)
|
2013-09-19 08:00:05 +02:00
|
|
|
final case class ClusterRouterPool(local: Pool, settings: ClusterRouterPoolSettings) extends Pool with ClusterRouterConfigBase {
|
|
|
|
|
|
|
|
|
|
require(local.resizer.isEmpty, "Resizer can't be used together with cluster router")
|
|
|
|
|
|
|
|
|
|
@transient private val childNameCounter = new AtomicInteger
|
2012-09-10 14:11:03 +02:00
|
|
|
|
|
|
|
|
/**
|
2013-09-19 08:00:05 +02:00
|
|
|
* INTERNAL API
|
2012-09-10 14:11:03 +02:00
|
|
|
*/
|
2013-09-19 08:00:05 +02:00
|
|
|
override private[akka] def newRoutee(routeeProps: Props, context: ActorContext): Routee = {
|
|
|
|
|
val name = "c" + childNameCounter.incrementAndGet
|
2013-10-16 13:02:35 +02:00
|
|
|
val ref = context.asInstanceOf[ActorCell].attachChild(
|
|
|
|
|
local.enrichWithPoolDispatcher(routeeProps, context), name, systemService = false)
|
2013-09-19 08:00:05 +02:00
|
|
|
ActorRefRoutee(ref)
|
|
|
|
|
}
|
2012-09-10 14:11:03 +02:00
|
|
|
|
|
|
|
|
/**
|
2013-09-19 08:00:05 +02:00
|
|
|
* Initial number of routee instances
|
2012-09-10 14:11:03 +02:00
|
|
|
*/
|
2013-09-19 08:00:05 +02:00
|
|
|
override def nrOfInstances: Int = if (settings.allowLocalRoutees) settings.maxInstancesPerNode else 0
|
2012-09-08 17:30:42 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
override def resizer: Option[Resizer] = local.resizer
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
override private[akka] def createRouterActor(): RouterActor = new ClusterRouterPoolActor(local.supervisorStrategy, settings)
|
|
|
|
|
|
|
|
|
|
override def supervisorStrategy: SupervisorStrategy = local.supervisorStrategy
|
2012-09-07 14:54:53 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
|
|
|
|
case ClusterRouterPool(_: ClusterRouterPool, _) ⇒ throw new IllegalStateException(
|
|
|
|
|
"ClusterRouterPool is not allowed to wrap a ClusterRouterPool")
|
|
|
|
|
case ClusterRouterPool(otherLocal, _) ⇒
|
|
|
|
|
copy(local = this.local.withFallback(otherLocal).asInstanceOf[Pool])
|
|
|
|
|
case ClusterRouterConfig(otherLocal, _) ⇒
|
|
|
|
|
copy(local = this.local.withFallback(otherLocal).asInstanceOf[Pool])
|
2012-09-10 14:41:51 +02:00
|
|
|
case _ ⇒
|
2013-09-19 08:00:05 +02:00
|
|
|
copy(local = this.local.withFallback(other).asInstanceOf[Pool])
|
2012-09-10 14:41:51 +02:00
|
|
|
}
|
|
|
|
|
|
2012-09-07 14:54:53 +02:00
|
|
|
}
|
2012-09-07 12:07:41 +02:00
|
|
|
|
2012-08-28 08:36:14 +02:00
|
|
|
/**
|
2012-08-30 12:13:50 +02:00
|
|
|
* INTERNAL API
|
2012-08-28 08:36:14 +02:00
|
|
|
*/
|
2013-09-19 08:00:05 +02:00
|
|
|
private[akka] trait ClusterRouterConfigBase extends RouterConfig {
|
|
|
|
|
def local: RouterConfig
|
|
|
|
|
def settings: ClusterRouterSettingsBase
|
|
|
|
|
override def createRouter(system: ActorSystem): Router = local.createRouter(system)
|
|
|
|
|
override def routerDispatcher: String = local.routerDispatcher
|
|
|
|
|
override def stopRouterWhenAllRouteesRemoved: Boolean = false
|
|
|
|
|
override def routingLogicController(routingLogic: RoutingLogic): Option[Props] =
|
|
|
|
|
local.routingLogicController(routingLogic)
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
// Intercept ClusterDomainEvent and route them to the ClusterRouterActor
|
|
|
|
|
override def isManagementMessage(msg: Any): Boolean =
|
2014-01-08 14:14:48 +01:00
|
|
|
(msg.isInstanceOf[ClusterDomainEvent]) || msg.isInstanceOf[CurrentClusterState] || super.isManagementMessage(msg)
|
2013-09-19 08:00:05 +02:00
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class ClusterRouterPoolActor(
|
|
|
|
|
supervisorStrategy: SupervisorStrategy, val settings: ClusterRouterPoolSettings)
|
|
|
|
|
extends RouterPoolActor(supervisorStrategy) with ClusterRouterActor {
|
|
|
|
|
|
|
|
|
|
override def receive = clusterReceive orElse super.receive
|
2012-08-28 20:54:16 +02:00
|
|
|
|
2012-08-30 10:06:12 +02:00
|
|
|
/**
|
2013-09-19 08:00:05 +02:00
|
|
|
* Adds routees based on totalInstances and maxInstancesPerNode settings
|
2012-08-30 10:06:12 +02:00
|
|
|
*/
|
2013-09-19 08:00:05 +02:00
|
|
|
override def addRoutees(): Unit = {
|
2012-09-17 08:42:47 +02:00
|
|
|
@tailrec
|
2013-09-19 08:00:05 +02:00
|
|
|
def doAddRoutees(): Unit = selectDeploymentTarget match {
|
2012-09-17 08:42:47 +02:00
|
|
|
case None ⇒ // done
|
|
|
|
|
case Some(target) ⇒
|
2013-09-19 08:00:05 +02:00
|
|
|
val routeeProps = cell.routeeProps
|
|
|
|
|
val deploy = Deploy(config = ConfigFactory.empty(), routerConfig = routeeProps.routerConfig,
|
|
|
|
|
scope = RemoteScope(target))
|
|
|
|
|
val routee = pool.newRoutee(routeeProps.withDeploy(deploy), context)
|
2012-09-17 08:42:47 +02:00
|
|
|
// must register each one, since registered routees are used in selectDeploymentTarget
|
2013-09-19 08:00:05 +02:00
|
|
|
cell.addRoutee(routee)
|
2012-09-17 08:42:47 +02:00
|
|
|
|
|
|
|
|
// recursion until all created
|
2013-09-19 08:00:05 +02:00
|
|
|
doAddRoutees()
|
2012-08-29 19:33:19 +02:00
|
|
|
}
|
2012-09-17 08:42:47 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
doAddRoutees()
|
2012-08-29 19:33:19 +02:00
|
|
|
}
|
|
|
|
|
|
2013-10-16 11:06:38 +02:00
|
|
|
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
|
|
|
|
|
}
|
|
|
|
|
}
|
2012-08-30 12:13:50 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
}
|
2013-04-08 15:04:36 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class ClusterRouterGroupActor(val settings: ClusterRouterGroupSettings)
|
|
|
|
|
extends RouterActor with ClusterRouterActor {
|
2012-08-29 19:33:19 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
val group = cell.routerConfig match {
|
|
|
|
|
case x: Group ⇒ x
|
|
|
|
|
case other ⇒
|
|
|
|
|
throw ActorInitializationException("ClusterRouterGroupActor can only be used with Nozle, not " + other.getClass)
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|
|
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
override def receive = clusterReceive orElse super.receive
|
2012-08-30 12:13:50 +02:00
|
|
|
|
2013-10-16 11:06:38 +02:00
|
|
|
var usedRouteePaths: Map[Address, Set[String]] =
|
|
|
|
|
if (settings.allowLocalRoutees)
|
|
|
|
|
Map(cluster.selfAddress -> settings.routeesPaths.toSet)
|
|
|
|
|
else
|
|
|
|
|
Map.empty
|
|
|
|
|
|
2012-08-29 19:33:19 +02:00
|
|
|
/**
|
2013-09-19 08:00:05 +02:00
|
|
|
* Adds routees based on totalInstances and maxInstancesPerNode settings
|
2012-08-29 19:33:19 +02:00
|
|
|
*/
|
2013-09-19 08:00:05 +02:00
|
|
|
override def addRoutees(): Unit = {
|
|
|
|
|
@tailrec
|
|
|
|
|
def doAddRoutees(): Unit = selectDeploymentTarget match {
|
|
|
|
|
case None ⇒ // done
|
2013-10-16 11:06:38 +02:00
|
|
|
case Some((address, path)) ⇒
|
|
|
|
|
val routee = group.routeeFor(address + path, context)
|
|
|
|
|
usedRouteePaths = usedRouteePaths.updated(address, usedRouteePaths.getOrElse(address, Set.empty) + path)
|
2013-09-19 08:00:05 +02:00
|
|
|
// must register each one, since registered routees are used in selectDeploymentTarget
|
|
|
|
|
cell.addRoutee(routee)
|
|
|
|
|
|
|
|
|
|
// recursion until all created
|
|
|
|
|
doAddRoutees()
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
doAddRoutees()
|
2012-08-29 19:33:19 +02:00
|
|
|
}
|
|
|
|
|
|
2013-10-16 11:06:38 +02:00
|
|
|
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) }
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2013-09-19 08:00:05 +02:00
|
|
|
|
2013-10-16 11:06:38 +02:00
|
|
|
override def removeMember(member: Member): Unit = {
|
|
|
|
|
usedRouteePaths -= member.address
|
|
|
|
|
super.removeMember(member)
|
|
|
|
|
}
|
2013-09-19 08:00:05 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* INTERNAL API
|
|
|
|
|
* The router actor, subscribes to cluster events and
|
|
|
|
|
* adjusts the routees.
|
|
|
|
|
*/
|
|
|
|
|
private[akka] trait ClusterRouterActor { this: RouterActor ⇒
|
|
|
|
|
|
|
|
|
|
def settings: ClusterRouterSettingsBase
|
|
|
|
|
|
|
|
|
|
if (!cell.routerConfig.isInstanceOf[Pool] && !cell.routerConfig.isInstanceOf[Group])
|
|
|
|
|
throw ActorInitializationException("Cluster router actor can only be used with Pool or Group, not with " +
|
|
|
|
|
cell.routerConfig.getClass)
|
|
|
|
|
|
|
|
|
|
def cluster: Cluster = Cluster(context.system)
|
|
|
|
|
|
|
|
|
|
// re-subscribe when restart
|
2014-01-08 14:14:48 +01:00
|
|
|
override def preStart(): Unit =
|
|
|
|
|
cluster.subscribe(self, classOf[MemberEvent], classOf[ReachabilityEvent])
|
|
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
override def postStop(): Unit = cluster.unsubscribe(self)
|
2012-09-10 14:41:51 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
var nodes: immutable.SortedSet[Address] = {
|
2012-09-10 14:41:51 +02:00
|
|
|
import Member.addressOrdering
|
|
|
|
|
cluster.readView.members.collect {
|
2012-10-10 10:05:41 -06:00
|
|
|
case m if isAvailable(m) ⇒ m.address
|
2012-09-10 14:41:51 +02:00
|
|
|
}
|
2012-09-07 12:07:41 +02:00
|
|
|
}
|
|
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
def isAvailable(m: Member): Boolean =
|
2013-03-14 20:32:43 +01:00
|
|
|
m.status == MemberStatus.Up &&
|
|
|
|
|
satisfiesRole(m.roles) &&
|
|
|
|
|
(settings.allowLocalRoutees || m.address != cluster.selfAddress)
|
|
|
|
|
|
|
|
|
|
private def satisfiesRole(memberRoles: Set[String]): Boolean = settings.useRole match {
|
|
|
|
|
case None ⇒ true
|
|
|
|
|
case Some(r) ⇒ memberRoles.contains(r)
|
|
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
def availableNodes: immutable.SortedSet[Address] = {
|
|
|
|
|
import Member.addressOrdering
|
|
|
|
|
val currentNodes = nodes
|
|
|
|
|
if (currentNodes.isEmpty && settings.allowLocalRoutees && satisfiesRole(cluster.selfRoles))
|
|
|
|
|
//use my own node, cluster information not updated yet
|
|
|
|
|
immutable.SortedSet(cluster.selfAddress)
|
|
|
|
|
else
|
|
|
|
|
currentNodes
|
2012-11-27 18:07:37 +01:00
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
/**
|
|
|
|
|
* Fills in self address for local ActorRef
|
|
|
|
|
*/
|
|
|
|
|
def fullAddress(routee: Routee): Address = {
|
|
|
|
|
val a = routee match {
|
|
|
|
|
case ActorRefRoutee(ref) ⇒ ref.path.address
|
|
|
|
|
case ActorSelectionRoutee(sel) ⇒ sel.anchor.path.address
|
|
|
|
|
}
|
|
|
|
|
a match {
|
|
|
|
|
case Address(_, _, None, None) ⇒ cluster.selfAddress
|
|
|
|
|
case a ⇒ a
|
|
|
|
|
}
|
2012-08-30 12:13:50 +02:00
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
/**
|
2013-10-16 11:06:38 +02:00
|
|
|
* Adds routees based on settings
|
2013-09-19 08:00:05 +02:00
|
|
|
*/
|
|
|
|
|
def addRoutees(): Unit
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-10-16 11:06:38 +02:00
|
|
|
def addMember(member: Member): Unit = {
|
2013-09-19 08:00:05 +02:00
|
|
|
nodes += member.address
|
|
|
|
|
addRoutees()
|
|
|
|
|
}
|
|
|
|
|
|
2013-10-16 11:06:38 +02:00
|
|
|
def removeMember(member: Member): Unit = {
|
2012-11-27 18:07:37 +01:00
|
|
|
val address = member.address
|
2013-09-19 08:00:05 +02:00
|
|
|
nodes -= address
|
2012-11-27 18:07:37 +01:00
|
|
|
|
|
|
|
|
// unregister routees that live on that node
|
2013-09-19 08:00:05 +02:00
|
|
|
val affectedRoutees = cell.router.routees.filter(fullAddress(_) == address)
|
|
|
|
|
cell.removeRoutees(affectedRoutees, stopChild = true)
|
2012-11-27 18:07:37 +01:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
// addRoutees will not create more than createRoutees and maxInstancesPerNode
|
2012-11-27 18:07:37 +01:00
|
|
|
// this is useful when totalInstances < upNodes.size
|
2013-09-19 08:00:05 +02:00
|
|
|
addRoutees()
|
2012-11-27 18:07:37 +01:00
|
|
|
}
|
|
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
def clusterReceive: Receive = {
|
2012-08-30 12:13:50 +02:00
|
|
|
case s: CurrentClusterState ⇒
|
|
|
|
|
import Member.addressOrdering
|
2013-09-19 08:00:05 +02:00
|
|
|
nodes = s.members.collect { case m if isAvailable(m) ⇒ m.address }
|
|
|
|
|
addRoutees()
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2013-09-19 08:00:05 +02:00
|
|
|
case m: MemberEvent if isAvailable(m.member) ⇒
|
|
|
|
|
addMember(m.member)
|
2012-08-28 08:36:14 +02:00
|
|
|
|
2012-08-30 12:13:50 +02:00
|
|
|
case other: MemberEvent ⇒
|
|
|
|
|
// other events means that it is no longer interesting, such as
|
2013-08-27 15:14:53 +02:00
|
|
|
// MemberExited, MemberRemoved
|
2013-09-19 08:00:05 +02:00
|
|
|
removeMember(other.member)
|
2012-08-30 12:13:50 +02:00
|
|
|
|
2012-11-27 18:07:37 +01:00
|
|
|
case UnreachableMember(m) ⇒
|
2013-09-19 08:00:05 +02:00
|
|
|
removeMember(m)
|
2013-08-27 15:14:53 +02:00
|
|
|
|
|
|
|
|
case ReachableMember(m) ⇒
|
2013-09-19 08:00:05 +02:00
|
|
|
if (isAvailable(m)) addMember(m)
|
2012-08-30 12:13:50 +02:00
|
|
|
}
|
2012-08-28 08:36:14 +02:00
|
|
|
}
|
2013-09-19 08:00:05 +02:00
|
|
|
|