Deploy to new members in cluster, see #2103

* Config max-nr-of-instances-per-node
* selectDeploymentTarget that takes max-nr-of-instances-per-node
  and nr-of-instances into account
* Deploy when new member added or removed
* Moved routeeProps to RouteeProvider constructor, needed for
  this feature, but also simplifies createRoute, createRoutee,
  and resize, since routeeProps doesn't have to be passed around.
This commit is contained in:
Patrik Nordwall 2012-08-29 19:33:19 +02:00
parent bf20ae5157
commit 695ce49727
9 changed files with 185 additions and 127 deletions

View file

@ -6,7 +6,6 @@ package akka.cluster.routing
import java.util.concurrent.atomic.AtomicInteger
import scala.collection.immutable.SortedSet
import com.typesafe.config.ConfigFactory
import akka.ConfigurationException
import akka.actor.Actor
import akka.actor.ActorContext
@ -27,6 +26,8 @@ import akka.routing.Route
import akka.routing.RouteeProvider
import akka.routing.Router
import akka.routing.RouterConfig
import java.lang.IllegalStateException
import akka.cluster.ClusterScope
/**
* [[akka.routing.RouterConfig]] implementation for deployment on cluster nodes.
@ -34,13 +35,12 @@ import akka.routing.RouterConfig
* which makes it possible to mix this with the built-in routers such as
* [[akka.routing.RoundRobinRouter]] or custom routers.
*/
case class ClusterRouterConfig(local: RouterConfig) extends RouterConfig {
case class ClusterRouterConfig(local: RouterConfig, totalInstances: Int, maxInstancesPerNode: Int) extends RouterConfig {
override def createRouteeProvider(context: ActorContext) = new ClusterRouteeProvider(context, resizer)
override def createRouteeProvider(context: ActorContext, routeeProps: Props) =
new ClusterRouteeProvider(context, routeeProps, resizer, totalInstances, maxInstancesPerNode)
override def createRoute(routeeProps: Props, routeeProvider: RouteeProvider): Route = {
local.createRoute(routeeProps, routeeProvider)
}
override def createRoute(routeeProvider: RouteeProvider): Route = local.createRoute(routeeProvider)
override def createActor(): Router = local.createActor()
@ -51,8 +51,8 @@ case class ClusterRouterConfig(local: RouterConfig) extends RouterConfig {
override def resizer: Option[Resizer] = local.resizer
override def withFallback(other: RouterConfig): RouterConfig = other match {
case ClusterRouterConfig(local) copy(local = this.local.withFallback(local))
case _ copy(local = this.local.withFallback(other))
case ClusterRouterConfig(local, _, _) copy(local = this.local.withFallback(local))
case _ copy(local = this.local.withFallback(other))
}
}
@ -60,8 +60,13 @@ case class ClusterRouterConfig(local: RouterConfig) extends RouterConfig {
* Factory and registry for routees of the router.
* Deploys new routees on the cluster nodes.
*/
class ClusterRouteeProvider(_context: ActorContext, _resizer: Option[Resizer])
extends RouteeProvider(_context, _resizer) {
class ClusterRouteeProvider(
_context: ActorContext,
_routeeProps: Props,
_resizer: Option[Resizer],
totalInstances: Int,
maxInstancesPerNode: Int)
extends RouteeProvider(_context, _routeeProps, _resizer) {
// need this counter as instance variable since Resizer may call createRoutees several times
private val childNameCounter = new AtomicInteger
@ -70,64 +75,80 @@ class ClusterRouteeProvider(_context: ActorContext, _resizer: Option[Resizer])
throw new ConfigurationException("Cluster deployment can not be combined with routees for [%s]"
format context.self.path.toString)
override def createRoutees(props: Props, nrOfInstances: Int): Unit = {
val nodes = upNodes
if (nodes.isEmpty) {
IndexedSeq.empty
} else {
val impl = context.system.asInstanceOf[ActorSystemImpl] //TODO ticket #1559
// FIXME We could count number of routees per node and select nodes with least routees first
val nodesIter: Iterator[Address] = Stream.continually(nodes).flatten.iterator
val refs = IndexedSeq.empty[ActorRef] ++ (for (i 1 to nrOfInstances) yield {
val name = "c" + childNameCounter.incrementAndGet
val deploy = Deploy("", ConfigFactory.empty(), props.routerConfig, RemoteScope(nodesIter.next))
impl.provider.actorOf(impl, props, context.self.asInstanceOf[InternalActorRef], context.self.path / name,
systemService = false, Some(deploy), lookupDeploy = false, async = false)
})
registerRoutees(refs)
override def createRoutees(nrOfInstances: Int): Unit = {
val impl = context.system.asInstanceOf[ActorSystemImpl] //TODO ticket #1559
for (i 1 to nrOfInstances; target selectDeploymentTarget) {
val name = "c" + childNameCounter.incrementAndGet
val deploy = Deploy("", ConfigFactory.empty(), routeeProps.routerConfig, RemoteScope(target))
var ref = impl.provider.actorOf(impl, routeeProps, context.self.asInstanceOf[InternalActorRef], context.self.path / name,
systemService = false, Some(deploy), lookupDeploy = false, async = false)
// must register each one, since registered routees are used in selectDeploymentTarget
registerRoutees(Some(ref))
}
}
// FIXME experimental hack to let the cluster initialize
// What should we do before we have full cluster information (startup phase)?
Cluster(context.system).readView
Thread.sleep(2000)
private def selectDeploymentTarget: Option[Address] = {
val currentRoutees = routees
val currentNodes = upNodes
if (currentRoutees.size >= totalInstances) {
None
} else if (currentNodes.isEmpty) {
// use my own node, cluster information not updated yet
Some(cluster.selfAddress)
} else {
val numberOfRouteesPerNode: Map[Address, Int] =
Map.empty[Address, Int] ++ currentNodes.toSeq.map(_ -> 0) ++
currentRoutees.groupBy(fullAddress).map {
case (address, refs) address -> refs.size
}
val (address, count) = numberOfRouteesPerNode.minBy(_._2)
if (count < maxInstancesPerNode) Some(address) else None
}
}
/**
* 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
}
private def cluster: Cluster = Cluster(context.system)
import Member.addressOrdering
@volatile
private var upNodes: SortedSet[Address] = Cluster(context.system).readView.members.collect {
private var upNodes: SortedSet[Address] = cluster.readView.members.collect {
case m if m.status == MemberStatus.Up m.address
}
// create actor that subscribes to the cluster eventBus
private val eventBusListener: ActorRef = {
private val eventBusListener: ActorRef = context.actorOf(Props(new Actor {
override def preStart(): Unit = cluster.subscribe(self, classOf[ClusterDomainEvent])
override def postStop(): Unit = cluster.unsubscribe(self)
// FIXME is this allowed, are we inside or outside of the actor?
context.actorOf(Props(new Actor {
override def preStart(): Unit = Cluster(context.system).subscribe(self, classOf[ClusterDomainEvent])
override def postStop(): Unit = Cluster(context.system).unsubscribe(self)
def receive = {
case s: CurrentClusterState
upNodes = s.members.collect { case m if m.status == MemberStatus.Up m.address }
def receive = {
case s: CurrentClusterState
upNodes = s.members.collect { case m if m.status == MemberStatus.Up m.address }
case MemberUp(m)
upNodes += m.address
// createRoutees will not create more than createRoutees and maxInstancesPerNode
createRoutees(totalInstances)
case MemberUp(m)
upNodes += m.address
// FIXME Here we could trigger a rebalance, by counting number of routees per node and unregister
// routees from nodes with many routees and deploy on this new node instead
case other: MemberEvent
// other events means that it is no longer interesting, such as
// MemberJoined, MemberLeft, MemberExited, MemberUnreachable, MemberRemoved
upNodes -= other.member.address
case other: MemberEvent
// other events means that it is no longer interesting, such as
// MemberJoined, MemberLeft, MemberExited, MemberUnreachable, MemberRemoved
upNodes -= other.member.address
// createRoutees will not create more than createRoutees and maxInstancesPerNode
createRoutees(totalInstances) // Here we
// FIXME Should we deploy new routees corresponding to the ones that goes away here?
// or is that a job for a special Cluster Resizer?
}
}
}), name = "cluster-listener")
}
}), name = "cluster-listener")
}