implement remote routers

- split out actual routing functionality into *Like traits to be
  reusable
- make RemoteRouterConfig which only differs in how remote placement is
  initiated
- teach RemoteDeployer how to convert local RouterConfig in case config
  key “target.nodes” is set
- IT WORKS! All tests GREEN
- oh, yeah, had to make ActorPath serializable ...
This commit is contained in:
Roland 2011-12-13 01:09:05 +01:00
parent 0a7e5fe296
commit d1a26a9e28
8 changed files with 226 additions and 29 deletions

View file

@ -1,3 +1,6 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.routing
import java.util.concurrent.atomic.AtomicInteger

View file

@ -27,7 +27,7 @@ object ActorPath {
* is sorted by path elements FROM RIGHT TO LEFT, where RootActorPath >
* ChildActorPath in case the number of elements is different.
*/
sealed trait ActorPath extends Comparable[ActorPath] {
sealed trait ActorPath extends Comparable[ActorPath] with Serializable {
/**
* The Address under which this path can be reached; walks up the tree to
* the RootActorPath.

View file

@ -72,7 +72,7 @@ trait ActorRefProvider {
* in case of remote supervision). If systemService is true, deployment is
* bypassed (local-only).
*/
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean): InternalActorRef
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean, deploy: Option[Deploy] = None): InternalActorRef
/**
* Create actor reference for a specified local or remote path. If no such
@ -518,16 +518,16 @@ class LocalActorRefProvider(
case x x
}
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean): InternalActorRef = {
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean, deploy: Option[Deploy] = None): InternalActorRef = {
props.routerConfig match {
case NoRouter new LocalActorRef(system, props, supervisor, path, systemService) // create a local actor
case router new RoutedActorRef(system, props.withRouting(adaptFromDeploy(router, path)), supervisor, path)
case router
val depl = deploy orElse {
val lookupPath = path.elements.drop(1).mkString("/", "/", "")
deployer.lookup(lookupPath)
}
new RoutedActorRef(system, props.withRouting(router.adaptFromDeploy(depl)), supervisor, path)
}
private def adaptFromDeploy(r: RouterConfig, p: ActorPath): RouterConfig = {
val lookupPath = p.elements.drop(1).mkString("/", "/", "")
r.adaptFromDeploy(deployer.lookup(lookupPath))
}
def ask(within: Timeout): Option[AskActorRef] = {

View file

@ -62,6 +62,12 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
*/
trait RouterConfig {
def nrOfInstances: Int
def targets: Iterable[String]
def createRoute(creator: () Actor, actorContext: ActorContext): Route
def createActor(): Router = new Router {}
def adaptFromDeploy(deploy: Option[Deploy]): RouterConfig = {
@ -72,8 +78,6 @@ trait RouterConfig {
}
}
def createRoute(creator: () Actor, actorContext: ActorContext): Route
protected def createRoutees(props: Props, context: ActorContext, nrOfInstances: Int, targets: Iterable[String]): Vector[ActorRef] = (nrOfInstances, targets) match {
case (0, Nil) throw new IllegalArgumentException("Insufficient information - missing configuration.")
case (x, Nil) (1 to x).map(_ context.actorOf(props))(scala.collection.breakOut)
@ -106,7 +110,9 @@ case class Broadcast(message: Any)
* Oxymoron style.
*/
case object NoRouter extends RouterConfig {
def createRoute(creator: () Actor, actorContext: ActorContext) = null
def nrOfInstances: Int = 0
def targets: Iterable[String] = Nil
def createRoute(creator: () Actor, actorContext: ActorContext): Route = null
}
object RoundRobinRouter {
@ -123,7 +129,7 @@ object RoundRobinRouter {
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class RoundRobinRouter(nrOfInstances: Int = 0, targets: Iterable[String] = Nil) extends RouterConfig {
case class RoundRobinRouter(nrOfInstances: Int = 0, targets: Iterable[String] = Nil) extends RouterConfig with RoundRobinLike {
/**
* Constructor that sets nrOfInstances to be created.
@ -140,7 +146,9 @@ case class RoundRobinRouter(nrOfInstances: Int = 0, targets: Iterable[String] =
def this(t: java.util.Collection[String]) = {
this(targets = collectionAsScalaIterable(t))
}
}
trait RoundRobinLike { this: RouterConfig
def createRoute(creator: () Actor, context: ActorContext): Route = {
val routees: Vector[ActorRef] =
createRoutees(context.props.copy(creator = creator, routerConfig = NoRouter), context, nrOfInstances, targets)
@ -175,7 +183,7 @@ object RandomRouter {
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class RandomRouter(nrOfInstances: Int = 0, targets: Iterable[String] = Nil) extends RouterConfig {
case class RandomRouter(nrOfInstances: Int = 0, targets: Iterable[String] = Nil) extends RouterConfig with RandomLike {
/**
* Constructor that sets nrOfInstances to be created.
@ -192,6 +200,9 @@ case class RandomRouter(nrOfInstances: Int = 0, targets: Iterable[String] = Nil)
def this(t: java.util.Collection[String]) = {
this(targets = collectionAsScalaIterable(t))
}
}
trait RandomLike { this: RouterConfig
import java.security.SecureRandom
@ -231,7 +242,7 @@ object BroadcastRouter {
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class BroadcastRouter(nrOfInstances: Int = 0, targets: Iterable[String] = Nil) extends RouterConfig {
case class BroadcastRouter(nrOfInstances: Int = 0, targets: Iterable[String] = Nil) extends RouterConfig with BroadcastLike {
/**
* Constructor that sets nrOfInstances to be created.
@ -248,7 +259,9 @@ case class BroadcastRouter(nrOfInstances: Int = 0, targets: Iterable[String] = N
def this(t: java.util.Collection[String]) = {
this(targets = collectionAsScalaIterable(t))
}
}
trait BroadcastLike { this: RouterConfig
def createRoute(creator: () Actor, context: ActorContext): Route = {
val routees: Vector[ActorRef] =
createRoutees(context.props.copy(creator = creator, routerConfig = NoRouter), context, nrOfInstances, targets)
@ -277,7 +290,8 @@ object ScatterGatherFirstCompletedRouter {
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, targets: Iterable[String] = Nil) extends RouterConfig {
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, targets: Iterable[String] = Nil)
extends RouterConfig with ScatterGatherFirstCompletedLike {
/**
* Constructor that sets nrOfInstances to be created.
@ -294,7 +308,9 @@ case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, targets: It
def this(t: java.util.Collection[String]) = {
this(targets = collectionAsScalaIterable(t))
}
}
trait ScatterGatherFirstCompletedLike { this: RouterConfig
def createRoute(creator: () Actor, context: ActorContext): Route = {
val routees: Vector[ActorRef] =
createRoutees(context.props.copy(creator = creator, routerConfig = NoRouter), context, nrOfInstances, targets)

View file

@ -57,7 +57,7 @@ class RemoteActorRefProvider(
terminationFuture.onComplete(_ remote.server.shutdown())
}
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean): InternalActorRef =
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean, deploy: Option[Deploy] = None): InternalActorRef =
if (systemService) local.actorOf(system, props, supervisor, path, systemService)
else {
@ -97,7 +97,7 @@ class RemoteActorRefProvider(
}
val elems = path.elements
val deployment = (elems.head match {
val deployment = deploy orElse (elems.head match {
case "user" deployer.lookup(elems.drop(1).mkString("/", "/", ""))
case "remote" lookupRemotes(elems)
case _ None

View file

@ -4,7 +4,7 @@
package akka.remote
import akka.actor._
import akka.event.EventStream
import akka.routing._
import com.typesafe.config._
import akka.config.ConfigurationException
@ -16,16 +16,26 @@ class RemoteDeployer(_settings: ActorSystem.Settings) extends Deployer(_settings
import scala.collection.JavaConverters._
import akka.util.ReflectiveAccess._
val deployment = config.withFallback(default)
val transform: Deploy Deploy =
if (deployment.hasPath("remote")) deployment.getString("remote") match {
case RemoteAddressExtractor(r) (d d.copy(scope = RemoteScope(r)))
case x identity
super.parseConfig(path, config) match {
case d @ Some(deploy)
deploy.config.getString("remote") match {
case RemoteAddressExtractor(r) Some(deploy.copy(scope = RemoteScope(r)))
case str
if (!str.isEmpty) throw new ConfigurationException("unparseable remote node name " + str)
val nodes = deploy.config.getStringList("target.nodes").asScala
if (nodes.isEmpty || deploy.routing == NoRouter) d
else {
val r = deploy.routing match {
case RoundRobinRouter(x, _) RemoteRoundRobinRouter(x, nodes)
case RandomRouter(x, _) RemoteRandomRouter(x, nodes)
case BroadcastRouter(x, _) RemoteBroadcastRouter(x, nodes)
case ScatterGatherFirstCompletedRouter(x, _) RemoteScatterGatherFirstCompletedRouter(x, nodes)
}
Some(deploy.copy(routing = r))
}
}
case None None
}
else identity
super.parseConfig(path, config) map transform
}
}

View file

@ -0,0 +1,110 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.routing
import akka.actor._
import akka.remote._
import scala.collection.JavaConverters._
import java.util.concurrent.atomic.AtomicInteger
import com.typesafe.config.ConfigFactory
import akka.config.ConfigurationException
trait RemoteRouterConfig extends RouterConfig {
override protected def createRoutees(props: Props, context: ActorContext, nrOfInstances: Int, targets: Iterable[String]): Vector[ActorRef] = (nrOfInstances, targets) match {
case (_, Nil) throw new ConfigurationException("must specify list of remote nodes")
case (n, xs)
val nodes = targets map {
case RemoteAddressExtractor(a) a
case x throw new ConfigurationException("unparseable remote node " + x)
}
val node = Stream.continually(nodes).flatten.iterator
val impl = context.system.asInstanceOf[ActorSystemImpl]
Vector.empty[ActorRef] ++ (for (i 1 to nrOfInstances) yield {
val name = "c" + i
val deploy = Deploy("", ConfigFactory.empty(), None, props.routerConfig, RemoteScope(node.next))
impl.provider.actorOf(impl, props, context.self.asInstanceOf[InternalActorRef], context.self.path / name, false, Some(deploy))
})
}
}
/**
* A Router that uses round-robin to select a connection. For concurrent calls, round robin is just a best effort.
* <br>
* Please note that providing both 'nrOfInstances' and 'targets' does not make logical sense as this means
* that the round robin should both create new actors and use the 'targets' actor(s).
* In this case the 'nrOfInstances' will be ignored and the 'targets' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class RemoteRoundRobinRouter(nrOfInstances: Int, targets: Iterable[String]) extends RemoteRouterConfig with RoundRobinLike {
/**
* Constructor that sets the targets to be used.
* Java API
*/
def this(n: Int, t: java.util.Collection[String]) = this(n, t.asScala)
}
/**
* A Router that randomly selects one of the target connections to send a message to.
* <br>
* Please note that providing both 'nrOfInstances' and 'targets' does not make logical sense as this means
* that the random router should both create new actors and use the 'targets' actor(s).
* In this case the 'nrOfInstances' will be ignored and the 'targets' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class RemoteRandomRouter(nrOfInstances: Int, targets: Iterable[String]) extends RemoteRouterConfig with RandomLike {
/**
* Constructor that sets the targets to be used.
* Java API
*/
def this(n: Int, t: java.util.Collection[String]) = this(n, t.asScala)
}
/**
* A Router that uses broadcasts a message to all its connections.
* <br>
* Please note that providing both 'nrOfInstances' and 'targets' does not make logical sense as this means
* that the random router should both create new actors and use the 'targets' actor(s).
* In this case the 'nrOfInstances' will be ignored and the 'targets' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class RemoteBroadcastRouter(nrOfInstances: Int, targets: Iterable[String]) extends RemoteRouterConfig with BroadcastLike {
/**
* Constructor that sets the targets to be used.
* Java API
*/
def this(n: Int, t: java.util.Collection[String]) = this(n, t.asScala)
}
/**
* Simple router that broadcasts the message to all routees, and replies with the first response.
* <br>
* Please note that providing both 'nrOfInstances' and 'targets' does not make logical sense as this means
* that the random router should both create new actors and use the 'targets' actor(s).
* In this case the 'nrOfInstances' will be ignored and the 'targets' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class RemoteScatterGatherFirstCompletedRouter(nrOfInstances: Int, targets: Iterable[String])
extends RemoteRouterConfig with ScatterGatherFirstCompletedLike {
/**
* Constructor that sets the targets to be used.
* Java API
*/
def this(n: Int, t: java.util.Collection[String]) = this(n, t.asScala)
}

View file

@ -0,0 +1,58 @@
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.remote
import akka.testkit._
import akka.routing._
import akka.actor._
import com.typesafe.config._
object RemoteRouterSpec {
class Echo extends Actor {
def receive = {
case _ sender ! self.path
}
}
}
class RemoteRouterSpec extends AkkaSpec("""
akka {
actor.provider = "akka.remote.RemoteActorRefProvider"
cluster.nodename = Nonsense
remote.server {
hostname = localhost
port = 12345
}
actor.deployment {
/blub {
router = "round-robin"
nr-of-instances = 2
target.nodes = ["akka://remote_sys@localhost:12346"]
}
}
}
""") with ImplicitSender {
import RemoteRouterSpec._
val conf = ConfigFactory.parseString("akka.remote.server.port=12346").withFallback(system.settings.config)
val other = ActorSystem("remote_sys", conf)
override def atTermination() {
other.stop()
}
"A Remote Router" must {
"deploy its children on remote host driven by configuration" in {
val router = system.actorOf(Props[Echo].withRouting(RoundRobinRouter(2)), "blub")
router ! ""
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c1"
router ! ""
expectMsgType[ActorPath].toString must be === "akka://remote_sys@localhost:12346/remote/RemoteRouterSpec@localhost:12345/user/blub/c2"
}
}
}