!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:
parent
81ca6fe8c8
commit
ebadd567b2
104 changed files with 9671 additions and 5006 deletions
|
|
@ -1,23 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing;
|
||||
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Props;
|
||||
import akka.testkit.ExtractRoute;
|
||||
|
||||
public class CustomRouteTest {
|
||||
|
||||
static private ActorSystem system;
|
||||
|
||||
// only to test compilability
|
||||
public void testRoute() {
|
||||
final ActorRef ref = system.actorOf(Props.empty().withRouter(new RoundRobinRouter(1)));
|
||||
final scala.Function1<scala.Tuple2<ActorRef, Object>,
|
||||
scala.collection.immutable.Iterable<Destination>> route = ExtractRoute.apply(ref);
|
||||
route.apply(null);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -44,27 +44,28 @@ class ActorConfigurationVerificationSpec extends AkkaSpec(ActorConfigurationVeri
|
|||
"An Actor configured with a BalancingDispatcher" must {
|
||||
"fail verification with a ConfigurationException if also configured with a RoundRobinRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(2).withDispatcher("balancing-dispatcher")))
|
||||
system.actorOf(RoundRobinRouter(2).withDispatcher("balancing-dispatcher").props(Props[TestActor]))
|
||||
}
|
||||
}
|
||||
"fail verification with a ConfigurationException if also configured with a BroadcastRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withRouter(BroadcastRouter(2).withDispatcher("balancing-dispatcher")))
|
||||
system.actorOf(BroadcastRouter(2).withDispatcher("balancing-dispatcher").props(Props[TestActor]))
|
||||
}
|
||||
}
|
||||
"fail verification with a ConfigurationException if also configured with a RandomRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withRouter(RandomRouter(2).withDispatcher("balancing-dispatcher")))
|
||||
system.actorOf(RandomRouter(2).withDispatcher("balancing-dispatcher").props(Props[TestActor]))
|
||||
}
|
||||
}
|
||||
"fail verification with a ConfigurationException if also configured with a SmallestMailboxRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withRouter(SmallestMailboxRouter(2).withDispatcher("balancing-dispatcher")))
|
||||
system.actorOf(SmallestMailboxRouter(2).withDispatcher("balancing-dispatcher").props(Props[TestActor]))
|
||||
}
|
||||
}
|
||||
"fail verification with a ConfigurationException if also configured with a ScatterGatherFirstCompletedRouter" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withRouter(ScatterGatherFirstCompletedRouter(nrOfInstances = 2, within = 2 seconds).withDispatcher("balancing-dispatcher")))
|
||||
system.actorOf(ScatterGatherFirstCompletedRouter(nrOfInstances = 2, within = 2 seconds).
|
||||
withDispatcher("balancing-dispatcher").props(Props[TestActor]))
|
||||
}
|
||||
}
|
||||
"not fail verification with a ConfigurationException also not configured with a Router" in {
|
||||
|
|
@ -73,7 +74,7 @@ class ActorConfigurationVerificationSpec extends AkkaSpec(ActorConfigurationVeri
|
|||
}
|
||||
"An Actor configured with a non-balancing dispatcher" must {
|
||||
"not fail verification with a ConfigurationException if also configured with a Router" in {
|
||||
system.actorOf(Props[TestActor].withDispatcher("pinned-dispatcher").withRouter(RoundRobinRouter(2)))
|
||||
system.actorOf(RoundRobinRouter(2).props(Props[TestActor].withDispatcher("pinned-dispatcher")))
|
||||
}
|
||||
|
||||
"fail verification if the dispatcher cannot be found" in {
|
||||
|
|
@ -84,13 +85,13 @@ class ActorConfigurationVerificationSpec extends AkkaSpec(ActorConfigurationVeri
|
|||
|
||||
"fail verification if the dispatcher cannot be found for the head of a router" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(1, routerDispatcher = "does not exist")))
|
||||
system.actorOf(RoundRobinRouter(1, routerDispatcher = "does not exist").props(Props[TestActor]))
|
||||
}
|
||||
}
|
||||
|
||||
"fail verification if the dispatcher cannot be found for the routees of a router" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withDispatcher("does not exist").withRouter(RoundRobinRouter(1)))
|
||||
system.actorOf(RoundRobinRouter(1).props(Props[TestActor].withDispatcher("does not exist")))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -162,37 +162,56 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
|
|||
}
|
||||
|
||||
"be able to parse 'akka.actor.deployment._' with round-robin router" in {
|
||||
assertRouting("/service-round-robin", RoundRobinRouter(1), "/service-round-robin")
|
||||
assertRouting("/service-round-robin", RoundRobinPool(1), "/service-round-robin")
|
||||
}
|
||||
|
||||
"be able to parse 'akka.actor.deployment._' with random router" in {
|
||||
assertRouting("/service-random", RandomRouter(1), "/service-random")
|
||||
assertRouting("/service-random", RandomPool(1), "/service-random")
|
||||
}
|
||||
|
||||
"be able to parse 'akka.actor.deployment._' with scatter-gather router" in {
|
||||
assertRouting("/service-scatter-gather", ScatterGatherFirstCompletedRouter(nrOfInstances = 1, within = 2 seconds), "/service-scatter-gather")
|
||||
assertRouting("/service-scatter-gather", ScatterGatherFirstCompletedPool(nrOfInstances = 1, within = 2 seconds), "/service-scatter-gather")
|
||||
}
|
||||
|
||||
"be able to parse 'akka.actor.deployment._' with consistent-hashing router" in {
|
||||
assertRouting("/service-consistent-hashing", ConsistentHashingRouter(1), "/service-consistent-hashing")
|
||||
assertRouting("/service-consistent-hashing", ConsistentHashingPool(1), "/service-consistent-hashing")
|
||||
}
|
||||
|
||||
"be able to parse 'akka.actor.deployment._' with router resizer" in {
|
||||
val resizer = DefaultResizer()
|
||||
assertRouting("/service-resizer", RoundRobinRouter(resizer = Some(resizer)), "/service-resizer")
|
||||
assertRouting("/service-resizer", RoundRobinPool(nrOfInstances = 0, resizer = Some(resizer)), "/service-resizer")
|
||||
}
|
||||
|
||||
"be able to use wildcards" in {
|
||||
assertRouting("/some/wildcardmatch", RandomRouter(1), "/some/*")
|
||||
assertRouting("/somewildcardmatch/some", ScatterGatherFirstCompletedRouter(nrOfInstances = 1, within = 2 seconds), "/*/some")
|
||||
assertRouting("/some/wildcardmatch", RandomPool(1), "/some/*")
|
||||
assertRouting("/somewildcardmatch/some", ScatterGatherFirstCompletedPool(nrOfInstances = 1, within = 2 seconds), "/*/some")
|
||||
}
|
||||
|
||||
"have correct router mappings" in {
|
||||
val mapping = system.asInstanceOf[ActorSystemImpl].provider.deployer.routerTypeMapping
|
||||
mapping("from-code") must be(classOf[akka.routing.NoRouter].getName)
|
||||
mapping("round-robin-pool") must be(classOf[akka.routing.RoundRobinPool].getName)
|
||||
mapping("round-robin-group") must be(classOf[akka.routing.RoundRobinGroup].getName)
|
||||
mapping("random-pool") must be(classOf[akka.routing.RandomPool].getName)
|
||||
mapping("random-group") must be(classOf[akka.routing.RandomGroup].getName)
|
||||
mapping("smallest-mailbox-pool") must be(classOf[akka.routing.SmallestMailboxPool].getName)
|
||||
mapping("broadcast-pool") must be(classOf[akka.routing.BroadcastPool].getName)
|
||||
mapping("broadcast-group") must be(classOf[akka.routing.BroadcastGroup].getName)
|
||||
mapping("scatter-gather-pool") must be(classOf[akka.routing.ScatterGatherFirstCompletedPool].getName)
|
||||
mapping("scatter-gather-group") must be(classOf[akka.routing.ScatterGatherFirstCompletedGroup].getName)
|
||||
mapping("consistent-hashing-pool") must be(classOf[akka.routing.ConsistentHashingPool].getName)
|
||||
mapping("consistent-hashing-group") must be(classOf[akka.routing.ConsistentHashingGroup].getName)
|
||||
}
|
||||
|
||||
def assertRouting(service: String, expected: RouterConfig, expectPath: String): Unit = {
|
||||
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1))
|
||||
deployment.map(_.path).getOrElse("NOT FOUND") must be(expectPath)
|
||||
deployment.get.routerConfig.getClass must be(expected.getClass)
|
||||
deployment.get.routerConfig.resizer must be(expected.resizer)
|
||||
deployment.get.scope must be(NoScopeGiven)
|
||||
expected match {
|
||||
case pool: Pool ⇒ deployment.get.routerConfig.asInstanceOf[Pool].resizer must be(pool.resizer)
|
||||
case _ ⇒
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,289 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import org.junit.runner.RunWith
|
||||
import akka.actor.{ Props, Deploy, Actor, ActorRef }
|
||||
import akka.ConfigurationException
|
||||
import scala.concurrent.Await
|
||||
import akka.pattern.{ ask, gracefulStop }
|
||||
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.UnstartedCell
|
||||
import akka.routing._
|
||||
|
||||
object ConfiguredLocalRoutingSpec {
|
||||
val config = """
|
||||
akka {
|
||||
actor {
|
||||
default-dispatcher {
|
||||
executor = "thread-pool-executor"
|
||||
thread-pool-executor {
|
||||
core-pool-size-min = 8
|
||||
core-pool-size-max = 16
|
||||
}
|
||||
}
|
||||
deployment {
|
||||
/config {
|
||||
router = random
|
||||
nr-of-instances = 4
|
||||
}
|
||||
/weird {
|
||||
router = round-robin
|
||||
nr-of-instances = 3
|
||||
}
|
||||
"/weird/*" {
|
||||
router = round-robin
|
||||
nr-of-instances = 2
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
"""
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.config) with DefaultTimeout with ImplicitSender {
|
||||
|
||||
def routerConfig(ref: ActorRef): RouterConfig = ref match {
|
||||
case r: RoutedActorRef ⇒
|
||||
r.underlying match {
|
||||
case c: RoutedActorCell ⇒ c.routerConfig
|
||||
case _: UnstartedCell ⇒ awaitCond(r.isStarted, 1 second, 10 millis); routerConfig(ref)
|
||||
}
|
||||
}
|
||||
|
||||
"RouterConfig" must {
|
||||
|
||||
"be picked up from Props" in {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "get" ⇒ sender ! context.props
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(12)), "someOther")
|
||||
routerConfig(actor) must be === RoundRobinRouter(12)
|
||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||
}
|
||||
|
||||
"be overridable in config" in {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "get" ⇒ sender ! context.props
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(12)), "config")
|
||||
routerConfig(actor) must be === RandomPool(4)
|
||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||
}
|
||||
|
||||
"be overridable in explicit deployment" in {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "get" ⇒ sender ! context.props
|
||||
}
|
||||
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "someOther")
|
||||
routerConfig(actor) must be === RoundRobinRouter(12)
|
||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||
}
|
||||
|
||||
"be overridable in config even with explicit deployment" in {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "get" ⇒ sender ! context.props
|
||||
}
|
||||
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "config")
|
||||
routerConfig(actor) must be === RandomPool(4)
|
||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||
}
|
||||
|
||||
"fail with an exception if not correct" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props.empty.withRouter(FromConfig))
|
||||
}
|
||||
}
|
||||
|
||||
"not get confused when trying to wildcard-configure children" in {
|
||||
val router = system.actorOf(Props(new Actor {
|
||||
testActor ! self
|
||||
def receive = { case _ ⇒ }
|
||||
}).withRouter(FromConfig), "weird")
|
||||
val recv = Set() ++ (for (_ ← 1 to 3) yield expectMsgType[ActorRef])
|
||||
val expc = Set('a', 'b', 'c') map (i ⇒ system.actorFor("/user/weird/$" + i))
|
||||
recv must be(expc)
|
||||
expectNoMsg(1 second)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
"round robin router" must {
|
||||
|
||||
"be able to shut down its instance" in {
|
||||
val helloLatch = new TestLatch(5)
|
||||
val stopLatch = new TestLatch(5)
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(5)), "round-robin-shutdown")
|
||||
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
Await.ready(helloLatch, 5 seconds)
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
|
||||
"deliver messages in a round robin fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 10
|
||||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
val counter = new AtomicInteger
|
||||
var replies = Map.empty[Int, Int]
|
||||
for (i ← 0 until connectionCount) {
|
||||
replies += i -> 0
|
||||
}
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
lazy val id = counter.getAndIncrement()
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! id
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(connectionCount)), "round-robin")
|
||||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
|
||||
replies = replies + (id -> (replies(id) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
counter.get must be(connectionCount)
|
||||
|
||||
actor ! Broadcast("end")
|
||||
Await.ready(doneLatch, 5 seconds)
|
||||
|
||||
replies.values foreach { _ must be(iterationCount) }
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val helloLatch = new TestLatch(5)
|
||||
val stopLatch = new TestLatch(5)
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(5)), "round-robin-broadcast")
|
||||
|
||||
actor ! Broadcast("hello")
|
||||
Await.ready(helloLatch, 5 seconds)
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
}
|
||||
|
||||
"random router" must {
|
||||
|
||||
"be able to shut down its instance" in {
|
||||
val stopLatch = new TestLatch(7)
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ sender ! "world"
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
}).withRouter(RandomRouter(7)), "random-shutdown")
|
||||
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
|
||||
within(2 seconds) {
|
||||
for (i ← 1 to 5) expectMsg("world")
|
||||
}
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
|
||||
"deliver messages in a random fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 100
|
||||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
val counter = new AtomicInteger
|
||||
var replies = Map.empty[Int, Int]
|
||||
for (i ← 0 until connectionCount) {
|
||||
replies = replies + (i -> 0)
|
||||
}
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
lazy val id = counter.getAndIncrement()
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! id
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
}
|
||||
}).withRouter(RandomRouter(connectionCount)), "random")
|
||||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
|
||||
replies = replies + (id -> (replies(id) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
counter.get must be(connectionCount)
|
||||
|
||||
actor ! Broadcast("end")
|
||||
Await.ready(doneLatch, 5 seconds)
|
||||
|
||||
replies.values foreach { _ must be > (0) }
|
||||
replies.values.sum must be === iterationCount * connectionCount
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val helloLatch = new TestLatch(6)
|
||||
val stopLatch = new TestLatch(6)
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
}).withRouter(RandomRouter(6)), "random-broadcast")
|
||||
|
||||
actor ! Broadcast("hello")
|
||||
Await.ready(helloLatch, 5 seconds)
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,105 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.oldrouting
|
||||
|
||||
import scala.concurrent.Await
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Props
|
||||
import akka.actor.actorRef2Scala
|
||||
import akka.pattern.ask
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashable
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.testkit._
|
||||
import akka.routing._
|
||||
|
||||
object ConsistentHashingRouterSpec {
|
||||
|
||||
val config = """
|
||||
akka.actor.deployment {
|
||||
/router1 {
|
||||
router = consistent-hashing
|
||||
nr-of-instances = 3
|
||||
virtual-nodes-factor = 17
|
||||
}
|
||||
/router2 {
|
||||
router = consistent-hashing
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
"""
|
||||
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
case class Msg(key: Any, data: String) extends ConsistentHashable {
|
||||
override def consistentHashKey = key
|
||||
}
|
||||
|
||||
case class MsgKey(name: String)
|
||||
|
||||
case class Msg2(key: Any, data: String)
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.config) with DefaultTimeout with ImplicitSender {
|
||||
import akka.routing.ConsistentHashingRouterSpec._
|
||||
implicit val ec = system.dispatcher
|
||||
|
||||
val router1 = system.actorOf(Props[Echo].withRouter(FromConfig()), "router1")
|
||||
|
||||
"consistent hashing router" must {
|
||||
"create routees from configuration" in {
|
||||
val currentRoutees = Await.result(router1 ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees]
|
||||
currentRoutees.routees.size must be(3)
|
||||
}
|
||||
|
||||
"select destination based on consistentHashKey of the message" in {
|
||||
router1 ! Msg("a", "A")
|
||||
val destinationA = expectMsgType[ActorRef]
|
||||
router1 ! ConsistentHashableEnvelope(message = "AA", hashKey = "a")
|
||||
expectMsg(destinationA)
|
||||
|
||||
router1 ! Msg(17, "B")
|
||||
val destinationB = expectMsgType[ActorRef]
|
||||
router1 ! ConsistentHashableEnvelope(message = "BB", hashKey = 17)
|
||||
expectMsg(destinationB)
|
||||
|
||||
router1 ! Msg(MsgKey("c"), "C")
|
||||
val destinationC = expectMsgType[ActorRef]
|
||||
router1 ! ConsistentHashableEnvelope(message = "CC", hashKey = MsgKey("c"))
|
||||
expectMsg(destinationC)
|
||||
}
|
||||
|
||||
"select destination with defined consistentHashRoute" in {
|
||||
def hashMapping: ConsistentHashMapping = {
|
||||
case Msg2(key, data) ⇒ key
|
||||
}
|
||||
val router2 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter(
|
||||
hashMapping = hashMapping)), "router2")
|
||||
|
||||
router2 ! Msg2("a", "A")
|
||||
val destinationA = expectMsgType[ActorRef]
|
||||
router2 ! ConsistentHashableEnvelope(message = "AA", hashKey = "a")
|
||||
expectMsg(destinationA)
|
||||
|
||||
router2 ! Msg2(17, "B")
|
||||
val destinationB = expectMsgType[ActorRef]
|
||||
router2 ! ConsistentHashableEnvelope(message = "BB", hashKey = 17)
|
||||
expectMsg(destinationB)
|
||||
|
||||
router2 ! Msg2(MsgKey("c"), "C")
|
||||
val destinationC = expectMsgType[ActorRef]
|
||||
router2 ! ConsistentHashableEnvelope(message = "CC", hashKey = MsgKey("c"))
|
||||
expectMsg(destinationC)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,221 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
import akka.actor.Actor
|
||||
import akka.testkit._
|
||||
import akka.testkit.TestEvent._
|
||||
import akka.actor.Props
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorRef
|
||||
import akka.pattern.ask
|
||||
import scala.util.Try
|
||||
import akka.routing._
|
||||
|
||||
object ResizerSpec {
|
||||
|
||||
val config = """
|
||||
akka.actor.serialize-messages = off
|
||||
akka.actor.deployment {
|
||||
/router1 {
|
||||
router = round-robin
|
||||
resizer {
|
||||
lower-bound = 2
|
||||
upper-bound = 3
|
||||
}
|
||||
}
|
||||
}
|
||||
bal-disp {
|
||||
type = BalancingDispatcher
|
||||
}
|
||||
"""
|
||||
|
||||
class TestActor extends Actor {
|
||||
def receive = {
|
||||
case latch: TestLatch ⇒ latch.countDown()
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with ImplicitSender {
|
||||
|
||||
import akka.routing.ResizerSpec._
|
||||
|
||||
override def atStartup: Unit = {
|
||||
// when shutting down some Resize messages might hang around
|
||||
system.eventStream.publish(Mute(EventFilter.warning(pattern = ".*Resize")))
|
||||
}
|
||||
|
||||
def routeeSize(router: ActorRef): Int =
|
||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size
|
||||
|
||||
"DefaultResizer" must {
|
||||
|
||||
"use settings to evaluate capacity" in {
|
||||
val resizer = DefaultResizer(
|
||||
lowerBound = 2,
|
||||
upperBound = 3)
|
||||
|
||||
val c1 = resizer.capacity(Vector.empty[Routee])
|
||||
c1 must be(2)
|
||||
|
||||
val current = Vector(
|
||||
ActorRefRoutee(system.actorOf(Props[TestActor])),
|
||||
ActorRefRoutee(system.actorOf(Props[TestActor])))
|
||||
val c2 = resizer.capacity(current)
|
||||
c2 must be(0)
|
||||
}
|
||||
|
||||
"use settings to evaluate rampUp" in {
|
||||
val resizer = DefaultResizer(
|
||||
lowerBound = 2,
|
||||
upperBound = 10,
|
||||
rampupRate = 0.2)
|
||||
|
||||
resizer.rampup(pressure = 9, capacity = 10) must be(0)
|
||||
resizer.rampup(pressure = 5, capacity = 5) must be(1)
|
||||
resizer.rampup(pressure = 6, capacity = 6) must be(2)
|
||||
}
|
||||
|
||||
"use settings to evaluate backoff" in {
|
||||
val resizer = DefaultResizer(
|
||||
lowerBound = 2,
|
||||
upperBound = 10,
|
||||
backoffThreshold = 0.3,
|
||||
backoffRate = 0.1)
|
||||
|
||||
resizer.backoff(pressure = 10, capacity = 10) must be(0)
|
||||
resizer.backoff(pressure = 4, capacity = 10) must be(0)
|
||||
resizer.backoff(pressure = 3, capacity = 10) must be(0)
|
||||
resizer.backoff(pressure = 2, capacity = 10) must be(-1)
|
||||
resizer.backoff(pressure = 0, capacity = 10) must be(-1)
|
||||
resizer.backoff(pressure = 1, capacity = 9) must be(-1)
|
||||
resizer.backoff(pressure = 0, capacity = 9) must be(-1)
|
||||
}
|
||||
|
||||
"be possible to define programmatically" in {
|
||||
val latch = new TestLatch(3)
|
||||
|
||||
val resizer = DefaultResizer(
|
||||
lowerBound = 2,
|
||||
upperBound = 3)
|
||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
||||
|
||||
router ! latch
|
||||
router ! latch
|
||||
router ! latch
|
||||
|
||||
Await.ready(latch, remaining)
|
||||
|
||||
// messagesPerResize is 10 so there is no risk of additional resize
|
||||
routeeSize(router) must be(2)
|
||||
}
|
||||
|
||||
"be possible to define in configuration" in {
|
||||
val latch = new TestLatch(3)
|
||||
|
||||
val router = system.actorOf(Props[TestActor].withRouter(FromConfig()), "router1")
|
||||
|
||||
router ! latch
|
||||
router ! latch
|
||||
router ! latch
|
||||
|
||||
Await.ready(latch, remaining)
|
||||
|
||||
routeeSize(router) must be(2)
|
||||
}
|
||||
|
||||
"grow as needed under pressure" in {
|
||||
// make sure the pool starts at the expected lower limit and grows to the upper as needed
|
||||
// as influenced by the backlog of blocking pooled actors
|
||||
|
||||
val resizer = DefaultResizer(
|
||||
lowerBound = 3,
|
||||
upperBound = 5,
|
||||
rampupRate = 0.1,
|
||||
backoffRate = 0.0,
|
||||
pressureThreshold = 1,
|
||||
messagesPerResize = 1,
|
||||
backoffThreshold = 0.0)
|
||||
|
||||
val router = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case d: FiniteDuration ⇒
|
||||
Thread.sleep(d.dilated.toMillis); sender ! "done"
|
||||
case "echo" ⇒ sender ! "reply"
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
||||
|
||||
// first message should create the minimum number of routees
|
||||
router ! "echo"
|
||||
expectMsg("reply")
|
||||
|
||||
routeeSize(router) must be(resizer.lowerBound)
|
||||
|
||||
def loop(loops: Int, d: FiniteDuration) = {
|
||||
for (m ← 0 until loops) {
|
||||
router ! d
|
||||
// sending in too quickly will result in skipped resize due to many resizeInProgress conflicts
|
||||
Thread.sleep(20.millis.dilated.toMillis)
|
||||
}
|
||||
within((d * loops / resizer.lowerBound) + 2.seconds.dilated) {
|
||||
for (m ← 0 until loops) expectMsg("done")
|
||||
}
|
||||
}
|
||||
|
||||
// 2 more should go thru without triggering more
|
||||
loop(2, 200 millis)
|
||||
routeeSize(router) must be(resizer.lowerBound)
|
||||
|
||||
// a whole bunch should max it out
|
||||
loop(20, 500 millis)
|
||||
routeeSize(router) must be(resizer.upperBound)
|
||||
}
|
||||
|
||||
"backoff" in within(10 seconds) {
|
||||
|
||||
val resizer = DefaultResizer(
|
||||
lowerBound = 2,
|
||||
upperBound = 5,
|
||||
rampupRate = 1.0,
|
||||
backoffRate = 1.0,
|
||||
backoffThreshold = 0.40,
|
||||
pressureThreshold = 1,
|
||||
messagesPerResize = 2)
|
||||
|
||||
val router = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case n: Int if n <= 0 ⇒ // done
|
||||
case n: Int ⇒ Thread.sleep((n millis).dilated.toMillis)
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
||||
|
||||
// put some pressure on the router
|
||||
for (m ← 0 until 15) {
|
||||
router ! 150
|
||||
Thread.sleep((20 millis).dilated.toMillis)
|
||||
}
|
||||
|
||||
val z = routeeSize(router)
|
||||
z must be > (2)
|
||||
|
||||
Thread.sleep((300 millis).dilated.toMillis)
|
||||
|
||||
// let it cool down
|
||||
awaitCond({
|
||||
router ! 0 // trigger resize
|
||||
Thread.sleep((20 millis).dilated.toMillis)
|
||||
routeeSize(router) < z
|
||||
}, interval = 500.millis.dilated)
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,52 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.oldrouting
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.Props
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.LocalActorRef
|
||||
import scala.concurrent.duration._
|
||||
import akka.routing._
|
||||
|
||||
class RouteeCreationSpec extends AkkaSpec {
|
||||
|
||||
"Creating Routees" must {
|
||||
|
||||
"result in visible routees" in {
|
||||
val N = 100
|
||||
system.actorOf(Props(new Actor {
|
||||
testActor ! system.actorFor(self.path)
|
||||
def receive = Actor.emptyBehavior
|
||||
}).withRouter(RoundRobinRouter(N)))
|
||||
for (i ← 1 to N) {
|
||||
expectMsgType[ActorRef] match {
|
||||
case _: LocalActorRef ⇒ // fine
|
||||
case x ⇒ fail(s"routee $i was a ${x.getClass}")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"allow sending to context.parent" in {
|
||||
val N = 100
|
||||
system.actorOf(Props(new Actor {
|
||||
context.parent ! "one"
|
||||
def receive = {
|
||||
case "one" ⇒ testActor forward "two"
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(N)))
|
||||
val gotit = receiveWhile(messages = N) {
|
||||
case "two" ⇒ lastSender.toString
|
||||
}
|
||||
expectNoMsg(100.millis)
|
||||
if (gotit.size != N) {
|
||||
fail(s"got only ${gotit.size} from \n${gotit mkString "\n"}")
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,586 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import akka.actor._
|
||||
import scala.collection.immutable
|
||||
import akka.testkit._
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.Await
|
||||
import akka.ConfigurationException
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.pattern.{ ask, pipe }
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import com.typesafe.config.Config
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.util.Collections.EmptyImmutableSeq
|
||||
import akka.util.Timeout
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.routing._
|
||||
|
||||
object RoutingSpec {
|
||||
|
||||
val config = """
|
||||
akka.actor.serialize-messages = off
|
||||
akka.actor.deployment {
|
||||
/router1 {
|
||||
router = round-robin
|
||||
nr-of-instances = 3
|
||||
}
|
||||
/router2 {
|
||||
router = round-robin
|
||||
nr-of-instances = 3
|
||||
}
|
||||
/router3 {
|
||||
router = round-robin
|
||||
nr-of-instances = 0
|
||||
}
|
||||
}
|
||||
"""
|
||||
|
||||
class TestActor extends Actor {
|
||||
def receive = { case _ ⇒ }
|
||||
}
|
||||
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with ImplicitSender {
|
||||
implicit val ec = system.dispatcher
|
||||
import RoutingSpec._
|
||||
|
||||
muteDeadLetters(classOf[akka.dispatch.sysmsg.DeathWatchNotification])()
|
||||
|
||||
"routers in general" must {
|
||||
|
||||
"evict terminated routees" in {
|
||||
val router = system.actorOf(Props[Echo].withRouter(RoundRobinRouter(2)))
|
||||
router ! ""
|
||||
router ! ""
|
||||
val c1, c2 = expectMsgType[ActorRef]
|
||||
watch(router)
|
||||
watch(c2)
|
||||
system.stop(c2)
|
||||
expectTerminated(c2).existenceConfirmed must be === true
|
||||
// it might take a while until the Router has actually processed the Terminated message
|
||||
awaitCond {
|
||||
router ! ""
|
||||
router ! ""
|
||||
val res = receiveWhile(100 millis, messages = 2) {
|
||||
case x: ActorRef ⇒ x
|
||||
}
|
||||
res == Seq(c1, c1)
|
||||
}
|
||||
system.stop(c1)
|
||||
expectTerminated(router).existenceConfirmed must be === true
|
||||
}
|
||||
|
||||
"not terminate when resizer is used" in {
|
||||
val latch = TestLatch(1)
|
||||
val resizer = new Resizer {
|
||||
def isTimeForResize(messageCounter: Long): Boolean = messageCounter == 0
|
||||
def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int = {
|
||||
latch.countDown()
|
||||
2
|
||||
}
|
||||
}
|
||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
||||
watch(router)
|
||||
Await.ready(latch, remaining)
|
||||
router ! CurrentRoutees
|
||||
val routees = expectMsgType[RouterRoutees].routees
|
||||
routees.size must be(2)
|
||||
routees foreach system.stop
|
||||
// expect no Terminated
|
||||
expectNoMsg(2.seconds)
|
||||
}
|
||||
|
||||
"be able to send their routees" in {
|
||||
case class TestRun(id: String, names: immutable.Iterable[String], actors: Int)
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case TestRun(id, names, actors) ⇒
|
||||
val routerProps = Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(
|
||||
routees = names map { context.actorOf(Props(new TestActor), _) },
|
||||
within = 5 seconds))
|
||||
|
||||
1 to actors foreach { i ⇒ context.actorOf(routerProps, id + i).tell(CurrentRoutees, testActor) }
|
||||
}
|
||||
}))
|
||||
|
||||
val actors = 15
|
||||
val names = 1 to 20 map { "routee" + _ } toSet
|
||||
|
||||
actor ! TestRun("test", names, actors)
|
||||
|
||||
1 to actors foreach { _ ⇒
|
||||
val routees = expectMsgType[RouterRoutees].routees
|
||||
routees.map(_.path.name).toSet must be === names
|
||||
}
|
||||
expectNoMsg(500.millis)
|
||||
}
|
||||
|
||||
"use configured nr-of-instances when FromConfig" in {
|
||||
val router = system.actorOf(Props[TestActor].withRouter(FromConfig), "router1")
|
||||
router ! CurrentRoutees
|
||||
expectMsgType[RouterRoutees].routees.size must be(3)
|
||||
watch(router)
|
||||
system.stop(router)
|
||||
expectTerminated(router)
|
||||
}
|
||||
|
||||
"use configured nr-of-instances when router is specified" in {
|
||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router2")
|
||||
router ! CurrentRoutees
|
||||
expectMsgType[RouterRoutees].routees.size must be(3)
|
||||
system.stop(router)
|
||||
}
|
||||
|
||||
"use specified resizer when resizer not configured" in {
|
||||
val latch = TestLatch(1)
|
||||
val resizer = new Resizer {
|
||||
def isTimeForResize(messageCounter: Long): Boolean = messageCounter == 0
|
||||
def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int = {
|
||||
latch.countDown()
|
||||
3
|
||||
}
|
||||
}
|
||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))), "router3")
|
||||
Await.ready(latch, remaining)
|
||||
router ! CurrentRoutees
|
||||
expectMsgType[RouterRoutees].routees.size must be(3)
|
||||
system.stop(router)
|
||||
}
|
||||
|
||||
"set supplied supervisorStrategy" in {
|
||||
//#supervision
|
||||
val escalator = OneForOneStrategy() {
|
||||
//#custom-strategy
|
||||
case e ⇒ testActor ! e; SupervisorStrategy.Escalate
|
||||
//#custom-strategy
|
||||
}
|
||||
val router = system.actorOf(Props.empty.withRouter(
|
||||
RoundRobinRouter(1, supervisorStrategy = escalator)))
|
||||
//#supervision
|
||||
router ! CurrentRoutees
|
||||
EventFilter[ActorKilledException](occurrences = 1) intercept {
|
||||
expectMsgType[RouterRoutees].routees.head ! Kill
|
||||
}
|
||||
expectMsgType[ActorKilledException]
|
||||
|
||||
val router2 = system.actorOf(Props.empty.withRouter(RoundRobinRouter(1).withSupervisorStrategy(escalator)))
|
||||
router2 ! CurrentRoutees
|
||||
EventFilter[ActorKilledException](occurrences = 1) intercept {
|
||||
expectMsgType[RouterRoutees].routees.head ! Kill
|
||||
}
|
||||
expectMsgType[ActorKilledException]
|
||||
}
|
||||
|
||||
"set supplied supervisorStrategy for FromConfig" in {
|
||||
val escalator = OneForOneStrategy() {
|
||||
case e ⇒ testActor ! e; SupervisorStrategy.Escalate
|
||||
}
|
||||
val router = system.actorOf(Props.empty.withRouter(FromConfig.withSupervisorStrategy(escalator)), "router1")
|
||||
router ! CurrentRoutees
|
||||
EventFilter[ActorKilledException](occurrences = 1) intercept {
|
||||
expectMsgType[RouterRoutees].routees.head ! Kill
|
||||
}
|
||||
expectMsgType[ActorKilledException]
|
||||
}
|
||||
|
||||
"default to all-for-one-always-escalate strategy" in {
|
||||
val restarter = OneForOneStrategy() {
|
||||
case e ⇒ testActor ! e; SupervisorStrategy.Restart
|
||||
}
|
||||
val supervisor = system.actorOf(Props(new Supervisor(restarter)))
|
||||
supervisor ! Props(new Actor {
|
||||
def receive = {
|
||||
case x: String ⇒ throw new Exception(x)
|
||||
}
|
||||
override def postRestart(reason: Throwable): Unit = testActor ! "restarted"
|
||||
}).withRouter(RoundRobinRouter(3))
|
||||
val router = expectMsgType[ActorRef]
|
||||
EventFilter[Exception]("die", occurrences = 1) intercept {
|
||||
router ! "die"
|
||||
}
|
||||
expectMsgType[Exception].getMessage must be("die")
|
||||
expectMsg("restarted")
|
||||
expectMsg("restarted")
|
||||
expectMsg("restarted")
|
||||
}
|
||||
|
||||
"start in-line for context.actorOf()" in {
|
||||
system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "start" ⇒
|
||||
context.actorOf(Props(new Actor {
|
||||
def receive = { case x ⇒ sender ! x }
|
||||
}).withRouter(RoundRobinRouter(2))) ? "hello" pipeTo sender
|
||||
}
|
||||
})) ! "start"
|
||||
expectMsg("hello")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
"no router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(NoRouter))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"send message to connection" in {
|
||||
class Actor1 extends Actor {
|
||||
def receive = {
|
||||
case msg ⇒ testActor forward msg
|
||||
}
|
||||
}
|
||||
|
||||
val routedActor = system.actorOf(Props(new Actor1).withRouter(NoRouter))
|
||||
routedActor ! "hello"
|
||||
routedActor ! "end"
|
||||
|
||||
expectMsg("hello")
|
||||
expectMsg("end")
|
||||
}
|
||||
}
|
||||
|
||||
"round robin router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 1)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
//In this test a bunch of actors are created and each actor has its own counter.
|
||||
//to test round robin, the routed actor receives the following sequence of messages 1 2 3 .. 1 2 3 .. 1 2 3 which it
|
||||
//uses to increment his counter.
|
||||
//So after n iteration, the first actor his counter should be 1*n, the second 2*n etc etc.
|
||||
"deliver messages in a round robin fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 10
|
||||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
//lets create some connections.
|
||||
@volatile var actors = immutable.IndexedSeq[ActorRef]()
|
||||
@volatile var counters = immutable.IndexedSeq[AtomicInteger]()
|
||||
for (i ← 0 until connectionCount) {
|
||||
counters = counters :+ new AtomicInteger()
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counters(i).addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
actors = actors :+ actor
|
||||
}
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(routees = actors)))
|
||||
|
||||
//send messages to the actor.
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
routedActor ! (k + 1)
|
||||
}
|
||||
}
|
||||
|
||||
routedActor ! Broadcast("end")
|
||||
//now wait some and do validations.
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
for (i ← 0 until connectionCount)
|
||||
counters(i).get must be((iterationCount * (i + 1)))
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(routees = List(actor1, actor2))))
|
||||
|
||||
routedActor ! Broadcast(1)
|
||||
routedActor ! Broadcast("end")
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
}
|
||||
|
||||
"random router" must {
|
||||
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RandomRouter(nrOfInstances = 1)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"deliver a broadcast message" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RandomRouter(routees = List(actor1, actor2))))
|
||||
|
||||
routedActor ! Broadcast(1)
|
||||
routedActor ! Broadcast("end")
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
}
|
||||
|
||||
"smallest mailbox router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(SmallestMailboxRouter(nrOfInstances = 1)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"deliver messages to idle actor" in {
|
||||
val usedActors = new ConcurrentHashMap[Int, String]()
|
||||
val router = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case (busy: TestLatch, receivedLatch: TestLatch) ⇒
|
||||
usedActors.put(0, self.path.toString)
|
||||
self ! "another in busy mailbox"
|
||||
receivedLatch.countDown()
|
||||
Await.ready(busy, TestLatch.DefaultTimeout)
|
||||
case (msg: Int, receivedLatch: TestLatch) ⇒
|
||||
usedActors.put(msg, self.path.toString)
|
||||
receivedLatch.countDown()
|
||||
case s: String ⇒
|
||||
}
|
||||
}).withRouter(SmallestMailboxRouter(3)))
|
||||
|
||||
val busy = TestLatch(1)
|
||||
val received0 = TestLatch(1)
|
||||
router ! ((busy, received0))
|
||||
Await.ready(received0, TestLatch.DefaultTimeout)
|
||||
|
||||
val received1 = TestLatch(1)
|
||||
router ! ((1, received1))
|
||||
Await.ready(received1, TestLatch.DefaultTimeout)
|
||||
|
||||
val received2 = TestLatch(1)
|
||||
router ! ((2, received2))
|
||||
Await.ready(received2, TestLatch.DefaultTimeout)
|
||||
|
||||
val received3 = TestLatch(1)
|
||||
router ! ((3, received3))
|
||||
Await.ready(received3, TestLatch.DefaultTimeout)
|
||||
|
||||
busy.countDown()
|
||||
|
||||
val busyPath = usedActors.get(0)
|
||||
busyPath must not be (null)
|
||||
|
||||
val path1 = usedActors.get(1)
|
||||
val path2 = usedActors.get(2)
|
||||
val path3 = usedActors.get(3)
|
||||
|
||||
path1 must not be (busyPath)
|
||||
path2 must not be (busyPath)
|
||||
path3 must not be (busyPath)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
"broadcast router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(BroadcastRouter(nrOfInstances = 1)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"broadcast message using !" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(BroadcastRouter(routees = List(actor1, actor2))))
|
||||
routedActor ! 1
|
||||
routedActor ! "end"
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
|
||||
"broadcast message using ?" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒
|
||||
counter1.addAndGet(msg)
|
||||
sender ! "ack"
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(BroadcastRouter(routees = List(actor1, actor2))))
|
||||
routedActor ? 1
|
||||
routedActor ! "end"
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
}
|
||||
|
||||
"Scatter-gather router" must {
|
||||
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(routees = List(newActor(0)), within = 1 seconds)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(routees = List(actor1, actor2), within = 1 seconds)))
|
||||
routedActor ! Broadcast(1)
|
||||
routedActor ! Broadcast("end")
|
||||
|
||||
Await.ready(doneLatch, TestLatch.DefaultTimeout)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
|
||||
"return response, even if one of the actors has stopped" in {
|
||||
val shutdownLatch = new TestLatch(1)
|
||||
val actor1 = newActor(1, Some(shutdownLatch))
|
||||
val actor2 = newActor(14, Some(shutdownLatch))
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(routees = List(actor1, actor2), within = 3 seconds)))
|
||||
|
||||
routedActor ! Broadcast(Stop(Some(1)))
|
||||
Await.ready(shutdownLatch, TestLatch.DefaultTimeout)
|
||||
Await.result(routedActor ? Broadcast(0), timeout.duration) must be(14)
|
||||
}
|
||||
|
||||
case class Stop(id: Option[Int] = None)
|
||||
|
||||
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case Stop(None) ⇒ context.stop(self)
|
||||
case Stop(Some(_id)) if (_id == id) ⇒ context.stop(self)
|
||||
case _id: Int if (_id == id) ⇒
|
||||
case x ⇒ {
|
||||
Thread sleep 100 * id
|
||||
sender ! id
|
||||
}
|
||||
}
|
||||
|
||||
override def postStop = {
|
||||
shudownLatch foreach (_.countDown())
|
||||
}
|
||||
}), "Actor:" + id)
|
||||
}
|
||||
|
||||
"router FromConfig" must {
|
||||
"throw suitable exception when not configured" in {
|
||||
val e = intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withRouter(FromConfig), "routerNotDefined")
|
||||
}
|
||||
e.getMessage must include("routerNotDefined")
|
||||
}
|
||||
|
||||
"allow external configuration" in {
|
||||
val sys = ActorSystem("FromConfig", ConfigFactory
|
||||
.parseString("akka.actor.deployment./routed.router=round-robin")
|
||||
.withFallback(system.settings.config))
|
||||
try {
|
||||
sys.actorOf(Props.empty.withRouter(FromConfig), "routed")
|
||||
} finally {
|
||||
shutdown(sys)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,88 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.{ Props, Actor }
|
||||
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
|
||||
import akka.pattern.ask
|
||||
|
||||
object BroadcastSpec {
|
||||
class TestActor extends Actor {
|
||||
def receive = { case _ ⇒ }
|
||||
}
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class BroadcastSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||
import BroadcastSpec._
|
||||
|
||||
"broadcast group" must {
|
||||
|
||||
"broadcast message using !" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val paths = List(actor1, actor2).map(_.path.toString)
|
||||
val routedActor = system.actorOf(BroadcastGroup(paths).props())
|
||||
routedActor ! 1
|
||||
routedActor ! "end"
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
|
||||
"broadcast message using ?" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒
|
||||
counter1.addAndGet(msg)
|
||||
sender ! "ack"
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val paths = List(actor1, actor2).map(_.path.toString)
|
||||
val routedActor = system.actorOf(BroadcastGroup(paths).props())
|
||||
routedActor ? 1
|
||||
routedActor ! "end"
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -3,17 +3,16 @@
|
|||
*/
|
||||
package akka.routing
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import org.junit.runner.RunWith
|
||||
import akka.actor.{ Props, Deploy, Actor, ActorRef }
|
||||
import akka.ConfigurationException
|
||||
import scala.concurrent.Await
|
||||
import akka.pattern.{ ask, gracefulStop }
|
||||
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import akka.ConfigurationException
|
||||
import akka.actor.{ Props, Deploy, Actor, ActorRef }
|
||||
import akka.actor.UnstartedCell
|
||||
import akka.testkit.{ ImplicitSender, DefaultTimeout, AkkaSpec }
|
||||
import akka.pattern.gracefulStop
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
object ConfiguredLocalRoutingSpec {
|
||||
val config = """
|
||||
|
|
@ -28,27 +27,62 @@ object ConfiguredLocalRoutingSpec {
|
|||
}
|
||||
deployment {
|
||||
/config {
|
||||
router = random
|
||||
router = random-pool
|
||||
nr-of-instances = 4
|
||||
}
|
||||
/paths {
|
||||
router = random-group
|
||||
routees.paths = ["/user/service1", "/user/service2"]
|
||||
}
|
||||
/weird {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 3
|
||||
}
|
||||
"/weird/*" {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 2
|
||||
}
|
||||
/myrouter {
|
||||
router = "akka.routing.ConfiguredLocalRoutingSpec$MyRouter"
|
||||
foo = bar
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
"""
|
||||
|
||||
class MyRouter(config: Config) extends CustomRouterConfig {
|
||||
override def createRouter(system: ActorSystem): Router = Router(MyRoutingLogic(config))
|
||||
}
|
||||
|
||||
case class MyRoutingLogic(config: Config) extends RoutingLogic {
|
||||
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
|
||||
MyRoutee(config.getString(message.toString))
|
||||
}
|
||||
|
||||
case class MyRoutee(reply: String) extends Routee {
|
||||
override def send(message: Any, sender: ActorRef): Unit =
|
||||
sender ! reply
|
||||
}
|
||||
|
||||
class EchoProps extends Actor {
|
||||
def receive = {
|
||||
case "get" ⇒ sender ! context.props
|
||||
}
|
||||
}
|
||||
|
||||
class SendRefAtStartup(testActor: ActorRef) extends Actor {
|
||||
testActor ! self
|
||||
def receive = { case _ ⇒ }
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.config) with DefaultTimeout with ImplicitSender {
|
||||
import ConfiguredLocalRoutingSpec._
|
||||
|
||||
def routerConfig(ref: ActorRef): RouterConfig = ref match {
|
||||
def routerConfig(ref: ActorRef): akka.routing.RouterConfig = ref match {
|
||||
case r: RoutedActorRef ⇒
|
||||
r.underlying match {
|
||||
case c: RoutedActorCell ⇒ c.routerConfig
|
||||
|
|
@ -59,230 +93,57 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con
|
|||
"RouterConfig" must {
|
||||
|
||||
"be picked up from Props" in {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "get" ⇒ sender ! context.props
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(12)), "someOther")
|
||||
routerConfig(actor) must be === RoundRobinRouter(12)
|
||||
val actor = system.actorOf(RoundRobinPool(12).props(routeeProps = Props[EchoProps]), "someOther")
|
||||
routerConfig(actor) must be === RoundRobinPool(12)
|
||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||
}
|
||||
|
||||
"be overridable in config" in {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "get" ⇒ sender ! context.props
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(12)), "config")
|
||||
routerConfig(actor) must be === RandomRouter(4)
|
||||
val actor = system.actorOf(RoundRobinPool(12).props(routeeProps = Props[EchoProps]), "config")
|
||||
routerConfig(actor) must be === RandomPool(4)
|
||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||
}
|
||||
|
||||
"use routees.paths from config" in {
|
||||
val actor = system.actorOf(RandomPool(12).props(routeeProps = Props[EchoProps]), "paths")
|
||||
routerConfig(actor) must be === RandomGroup(List("/user/service1", "/user/service2"))
|
||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||
}
|
||||
|
||||
"be overridable in explicit deployment" in {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "get" ⇒ sender ! context.props
|
||||
}
|
||||
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "someOther")
|
||||
routerConfig(actor) must be === RoundRobinRouter(12)
|
||||
val actor = system.actorOf(FromConfig.props(routeeProps = Props[EchoProps]).
|
||||
withDeploy(Deploy(routerConfig = RoundRobinPool(12))), "someOther")
|
||||
routerConfig(actor) must be === RoundRobinPool(12)
|
||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||
}
|
||||
|
||||
"be overridable in config even with explicit deployment" in {
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "get" ⇒ sender ! context.props
|
||||
}
|
||||
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "config")
|
||||
routerConfig(actor) must be === RandomRouter(4)
|
||||
val actor = system.actorOf(FromConfig.props(routeeProps = Props[EchoProps]).
|
||||
withDeploy(Deploy(routerConfig = RoundRobinPool(12))), "config")
|
||||
routerConfig(actor) must be === RandomPool(4)
|
||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||
}
|
||||
|
||||
"fail with an exception if not correct" in {
|
||||
intercept[ConfigurationException] {
|
||||
system.actorOf(Props.empty.withRouter(FromConfig))
|
||||
system.actorOf(FromConfig.props())
|
||||
}
|
||||
}
|
||||
|
||||
"not get confused when trying to wildcard-configure children" in {
|
||||
val router = system.actorOf(Props(new Actor {
|
||||
testActor ! self
|
||||
def receive = { case _ ⇒ }
|
||||
}).withRouter(FromConfig), "weird")
|
||||
val router = system.actorOf(FromConfig.props(routeeProps = Props(classOf[SendRefAtStartup], testActor)), "weird")
|
||||
val recv = Set() ++ (for (_ ← 1 to 3) yield expectMsgType[ActorRef])
|
||||
val expc = Set('a', 'b', 'c') map (i ⇒ system.actorFor("/user/weird/$" + i))
|
||||
recv must be(expc)
|
||||
expectNoMsg(1 second)
|
||||
}
|
||||
|
||||
"support custom router" in {
|
||||
val myrouter = system.actorOf(FromConfig.props(), "myrouter")
|
||||
myrouter ! "foo"
|
||||
expectMsg("bar")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
"round robin router" must {
|
||||
|
||||
"be able to shut down its instance" in {
|
||||
val helloLatch = new TestLatch(5)
|
||||
val stopLatch = new TestLatch(5)
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(5)), "round-robin-shutdown")
|
||||
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
Await.ready(helloLatch, 5 seconds)
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
|
||||
"deliver messages in a round robin fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 10
|
||||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
val counter = new AtomicInteger
|
||||
var replies = Map.empty[Int, Int]
|
||||
for (i ← 0 until connectionCount) {
|
||||
replies += i -> 0
|
||||
}
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
lazy val id = counter.getAndIncrement()
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! id
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(connectionCount)), "round-robin")
|
||||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
|
||||
replies = replies + (id -> (replies(id) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
counter.get must be(connectionCount)
|
||||
|
||||
actor ! Broadcast("end")
|
||||
Await.ready(doneLatch, 5 seconds)
|
||||
|
||||
replies.values foreach { _ must be(iterationCount) }
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val helloLatch = new TestLatch(5)
|
||||
val stopLatch = new TestLatch(5)
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(5)), "round-robin-broadcast")
|
||||
|
||||
actor ! Broadcast("hello")
|
||||
Await.ready(helloLatch, 5 seconds)
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
}
|
||||
|
||||
"random router" must {
|
||||
|
||||
"be able to shut down its instance" in {
|
||||
val stopLatch = new TestLatch(7)
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ sender ! "world"
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
}).withRouter(RandomRouter(7)), "random-shutdown")
|
||||
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
|
||||
within(2 seconds) {
|
||||
for (i ← 1 to 5) expectMsg("world")
|
||||
}
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
|
||||
"deliver messages in a random fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 100
|
||||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
val counter = new AtomicInteger
|
||||
var replies = Map.empty[Int, Int]
|
||||
for (i ← 0 until connectionCount) {
|
||||
replies = replies + (i -> 0)
|
||||
}
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
lazy val id = counter.getAndIncrement()
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! id
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
}
|
||||
}).withRouter(RandomRouter(connectionCount)), "random")
|
||||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
|
||||
replies = replies + (id -> (replies(id) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
counter.get must be(connectionCount)
|
||||
|
||||
actor ! Broadcast("end")
|
||||
Await.ready(doneLatch, 5 seconds)
|
||||
|
||||
replies.values foreach { _ must be > (0) }
|
||||
replies.values.sum must be === iterationCount * connectionCount
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val helloLatch = new TestLatch(6)
|
||||
val stopLatch = new TestLatch(6)
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
}).withRouter(RandomRouter(6)), "random-broadcast")
|
||||
|
||||
actor ! Broadcast("hello")
|
||||
Await.ready(helloLatch, 5 seconds)
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -4,11 +4,9 @@
|
|||
package akka.routing
|
||||
|
||||
import scala.concurrent.Await
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Props
|
||||
import akka.actor.actorRef2Scala
|
||||
import akka.pattern.ask
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashable
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
|
||||
|
|
@ -21,12 +19,12 @@ object ConsistentHashingRouterSpec {
|
|||
val config = """
|
||||
akka.actor.deployment {
|
||||
/router1 {
|
||||
router = consistent-hashing
|
||||
router = consistent-hashing-pool
|
||||
nr-of-instances = 3
|
||||
virtual-nodes-factor = 17
|
||||
}
|
||||
/router2 {
|
||||
router = consistent-hashing
|
||||
router = consistent-hashing-pool
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
|
|
@ -34,7 +32,8 @@ object ConsistentHashingRouterSpec {
|
|||
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! self
|
||||
case x: ConsistentHashableEnvelope ⇒ sender ! s"Unexpected envelope: $x"
|
||||
case _ ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -49,14 +48,14 @@ object ConsistentHashingRouterSpec {
|
|||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.config) with DefaultTimeout with ImplicitSender {
|
||||
import akka.routing.ConsistentHashingRouterSpec._
|
||||
import ConsistentHashingRouterSpec._
|
||||
implicit val ec = system.dispatcher
|
||||
|
||||
val router1 = system.actorOf(Props[Echo].withRouter(FromConfig()), "router1")
|
||||
val router1 = system.actorOf(FromConfig.props(Props[Echo]), "router1")
|
||||
|
||||
"consistent hashing router" must {
|
||||
"create routees from configuration" in {
|
||||
val currentRoutees = Await.result(router1 ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees]
|
||||
val currentRoutees = Await.result(router1 ? GetRoutees, remaining).asInstanceOf[Routees]
|
||||
currentRoutees.routees.size must be(3)
|
||||
}
|
||||
|
||||
|
|
@ -77,12 +76,12 @@ class ConsistentHashingRouterSpec extends AkkaSpec(ConsistentHashingRouterSpec.c
|
|||
expectMsg(destinationC)
|
||||
}
|
||||
|
||||
"select destination with defined consistentHashRoute" in {
|
||||
"select destination with defined hashMapping" in {
|
||||
def hashMapping: ConsistentHashMapping = {
|
||||
case Msg2(key, data) ⇒ key
|
||||
}
|
||||
val router2 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter(
|
||||
hashMapping = hashMapping)), "router2")
|
||||
val router2 = system.actorOf(ConsistentHashingPool(nrOfInstances = 1, hashMapping = hashMapping).
|
||||
props(Props[Echo]), "router2")
|
||||
|
||||
router2 ! Msg2("a", "A")
|
||||
val destinationA = expectMsgType[ActorRef]
|
||||
|
|
|
|||
|
|
@ -1,52 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class CustomRouteSpec extends AkkaSpec {
|
||||
|
||||
//#custom-router
|
||||
import akka.actor.{ ActorRef, Props, SupervisorStrategy }
|
||||
import akka.dispatch.Dispatchers
|
||||
|
||||
class MyRouter(target: ActorRef) extends RouterConfig {
|
||||
override def createRoute(provider: RouteeProvider): Route = {
|
||||
provider.createRoutees(1)
|
||||
|
||||
{
|
||||
case (sender, message: String) ⇒ List(Destination(sender, target))
|
||||
case (sender, message) ⇒ toAll(sender, provider.routees)
|
||||
}
|
||||
}
|
||||
override def supervisorStrategy = SupervisorStrategy.defaultStrategy
|
||||
override def routerDispatcher = Dispatchers.DefaultDispatcherId
|
||||
}
|
||||
//#custom-router
|
||||
|
||||
"A custom RouterConfig" must {
|
||||
|
||||
"be testable" in {
|
||||
//#test-route
|
||||
import akka.pattern.ask
|
||||
import akka.testkit.ExtractRoute
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
|
||||
val target = system.actorOf(Props.empty)
|
||||
val router = system.actorOf(Props.empty.withRouter(new MyRouter(target)))
|
||||
val route = ExtractRoute(router)
|
||||
val r = Await.result(router.ask(CurrentRoutees)(1 second).
|
||||
mapTo[RouterRoutees], 1 second)
|
||||
r.routees.size must be(1)
|
||||
route(testActor -> "hallo") must be(List(Destination(testActor, target)))
|
||||
route(testActor -> 12) must be(List(Destination(testActor, r.routees.head)))
|
||||
//#test-route
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
103
akka-actor-tests/src/test/scala/akka/routing/RandomSpec.scala
Normal file
103
akka-actor-tests/src/test/scala/akka/routing/RandomSpec.scala
Normal file
|
|
@ -0,0 +1,103 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import language.postfixOps
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.{ Props, Actor }
|
||||
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
|
||||
import akka.pattern.ask
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RandomSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||
|
||||
"random pool" must {
|
||||
|
||||
"be able to shut down its instance" in {
|
||||
val stopLatch = new TestLatch(7)
|
||||
|
||||
val actor = system.actorOf(RandomPool(7).props(Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ sender ! "world"
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
})), "random-shutdown")
|
||||
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
|
||||
within(2 seconds) {
|
||||
for (i ← 1 to 5) expectMsg("world")
|
||||
}
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
|
||||
"deliver messages in a random fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 100
|
||||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
val counter = new AtomicInteger
|
||||
var replies = Map.empty[Int, Int]
|
||||
for (i ← 0 until connectionCount) {
|
||||
replies = replies + (i -> 0)
|
||||
}
|
||||
|
||||
val actor = system.actorOf(RandomPool(connectionCount).props(routeeProps =
|
||||
Props(new Actor {
|
||||
lazy val id = counter.getAndIncrement()
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! id
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
}
|
||||
})), name = "random")
|
||||
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
|
||||
replies = replies + (id -> (replies(id) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
counter.get must be(connectionCount)
|
||||
|
||||
actor ! akka.routing.Broadcast("end")
|
||||
Await.ready(doneLatch, 5 seconds)
|
||||
|
||||
replies.values foreach { _ must be > (0) }
|
||||
replies.values.sum must be === iterationCount * connectionCount
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val helloLatch = new TestLatch(6)
|
||||
val stopLatch = new TestLatch(6)
|
||||
|
||||
val actor = system.actorOf(RandomPool(6).props(routeeProps = Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
})), "random-broadcast")
|
||||
|
||||
actor ! akka.routing.Broadcast("hello")
|
||||
Await.ready(helloLatch, 5 seconds)
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -10,7 +10,6 @@ import akka.testkit.TestEvent._
|
|||
import akka.actor.Props
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorRef
|
||||
import akka.pattern.ask
|
||||
import scala.util.Try
|
||||
|
|
@ -21,7 +20,7 @@ object ResizerSpec {
|
|||
akka.actor.serialize-messages = off
|
||||
akka.actor.deployment {
|
||||
/router1 {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
resizer {
|
||||
lower-bound = 2
|
||||
upper-bound = 3
|
||||
|
|
@ -52,7 +51,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
|||
}
|
||||
|
||||
def routeeSize(router: ActorRef): Int =
|
||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees.size
|
||||
Await.result(router ? GetRoutees, remaining).asInstanceOf[Routees].routees.size
|
||||
|
||||
"DefaultResizer" must {
|
||||
|
||||
|
|
@ -61,10 +60,12 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
|||
lowerBound = 2,
|
||||
upperBound = 3)
|
||||
|
||||
val c1 = resizer.capacity(immutable.IndexedSeq.empty[ActorRef])
|
||||
val c1 = resizer.capacity(Vector.empty[Routee])
|
||||
c1 must be(2)
|
||||
|
||||
val current = immutable.IndexedSeq(system.actorOf(Props[TestActor]), system.actorOf(Props[TestActor]))
|
||||
val current = Vector(
|
||||
ActorRefRoutee(system.actorOf(Props[TestActor])),
|
||||
ActorRefRoutee(system.actorOf(Props[TestActor])))
|
||||
val c2 = resizer.capacity(current)
|
||||
c2 must be(0)
|
||||
}
|
||||
|
|
@ -102,7 +103,8 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
|||
val resizer = DefaultResizer(
|
||||
lowerBound = 2,
|
||||
upperBound = 3)
|
||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
||||
val router = system.actorOf(RoundRobinPool(nrOfInstances = 0, resizer = Some(resizer)).
|
||||
props(Props[TestActor]))
|
||||
|
||||
router ! latch
|
||||
router ! latch
|
||||
|
|
@ -117,7 +119,7 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
|||
"be possible to define in configuration" in {
|
||||
val latch = new TestLatch(3)
|
||||
|
||||
val router = system.actorOf(Props[TestActor].withRouter(FromConfig()), "router1")
|
||||
val router = system.actorOf(FromConfig.props(Props[TestActor]), "router1")
|
||||
|
||||
router ! latch
|
||||
router ! latch
|
||||
|
|
@ -141,12 +143,14 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
|||
messagesPerResize = 1,
|
||||
backoffThreshold = 0.0)
|
||||
|
||||
val router = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case d: FiniteDuration ⇒ Thread.sleep(d.dilated.toMillis); sender ! "done"
|
||||
case "echo" ⇒ sender ! "reply"
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
||||
val router = system.actorOf(RoundRobinPool(nrOfInstances = 0, resizer = Some(resizer)).props(
|
||||
Props(new Actor {
|
||||
def receive = {
|
||||
case d: FiniteDuration ⇒
|
||||
Thread.sleep(d.dilated.toMillis); sender ! "done"
|
||||
case "echo" ⇒ sender ! "reply"
|
||||
}
|
||||
})))
|
||||
|
||||
// first message should create the minimum number of routees
|
||||
router ! "echo"
|
||||
|
|
@ -175,7 +179,6 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
|||
}
|
||||
|
||||
"backoff" in within(10 seconds) {
|
||||
|
||||
val resizer = DefaultResizer(
|
||||
lowerBound = 2,
|
||||
upperBound = 5,
|
||||
|
|
@ -185,12 +188,13 @@ class ResizerSpec extends AkkaSpec(ResizerSpec.config) with DefaultTimeout with
|
|||
pressureThreshold = 1,
|
||||
messagesPerResize = 2)
|
||||
|
||||
val router = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case n: Int if n <= 0 ⇒ // done
|
||||
case n: Int ⇒ Thread.sleep((n millis).dilated.toMillis)
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
||||
val router = system.actorOf(RoundRobinPool(nrOfInstances = 0, resizer = Some(resizer)).props(
|
||||
Props(new Actor {
|
||||
def receive = {
|
||||
case n: Int if n <= 0 ⇒ // done
|
||||
case n: Int ⇒ Thread.sleep((n millis).dilated.toMillis)
|
||||
}
|
||||
})))
|
||||
|
||||
// put some pressure on the router
|
||||
for (m ← 0 until 15) {
|
||||
|
|
|
|||
|
|
@ -0,0 +1,199 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import language.postfixOps
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Props
|
||||
import akka.actor.Actor
|
||||
import akka.testkit._
|
||||
import akka.pattern.ask
|
||||
import akka.actor.Terminated
|
||||
import akka.actor.ActorRef
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RoundRobinSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||
|
||||
def routeeSize(router: ActorRef): Int =
|
||||
Await.result(router ? GetRoutees, remaining).asInstanceOf[Routees].routees.size
|
||||
|
||||
"round robin pool" must {
|
||||
|
||||
"be able to shut down its instance" in {
|
||||
val helloLatch = new TestLatch(5)
|
||||
val stopLatch = new TestLatch(5)
|
||||
|
||||
val actor = system.actorOf(RoundRobinPool(5).props(routeeProps = Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
})), "round-robin-shutdown")
|
||||
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
actor ! "hello"
|
||||
Await.ready(helloLatch, 5 seconds)
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
|
||||
"deliver messages in a round robin fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 10
|
||||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
val counter = new AtomicInteger
|
||||
var replies = Map.empty[Int, Int].withDefaultValue(0)
|
||||
|
||||
val actor = system.actorOf(RoundRobinPool(connectionCount).props(routeeProps = Props(new Actor {
|
||||
lazy val id = counter.getAndIncrement()
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! id
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
}
|
||||
})), "round-robin")
|
||||
|
||||
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
||||
val id = Await.result((actor ? "hit").mapTo[Int], timeout.duration)
|
||||
replies = replies + (id -> (replies(id) + 1))
|
||||
}
|
||||
|
||||
counter.get must be(connectionCount)
|
||||
|
||||
actor ! akka.routing.Broadcast("end")
|
||||
Await.ready(doneLatch, 5 seconds)
|
||||
|
||||
replies.values foreach { _ must be(iterationCount) }
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val helloLatch = new TestLatch(5)
|
||||
val stopLatch = new TestLatch(5)
|
||||
|
||||
val actor = system.actorOf(RoundRobinPool(5).props(routeeProps = Props(new Actor {
|
||||
def receive = {
|
||||
case "hello" ⇒ helloLatch.countDown()
|
||||
}
|
||||
|
||||
override def postStop() {
|
||||
stopLatch.countDown()
|
||||
}
|
||||
})), "round-robin-broadcast")
|
||||
|
||||
actor ! akka.routing.Broadcast("hello")
|
||||
Await.ready(helloLatch, 5 seconds)
|
||||
|
||||
system.stop(actor)
|
||||
Await.ready(stopLatch, 5 seconds)
|
||||
}
|
||||
|
||||
"be controlled with management messages" in {
|
||||
val actor = system.actorOf(RoundRobinPool(3).props(routeeProps = Props(new Actor {
|
||||
def receive = Actor.emptyBehavior
|
||||
})), "round-robin-managed")
|
||||
|
||||
routeeSize(actor) must be(3)
|
||||
actor ! AdjustPoolSize(+4)
|
||||
routeeSize(actor) must be(7)
|
||||
actor ! AdjustPoolSize(-2)
|
||||
routeeSize(actor) must be(5)
|
||||
|
||||
val other = ActorSelectionRoutee(system.actorSelection("/user/other"))
|
||||
actor ! AddRoutee(other)
|
||||
routeeSize(actor) must be(6)
|
||||
actor ! RemoveRoutee(other)
|
||||
routeeSize(actor) must be(5)
|
||||
}
|
||||
}
|
||||
|
||||
"round robin group" must {
|
||||
|
||||
"deliver messages in a round robin fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 10
|
||||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
var replies = Map.empty[String, Int].withDefaultValue(0)
|
||||
|
||||
val paths = (1 to connectionCount) map { n ⇒
|
||||
val ref = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self.path.name
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
}
|
||||
}), name = "target-" + n)
|
||||
ref.path.elements.mkString("/", "/", "")
|
||||
}
|
||||
|
||||
val actor = system.actorOf(RoundRobinGroup(paths).props(), "round-robin-group1")
|
||||
|
||||
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
||||
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
|
||||
replies = replies + (id -> (replies(id) + 1))
|
||||
}
|
||||
|
||||
actor ! akka.routing.Broadcast("end")
|
||||
Await.ready(doneLatch, 5 seconds)
|
||||
|
||||
replies.values foreach { _ must be(iterationCount) }
|
||||
}
|
||||
}
|
||||
|
||||
"round robin logic used in actor" must {
|
||||
"deliver messages in a round robin fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 10
|
||||
|
||||
var replies = Map.empty[String, Int].withDefaultValue(0)
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
var n = 0
|
||||
var router = Router(RoundRobinRoutingLogic())
|
||||
|
||||
def receive = {
|
||||
case p: Props ⇒
|
||||
n += 1
|
||||
val c = context.actorOf(p, name = "child-" + n)
|
||||
context.watch(c)
|
||||
router = router.addRoutee(c)
|
||||
case Terminated(c) ⇒
|
||||
router = router.removeRoutee(c)
|
||||
if (router.routees.isEmpty)
|
||||
context.stop(self)
|
||||
case other ⇒ router.route(other, sender)
|
||||
}
|
||||
}))
|
||||
|
||||
val childProps = Props(new Actor {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self.path.name
|
||||
case "end" ⇒ context.stop(self)
|
||||
}
|
||||
})
|
||||
|
||||
(1 to connectionCount) foreach { _ ⇒ actor ! childProps }
|
||||
|
||||
for (_ ← 1 to iterationCount; _ ← 1 to connectionCount) {
|
||||
val id = Await.result((actor ? "hit").mapTo[String], timeout.duration)
|
||||
replies = replies + (id -> (replies(id) + 1))
|
||||
}
|
||||
|
||||
watch(actor)
|
||||
actor ! akka.routing.Broadcast("end")
|
||||
expectTerminated(actor)
|
||||
|
||||
replies.values foreach { _ must be(iterationCount) }
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -10,39 +10,42 @@ import akka.actor.Actor
|
|||
import akka.actor.ActorRef
|
||||
import akka.actor.LocalActorRef
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Identify
|
||||
import akka.actor.ActorIdentity
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RouteeCreationSpec extends AkkaSpec {
|
||||
|
||||
"Creating Routees" must {
|
||||
|
||||
"result in visible routees" in {
|
||||
val N = 100
|
||||
system.actorOf(Props(new Actor {
|
||||
testActor ! system.actorFor(self.path)
|
||||
system.actorOf(RoundRobinPool(N).props(Props(new Actor {
|
||||
system.actorSelection(self.path).tell(Identify(self.path), testActor)
|
||||
def receive = Actor.emptyBehavior
|
||||
}).withRouter(RoundRobinRouter(N)))
|
||||
})))
|
||||
for (i ← 1 to N) {
|
||||
expectMsgType[ActorRef] match {
|
||||
case _: LocalActorRef ⇒ // fine
|
||||
case x ⇒ fail(s"routee $i was a ${x.getClass}")
|
||||
expectMsgType[ActorIdentity] match {
|
||||
case ActorIdentity(_, Some(_)) ⇒ // fine
|
||||
case x ⇒ fail(s"routee $i was not found $x")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
"allow sending to context.parent" in {
|
||||
val N = 100
|
||||
system.actorOf(Props(new Actor {
|
||||
system.actorOf(RoundRobinPool(N).props(Props(new Actor {
|
||||
context.parent ! "one"
|
||||
def receive = {
|
||||
case "one" ⇒ testActor forward "two"
|
||||
}
|
||||
}).withRouter(RoundRobinRouter(N)))
|
||||
})))
|
||||
val gotit = receiveWhile(messages = N) {
|
||||
case "two" ⇒ lastSender.toString
|
||||
}
|
||||
expectNoMsg(100.millis)
|
||||
if (gotit.size != N) {
|
||||
fail(s"got only ${gotit.size} from \n${gotit mkString "\n"}")
|
||||
fail(s"got only ${gotit.size} from [${gotit mkString ", "}]")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -19,6 +19,7 @@ import akka.dispatch.Dispatchers
|
|||
import akka.util.Collections.EmptyImmutableSeq
|
||||
import akka.util.Timeout
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.routing._
|
||||
|
||||
object RoutingSpec {
|
||||
|
||||
|
|
@ -26,19 +27,16 @@ object RoutingSpec {
|
|||
akka.actor.serialize-messages = off
|
||||
akka.actor.deployment {
|
||||
/router1 {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 3
|
||||
}
|
||||
/router2 {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 3
|
||||
}
|
||||
/router3 {
|
||||
router = round-robin
|
||||
}
|
||||
/myrouter {
|
||||
router = "akka.routing.RoutingSpec$MyRouter"
|
||||
foo = bar
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 0
|
||||
}
|
||||
}
|
||||
"""
|
||||
|
|
@ -53,32 +51,19 @@ object RoutingSpec {
|
|||
}
|
||||
}
|
||||
|
||||
class MyRouter(config: Config) extends RouterConfig {
|
||||
val foo = config.getString("foo")
|
||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
routeeProvider.registerRoutees(List(routeeProvider.context.actorOf(Props[Echo])))
|
||||
|
||||
{
|
||||
case (sender, message) ⇒ EmptyImmutableSeq
|
||||
}
|
||||
}
|
||||
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
|
||||
def supervisorStrategy: SupervisorStrategy = SupervisorStrategy.defaultStrategy
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with ImplicitSender {
|
||||
implicit val ec = system.dispatcher
|
||||
import akka.routing.RoutingSpec._
|
||||
import RoutingSpec._
|
||||
|
||||
muteDeadLetters(classOf[akka.dispatch.sysmsg.DeathWatchNotification])()
|
||||
|
||||
"routers in general" must {
|
||||
|
||||
"evict terminated routees" in {
|
||||
val router = system.actorOf(Props[Echo].withRouter(RoundRobinRouter(2)))
|
||||
val router = system.actorOf(RoundRobinPool(2).props(routeeProps = Props[Echo]))
|
||||
router ! ""
|
||||
router ! ""
|
||||
val c1, c2 = expectMsgType[ActorRef]
|
||||
|
|
@ -103,61 +88,36 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
val latch = TestLatch(1)
|
||||
val resizer = new Resizer {
|
||||
def isTimeForResize(messageCounter: Long): Boolean = messageCounter == 0
|
||||
def resize(routeeProvider: RouteeProvider): Unit = {
|
||||
routeeProvider.createRoutees(nrOfInstances = 2)
|
||||
def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int = {
|
||||
latch.countDown()
|
||||
2
|
||||
}
|
||||
}
|
||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))))
|
||||
val router = system.actorOf(RoundRobinPool(nrOfInstances = 0, resizer = Some(resizer)).props(
|
||||
routeeProps = Props[TestActor]))
|
||||
watch(router)
|
||||
Await.ready(latch, remaining)
|
||||
router ! CurrentRoutees
|
||||
val routees = expectMsgType[RouterRoutees].routees
|
||||
router ! GetRoutees
|
||||
val routees = expectMsgType[Routees].routees
|
||||
routees.size must be(2)
|
||||
routees foreach system.stop
|
||||
routees foreach { _.send(PoisonPill, testActor) }
|
||||
// expect no Terminated
|
||||
expectNoMsg(2.seconds)
|
||||
}
|
||||
|
||||
"be able to send their routees" in {
|
||||
case class TestRun(id: String, names: immutable.Iterable[String], actors: Int)
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case TestRun(id, names, actors) ⇒
|
||||
val routerProps = Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(
|
||||
routees = names map { context.actorOf(Props(new TestActor), _) },
|
||||
within = 5 seconds))
|
||||
|
||||
1 to actors foreach { i ⇒ context.actorOf(routerProps, id + i).tell(CurrentRoutees, testActor) }
|
||||
}
|
||||
}))
|
||||
|
||||
val actors = 15
|
||||
val names = 1 to 20 map { "routee" + _ } toList
|
||||
|
||||
actor ! TestRun("test", names, actors)
|
||||
|
||||
1 to actors foreach { _ ⇒
|
||||
val routees = expectMsgType[RouterRoutees].routees
|
||||
routees.map(_.path.name) must be === names
|
||||
}
|
||||
expectNoMsg(500.millis)
|
||||
}
|
||||
|
||||
"use configured nr-of-instances when FromConfig" in {
|
||||
val router = system.actorOf(Props[TestActor].withRouter(FromConfig), "router1")
|
||||
router ! CurrentRoutees
|
||||
expectMsgType[RouterRoutees].routees.size must be(3)
|
||||
val router = system.actorOf(FromConfig.props(routeeProps = Props[TestActor]), "router1")
|
||||
router ! GetRoutees
|
||||
expectMsgType[Routees].routees.size must be(3)
|
||||
watch(router)
|
||||
system.stop(router)
|
||||
expectTerminated(router)
|
||||
}
|
||||
|
||||
"use configured nr-of-instances when router is specified" in {
|
||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 2)), "router2")
|
||||
router ! CurrentRoutees
|
||||
expectMsgType[RouterRoutees].routees.size must be(3)
|
||||
val router = system.actorOf(RoundRobinPool(nrOfInstances = 2).props(routeeProps = Props[TestActor]), "router2")
|
||||
router ! GetRoutees
|
||||
expectMsgType[Routees].routees.size must be(3)
|
||||
system.stop(router)
|
||||
}
|
||||
|
||||
|
|
@ -165,15 +125,16 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
val latch = TestLatch(1)
|
||||
val resizer = new Resizer {
|
||||
def isTimeForResize(messageCounter: Long): Boolean = messageCounter == 0
|
||||
def resize(routeeProvider: RouteeProvider): Unit = {
|
||||
routeeProvider.createRoutees(nrOfInstances = 3)
|
||||
def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int = {
|
||||
latch.countDown()
|
||||
3
|
||||
}
|
||||
}
|
||||
val router = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(resizer = Some(resizer))), "router3")
|
||||
val router = system.actorOf(RoundRobinPool(nrOfInstances = 0, resizer = Some(resizer)).props(
|
||||
routeeProps = Props[TestActor]), "router3")
|
||||
Await.ready(latch, remaining)
|
||||
router ! CurrentRoutees
|
||||
expectMsgType[RouterRoutees].routees.size must be(3)
|
||||
router ! GetRoutees
|
||||
expectMsgType[Routees].routees.size must be(3)
|
||||
system.stop(router)
|
||||
}
|
||||
|
||||
|
|
@ -184,19 +145,20 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
case e ⇒ testActor ! e; SupervisorStrategy.Escalate
|
||||
//#custom-strategy
|
||||
}
|
||||
val router = system.actorOf(Props.empty.withRouter(
|
||||
RoundRobinRouter(1, supervisorStrategy = escalator)))
|
||||
val router = system.actorOf(RoundRobinPool(1, supervisorStrategy = escalator).props(
|
||||
routeeProps = Props[TestActor]))
|
||||
//#supervision
|
||||
router ! CurrentRoutees
|
||||
router ! GetRoutees
|
||||
EventFilter[ActorKilledException](occurrences = 1) intercept {
|
||||
expectMsgType[RouterRoutees].routees.head ! Kill
|
||||
expectMsgType[Routees].routees.head.send(Kill, testActor)
|
||||
}
|
||||
expectMsgType[ActorKilledException]
|
||||
|
||||
val router2 = system.actorOf(Props.empty.withRouter(RoundRobinRouter(1).withSupervisorStrategy(escalator)))
|
||||
router2 ! CurrentRoutees
|
||||
val router2 = system.actorOf(RoundRobinPool(1).withSupervisorStrategy(escalator).props(
|
||||
routeeProps = Props[TestActor]))
|
||||
router2 ! GetRoutees
|
||||
EventFilter[ActorKilledException](occurrences = 1) intercept {
|
||||
expectMsgType[RouterRoutees].routees.head ! Kill
|
||||
expectMsgType[Routees].routees.head.send(Kill, testActor)
|
||||
}
|
||||
expectMsgType[ActorKilledException]
|
||||
}
|
||||
|
|
@ -205,10 +167,11 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
val escalator = OneForOneStrategy() {
|
||||
case e ⇒ testActor ! e; SupervisorStrategy.Escalate
|
||||
}
|
||||
val router = system.actorOf(Props.empty.withRouter(FromConfig.withSupervisorStrategy(escalator)), "router1")
|
||||
router ! CurrentRoutees
|
||||
val router = system.actorOf(FromConfig.withSupervisorStrategy(escalator).props(
|
||||
routeeProps = Props[TestActor]), "router1")
|
||||
router ! GetRoutees
|
||||
EventFilter[ActorKilledException](occurrences = 1) intercept {
|
||||
expectMsgType[RouterRoutees].routees.head ! Kill
|
||||
expectMsgType[Routees].routees.head.send(Kill, testActor)
|
||||
}
|
||||
expectMsgType[ActorKilledException]
|
||||
}
|
||||
|
|
@ -218,12 +181,12 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
case e ⇒ testActor ! e; SupervisorStrategy.Restart
|
||||
}
|
||||
val supervisor = system.actorOf(Props(new Supervisor(restarter)))
|
||||
supervisor ! Props(new Actor {
|
||||
supervisor ! RoundRobinPool(3).props(routeeProps = Props(new Actor {
|
||||
def receive = {
|
||||
case x: String ⇒ throw new Exception(x)
|
||||
}
|
||||
override def postRestart(reason: Throwable): Unit = testActor ! "restarted"
|
||||
}).withRouter(RoundRobinRouter(3))
|
||||
}))
|
||||
val router = expectMsgType[ActorRef]
|
||||
EventFilter[Exception]("die", occurrences = 1) intercept {
|
||||
router ! "die"
|
||||
|
|
@ -238,9 +201,9 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "start" ⇒
|
||||
context.actorOf(Props(new Actor {
|
||||
context.actorOf(RoundRobinPool(2).props(routeeProps = Props(new Actor {
|
||||
def receive = { case x ⇒ sender ! x }
|
||||
}).withRouter(RoundRobinRouter(2))) ? "hello" pipeTo sender
|
||||
}))) ? "hello" pipeTo sender
|
||||
}
|
||||
})) ! "start"
|
||||
expectMsg("hello")
|
||||
|
|
@ -249,10 +212,6 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
}
|
||||
|
||||
"no router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(NoRouter))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"send message to connection" in {
|
||||
class Actor1 extends Actor {
|
||||
|
|
@ -261,7 +220,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
}
|
||||
}
|
||||
|
||||
val routedActor = system.actorOf(Props(new Actor1).withRouter(NoRouter))
|
||||
val routedActor = system.actorOf(NoRouter.props(routeeProps = Props(new Actor1)))
|
||||
routedActor ! "hello"
|
||||
routedActor ! "end"
|
||||
|
||||
|
|
@ -270,317 +229,10 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
}
|
||||
}
|
||||
|
||||
"round robin router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(nrOfInstances = 1)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
//In this test a bunch of actors are created and each actor has its own counter.
|
||||
//to test round robin, the routed actor receives the following sequence of messages 1 2 3 .. 1 2 3 .. 1 2 3 which it
|
||||
//uses to increment his counter.
|
||||
//So after n iteration, the first actor his counter should be 1*n, the second 2*n etc etc.
|
||||
"deliver messages in a round robin fashion" in {
|
||||
val connectionCount = 10
|
||||
val iterationCount = 10
|
||||
val doneLatch = new TestLatch(connectionCount)
|
||||
|
||||
//lets create some connections.
|
||||
@volatile var actors = immutable.IndexedSeq[ActorRef]()
|
||||
@volatile var counters = immutable.IndexedSeq[AtomicInteger]()
|
||||
for (i ← 0 until connectionCount) {
|
||||
counters = counters :+ new AtomicInteger()
|
||||
|
||||
val actor = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counters(i).addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
actors = actors :+ actor
|
||||
}
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(routees = actors)))
|
||||
|
||||
//send messages to the actor.
|
||||
for (i ← 0 until iterationCount) {
|
||||
for (k ← 0 until connectionCount) {
|
||||
routedActor ! (k + 1)
|
||||
}
|
||||
}
|
||||
|
||||
routedActor ! Broadcast("end")
|
||||
//now wait some and do validations.
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
for (i ← 0 until connectionCount)
|
||||
counters(i).get must be((iterationCount * (i + 1)))
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RoundRobinRouter(routees = List(actor1, actor2))))
|
||||
|
||||
routedActor ! Broadcast(1)
|
||||
routedActor ! Broadcast("end")
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
}
|
||||
|
||||
"random router" must {
|
||||
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RandomRouter(nrOfInstances = 1)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"deliver a broadcast message" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(RandomRouter(routees = List(actor1, actor2))))
|
||||
|
||||
routedActor ! Broadcast(1)
|
||||
routedActor ! Broadcast("end")
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
}
|
||||
|
||||
"smallest mailbox router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(SmallestMailboxRouter(nrOfInstances = 1)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"deliver messages to idle actor" in {
|
||||
val usedActors = new ConcurrentHashMap[Int, String]()
|
||||
val router = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case (busy: TestLatch, receivedLatch: TestLatch) ⇒
|
||||
usedActors.put(0, self.path.toString)
|
||||
self ! "another in busy mailbox"
|
||||
receivedLatch.countDown()
|
||||
Await.ready(busy, TestLatch.DefaultTimeout)
|
||||
case (msg: Int, receivedLatch: TestLatch) ⇒
|
||||
usedActors.put(msg, self.path.toString)
|
||||
receivedLatch.countDown()
|
||||
case s: String ⇒
|
||||
}
|
||||
}).withRouter(SmallestMailboxRouter(3)))
|
||||
|
||||
val busy = TestLatch(1)
|
||||
val received0 = TestLatch(1)
|
||||
router ! ((busy, received0))
|
||||
Await.ready(received0, TestLatch.DefaultTimeout)
|
||||
|
||||
val received1 = TestLatch(1)
|
||||
router ! ((1, received1))
|
||||
Await.ready(received1, TestLatch.DefaultTimeout)
|
||||
|
||||
val received2 = TestLatch(1)
|
||||
router ! ((2, received2))
|
||||
Await.ready(received2, TestLatch.DefaultTimeout)
|
||||
|
||||
val received3 = TestLatch(1)
|
||||
router ! ((3, received3))
|
||||
Await.ready(received3, TestLatch.DefaultTimeout)
|
||||
|
||||
busy.countDown()
|
||||
|
||||
val busyPath = usedActors.get(0)
|
||||
busyPath must not be (null)
|
||||
|
||||
val path1 = usedActors.get(1)
|
||||
val path2 = usedActors.get(2)
|
||||
val path3 = usedActors.get(3)
|
||||
|
||||
path1 must not be (busyPath)
|
||||
path2 must not be (busyPath)
|
||||
path3 must not be (busyPath)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
"broadcast router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(BroadcastRouter(nrOfInstances = 1)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"broadcast message using !" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(BroadcastRouter(routees = List(actor1, actor2))))
|
||||
routedActor ! 1
|
||||
routedActor ! "end"
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
|
||||
"broadcast message using ?" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒
|
||||
counter1.addAndGet(msg)
|
||||
sender ! "ack"
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(BroadcastRouter(routees = List(actor1, actor2))))
|
||||
routedActor ? 1
|
||||
routedActor ! "end"
|
||||
|
||||
Await.ready(doneLatch, remaining)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
}
|
||||
|
||||
"Scatter-gather router" must {
|
||||
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(routees = List(newActor(0)), within = 1 seconds)))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(routees = List(actor1, actor2), within = 1 seconds)))
|
||||
routedActor ! Broadcast(1)
|
||||
routedActor ! Broadcast("end")
|
||||
|
||||
Await.ready(doneLatch, TestLatch.DefaultTimeout)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
|
||||
"return response, even if one of the actors has stopped" in {
|
||||
val shutdownLatch = new TestLatch(1)
|
||||
val actor1 = newActor(1, Some(shutdownLatch))
|
||||
val actor2 = newActor(14, Some(shutdownLatch))
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(
|
||||
ScatterGatherFirstCompletedRouter(routees = List(actor1, actor2), within = 3 seconds)))
|
||||
|
||||
routedActor ! Broadcast(Stop(Some(1)))
|
||||
Await.ready(shutdownLatch, TestLatch.DefaultTimeout)
|
||||
Await.result(routedActor ? Broadcast(0), timeout.duration) must be(14)
|
||||
}
|
||||
|
||||
case class Stop(id: Option[Int] = None)
|
||||
|
||||
def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case Stop(None) ⇒ context.stop(self)
|
||||
case Stop(Some(_id)) if (_id == id) ⇒ context.stop(self)
|
||||
case _id: Int if (_id == id) ⇒
|
||||
case x ⇒ {
|
||||
Thread sleep 100 * id
|
||||
sender ! id
|
||||
}
|
||||
}
|
||||
|
||||
override def postStop = {
|
||||
shudownLatch foreach (_.countDown())
|
||||
}
|
||||
}), "Actor:" + id)
|
||||
}
|
||||
|
||||
"router FromConfig" must {
|
||||
"throw suitable exception when not configured" in {
|
||||
val e = intercept[ConfigurationException] {
|
||||
system.actorOf(Props[TestActor].withRouter(FromConfig), "routerNotDefined")
|
||||
system.actorOf(FromConfig.props(routeeProps = Props[TestActor]), "routerNotDefined")
|
||||
}
|
||||
e.getMessage must include("routerNotDefined")
|
||||
}
|
||||
|
|
@ -590,102 +242,12 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
|||
.parseString("akka.actor.deployment./routed.router=round-robin")
|
||||
.withFallback(system.settings.config))
|
||||
try {
|
||||
sys.actorOf(Props.empty.withRouter(FromConfig), "routed")
|
||||
sys.actorOf(FromConfig.props(), "routed")
|
||||
} finally {
|
||||
shutdown(sys)
|
||||
}
|
||||
}
|
||||
|
||||
"support custom router" in {
|
||||
val myrouter = system.actorOf(Props.empty.withRouter(FromConfig), "myrouter")
|
||||
myrouter.isTerminated must be(false)
|
||||
}
|
||||
}
|
||||
|
||||
"custom router" must {
|
||||
"be started when constructed" in {
|
||||
val routedActor = system.actorOf(Props[TestActor].withRouter(VoteCountRouter()))
|
||||
routedActor.isTerminated must be(false)
|
||||
}
|
||||
|
||||
"count votes as intended - not as in Florida" in {
|
||||
val routedActor = system.actorOf(Props.empty.withRouter(VoteCountRouter()))
|
||||
routedActor ! DemocratVote
|
||||
routedActor ! DemocratVote
|
||||
routedActor ! RepublicanVote
|
||||
routedActor ! DemocratVote
|
||||
routedActor ! RepublicanVote
|
||||
val democratsResult = (routedActor ? DemocratCountResult)
|
||||
val republicansResult = (routedActor ? RepublicanCountResult)
|
||||
|
||||
Await.result(democratsResult, 1 seconds) === 3
|
||||
Await.result(republicansResult, 1 seconds) === 2
|
||||
}
|
||||
|
||||
// DO NOT CHANGE THE COMMENTS BELOW AS THEY ARE USED IN THE DOCUMENTATION
|
||||
|
||||
//#CustomRouter
|
||||
//#crMessages
|
||||
case object DemocratVote
|
||||
case object DemocratCountResult
|
||||
case object RepublicanVote
|
||||
case object RepublicanCountResult
|
||||
//#crMessages
|
||||
|
||||
//#crActors
|
||||
class DemocratActor extends Actor {
|
||||
var counter = 0
|
||||
|
||||
def receive = {
|
||||
case DemocratVote ⇒ counter += 1
|
||||
case DemocratCountResult ⇒ sender ! counter
|
||||
}
|
||||
}
|
||||
|
||||
class RepublicanActor extends Actor {
|
||||
var counter = 0
|
||||
|
||||
def receive = {
|
||||
case RepublicanVote ⇒ counter += 1
|
||||
case RepublicanCountResult ⇒ sender ! counter
|
||||
}
|
||||
}
|
||||
//#crActors
|
||||
|
||||
//#crRouter
|
||||
case class VoteCountRouter() extends RouterConfig {
|
||||
|
||||
def routerDispatcher: String = Dispatchers.DefaultDispatcherId
|
||||
def supervisorStrategy: SupervisorStrategy = SupervisorStrategy.defaultStrategy
|
||||
|
||||
//#crRoute
|
||||
def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
val democratActor =
|
||||
routeeProvider.context.actorOf(Props(new DemocratActor()), "d")
|
||||
val republicanActor =
|
||||
routeeProvider.context.actorOf(Props(new RepublicanActor()), "r")
|
||||
val routees = Vector[ActorRef](democratActor, republicanActor)
|
||||
|
||||
//#crRegisterRoutees
|
||||
routeeProvider.registerRoutees(routees)
|
||||
//#crRegisterRoutees
|
||||
|
||||
//#crRoutingLogic
|
||||
{
|
||||
case (sender, message) ⇒
|
||||
message match {
|
||||
case DemocratVote | DemocratCountResult ⇒
|
||||
List(Destination(sender, democratActor))
|
||||
case RepublicanVote | RepublicanCountResult ⇒
|
||||
List(Destination(sender, republicanActor))
|
||||
}
|
||||
}
|
||||
//#crRoutingLogic
|
||||
}
|
||||
//#crRoute
|
||||
|
||||
}
|
||||
//#crRouter
|
||||
//#CustomRouter
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,89 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.{ Props, Actor }
|
||||
import akka.pattern.ask
|
||||
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
object ScatterGatherFirstCompletedSpec {
|
||||
class TestActor extends Actor {
|
||||
def receive = { case _ ⇒ }
|
||||
}
|
||||
|
||||
case class Stop(id: Option[Int] = None)
|
||||
|
||||
def newActor(id: Int, shudownLatch: Option[TestLatch] = None)(implicit system: ActorSystem) =
|
||||
system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case Stop(None) ⇒ context.stop(self)
|
||||
case Stop(Some(_id)) if (_id == id) ⇒ context.stop(self)
|
||||
case _id: Int if (_id == id) ⇒
|
||||
case x ⇒ {
|
||||
Thread sleep 100 * id
|
||||
sender ! id
|
||||
}
|
||||
}
|
||||
|
||||
override def postStop = {
|
||||
shudownLatch foreach (_.countDown())
|
||||
}
|
||||
}), "Actor:" + id)
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ScatterGatherFirstCompletedSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
||||
import ScatterGatherFirstCompletedSpec._
|
||||
|
||||
"Scatter-gather group" must {
|
||||
|
||||
"deliver a broadcast message using the !" in {
|
||||
val doneLatch = new TestLatch(2)
|
||||
|
||||
val counter1 = new AtomicInteger
|
||||
val actor1 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter1.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val counter2 = new AtomicInteger
|
||||
val actor2 = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case "end" ⇒ doneLatch.countDown()
|
||||
case msg: Int ⇒ counter2.addAndGet(msg)
|
||||
}
|
||||
}))
|
||||
|
||||
val paths = List(actor1, actor2).map(_.path.toString)
|
||||
val routedActor = system.actorOf(ScatterGatherFirstCompletedGroup(paths, within = 1.second).props())
|
||||
routedActor ! Broadcast(1)
|
||||
routedActor ! Broadcast("end")
|
||||
|
||||
Await.ready(doneLatch, TestLatch.DefaultTimeout)
|
||||
|
||||
counter1.get must be(1)
|
||||
counter2.get must be(1)
|
||||
}
|
||||
|
||||
"return response, even if one of the actors has stopped" in {
|
||||
val shutdownLatch = new TestLatch(1)
|
||||
val actor1 = newActor(1, Some(shutdownLatch))
|
||||
val actor2 = newActor(14, Some(shutdownLatch))
|
||||
val paths = List(actor1, actor2).map(_.path.toString)
|
||||
val routedActor = system.actorOf(ScatterGatherFirstCompletedGroup(paths, within = 3.seconds).props())
|
||||
|
||||
routedActor ! Broadcast(Stop(Some(1)))
|
||||
Await.ready(shutdownLatch, TestLatch.DefaultTimeout)
|
||||
Await.result(routedActor ? Broadcast(0), timeout.duration) must be(14)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,67 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
import scala.concurrent.Await
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.{ Props, Actor }
|
||||
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class SmallestMailboxSpec extends AkkaSpec("akka.actor.serialize-messages = off")
|
||||
with DefaultTimeout with ImplicitSender {
|
||||
|
||||
"smallest mailbox pool" must {
|
||||
|
||||
"deliver messages to idle actor" in {
|
||||
val usedActors = new ConcurrentHashMap[Int, String]()
|
||||
val router = system.actorOf(SmallestMailboxPool(3).props(routeeProps = Props(new Actor {
|
||||
def receive = {
|
||||
case (busy: TestLatch, receivedLatch: TestLatch) ⇒
|
||||
usedActors.put(0, self.path.toString)
|
||||
self ! "another in busy mailbox"
|
||||
receivedLatch.countDown()
|
||||
Await.ready(busy, TestLatch.DefaultTimeout)
|
||||
case (msg: Int, receivedLatch: TestLatch) ⇒
|
||||
usedActors.put(msg, self.path.toString)
|
||||
receivedLatch.countDown()
|
||||
case s: String ⇒
|
||||
}
|
||||
})))
|
||||
|
||||
val busy = TestLatch(1)
|
||||
val received0 = TestLatch(1)
|
||||
router ! ((busy, received0))
|
||||
Await.ready(received0, TestLatch.DefaultTimeout)
|
||||
|
||||
val received1 = TestLatch(1)
|
||||
router ! ((1, received1))
|
||||
Await.ready(received1, TestLatch.DefaultTimeout)
|
||||
|
||||
val received2 = TestLatch(1)
|
||||
router ! ((2, received2))
|
||||
Await.ready(received2, TestLatch.DefaultTimeout)
|
||||
|
||||
val received3 = TestLatch(1)
|
||||
router ! ((3, received3))
|
||||
Await.ready(received3, TestLatch.DefaultTimeout)
|
||||
|
||||
busy.countDown()
|
||||
|
||||
val busyPath = usedActors.get(0)
|
||||
busyPath must not be (null)
|
||||
|
||||
val path1 = usedActors.get(1)
|
||||
val path2 = usedActors.get(2)
|
||||
val path3 = usedActors.get(3)
|
||||
|
||||
path1 must not be (busyPath)
|
||||
path2 must not be (busyPath)
|
||||
path3 must not be (busyPath)
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -100,12 +100,28 @@ akka {
|
|||
# Default timeout for typed actor methods with non-void return type
|
||||
timeout = 5s
|
||||
}
|
||||
|
||||
# Mapping between ´deployment.router' short names to fully qualified class names
|
||||
router.type-mapping {
|
||||
from-code = "akka.routing.NoRouter"
|
||||
round-robin-pool = "akka.routing.RoundRobinPool"
|
||||
round-robin-group = "akka.routing.RoundRobinGroup"
|
||||
random-pool = "akka.routing.RandomPool"
|
||||
random-group = "akka.routing.RandomGroup"
|
||||
smallest-mailbox-pool = "akka.routing.SmallestMailboxPool"
|
||||
broadcast-pool = "akka.routing.BroadcastPool"
|
||||
broadcast-group = "akka.routing.BroadcastGroup"
|
||||
scatter-gather-pool = "akka.routing.ScatterGatherFirstCompletedPool"
|
||||
scatter-gather-group = "akka.routing.ScatterGatherFirstCompletedGroup"
|
||||
consistent-hashing-pool = "akka.routing.ConsistentHashingPool"
|
||||
consistent-hashing-group = "akka.routing.ConsistentHashingGroup"
|
||||
}
|
||||
|
||||
deployment {
|
||||
|
||||
# deployment id pattern - on the format: /parent/child etc.
|
||||
default {
|
||||
|
||||
|
||||
# The id of the dispatcher to use for this actor.
|
||||
# If undefined or empty the dispatcher specified in code
|
||||
# (Props.withDispatcher) is used, or default-dispatcher if not
|
||||
|
|
@ -126,7 +142,7 @@ akka {
|
|||
# - or: Fully qualified class name of the router class.
|
||||
# The class must extend akka.routing.CustomRouterConfig and
|
||||
# have a public constructor with com.typesafe.config.Config
|
||||
# parameter.
|
||||
# and optional akka.actor.DynamicAccess parameter.
|
||||
# - default is "from-code";
|
||||
# Whether or not an actor is transformed to a Router is decided in code
|
||||
# only (Props.withRouter). The type of router can be overridden in the
|
||||
|
|
@ -161,6 +177,8 @@ akka {
|
|||
# Routers with dynamically resizable number of routees; this feature is
|
||||
# enabled by including (parts of) this section in the deployment
|
||||
resizer {
|
||||
|
||||
enabled = off
|
||||
|
||||
# The fewest number of routees the router should ever have.
|
||||
lower-bound = 1
|
||||
|
|
@ -198,13 +216,6 @@ akka {
|
|||
# capacity is 9 it will request an decrease of 1 routee.
|
||||
backoff-rate = 0.1
|
||||
|
||||
# When the resizer reduce the capacity the abandoned routee actors are
|
||||
# stopped with PoisonPill after this delay. The reason for the delay is
|
||||
# to give concurrent messages a chance to be placed in mailbox before
|
||||
# sending PoisonPill.
|
||||
# Use 0s to skip delay.
|
||||
stop-delay = 1s
|
||||
|
||||
# Number of messages between resize operation.
|
||||
# Use 1 to resize before each message.
|
||||
messages-per-resize = 10
|
||||
|
|
|
|||
|
|
@ -735,16 +735,15 @@ private[akka] class LocalActorRefProvider private[akka] (
|
|||
if (!system.dispatchers.hasDispatcher(d.routerConfig.routerDispatcher))
|
||||
throw new ConfigurationException(s"Dispatcher [${p.dispatcher}] not configured for router of $path")
|
||||
|
||||
val routerProps =
|
||||
Props(p.deploy.copy(dispatcher = p.routerConfig.routerDispatcher),
|
||||
classOf[RoutedActorCell.RouterCreator], Vector(p.routerConfig))
|
||||
val routerProps = Props(p.deploy.copy(dispatcher = p.routerConfig.routerDispatcher),
|
||||
classOf[RoutedActorCell.RouterActorCreator], Vector(p.routerConfig))
|
||||
val routeeProps = p.withRouter(NoRouter)
|
||||
|
||||
try {
|
||||
val routerDispatcher = system.dispatchers.lookup(p.routerConfig.routerDispatcher)
|
||||
val routerMailbox = system.mailboxes.getMailboxType(routerProps, routerDispatcher.configurator.config)
|
||||
|
||||
// the RouteeProvider uses context.actorOf() to create the routees, which does not allow us to pass
|
||||
// routers use context.actorOf() to create the routees, which does not allow us to pass
|
||||
// these through, but obtain them here for early verification
|
||||
val routeeDispatcher = system.dispatchers.lookup(p.dispatcher)
|
||||
val routeeMailbox = system.mailboxes.getMailboxType(routeeProps, routeeDispatcher.configurator.config)
|
||||
|
|
|
|||
|
|
@ -132,9 +132,14 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
|||
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
private val resizerEnabled: Config = ConfigFactory.parseString("resizer.enabled=on")
|
||||
private val deployments = new AtomicReference(WildcardTree[Deploy]())
|
||||
private val config = settings.config.getConfig("akka.actor.deployment")
|
||||
protected val default = config.getConfig("default")
|
||||
val routerTypeMapping: Map[String, String] =
|
||||
settings.config.getConfig("akka.actor.router.type-mapping").root.unwrapped.asScala.collect {
|
||||
case (key, value: String) ⇒ (key -> value)
|
||||
}.toMap
|
||||
|
||||
config.root.asScala flatMap {
|
||||
case ("default", _) ⇒ None
|
||||
|
|
@ -181,33 +186,39 @@ private[akka] class Deployer(val settings: ActorSystem.Settings, val dynamicAcce
|
|||
* @param config the user defined config of the deployment, without defaults
|
||||
* @param deployment the deployment config, with defaults
|
||||
*/
|
||||
protected def createRouterConfig(routerType: String, key: String, config: Config, deployment: Config): RouterConfig = {
|
||||
val routees = immutableSeq(deployment.getStringList("routees.paths"))
|
||||
val nrOfInstances = deployment.getInt("nr-of-instances")
|
||||
val resizer = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None
|
||||
protected def createRouterConfig(routerType: String, key: String, config: Config, deployment: Config): RouterConfig =
|
||||
if (routerType == "from-code") NoRouter
|
||||
else {
|
||||
// need this for backwards compatibility, resizer enabled when including (parts of) resizer section in the deployment
|
||||
val deployment2 =
|
||||
if (config.hasPath("resizer") && !deployment.getBoolean("resizer.enabled"))
|
||||
resizerEnabled.withFallback(deployment)
|
||||
else deployment
|
||||
|
||||
routerType match {
|
||||
case "from-code" ⇒ NoRouter
|
||||
case "round-robin" ⇒ RoundRobinRouter(nrOfInstances, routees, resizer)
|
||||
case "random" ⇒ RandomRouter(nrOfInstances, routees, resizer)
|
||||
case "smallest-mailbox" ⇒ SmallestMailboxRouter(nrOfInstances, routees, resizer)
|
||||
case "broadcast" ⇒ BroadcastRouter(nrOfInstances, routees, resizer)
|
||||
case "scatter-gather" ⇒
|
||||
val within = Duration(deployment.getMilliseconds("within"), TimeUnit.MILLISECONDS)
|
||||
ScatterGatherFirstCompletedRouter(nrOfInstances, routees, within, resizer)
|
||||
case "consistent-hashing" ⇒
|
||||
val vnodes = deployment.getInt("virtual-nodes-factor")
|
||||
ConsistentHashingRouter(nrOfInstances, routees, resizer, virtualNodesFactor = vnodes)
|
||||
case fqn ⇒
|
||||
val args = List(classOf[Config] -> deployment)
|
||||
dynamicAccess.createInstanceFor[RouterConfig](fqn, args).recover({
|
||||
case exception ⇒ throw new IllegalArgumentException(
|
||||
("Cannot instantiate router [%s], defined in [%s], " +
|
||||
"make sure it extends [akka.routing.RouterConfig] and has constructor with " +
|
||||
"[com.typesafe.config.Config] parameter")
|
||||
.format(fqn, key), exception)
|
||||
}).get
|
||||
val fqn = routerTypeMapping.getOrElse(routerType,
|
||||
if (deployment.getStringList("routees.paths").isEmpty())
|
||||
routerTypeMapping.getOrElse(routerType + "-pool", routerType)
|
||||
else
|
||||
routerTypeMapping.getOrElse(routerType + "-group", routerType))
|
||||
|
||||
def throwCannotInstantiateRouter(args: Seq[(Class[_], AnyRef)], cause: Throwable) =
|
||||
throw new IllegalArgumentException(
|
||||
s"Cannot instantiate router [$fqn], defined in [$key], " +
|
||||
s"make sure it extends [${classOf[RouterConfig]}] and has constructor with " +
|
||||
s"[${args(0)._1.getName}] and optional [${args(1)._1.getName}] parameter", cause)
|
||||
|
||||
// first try with Config param, and then with Config and DynamicAccess parameters
|
||||
val args1 = List(classOf[Config] -> deployment2)
|
||||
val args2 = List(classOf[Config] -> deployment2, classOf[DynamicAccess] -> dynamicAccess)
|
||||
dynamicAccess.createInstanceFor[RouterConfig](fqn, args1).recover({
|
||||
case e @ (_: IllegalArgumentException | _: ConfigException) ⇒ throw e
|
||||
case e: NoSuchMethodException ⇒
|
||||
dynamicAccess.createInstanceFor[RouterConfig](fqn, args2).recover({
|
||||
case e @ (_: IllegalArgumentException | _: ConfigException) ⇒ throw e
|
||||
case e2 ⇒ throwCannotInstantiateRouter(args2, e)
|
||||
}).get
|
||||
case e ⇒ throwCannotInstantiateRouter(args2, e)
|
||||
}).get
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -80,7 +80,7 @@ private[io] object SelectionHandler {
|
|||
override def supervisorStrategy = connectionSupervisorStrategy
|
||||
|
||||
val selectorPool = context.actorOf(
|
||||
props = Props(classOf[SelectionHandler], selectorSettings).withRouter(RandomRouter(nrOfSelectors)).withDeploy(Deploy.local),
|
||||
props = RandomRouter(nrOfSelectors).props(Props(classOf[SelectionHandler], selectorSettings)).withDeploy(Deploy.local),
|
||||
name = "selectors")
|
||||
|
||||
final def workerForCommandHandler(pf: PartialFunction[HasFailureMessage, ChannelRegistry ⇒ Props]): Receive = {
|
||||
|
|
|
|||
|
|
@ -210,4 +210,10 @@ object Util {
|
|||
}
|
||||
|
||||
def immutableSingletonSeq[T](value: T): immutable.Seq[T] = value :: Nil
|
||||
|
||||
/**
|
||||
* Turns an [[java.lang.Iterable]] into an immutable Scala IndexedSeq (by copying it).
|
||||
*/
|
||||
def immutableIndexedSeq[T](iterable: java.lang.Iterable[T]): immutable.IndexedSeq[T] =
|
||||
immutableSeq(iterable).toVector
|
||||
}
|
||||
|
|
|
|||
141
akka-actor/src/main/scala/akka/routing/Broadcast.scala
Normal file
141
akka-actor/src/main/scala/akka/routing/Broadcast.scala
Normal file
|
|
@ -0,0 +1,141 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.Props
|
||||
import akka.dispatch.Dispatchers
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
object BroadcastRoutingLogic {
|
||||
def apply(): BroadcastRoutingLogic = new BroadcastRoutingLogic
|
||||
}
|
||||
|
||||
/**
|
||||
* Broadcasts a message to all its routees.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final class BroadcastRoutingLogic extends RoutingLogic {
|
||||
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
|
||||
if (routees.isEmpty) NoRoutee
|
||||
else SeveralRoutees(routees)
|
||||
}
|
||||
|
||||
/**
|
||||
* A router pool that broadcasts a message to all its routees.
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `nrOfInstances` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param nrOfInstances initial number of routees in the pool
|
||||
*
|
||||
* @param resizer optional resizer that dynamically adjust the pool size
|
||||
*
|
||||
* @param supervisorStrategy strategy for supervising the routees, see 'Supervision Setup'
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* supervision, death watch and router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class BroadcastPool(
|
||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Pool with PoolOverrideUnsetConfig[BroadcastPool] {
|
||||
|
||||
def this(config: Config) =
|
||||
this(
|
||||
nrOfInstances = config.getInt("nr-of-instances"),
|
||||
resizer = DefaultResizer.fromConfig(config))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param nr initial number of routees in the pool
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(BroadcastRoutingLogic())
|
||||
|
||||
/**
|
||||
* Setting the supervisor strategy to be used for the “head” Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): BroadcastPool = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): BroadcastPool = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* supervision, death watch and router management messages.
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): BroadcastPool = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* A router group that broadcasts a message to all its routees.
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `paths` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* @param paths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class BroadcastGroup(
|
||||
paths: immutable.Iterable[String],
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Group {
|
||||
|
||||
def this(config: Config) =
|
||||
this(paths = immutableSeq(config.getStringList("routees.paths")))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param routeePaths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(paths = immutableSeq(routeePaths))
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(BroadcastRoutingLogic())
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): BroadcastGroup = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
}
|
||||
568
akka-actor/src/main/scala/akka/routing/ConsistentHashing.scala
Normal file
568
akka-actor/src/main/scala/akka/routing/ConsistentHashing.scala
Normal file
|
|
@ -0,0 +1,568 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.Props
|
||||
import akka.dispatch.Dispatchers
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.actor.Address
|
||||
import akka.actor.ExtendedActorSystem
|
||||
import akka.actor.ActorSystem
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import akka.actor.ActorRef
|
||||
import akka.serialization.SerializationExtension
|
||||
import scala.util.control.NonFatal
|
||||
import akka.event.Logging
|
||||
|
||||
object ConsistentHashingRouter {
|
||||
|
||||
/**
|
||||
* If you don't define the `hashMapping` when
|
||||
* constructing the [[akka.routing.ConsistentHashingRouter]]
|
||||
* the messages need to implement this interface to define what
|
||||
* data to use for the consistent hash key. Note that it's not
|
||||
* the hash, but the data to be hashed.
|
||||
*
|
||||
* If returning an `Array[Byte]` or String it will be used as is,
|
||||
* otherwise the configured [[akka.serialization.Serializer]]
|
||||
* will be applied to the returned data.
|
||||
*
|
||||
* If messages can't implement this interface themselves,
|
||||
* it's possible to wrap the messages in
|
||||
* [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]],
|
||||
* or use [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]]
|
||||
*/
|
||||
trait ConsistentHashable {
|
||||
def consistentHashKey: Any
|
||||
}
|
||||
|
||||
/**
|
||||
* If you don't define the `hashMapping` when
|
||||
* constructing the [[akka.routing.ConsistentHashingRouter]]
|
||||
* and messages can't implement [[akka.routing.ConsistentHashingRouter.ConsistentHashable]]
|
||||
* themselves they can we wrapped by this envelope instead. The
|
||||
* router will only send the wrapped message to the destination,
|
||||
* i.e. the envelope will be stripped off.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class ConsistentHashableEnvelope(message: Any, hashKey: Any)
|
||||
extends ConsistentHashable with RouterEnvelope {
|
||||
override def consistentHashKey: Any = hashKey
|
||||
}
|
||||
|
||||
/**
|
||||
* Partial function from message to the data to
|
||||
* use for the consistent hash key. Note that it's not
|
||||
* the hash that is to be returned, but the data to be hashed.
|
||||
*
|
||||
* If returning an `Array[Byte]` or String it will be used as is,
|
||||
* otherwise the configured [[akka.serialization.Serializer]]
|
||||
* will be applied to the returned data.
|
||||
*/
|
||||
type ConsistentHashMapping = PartialFunction[Any, Any]
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
object emptyConsistentHashMapping extends ConsistentHashMapping {
|
||||
def isDefinedAt(x: Any) = false
|
||||
def apply(x: Any) = throw new UnsupportedOperationException("Empty ConsistentHashMapping apply()")
|
||||
}
|
||||
|
||||
/**
|
||||
* JAVA API
|
||||
* Mapping from message to the data to use for the consistent hash key.
|
||||
* Note that it's not the hash that is to be returned, but the data to be
|
||||
* hashed.
|
||||
*
|
||||
* May return `null` to indicate that the message is not handled by
|
||||
* this mapping.
|
||||
*
|
||||
* If returning an `Array[Byte]` or String it will be used as is,
|
||||
* otherwise the configured [[akka.serialization.Serializer]]
|
||||
* will be applied to the returned data.
|
||||
*/
|
||||
trait ConsistentHashMapper {
|
||||
def hashKey(message: Any): Any
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def hashMappingAdapter(mapper: ConsistentHashMapper): ConsistentHashMapping = {
|
||||
case message if (mapper.hashKey(message).asInstanceOf[AnyRef] ne null) ⇒
|
||||
mapper.hashKey(message)
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates a new ConsistentHashingRouter, routing to the specified routees
|
||||
*/
|
||||
@deprecated("Use ConsistentHashingGroup", "2.3")
|
||||
def apply(routees: immutable.Iterable[ActorRef]): ConsistentHashingRouter =
|
||||
new ConsistentHashingRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
@deprecated("Use ConsistentHashingGroup", "2.3")
|
||||
def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = apply(immutableSeq(routees))
|
||||
|
||||
}
|
||||
|
||||
object ConsistentHashingRoutingLogic {
|
||||
/**
|
||||
* Address to use for the selfAddress parameter
|
||||
*/
|
||||
def defaultAddress(system: ActorSystem): Address =
|
||||
system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
|
||||
}
|
||||
|
||||
/**
|
||||
* Uses consistent hashing to select a routee based on the sent message.
|
||||
*
|
||||
* There is 3 ways to define what data to use for the consistent hash key.
|
||||
*
|
||||
* 1. You can define `hashMapping` / `withHashMapper`
|
||||
* of the router to map incoming messages to their consistent hash key.
|
||||
* This makes the decision transparent for the sender.
|
||||
*
|
||||
* 2. The messages may implement [[akka.routing.ConsistentHashingRouter.ConsistentHashable]].
|
||||
* The key is part of the message and it's convenient to define it together
|
||||
* with the message definition.
|
||||
*
|
||||
* 3. The messages can be be wrapped in a [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]]
|
||||
* to define what data to use for the consistent hash key. The sender knows
|
||||
* the key to use.
|
||||
*
|
||||
* These ways to define the consistent hash key can be use together and at
|
||||
* the same time for one router. The `hashMapping` is tried first.
|
||||
*
|
||||
* @param virtualNodesFactor number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
*
|
||||
* @param hashMapping partial function from message to the data to
|
||||
* use for the consistent hash key
|
||||
*
|
||||
* @param system the actor system hosting this router
|
||||
*
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class ConsistentHashingRoutingLogic(
|
||||
system: ActorSystem,
|
||||
virtualNodesFactor: Int = 0,
|
||||
hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping)
|
||||
extends RoutingLogic {
|
||||
|
||||
import ConsistentHashingRouter._
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param system the actor system hosting this router
|
||||
*/
|
||||
def this(system: ActorSystem) =
|
||||
this(system, virtualNodesFactor = 0, hashMapping = ConsistentHashingRouter.emptyConsistentHashMapping)
|
||||
|
||||
private val selfAddress = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
|
||||
val vnodes =
|
||||
if (virtualNodesFactor == 0) system.settings.DefaultVirtualNodesFactor
|
||||
else virtualNodesFactor
|
||||
|
||||
private lazy val log = Logging(system, getClass)
|
||||
|
||||
/**
|
||||
* Setting the number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
*/
|
||||
def withVirtualNodesFactor(vnodes: Int): ConsistentHashingRoutingLogic = copy(virtualNodesFactor = vnodes)
|
||||
|
||||
/**
|
||||
* Java API: Setting the mapping from message to the data to use for the consistent hash key.
|
||||
*/
|
||||
def withHashMapper(mapper: ConsistentHashingRouter.ConsistentHashMapper): ConsistentHashingRoutingLogic =
|
||||
copy(hashMapping = ConsistentHashingRouter.hashMappingAdapter(mapper))
|
||||
|
||||
// tuple of routees and the ConsistentHash, updated together in updateConsistentHash
|
||||
private val consistentHashRef = new AtomicReference[(immutable.IndexedSeq[Routee], ConsistentHash[ConsistentRoutee])]((null, null))
|
||||
|
||||
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
|
||||
if (routees.isEmpty) NoRoutee
|
||||
else {
|
||||
|
||||
// update consistentHash when routees has changed
|
||||
// changes to routees are rare and when no changes this is a quick operation
|
||||
def updateConsistentHash(): ConsistentHash[ConsistentRoutee] = {
|
||||
val oldConsistentHashTuple = consistentHashRef.get
|
||||
val (oldRoutees, oldConsistentHash) = oldConsistentHashTuple
|
||||
|
||||
if (routees ne oldRoutees) {
|
||||
// when other instance, same content, no need to re-hash, but try to set routees
|
||||
val consistentHash =
|
||||
if (routees == oldRoutees) oldConsistentHash
|
||||
else ConsistentHash(routees.map(ConsistentRoutee(_, selfAddress)), vnodes) // re-hash
|
||||
// ignore, don't update, in case of CAS failure
|
||||
consistentHashRef.compareAndSet(oldConsistentHashTuple, (routees, consistentHash))
|
||||
consistentHash
|
||||
} else oldConsistentHash
|
||||
}
|
||||
|
||||
def target(hashData: Any): Routee = try {
|
||||
val currentConsistenHash = updateConsistentHash()
|
||||
if (currentConsistenHash.isEmpty) NoRoutee
|
||||
else hashData match {
|
||||
case bytes: Array[Byte] ⇒ currentConsistenHash.nodeFor(bytes).routee
|
||||
case str: String ⇒ currentConsistenHash.nodeFor(str).routee
|
||||
case x: AnyRef ⇒ currentConsistenHash.nodeFor(SerializationExtension(system).serialize(x).get).routee
|
||||
}
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
// serialization failed
|
||||
log.warning("Couldn't route message with consistent hash key [{}] due to [{}]", hashData, e.getMessage)
|
||||
NoRoutee
|
||||
}
|
||||
|
||||
message match {
|
||||
case _ if hashMapping.isDefinedAt(message) ⇒ target(hashMapping(message))
|
||||
case hashable: ConsistentHashable ⇒ target(hashable.consistentHashKey)
|
||||
case other ⇒
|
||||
log.warning("Message [{}] must be handled by hashMapping, or implement [{}] or be wrapped in [{}]",
|
||||
message.getClass.getName, classOf[ConsistentHashable].getName,
|
||||
classOf[ConsistentHashableEnvelope].getName)
|
||||
NoRoutee
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* A router pool that uses consistent hashing to select a routee based on the
|
||||
* sent message. The selection is described in [[akka.routing.ConsistentHashingRoutingLogic]].
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `nrOfInstances` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param nrOfInstances initial number of routees in the pool
|
||||
*
|
||||
* @param resizer optional resizer that dynamically adjust the pool size
|
||||
*
|
||||
* @param virtualNodesFactor number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
*
|
||||
* @param hashMapping partial function from message to the data to
|
||||
* use for the consistent hash key
|
||||
*
|
||||
* @param supervisorStrategy strategy for supervising the routees, see 'Supervision Setup'
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* supervision, death watch and router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class ConsistentHashingPool(
|
||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||
val virtualNodesFactor: Int = 0,
|
||||
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Pool with PoolOverrideUnsetConfig[ConsistentHashingPool] {
|
||||
|
||||
def this(config: Config) =
|
||||
this(
|
||||
nrOfInstances = config.getInt("nr-of-instances"),
|
||||
resizer = DefaultResizer.fromConfig(config))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param nr initial number of routees in the pool
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router =
|
||||
new Router(ConsistentHashingRoutingLogic(system, virtualNodesFactor, hashMapping))
|
||||
|
||||
/**
|
||||
* Setting the supervisor strategy to be used for the “head” Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): ConsistentHashingPool = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): ConsistentHashingPool = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* supervision, death watch and router management messages.
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): ConsistentHashingPool = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Setting the number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
*/
|
||||
def withVirtualNodesFactor(vnodes: Int): ConsistentHashingPool = copy(virtualNodesFactor = vnodes)
|
||||
|
||||
/**
|
||||
* Java API: Setting the mapping from message to the data to use for the consistent hash key.
|
||||
*/
|
||||
def withHashMapper(mapper: ConsistentHashingRouter.ConsistentHashMapper): ConsistentHashingPool =
|
||||
copy(hashMapping = ConsistentHashingRouter.hashMappingAdapter(mapper))
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
* Uses the the `hashMapping` defined in code, since that can't be defined in configuration.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||
case _: FromConfig | _: NoRouter ⇒ this.overrideUnsetConfig(other)
|
||||
case otherRouter: ConsistentHashingPool ⇒ (copy(hashMapping = otherRouter.hashMapping)).overrideUnsetConfig(other)
|
||||
case otherRouter: ConsistentHashingRouter ⇒ (copy(hashMapping = otherRouter.hashMapping)).overrideUnsetConfig(other)
|
||||
case _ ⇒ throw new IllegalArgumentException("Expected ConsistentHashingPool, got [%s]".format(other))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* A router group that uses consistent hashing to select a routee based on the
|
||||
* sent message. The selection is described in [[akka.routing.ConsistentHashingRoutingLogic]].
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `paths` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* @param paths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*
|
||||
* @param virtualNodesFactor number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
*
|
||||
* @param hashMapping partial function from message to the data to
|
||||
* use for the consistent hash key
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class ConsistentHashingGroup(
|
||||
override val paths: immutable.Iterable[String],
|
||||
val virtualNodesFactor: Int = 0,
|
||||
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Group {
|
||||
|
||||
def this(config: Config) =
|
||||
this(paths = immutableSeq(config.getStringList("routees.paths")))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param routeePaths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(paths = immutableSeq(routeePaths))
|
||||
|
||||
override def createRouter(system: ActorSystem): Router =
|
||||
new Router(ConsistentHashingRoutingLogic(system, virtualNodesFactor, hashMapping))
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): ConsistentHashingGroup = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Setting the number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
*/
|
||||
def withVirtualNodesFactor(vnodes: Int): ConsistentHashingGroup = copy(virtualNodesFactor = vnodes)
|
||||
|
||||
/**
|
||||
* Java API: Setting the mapping from message to the data to use for the consistent hash key.
|
||||
*/
|
||||
def withHashMapper(mapper: ConsistentHashingRouter.ConsistentHashMapper): ConsistentHashingGroup =
|
||||
copy(hashMapping = ConsistentHashingRouter.hashMappingAdapter(mapper))
|
||||
|
||||
/**
|
||||
* Uses the the `hashMapping` defined in code, since that can't be defined in configuration.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||
case _: FromConfig | _: NoRouter ⇒ super.withFallback(other)
|
||||
case otherRouter: ConsistentHashingGroup ⇒ copy(hashMapping = otherRouter.hashMapping)
|
||||
case _ ⇒ throw new IllegalArgumentException("Expected ConsistentHashingGroup, got [%s]".format(other))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Important to use ActorRef with full address, with host and port, in the hash ring,
|
||||
* so that same ring is produced on different nodes.
|
||||
* The ConsistentHash uses toString of the ring nodes, and the ActorRef itself
|
||||
* isn't a good representation, because LocalActorRef doesn't include the
|
||||
* host and port.
|
||||
*/
|
||||
private[akka] case class ConsistentRoutee(routee: Routee, selfAddress: Address) {
|
||||
|
||||
override def toString: String = routee match {
|
||||
case ActorRefRoutee(ref) ⇒ toStringWithfullAddress(ref)
|
||||
case ActorSelectionRoutee(sel) ⇒ toStringWithfullAddress(sel.anchor)
|
||||
case other ⇒ other.toString
|
||||
}
|
||||
|
||||
private def toStringWithfullAddress(ref: ActorRef): String = {
|
||||
ref.path.address match {
|
||||
case Address(_, _, None, None) ⇒ ref.path.toStringWithAddress(selfAddress)
|
||||
case a ⇒ ref.path.toString
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A Router that uses consistent hashing to select a connection based on the
|
||||
* sent message.
|
||||
*
|
||||
* There is 3 ways to define what data to use for the consistent hash key.
|
||||
*
|
||||
* 1. You can define `hashMapping` / `withHashMapper`
|
||||
* of the router to map incoming messages to their consistent hash key.
|
||||
* This makes the decision transparent for the sender.
|
||||
*
|
||||
* 2. The messages may implement [[akka.routing.ConsistentHashingRouter.ConsistentHashable]].
|
||||
* The key is part of the message and it's convenient to define it together
|
||||
* with the message definition.
|
||||
*
|
||||
* 3. The messages can be be wrapped in a [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]]
|
||||
* to define what data to use for the consistent hash key. The sender knows
|
||||
* the key to use.
|
||||
*
|
||||
* These ways to define the consistent hash key can be use together and at
|
||||
* the same time for one router. The `hashMapping` is tried first.
|
||||
*
|
||||
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical
|
||||
* sense as this means that the router should both create new actors and use the 'routees'
|
||||
* actor(s). In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
|
||||
* <br>
|
||||
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
|
||||
* be ignored if the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param routees string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
* @param virtualNodesFactor number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
* @param hashMapping partial function from message to the data to
|
||||
* use for the consistent hash key
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
@deprecated("Use ConsistentHashingPool or ConsistentHashingGroup", "2.3")
|
||||
case class ConsistentHashingRouter(
|
||||
nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
val virtualNodesFactor: Int = 0,
|
||||
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping)
|
||||
extends DeprecatedRouterConfig with PoolOverrideUnsetConfig[ConsistentHashingRouter] {
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets nrOfInstances to be created.
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the routees to be used.
|
||||
*
|
||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the resizer to be used.
|
||||
*/
|
||||
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||
|
||||
override def paths: immutable.Iterable[String] = routees
|
||||
|
||||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): ConsistentHashingRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): ConsistentHashingRouter = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API for setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): ConsistentHashingRouter = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Java API for setting the number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
*/
|
||||
def withVirtualNodesFactor(vnodes: Int): ConsistentHashingRouter = copy(virtualNodesFactor = vnodes)
|
||||
|
||||
/**
|
||||
* Java API for setting the mapping from message to the data to use for the consistent hash key.
|
||||
*/
|
||||
def withHashMapper(mapping: ConsistentHashingRouter.ConsistentHashMapper) =
|
||||
copy(hashMapping = ConsistentHashingRouter.hashMappingAdapter(mapping))
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
* Uses the the `hashMapping` defined in code, since that can't be defined in configuration.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||
case _: FromConfig | _: NoRouter ⇒ this.overrideUnsetConfig(other)
|
||||
case otherRouter: ConsistentHashingRouter ⇒ (copy(hashMapping = otherRouter.hashMapping)).overrideUnsetConfig(other)
|
||||
case _ ⇒ throw new IllegalArgumentException("Expected ConsistentHashingRouter, got [%s]".format(other))
|
||||
}
|
||||
|
||||
override def createRouter(system: ActorSystem): Router =
|
||||
new Router(ConsistentHashingRoutingLogic(system, virtualNodesFactor, hashMapping))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Important to use ActorRef with full address, with host and port, in the hash ring,
|
||||
* so that same ring is produced on different nodes.
|
||||
* The ConsistentHash uses toString of the ring nodes, and the ActorRef itself
|
||||
* isn't a good representation, because LocalActorRef doesn't include the
|
||||
* host and port.
|
||||
*/
|
||||
@deprecated("Replaced by ConsistentRoutee", "2.3")
|
||||
private[akka] case class ConsistentActorRef(actorRef: ActorRef, selfAddress: Address) {
|
||||
override def toString: String = {
|
||||
actorRef.path.address match {
|
||||
case Address(_, _, None, None) ⇒ actorRef.path.toStringWithAddress(selfAddress)
|
||||
case a ⇒ actorRef.path.toString
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1,317 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.japi.Util.immutableSeq
|
||||
import scala.util.control.NonFatal
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.event.Logging
|
||||
import akka.serialization.SerializationExtension
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
import akka.actor.Address
|
||||
import akka.actor.ExtendedActorSystem
|
||||
|
||||
object ConsistentHashingRouter {
|
||||
/**
|
||||
* Creates a new ConsistentHashingRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: immutable.Iterable[ActorRef]): ConsistentHashingRouter =
|
||||
new ConsistentHashingRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): ConsistentHashingRouter = apply(immutableSeq(routees))
|
||||
|
||||
/**
|
||||
* If you don't define the `hashMapping` when
|
||||
* constructing the [[akka.routing.ConsistentHashingRouter]]
|
||||
* the messages need to implement this interface to define what
|
||||
* data to use for the consistent hash key. Note that it's not
|
||||
* the hash, but the data to be hashed.
|
||||
*
|
||||
* If returning an `Array[Byte]` or String it will be used as is,
|
||||
* otherwise the configured [[akka.serialization.Serializer]]
|
||||
* will be applied to the returned data.
|
||||
*
|
||||
* If messages can't implement this interface themselves,
|
||||
* it's possible to wrap the messages in
|
||||
* [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]],
|
||||
* or use [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]]
|
||||
*/
|
||||
trait ConsistentHashable {
|
||||
def consistentHashKey: Any
|
||||
}
|
||||
|
||||
/**
|
||||
* If you don't define the `hashMapping` when
|
||||
* constructing the [[akka.routing.ConsistentHashingRouter]]
|
||||
* and messages can't implement [[akka.routing.ConsistentHashingRouter.ConsistentHashable]]
|
||||
* themselves they can we wrapped by this envelope instead. The
|
||||
* router will only send the wrapped message to the destination,
|
||||
* i.e. the envelope will be stripped off.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class ConsistentHashableEnvelope(message: Any, hashKey: Any)
|
||||
extends ConsistentHashable with RouterEnvelope {
|
||||
override def consistentHashKey: Any = hashKey
|
||||
}
|
||||
|
||||
/**
|
||||
* Partial function from message to the data to
|
||||
* use for the consistent hash key. Note that it's not
|
||||
* the hash that is to be returned, but the data to be hashed.
|
||||
*
|
||||
* If returning an `Array[Byte]` or String it will be used as is,
|
||||
* otherwise the configured [[akka.serialization.Serializer]]
|
||||
* will be applied to the returned data.
|
||||
*/
|
||||
type ConsistentHashMapping = PartialFunction[Any, Any]
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
object emptyConsistentHashMapping extends ConsistentHashMapping {
|
||||
def isDefinedAt(x: Any) = false
|
||||
def apply(x: Any) = throw new UnsupportedOperationException("Empty ConsistentHashMapping apply()")
|
||||
}
|
||||
|
||||
/**
|
||||
* JAVA API
|
||||
* Mapping from message to the data to use for the consistent hash key.
|
||||
* Note that it's not the hash that is to be returned, but the data to be
|
||||
* hashed.
|
||||
*
|
||||
* May return `null` to indicate that the message is not handled by
|
||||
* this mapping.
|
||||
*
|
||||
* If returning an `Array[Byte]` or String it will be used as is,
|
||||
* otherwise the configured [[akka.serialization.Serializer]]
|
||||
* will be applied to the returned data.
|
||||
*/
|
||||
trait ConsistentHashMapper {
|
||||
def hashKey(message: Any): Any
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A Router that uses consistent hashing to select a connection based on the
|
||||
* sent message.
|
||||
*
|
||||
* There is 3 ways to define what data to use for the consistent hash key.
|
||||
*
|
||||
* 1. You can define `hashMapping` / `withHashMapper`
|
||||
* of the router to map incoming messages to their consistent hash key.
|
||||
* This makes the decision transparent for the sender.
|
||||
*
|
||||
* 2. The messages may implement [[akka.routing.ConsistentHashingRouter.ConsistentHashable]].
|
||||
* The key is part of the message and it's convenient to define it together
|
||||
* with the message definition.
|
||||
*
|
||||
* 3. The messages can be be wrapped in a [[akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope]]
|
||||
* to define what data to use for the consistent hash key. The sender knows
|
||||
* the key to use.
|
||||
*
|
||||
* These ways to define the consistent hash key can be use together and at
|
||||
* the same time for one router. The `hashMapping` is tried first.
|
||||
*
|
||||
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical
|
||||
* sense as this means that the router should both create new actors and use the 'routees'
|
||||
* actor(s). In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
|
||||
* <br>
|
||||
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
|
||||
* be ignored if the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param routees string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
* @param virtualNodesFactor number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
* @param hashMapping partial function from message to the data to
|
||||
* use for the consistent hash key
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class ConsistentHashingRouter(
|
||||
nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy,
|
||||
val virtualNodesFactor: Int = 0,
|
||||
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping)
|
||||
extends RouterConfig with ConsistentHashingLike with OverrideUnsetConfig[ConsistentHashingRouter] {
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets nrOfInstances to be created.
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the routees to be used.
|
||||
*
|
||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the resizer to be used.
|
||||
*/
|
||||
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): ConsistentHashingRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): ConsistentHashingRouter = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API for setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): ConsistentHashingRouter = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Java API for setting the number of virtual nodes per node, used in [[akka.routing.ConsistentHash]]
|
||||
*/
|
||||
def withVirtualNodesFactor(vnodes: Int): ConsistentHashingRouter = copy(virtualNodesFactor = vnodes)
|
||||
|
||||
/**
|
||||
* Java API for setting the mapping from message to the data to use for the consistent hash key.
|
||||
*/
|
||||
def withHashMapper(mapping: ConsistentHashingRouter.ConsistentHashMapper) = {
|
||||
copy(hashMapping = {
|
||||
case message if (mapping.hashKey(message).asInstanceOf[AnyRef] ne null) ⇒
|
||||
mapping.hashKey(message)
|
||||
})
|
||||
}
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
* Uses the the `hashMapping` defined in code, since that can't be defined in configuration.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||
case _: FromConfig | _: NoRouter ⇒ this.overrideUnsetConfig(other)
|
||||
case otherRouter: ConsistentHashingRouter ⇒ (copy(hashMapping = otherRouter.hashMapping)).overrideUnsetConfig(other)
|
||||
case _ ⇒ throw new IllegalArgumentException("Expected ConsistentHashingRouter, got [%s]".format(other))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* The core pieces of the routing logic is located in this
|
||||
* trait to be able to extend.
|
||||
*/
|
||||
trait ConsistentHashingLike { this: RouterConfig ⇒
|
||||
|
||||
import ConsistentHashingRouter._
|
||||
|
||||
def nrOfInstances: Int
|
||||
|
||||
def routees: immutable.Iterable[String]
|
||||
|
||||
def virtualNodesFactor: Int
|
||||
|
||||
def hashMapping: ConsistentHashMapping
|
||||
|
||||
override def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
if (resizer.isEmpty) {
|
||||
if (routees.isEmpty) routeeProvider.createRoutees(nrOfInstances)
|
||||
else routeeProvider.registerRouteesFor(routees)
|
||||
}
|
||||
|
||||
val log = Logging(routeeProvider.context.system, routeeProvider.context.self)
|
||||
val selfAddress = routeeProvider.context.system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress
|
||||
val vnodes =
|
||||
if (virtualNodesFactor == 0) routeeProvider.context.system.settings.DefaultVirtualNodesFactor
|
||||
else virtualNodesFactor
|
||||
|
||||
// tuple of routees and the ConsistentHash, updated together in updateConsistentHash
|
||||
val consistentHashRef = new AtomicReference[(IndexedSeq[ConsistentActorRef], ConsistentHash[ConsistentActorRef])]((null, null))
|
||||
updateConsistentHash()
|
||||
|
||||
// update consistentHash when routees has changed
|
||||
// changes to routees are rare and when no changes this is a quick operation
|
||||
def updateConsistentHash(): ConsistentHash[ConsistentActorRef] = {
|
||||
val oldConsistentHashTuple = consistentHashRef.get
|
||||
val (oldConsistentHashRoutees, oldConsistentHash) = oldConsistentHashTuple
|
||||
val currentRoutees = routeeProvider.routees map { ConsistentActorRef(_, selfAddress) }
|
||||
|
||||
if (currentRoutees ne oldConsistentHashRoutees) {
|
||||
// when other instance, same content, no need to re-hash, but try to set routees
|
||||
val consistentHash =
|
||||
if (currentRoutees == oldConsistentHashRoutees) oldConsistentHash
|
||||
else ConsistentHash(currentRoutees, vnodes) // re-hash
|
||||
// ignore, don't update, in case of CAS failure
|
||||
consistentHashRef.compareAndSet(oldConsistentHashTuple, (currentRoutees, consistentHash))
|
||||
consistentHash
|
||||
} else oldConsistentHash
|
||||
}
|
||||
|
||||
def target(hashData: Any): ActorRef = try {
|
||||
val currentConsistenHash = updateConsistentHash()
|
||||
if (currentConsistenHash.isEmpty) routeeProvider.context.system.deadLetters
|
||||
else hashData match {
|
||||
case bytes: Array[Byte] ⇒ currentConsistenHash.nodeFor(bytes).actorRef
|
||||
case str: String ⇒ currentConsistenHash.nodeFor(str).actorRef
|
||||
case x: AnyRef ⇒ currentConsistenHash.nodeFor(SerializationExtension(routeeProvider.context.system).serialize(x).get).actorRef
|
||||
}
|
||||
} catch {
|
||||
case NonFatal(e) ⇒
|
||||
// serialization failed
|
||||
log.warning("Couldn't route message with consistent hash key [{}] due to [{}]", hashData, e.getMessage)
|
||||
routeeProvider.context.system.deadLetters
|
||||
}
|
||||
|
||||
{
|
||||
case (sender, message) ⇒
|
||||
message match {
|
||||
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
||||
case _ if hashMapping.isDefinedAt(message) ⇒
|
||||
List(Destination(sender, target(hashMapping(message))))
|
||||
case hashable: ConsistentHashable ⇒ List(Destination(sender, target(hashable.consistentHashKey)))
|
||||
case other ⇒
|
||||
log.warning("Message [{}] must be handled by hashMapping, or implement [{}] or be wrapped in [{}]",
|
||||
message.getClass.getName, classOf[ConsistentHashable].getName,
|
||||
classOf[ConsistentHashableEnvelope].getName)
|
||||
List(Destination(sender, routeeProvider.context.system.deadLetters))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Important to use ActorRef with full address, with host and port, in the hash ring,
|
||||
* so that same ring is produced on different nodes.
|
||||
* The ConsistentHash uses toString of the ring nodes, and the ActorRef itself
|
||||
* isn't a good representation, because LocalActorRef doesn't include the
|
||||
* host and port.
|
||||
*/
|
||||
private[akka] case class ConsistentActorRef(actorRef: ActorRef, selfAddress: Address) {
|
||||
override def toString: String = {
|
||||
actorRef.path.address match {
|
||||
case Address(_, _, None, None) ⇒ actorRef.path.toStringWithAddress(selfAddress)
|
||||
case a ⇒ actorRef.path.toString
|
||||
}
|
||||
}
|
||||
}
|
||||
546
akka-actor/src/main/scala/akka/routing/DeprecatedRouting.scala
Normal file
546
akka-actor/src/main/scala/akka/routing/DeprecatedRouting.scala
Normal file
|
|
@ -0,0 +1,546 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import language.implicitConversions
|
||||
import language.postfixOps
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor._
|
||||
import akka.ConfigurationException
|
||||
import akka.dispatch.{ Envelope, Dispatchers }
|
||||
import akka.pattern.pipe
|
||||
import akka.japi.Util.immutableSeq
|
||||
import com.typesafe.config.Config
|
||||
import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.event.Logging.Warning
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import scala.annotation.tailrec
|
||||
import akka.event.Logging.Warning
|
||||
import akka.dispatch.{ MailboxType, MessageDispatcher }
|
||||
import akka.dispatch.BalancingDispatcher
|
||||
|
||||
/**
|
||||
* Sending this message to a router will make it send back its currently used routees.
|
||||
* A RouterRoutees message is sent asynchronously to the "requester" containing information
|
||||
* about what routees the router is routing over.
|
||||
*/
|
||||
@deprecated("Use GetRoutees", "2.3")
|
||||
@SerialVersionUID(1L) abstract class CurrentRoutees extends RouterManagementMesssage
|
||||
|
||||
@deprecated("Use GetRoutees", "2.3")
|
||||
@SerialVersionUID(1L) case object CurrentRoutees extends CurrentRoutees {
|
||||
/**
|
||||
* Java API: get the singleton instance
|
||||
*/
|
||||
def getInstance = this
|
||||
}
|
||||
|
||||
/**
|
||||
* Message used to carry information about what routees the router is currently using.
|
||||
*/
|
||||
@deprecated("Use GetRoutees", "2.3")
|
||||
@SerialVersionUID(1L)
|
||||
case class RouterRoutees(routees: immutable.IndexedSeq[ActorRef]) {
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def getRoutees: java.util.List[ActorRef] = {
|
||||
import scala.collection.JavaConverters._
|
||||
routees.asJava
|
||||
}
|
||||
}
|
||||
|
||||
@deprecated("Use Pool or Group", "2.3")
|
||||
trait DeprecatedRouterConfig extends Group with Pool
|
||||
|
||||
@deprecated("Use RoundRobinPool or RoundRobinGroup", "2.3")
|
||||
object RoundRobinRouter {
|
||||
/**
|
||||
* Creates a new RoundRobinRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: immutable.Iterable[ActorRef]): RoundRobinRouter =
|
||||
new RoundRobinRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter =
|
||||
apply(immutableSeq(routees))
|
||||
}
|
||||
/**
|
||||
* 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 'routees' does not make logical sense as this means
|
||||
* that the router should both create new actors and use the 'routees' actor(s).
|
||||
* In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
|
||||
* <br>
|
||||
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
|
||||
* be ignored if the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param routees string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
@deprecated("Use RoundRobinPool or RoundRobinGroup", "2.3")
|
||||
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy)
|
||||
extends DeprecatedRouterConfig with PoolOverrideUnsetConfig[RoundRobinRouter] {
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets nrOfInstances to be created.
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the routees to be used.
|
||||
*
|
||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the resizer to be used.
|
||||
*/
|
||||
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||
|
||||
override def paths: immutable.Iterable[String] = routees
|
||||
|
||||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): RoundRobinRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): RoundRobinRouter = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API for setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): RoundRobinRouter = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(RoundRobinRoutingLogic())
|
||||
}
|
||||
|
||||
@deprecated("Use RandomPool or RandomGroup", "2.3")
|
||||
object RandomRouter {
|
||||
/**
|
||||
* Creates a new RandomRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: immutable.Iterable[ActorRef]): RandomRouter = new RandomRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): RandomRouter =
|
||||
apply(immutableSeq(routees))
|
||||
}
|
||||
/**
|
||||
* A Router that randomly selects one of the target connections to send a message to.
|
||||
* <br>
|
||||
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical sense as this means
|
||||
* that the router should both create new actors and use the 'routees' actor(s).
|
||||
* In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
|
||||
* <br>
|
||||
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
|
||||
* be ignored if the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param routees string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
@deprecated("Use RandomPool or RandomGroup", "2.3")
|
||||
case class RandomRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy)
|
||||
extends DeprecatedRouterConfig with PoolOverrideUnsetConfig[RandomRouter] {
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets nrOfInstances to be created.
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the routees to be used.
|
||||
*
|
||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the resizer to be used.
|
||||
*/
|
||||
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||
|
||||
override def paths: immutable.Iterable[String] = routees
|
||||
|
||||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): RandomRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): RandomRouter = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API for setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): RandomRouter = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(RandomRoutingLogic())
|
||||
}
|
||||
|
||||
@deprecated("Use SmallestMailboxPool", "2.3")
|
||||
object SmallestMailboxRouter {
|
||||
/**
|
||||
* Creates a new SmallestMailboxRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: immutable.Iterable[ActorRef]): SmallestMailboxRouter =
|
||||
new SmallestMailboxRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): SmallestMailboxRouter =
|
||||
apply(immutableSeq(routees))
|
||||
}
|
||||
/**
|
||||
* A Router that tries to send to the non-suspended routee with fewest messages in mailbox.
|
||||
* The selection is done in this order:
|
||||
* <ul>
|
||||
* <li>pick any idle routee (not processing message) with empty mailbox</li>
|
||||
* <li>pick any routee with empty mailbox</li>
|
||||
* <li>pick routee with fewest pending messages in mailbox</li>
|
||||
* <li>pick any remote routee, remote actors are consider lowest priority,
|
||||
* since their mailbox size is unknown</li>
|
||||
* </ul>
|
||||
*
|
||||
* <br>
|
||||
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical sense as this means
|
||||
* that the router should both create new actors and use the 'routees' actor(s).
|
||||
* In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
|
||||
* <br>
|
||||
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
|
||||
* be ignored if the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param routees string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
@deprecated("Use SmallestMailboxPool", "2.3")
|
||||
case class SmallestMailboxRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy)
|
||||
extends DeprecatedRouterConfig with PoolOverrideUnsetConfig[SmallestMailboxRouter] {
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets nrOfInstances to be created.
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the routees to be used.
|
||||
*
|
||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the resizer to be used.
|
||||
*/
|
||||
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||
|
||||
override def paths: immutable.Iterable[String] = routees
|
||||
|
||||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): SmallestMailboxRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): SmallestMailboxRouter = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API for setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): SmallestMailboxRouter = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(SmallestMailboxRoutingLogic())
|
||||
}
|
||||
|
||||
@deprecated("Use BroadcastPool or BroadcastGroup", "2.3")
|
||||
object BroadcastRouter {
|
||||
/**
|
||||
* Creates a new BroadcastRouter, routing to the specified routees
|
||||
*/
|
||||
def apply(routees: immutable.Iterable[ActorRef]): BroadcastRouter = new BroadcastRouter(routees = routees map (_.path.toString))
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter =
|
||||
apply(immutableSeq(routees))
|
||||
}
|
||||
/**
|
||||
* A Router that uses broadcasts a message to all its connections.
|
||||
* <br>
|
||||
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical sense as this means
|
||||
* that the router should both create new actors and use the 'routees' actor(s).
|
||||
* In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
|
||||
* <br>
|
||||
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
|
||||
* be ignored if the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param routees string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
@deprecated("Use BroadcastPool or BroadcastGroup", "2.3")
|
||||
case class BroadcastRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy)
|
||||
extends DeprecatedRouterConfig with PoolOverrideUnsetConfig[BroadcastRouter] {
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets nrOfInstances to be created.
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the routees to be used.
|
||||
*
|
||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the resizer to be used.
|
||||
*/
|
||||
def this(resizer: Resizer) = this(resizer = Some(resizer))
|
||||
|
||||
override def paths: immutable.Iterable[String] = routees
|
||||
|
||||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): BroadcastRouter = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): BroadcastRouter = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API for setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): BroadcastRouter = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(BroadcastRoutingLogic())
|
||||
}
|
||||
|
||||
@deprecated("Use ScatterGatherFirstCompletedPool or ScatterGatherFirstCompletedGroup", "2.3")
|
||||
object ScatterGatherFirstCompletedRouter {
|
||||
/**
|
||||
* Creates a new ScatterGatherFirstCompletedRouter, routing to the specified routees, timing out after the specified Duration
|
||||
*/
|
||||
def apply(routees: immutable.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
|
||||
new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
|
||||
|
||||
/**
|
||||
* Java API to create router with the supplied 'routees' actors.
|
||||
*/
|
||||
def create(routees: java.lang.Iterable[ActorRef], within: FiniteDuration): ScatterGatherFirstCompletedRouter =
|
||||
apply(immutableSeq(routees), within)
|
||||
}
|
||||
/**
|
||||
* Simple router that broadcasts the message to all routees, and replies with the first response.
|
||||
* <br/>
|
||||
* You have to defin the 'within: Duration' parameter (f.e: within = 10 seconds).
|
||||
* <br/>
|
||||
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical sense as this means
|
||||
* that the router should both create new actors and use the 'routees' actor(s).
|
||||
* In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
|
||||
* <br/>
|
||||
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
|
||||
* be ignored if the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param routees string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
@deprecated("Use ScatterGatherFirstCompletedPool or ScatterGatherFirstCompletedGroup", "2.3")
|
||||
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil, within: FiniteDuration,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy)
|
||||
extends DeprecatedRouterConfig with PoolOverrideUnsetConfig[ScatterGatherFirstCompletedRouter] {
|
||||
|
||||
if (within <= Duration.Zero) throw new IllegalArgumentException(
|
||||
"[within: Duration] can not be zero or negative, was [" + within + "]")
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets nrOfInstances to be created.
|
||||
*/
|
||||
def this(nr: Int, w: FiniteDuration) = this(nrOfInstances = nr, within = w)
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the routees to be used.
|
||||
*
|
||||
* @param routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String], w: FiniteDuration) = this(routees = immutableSeq(routeePaths), within = w)
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the resizer to be used.
|
||||
*/
|
||||
def this(resizer: Resizer, w: FiniteDuration) = this(resizer = Some(resizer), within = w)
|
||||
|
||||
override def paths: immutable.Iterable[String] = routees
|
||||
|
||||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String) = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): ScatterGatherFirstCompletedRouter = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API for setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): ScatterGatherFirstCompletedRouter = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(SmallestMailboxRoutingLogic())
|
||||
}
|
||||
|
||||
142
akka-actor/src/main/scala/akka/routing/Random.scala
Normal file
142
akka-actor/src/main/scala/akka/routing/Random.scala
Normal file
|
|
@ -0,0 +1,142 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.Props
|
||||
import akka.dispatch.Dispatchers
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
object RandomRoutingLogic {
|
||||
def apply(): RandomRoutingLogic = new RandomRoutingLogic
|
||||
}
|
||||
|
||||
/**
|
||||
* Randomly selects one of the target routees to send a message to
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final class RandomRoutingLogic extends RoutingLogic {
|
||||
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
|
||||
if (routees.isEmpty) NoRoutee
|
||||
else routees(ThreadLocalRandom.current.nextInt(routees.size))
|
||||
}
|
||||
|
||||
/**
|
||||
* A router pool that randomly selects one of the target routees to send a message to.
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `nrOfInstances` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param nrOfInstances initial number of routees in the pool
|
||||
*
|
||||
* @param resizer optional resizer that dynamically adjust the pool size
|
||||
*
|
||||
* @param supervisorStrategy strategy for supervising the routees, see 'Supervision Setup'
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* supervision, death watch and router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class RandomPool(
|
||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Pool with PoolOverrideUnsetConfig[RandomPool] {
|
||||
|
||||
def this(config: Config) =
|
||||
this(
|
||||
nrOfInstances = config.getInt("nr-of-instances"),
|
||||
resizer = DefaultResizer.fromConfig(config))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param nr initial number of routees in the pool
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(RandomRoutingLogic())
|
||||
|
||||
/**
|
||||
* Setting the supervisor strategy to be used for the “head” Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): RandomPool = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): RandomPool = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* supervision, death watch and router management messages.
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): RandomPool = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* A router group that randomly selects one of the target routees to send a message to.
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `paths` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* @param paths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class RandomGroup(
|
||||
override val paths: immutable.Iterable[String],
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Group {
|
||||
|
||||
def this(config: Config) =
|
||||
this(paths = immutableSeq(config.getStringList("routees.paths")))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param routeePaths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(paths = immutableSeq(routeePaths))
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(RandomRoutingLogic())
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): RandomGroup = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
}
|
||||
318
akka-actor/src/main/scala/akka/routing/Resizer.scala
Normal file
318
akka-actor/src/main/scala/akka/routing/Resizer.scala
Normal file
|
|
@ -0,0 +1,318 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import java.util.concurrent.TimeUnit
|
||||
import java.util.concurrent.atomic.AtomicBoolean
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
|
||||
import com.typesafe.config.Config
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorCell
|
||||
import akka.actor.ActorInitializationException
|
||||
import akka.actor.ActorRefWithCell
|
||||
import akka.actor.ActorSystemImpl
|
||||
import akka.actor.InternalActorRef
|
||||
import akka.actor.PoisonPill
|
||||
import akka.actor.Props
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.dispatch.Envelope
|
||||
import akka.dispatch.MessageDispatcher
|
||||
|
||||
/**
|
||||
* [[Pool]] routers with dynamically resizable number of routees are implemented by providing a Resizer
|
||||
* implementation in the [[akka.routing.Pool]] configuration.
|
||||
*/
|
||||
trait Resizer {
|
||||
/**
|
||||
* Is it time for resizing. Typically implemented with modulo of nth message, but
|
||||
* could be based on elapsed time or something else. The messageCounter starts with 0
|
||||
* for the initial resize and continues with 1 for the first message. Make sure to perform
|
||||
* initial resize before first message (messageCounter == 0), because there is no guarantee
|
||||
* that resize will be done when concurrent messages are in play.
|
||||
*
|
||||
* CAUTION: this method is invoked from the thread which tries to send a
|
||||
* message to the pool, i.e. the ActorRef.!() method, hence it may be called
|
||||
* concurrently.
|
||||
*/
|
||||
def isTimeForResize(messageCounter: Long): Boolean
|
||||
|
||||
/**
|
||||
* Decide if the capacity of the router need to be changed. Will be invoked when `isTimeForResize`
|
||||
* returns true and no other resize is in progress.
|
||||
*
|
||||
* Return the number of routees to add or remove. Negative value will remove that number of routees.
|
||||
* Positive value will add that number of routess. 0 will not change the routees.
|
||||
*
|
||||
* This method is invoked only in the context of the Router actor.
|
||||
*/
|
||||
def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int
|
||||
}
|
||||
|
||||
case object DefaultResizer {
|
||||
|
||||
/**
|
||||
* Creates a new DefaultResizer from the given configuration
|
||||
*/
|
||||
def apply(resizerConfig: Config): DefaultResizer =
|
||||
DefaultResizer(
|
||||
lowerBound = resizerConfig.getInt("lower-bound"),
|
||||
upperBound = resizerConfig.getInt("upper-bound"),
|
||||
pressureThreshold = resizerConfig.getInt("pressure-threshold"),
|
||||
rampupRate = resizerConfig.getDouble("rampup-rate"),
|
||||
backoffThreshold = resizerConfig.getDouble("backoff-threshold"),
|
||||
backoffRate = resizerConfig.getDouble("backoff-rate"),
|
||||
messagesPerResize = resizerConfig.getInt("messages-per-resize"))
|
||||
|
||||
def fromConfig(resizerConfig: Config): Option[DefaultResizer] =
|
||||
if (resizerConfig.getBoolean("resizer.enabled"))
|
||||
Some(DefaultResizer(resizerConfig.getConfig("resizer")))
|
||||
else
|
||||
None
|
||||
}
|
||||
|
||||
/**
|
||||
* Implementation of [[Resizer]] that adjust the [[Pool]] based on specified
|
||||
* thresholds.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class DefaultResizer(
|
||||
/**
|
||||
* The fewest number of routees the router should ever have.
|
||||
*/
|
||||
lowerBound: Int = 1,
|
||||
/**
|
||||
* The most number of routees the router should ever have.
|
||||
* Must be greater than or equal to `lowerBound`.
|
||||
*/
|
||||
upperBound: Int = 10,
|
||||
/**
|
||||
* Threshold to evaluate if routee is considered to be busy (under pressure).
|
||||
* Implementation depends on this value (default is 1).
|
||||
* <ul>
|
||||
* <li> 0: number of routees currently processing a message.</li>
|
||||
* <li> 1: number of routees currently processing a message has
|
||||
* some messages in mailbox.</li>
|
||||
* <li> > 1: number of routees with at least the configured `pressureThreshold`
|
||||
* messages in their mailbox. Note that estimating mailbox size of
|
||||
* default UnboundedMailbox is O(N) operation.</li>
|
||||
* </ul>
|
||||
*/
|
||||
pressureThreshold: Int = 1,
|
||||
/**
|
||||
* Percentage to increase capacity whenever all routees are busy.
|
||||
* For example, 0.2 would increase 20% (rounded up), i.e. if current
|
||||
* capacity is 6 it will request an increase of 2 more routees.
|
||||
*/
|
||||
rampupRate: Double = 0.2,
|
||||
/**
|
||||
* Minimum fraction of busy routees before backing off.
|
||||
* For example, if this is 0.3, then we'll remove some routees only when
|
||||
* less than 30% of routees are busy, i.e. if current capacity is 10 and
|
||||
* 3 are busy then the capacity is unchanged, but if 2 or less are busy
|
||||
* the capacity is decreased.
|
||||
*
|
||||
* Use 0.0 or negative to avoid removal of routees.
|
||||
*/
|
||||
backoffThreshold: Double = 0.3,
|
||||
/**
|
||||
* Fraction of routees to be removed when the resizer reaches the
|
||||
* backoffThreshold.
|
||||
* For example, 0.1 would decrease 10% (rounded up), i.e. if current
|
||||
* capacity is 9 it will request an decrease of 1 routee.
|
||||
*/
|
||||
backoffRate: Double = 0.1,
|
||||
/**
|
||||
* Number of messages between resize operation.
|
||||
* Use 1 to resize before each message.
|
||||
*/
|
||||
messagesPerResize: Int = 10) extends Resizer {
|
||||
|
||||
/**
|
||||
* Java API constructor for default values except bounds.
|
||||
*/
|
||||
def this(lower: Int, upper: Int) = this(lowerBound = lower, upperBound = upper)
|
||||
|
||||
if (lowerBound < 0) throw new IllegalArgumentException("lowerBound must be >= 0, was: [%s]".format(lowerBound))
|
||||
if (upperBound < 0) throw new IllegalArgumentException("upperBound must be >= 0, was: [%s]".format(upperBound))
|
||||
if (upperBound < lowerBound) throw new IllegalArgumentException("upperBound must be >= lowerBound, was: [%s] < [%s]".format(upperBound, lowerBound))
|
||||
if (rampupRate < 0.0) throw new IllegalArgumentException("rampupRate must be >= 0.0, was [%s]".format(rampupRate))
|
||||
if (backoffThreshold > 1.0) throw new IllegalArgumentException("backoffThreshold must be <= 1.0, was [%s]".format(backoffThreshold))
|
||||
if (backoffRate < 0.0) throw new IllegalArgumentException("backoffRate must be >= 0.0, was [%s]".format(backoffRate))
|
||||
if (messagesPerResize <= 0) throw new IllegalArgumentException("messagesPerResize must be > 0, was [%s]".format(messagesPerResize))
|
||||
|
||||
def isTimeForResize(messageCounter: Long): Boolean = (messageCounter % messagesPerResize == 0)
|
||||
|
||||
override def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int =
|
||||
capacity(currentRoutees)
|
||||
|
||||
/**
|
||||
* Returns the overall desired change in resizer capacity. Positive value will
|
||||
* add routees to the resizer. Negative value will remove routees from the
|
||||
* resizer.
|
||||
* @param routees The current actor in the resizer
|
||||
* @return the number of routees by which the resizer should be adjusted (positive, negative or zero)
|
||||
*/
|
||||
def capacity(routees: immutable.IndexedSeq[Routee]): Int = {
|
||||
val currentSize = routees.size
|
||||
val press = pressure(routees)
|
||||
val delta = filter(press, currentSize)
|
||||
val proposed = currentSize + delta
|
||||
|
||||
if (proposed < lowerBound) delta + (lowerBound - proposed)
|
||||
else if (proposed > upperBound) delta - (proposed - upperBound)
|
||||
else delta
|
||||
}
|
||||
|
||||
/**
|
||||
* Number of routees considered busy, or above 'pressure level'.
|
||||
*
|
||||
* Implementation depends on the value of `pressureThreshold`
|
||||
* (default is 1).
|
||||
* <ul>
|
||||
* <li> 0: number of routees currently processing a message.</li>
|
||||
* <li> 1: number of routees currently processing a message has
|
||||
* some messages in mailbox.</li>
|
||||
* <li> > 1: number of routees with at least the configured `pressureThreshold`
|
||||
* messages in their mailbox. Note that estimating mailbox size of
|
||||
* default UnboundedMailbox is O(N) operation.</li>
|
||||
* </ul>
|
||||
*
|
||||
* @param routees the current resizer of routees
|
||||
* @return number of busy routees, between 0 and routees.size
|
||||
*/
|
||||
def pressure(routees: immutable.IndexedSeq[Routee]): Int = {
|
||||
routees count {
|
||||
case ActorRefRoutee(a: ActorRefWithCell) ⇒
|
||||
a.underlying match {
|
||||
case cell: ActorCell ⇒
|
||||
pressureThreshold match {
|
||||
case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.hasMessages
|
||||
case i if i < 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null
|
||||
case threshold ⇒ cell.mailbox.numberOfMessages >= threshold
|
||||
}
|
||||
case cell ⇒
|
||||
pressureThreshold match {
|
||||
case 1 ⇒ cell.hasMessages
|
||||
case i if i < 1 ⇒ true // unstarted cells are always busy, for example
|
||||
case threshold ⇒ cell.numberOfMessages >= threshold
|
||||
}
|
||||
}
|
||||
case x ⇒
|
||||
false
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method can be used to smooth the capacity delta by considering
|
||||
* the current pressure and current capacity.
|
||||
*
|
||||
* @param pressure current number of busy routees
|
||||
* @param capacity current number of routees
|
||||
* @return proposed change in the capacity
|
||||
*/
|
||||
def filter(pressure: Int, capacity: Int): Int = rampup(pressure, capacity) + backoff(pressure, capacity)
|
||||
|
||||
/**
|
||||
* Computes a proposed positive (or zero) capacity delta using
|
||||
* the configured `rampupRate`.
|
||||
* @param pressure the current number of busy routees
|
||||
* @param capacity the current number of total routees
|
||||
* @return proposed increase in capacity
|
||||
*/
|
||||
def rampup(pressure: Int, capacity: Int): Int =
|
||||
if (pressure < capacity) 0 else math.ceil(rampupRate * capacity).toInt
|
||||
|
||||
/**
|
||||
* Computes a proposed negative (or zero) capacity delta using
|
||||
* the configured `backoffThreshold` and `backoffRate`
|
||||
* @param pressure the current number of busy routees
|
||||
* @param capacity the current number of total routees
|
||||
* @return proposed decrease in capacity (as a negative number)
|
||||
*/
|
||||
def backoff(pressure: Int, capacity: Int): Int =
|
||||
if (backoffThreshold > 0.0 && backoffRate > 0.0 && capacity > 0 && pressure.toDouble / capacity < backoffThreshold)
|
||||
math.floor(-1.0 * backoffRate * capacity).toInt
|
||||
else 0
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] final class ResizablePoolCell(
|
||||
_system: ActorSystemImpl,
|
||||
_ref: InternalActorRef,
|
||||
_routerProps: Props,
|
||||
_routerDispatcher: MessageDispatcher,
|
||||
_routeeProps: Props,
|
||||
_supervisor: InternalActorRef,
|
||||
val pool: Pool)
|
||||
extends RoutedActorCell(_system, _ref, _routerProps, _routerDispatcher, _routeeProps, _supervisor) {
|
||||
|
||||
require(pool.resizer.isDefined, "RouterConfig must be a Pool with defined resizer")
|
||||
val resizer = pool.resizer.get
|
||||
private val resizeInProgress = new AtomicBoolean
|
||||
private val resizeCounter = new AtomicLong
|
||||
|
||||
override protected def preSuperStart(): Unit = {
|
||||
// initial resize, before message send
|
||||
if (resizer.isTimeForResize(resizeCounter.getAndIncrement())) {
|
||||
resize(initial = true)
|
||||
}
|
||||
}
|
||||
|
||||
override def sendMessage(envelope: Envelope): Unit = {
|
||||
if (!routerConfig.isManagementMessage(envelope.message) &&
|
||||
resizer.isTimeForResize(resizeCounter.getAndIncrement()) && resizeInProgress.compareAndSet(false, true)) {
|
||||
super.sendMessage(Envelope(ResizablePoolActor.Resize, self, system))
|
||||
}
|
||||
super.sendMessage(envelope)
|
||||
}
|
||||
|
||||
private[akka] def resize(initial: Boolean): Unit = {
|
||||
if (resizeInProgress.get || initial) try {
|
||||
val requestedCapacity = resizer.resize(router.routees)
|
||||
if (requestedCapacity > 0) {
|
||||
val newRoutees = Vector.fill(requestedCapacity)(pool.newRoutee(routeeProps, this))
|
||||
addRoutees(newRoutees)
|
||||
} else if (requestedCapacity < 0) {
|
||||
val currentRoutees = router.routees
|
||||
val abandon = currentRoutees.drop(currentRoutees.length + requestedCapacity)
|
||||
removeRoutees(abandon, stopChild = true)
|
||||
}
|
||||
} finally resizeInProgress.set(false)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object ResizablePoolActor {
|
||||
case object Resize extends RouterManagementMesssage
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ResizablePoolActor(supervisorStrategy: SupervisorStrategy)
|
||||
extends RouterPoolActor(supervisorStrategy) {
|
||||
import ResizablePoolActor._
|
||||
|
||||
val resizerCell = context match {
|
||||
case x: ResizablePoolCell ⇒ x
|
||||
case _ ⇒
|
||||
throw ActorInitializationException("Router actor can only be used in RoutedActorRef, not in " + context.getClass)
|
||||
}
|
||||
|
||||
override def receive = ({
|
||||
case Resize ⇒ resizerCell.resize(initial = false)
|
||||
}: Actor.Receive) orElse super.receive
|
||||
}
|
||||
147
akka-actor/src/main/scala/akka/routing/RoundRobin.scala
Normal file
147
akka-actor/src/main/scala/akka/routing/RoundRobin.scala
Normal file
|
|
@ -0,0 +1,147 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import java.util.concurrent.atomic.AtomicLong
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.Props
|
||||
import akka.dispatch.Dispatchers
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
object RoundRobinRoutingLogic {
|
||||
def apply(): RoundRobinRoutingLogic = new RoundRobinRoutingLogic
|
||||
}
|
||||
|
||||
/**
|
||||
* Uses round-robin to select a routee. For concurrent calls,
|
||||
* round robin is just a best effort.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final class RoundRobinRoutingLogic extends RoutingLogic {
|
||||
val next = new AtomicLong(0)
|
||||
|
||||
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
|
||||
if (routees.isEmpty) NoRoutee
|
||||
else routees((next.getAndIncrement % routees.size).asInstanceOf[Int])
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* A router pool that uses round-robin to select a routee. For concurrent calls,
|
||||
* round robin is just a best effort.
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `nrOfInstances` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param nrOfInstances initial number of routees in the pool
|
||||
*
|
||||
* @param resizer optional resizer that dynamically adjust the pool size
|
||||
*
|
||||
* @param supervisorStrategy strategy for supervising the routees, see 'Supervision Setup'
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* supervision, death watch and router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class RoundRobinPool(
|
||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Pool with PoolOverrideUnsetConfig[RoundRobinPool] {
|
||||
|
||||
def this(config: Config) =
|
||||
this(nrOfInstances = config.getInt("nr-of-instances"),
|
||||
resizer = DefaultResizer.fromConfig(config))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param nr initial number of routees in the pool
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(RoundRobinRoutingLogic())
|
||||
|
||||
/**
|
||||
* Setting the supervisor strategy to be used for the “head” Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): RoundRobinPool = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): RoundRobinPool = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* supervision, death watch and router management messages.
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): RoundRobinPool = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* A router group that uses round-robin to select a routee. For concurrent calls,
|
||||
* round robin is just a best effort.
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `paths` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* @param paths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class RoundRobinGroup(
|
||||
override val paths: immutable.Iterable[String],
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Group {
|
||||
|
||||
def this(config: Config) =
|
||||
this(paths = immutableSeq(config.getStringList("routees.paths")))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param routeePaths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String]) = this(paths = immutableSeq(routeePaths))
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(RoundRobinRoutingLogic())
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): RoundRobinGroup = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
}
|
||||
245
akka-actor/src/main/scala/akka/routing/RoutedActorCell.scala
Normal file
245
akka-actor/src/main/scala/akka/routing/RoutedActorCell.scala
Normal file
|
|
@ -0,0 +1,245 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorCell
|
||||
import akka.actor.ActorInitializationException
|
||||
import akka.actor.ActorSystemImpl
|
||||
import akka.actor.AutoReceivedMessage
|
||||
import akka.actor.IndirectActorProducer
|
||||
import akka.actor.InternalActorRef
|
||||
import akka.actor.Props
|
||||
import akka.actor.Terminated
|
||||
import akka.dispatch.Envelope
|
||||
import akka.dispatch.MessageDispatcher
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.PoisonPill
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.ReceiveTimeout
|
||||
import akka.actor.Identify
|
||||
import akka.actor.ActorIdentity
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] object RoutedActorCell {
|
||||
class RouterActorCreator(routerConfig: RouterConfig) extends IndirectActorProducer {
|
||||
override def actorClass = classOf[RouterActor]
|
||||
override def produce() = routerConfig.createRouterActor()
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class RoutedActorCell(
|
||||
_system: ActorSystemImpl,
|
||||
_ref: InternalActorRef,
|
||||
_routerProps: Props,
|
||||
_routerDispatcher: MessageDispatcher,
|
||||
val routeeProps: Props,
|
||||
_supervisor: InternalActorRef)
|
||||
extends ActorCell(_system, _ref, _routerProps, _routerDispatcher, _supervisor) {
|
||||
|
||||
private[akka] val routerConfig = _routerProps.routerConfig
|
||||
|
||||
@volatile
|
||||
private var _router: Router = null // initialized in start, and then only updated from the actor
|
||||
def router: Router = _router
|
||||
|
||||
def addRoutee(routee: Routee): Unit = addRoutees(List(routee))
|
||||
|
||||
/**
|
||||
* Add routees to the `Router`. Messages in flight may still be routed to
|
||||
* the old `Router` instance containing the old routees.
|
||||
*/
|
||||
def addRoutees(routees: immutable.Iterable[Routee]): Unit = {
|
||||
routees foreach watch
|
||||
val r = _router
|
||||
_router = r.withRoutees(r.routees ++ routees)
|
||||
}
|
||||
|
||||
def removeRoutee(routee: Routee, stopChild: Boolean): Unit =
|
||||
removeRoutees(List(routee), stopChild)
|
||||
|
||||
/**
|
||||
* Remove routees from the `Router`. Messages in flight may still be routed to
|
||||
* the old `Router` instance containing the old routees.
|
||||
*/
|
||||
def removeRoutees(routees: immutable.Iterable[Routee], stopChild: Boolean): Unit = {
|
||||
val r = _router
|
||||
val newRoutees = routees.foldLeft(r.routees) { (xs, x) ⇒ unwatch(x); xs.filterNot(_ == x) }
|
||||
_router = r.withRoutees(newRoutees)
|
||||
if (stopChild) routees foreach stopIfChild
|
||||
}
|
||||
|
||||
private def watch(routee: Routee): Unit = routee match {
|
||||
case ActorRefRoutee(ref) ⇒ watch(ref)
|
||||
case _ ⇒
|
||||
}
|
||||
|
||||
private def unwatch(routee: Routee): Unit = routee match {
|
||||
case ActorRefRoutee(ref) ⇒ unwatch(ref)
|
||||
case _ ⇒
|
||||
}
|
||||
|
||||
private def stopIfChild(routee: Routee): Unit = routee match {
|
||||
case ActorRefRoutee(ref) ⇒ child(ref.path.name) match {
|
||||
case Some(`ref`) ⇒
|
||||
// The reason for the delay is to give concurrent
|
||||
// messages a chance to be placed in mailbox before sending PoisonPill,
|
||||
// best effort.
|
||||
system.scheduler.scheduleOnce(100.milliseconds, ref, PoisonPill)(dispatcher)
|
||||
case _ ⇒
|
||||
}
|
||||
case _ ⇒
|
||||
}
|
||||
|
||||
override def start(): this.type = {
|
||||
// create the initial routees before scheduling the Router actor
|
||||
_router = routerConfig.createRouter(system)
|
||||
routerConfig match {
|
||||
case old: DeprecatedRouterConfig ⇒
|
||||
if (old.nrOfInstances > 0)
|
||||
addRoutees(Vector.fill(old.nrOfInstances)(old.newRoutee(routeeProps, this)))
|
||||
val paths = old.paths
|
||||
if (paths.nonEmpty)
|
||||
addRoutees(paths.map(p ⇒ old.routeeFor(p, this))(collection.breakOut))
|
||||
case pool: Pool ⇒
|
||||
if (pool.nrOfInstances > 0)
|
||||
addRoutees(Vector.fill(pool.nrOfInstances)(pool.newRoutee(routeeProps, this)))
|
||||
case group: Group ⇒
|
||||
val paths = group.paths
|
||||
if (paths.nonEmpty)
|
||||
addRoutees(paths.map(p ⇒ group.routeeFor(p, this))(collection.breakOut))
|
||||
case _ ⇒
|
||||
}
|
||||
preSuperStart()
|
||||
super.start()
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when `router` is initalized but before `super.start()` to
|
||||
* be able to do extra initialization in subclass.
|
||||
*/
|
||||
protected def preSuperStart(): Unit = ()
|
||||
|
||||
/*
|
||||
* end of construction
|
||||
*/
|
||||
|
||||
/**
|
||||
* Route the message via the router to the selected destination.
|
||||
*
|
||||
* When [[akka.routing.CurrentRoutees]] is sent to the RoutedActorRef it
|
||||
* replies with [[akka.routing.RouterRoutees]].
|
||||
*/
|
||||
override def sendMessage(envelope: Envelope): Unit = {
|
||||
if (routerConfig.isManagementMessage(envelope.message))
|
||||
super.sendMessage(envelope)
|
||||
else
|
||||
router.route(envelope.message, envelope.sender)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class RouterActor extends Actor {
|
||||
|
||||
val cell = context match {
|
||||
case x: RoutedActorCell ⇒ x
|
||||
case _ ⇒
|
||||
throw ActorInitializationException("Router actor can only be used in RoutedActorRef, not in " + context.getClass)
|
||||
}
|
||||
|
||||
val routingLogicController: Option[ActorRef] = cell.routerConfig.routingLogicController(
|
||||
cell.router.logic).map(props ⇒ context.actorOf(props.withDispatcher(context.props.dispatcher),
|
||||
name = "routingLogicController"))
|
||||
|
||||
def receive = {
|
||||
case GetRoutees ⇒
|
||||
sender ! Routees(cell.router.routees)
|
||||
case CurrentRoutees ⇒
|
||||
context.actorOf(Props(classOf[CollectRouteeRefs], cell.router.routees, sender))
|
||||
case AddRoutee(routee) ⇒
|
||||
cell.addRoutee(routee)
|
||||
case RemoveRoutee(routee) ⇒
|
||||
cell.removeRoutee(routee, stopChild = true)
|
||||
stopIfAllRouteesRemoved()
|
||||
case other if routingLogicController.isDefined ⇒
|
||||
routingLogicController.foreach(_.forward(other))
|
||||
}
|
||||
|
||||
def stopIfAllRouteesRemoved(): Unit =
|
||||
if (cell.router.routees.isEmpty && cell.routerConfig.stopRouterWhenAllRouteesRemoved)
|
||||
context.stop(self)
|
||||
|
||||
override def preRestart(cause: Throwable, msg: Option[Any]): Unit = {
|
||||
// do not scrap children
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* Backwards compatibility glue to support CurrentRoutees/RouterRoutees containing refs of
|
||||
* the routees. This class is not needed when CurrentRoutees/RouterRoutees are removed.
|
||||
*/
|
||||
private[akka] class CollectRouteeRefs(routees: immutable.IndexedSeq[Routee], replyTo: ActorRef) extends Actor {
|
||||
|
||||
var collected = Vector.empty[ActorRef]
|
||||
var count = 0
|
||||
routees.foreach(_.send(Identify(None), self))
|
||||
|
||||
import context.dispatcher
|
||||
context.system.scheduler.scheduleOnce(10.seconds, self, ReceiveTimeout)
|
||||
|
||||
def receive = {
|
||||
case ActorIdentity(_, refOption) ⇒
|
||||
refOption foreach { ref ⇒ collected = collected :+ ref }
|
||||
count += 1
|
||||
if (count == routees.size) done()
|
||||
case ReceiveTimeout ⇒ done()
|
||||
}
|
||||
|
||||
def done(): Unit = {
|
||||
replyTo ! RouterRoutees(collected)
|
||||
context.stop(self)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class RouterPoolActor(override val supervisorStrategy: SupervisorStrategy) extends RouterActor {
|
||||
|
||||
val pool = cell.routerConfig match {
|
||||
case x: Pool ⇒ x
|
||||
case other ⇒
|
||||
throw ActorInitializationException("RouterPoolActor can only be used with Pool, not " + other.getClass)
|
||||
}
|
||||
|
||||
override def receive = ({
|
||||
case Terminated(child) ⇒
|
||||
cell.removeRoutee(ActorRefRoutee(child), stopChild = false)
|
||||
stopIfAllRouteesRemoved()
|
||||
case AdjustPoolSize(change: Int) ⇒
|
||||
if (change > 0) {
|
||||
val newRoutees = Vector.fill(change)(pool.newRoutee(cell.routeeProps, context))
|
||||
cell.addRoutees(newRoutees)
|
||||
} else if (change < 0) {
|
||||
val currentRoutees = cell.router.routees
|
||||
val abandon = currentRoutees.drop(currentRoutees.length + change)
|
||||
cell.removeRoutees(abandon, stopChild = true)
|
||||
}
|
||||
}: Actor.Receive) orElse super.receive
|
||||
|
||||
}
|
||||
|
||||
52
akka-actor/src/main/scala/akka/routing/RoutedActorRef.scala
Normal file
52
akka-actor/src/main/scala/akka/routing/RoutedActorRef.scala
Normal file
|
|
@ -0,0 +1,52 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.ConfigurationException
|
||||
import akka.actor.ActorPath
|
||||
import akka.actor.ActorSystemImpl
|
||||
import akka.actor.Cell
|
||||
import akka.actor.InternalActorRef
|
||||
import akka.actor.Props
|
||||
import akka.actor.RepointableActorRef
|
||||
import akka.actor.UnstartedCell
|
||||
import akka.dispatch.BalancingDispatcher
|
||||
import akka.dispatch.MailboxType
|
||||
import akka.dispatch.MessageDispatcher
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to
|
||||
* send a message to one (or more) of these actors.
|
||||
*/
|
||||
private[akka] class RoutedActorRef(
|
||||
_system: ActorSystemImpl,
|
||||
_routerProps: Props,
|
||||
_routerDispatcher: MessageDispatcher,
|
||||
_routerMailbox: MailboxType,
|
||||
_routeeProps: Props,
|
||||
_supervisor: InternalActorRef,
|
||||
_path: ActorPath)
|
||||
extends RepointableActorRef(_system, _routerProps, _routerDispatcher, _routerMailbox, _supervisor, _path) {
|
||||
|
||||
// verify that a BalancingDispatcher is not used with a Router
|
||||
if (_routerProps.routerConfig != NoRouter && _routerDispatcher.isInstanceOf[BalancingDispatcher]) {
|
||||
throw new ConfigurationException(
|
||||
"Configuration for " + this +
|
||||
" is invalid - you can not use a 'BalancingDispatcher' as a Router's dispatcher, you can however use it for the routees.")
|
||||
} else _routerProps.routerConfig.verifyConfig(_path)
|
||||
|
||||
override def newCell(old: UnstartedCell): Cell = {
|
||||
val cell = props.routerConfig match {
|
||||
case pool: Pool if pool.resizer.isDefined ⇒
|
||||
new ResizablePoolCell(system, this, props, dispatcher, _routeeProps, supervisor, pool)
|
||||
case _ ⇒
|
||||
new RoutedActorCell(system, this, props, dispatcher, _routeeProps, supervisor)
|
||||
}
|
||||
cell.init(sendSupervise = false, mailboxType)
|
||||
}
|
||||
|
||||
}
|
||||
186
akka-actor/src/main/scala/akka/routing/Router.scala
Normal file
186
akka-actor/src/main/scala/akka/routing/Router.scala
Normal file
|
|
@ -0,0 +1,186 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.ActorSelection
|
||||
import akka.actor.InternalActorRef
|
||||
import akka.japi.Util.immutableSeq
|
||||
|
||||
/**
|
||||
* The interface of the routing logic that is used in a [[Router]] to select
|
||||
* destination routed messages.
|
||||
*
|
||||
* The implementation must be thread safe.
|
||||
*/
|
||||
trait RoutingLogic {
|
||||
/**
|
||||
* Pick the destination for a given message. Normally it picks one of the
|
||||
* passed `routees`, but in the end it is up to the implementation to
|
||||
* return whatever [[Routee]] to use for sending a specific message.
|
||||
*
|
||||
* When implemented from Java it can be good to know that
|
||||
* `routees.apply(index)` can be used to get an element
|
||||
* from the `IndexedSeq`.
|
||||
*/
|
||||
def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Abstraction of a destination for messages routed via a [[Router]].
|
||||
*/
|
||||
trait Routee {
|
||||
def send(message: Any, sender: ActorRef): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
* [[Routee]] that sends the messages to an [[akka.actor.ActorRefRoutee]].
|
||||
*/
|
||||
case class ActorRefRoutee(ref: ActorRef) extends Routee {
|
||||
override def send(message: Any, sender: ActorRef): Unit =
|
||||
ref.tell(message, sender)
|
||||
}
|
||||
|
||||
/**
|
||||
* [[Routee]] that sends the messages to an [[akka.actor.ActorSelection]].
|
||||
*/
|
||||
case class ActorSelectionRoutee(selection: ActorSelection) extends Routee {
|
||||
override def send(message: Any, sender: ActorRef): Unit =
|
||||
selection.tell(message, sender)
|
||||
}
|
||||
|
||||
/**
|
||||
* [[Routee]] that doesn't send the message to any routee.
|
||||
* The [[Router]] will send the message to `deadLetters` if
|
||||
* `NoRoutee` is returned from [[RoutingLogic#select]]
|
||||
*/
|
||||
object NoRoutee extends Routee {
|
||||
override def send(message: Any, sender: ActorRef): Unit = ()
|
||||
}
|
||||
|
||||
/**
|
||||
* [[Routee]] that sends each message to all `routees`.
|
||||
*/
|
||||
case class SeveralRoutees(routees: immutable.IndexedSeq[Routee]) extends Routee {
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def this(rs: java.lang.Iterable[Routee]) = this(routees = immutableSeq(rs).toVector)
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def getRoutees(): java.util.List[Routee] = {
|
||||
import scala.collection.JavaConverters._
|
||||
routees.asJava
|
||||
}
|
||||
|
||||
override def send(message: Any, sender: ActorRef): Unit =
|
||||
routees.foreach(_.send(message, sender))
|
||||
}
|
||||
|
||||
/**
|
||||
* For each message that is sent through the router via the [[#route]] method the
|
||||
* [[RoutingLogic]] decides to which [[Routee]] to send the message. The [[Routee]] itself
|
||||
* knows how to perform the actual sending. Normally the [[RoutingLogic]] picks one of the
|
||||
* contained `routees`, but that is up to the implementation of the [[RoutingLogic]].
|
||||
*
|
||||
* A `Router` is immutable and the [[RoutingLogic]] must be thread safe.
|
||||
*/
|
||||
final case class Router(val logic: RoutingLogic, val routees: immutable.IndexedSeq[Routee] = Vector.empty) {
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def this(logic: RoutingLogic) = this(logic, Vector.empty)
|
||||
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def this(logic: RoutingLogic, routees: java.lang.Iterable[Routee]) = this(logic, immutableSeq(routees).toVector)
|
||||
|
||||
/**
|
||||
* Send the message to the destination [[Routee]] selected by the [[RoutingLogic]].
|
||||
* If the message is a [[akka.routing.RouterEnvelope]] it will be unwrapped
|
||||
* before sent to the destinations.
|
||||
* Messages wrapped in a [[Broadcast]] envelope are always sent to all `routees`.
|
||||
*/
|
||||
def route(message: Any, sender: ActorRef): Unit =
|
||||
message match {
|
||||
case akka.routing.Broadcast(msg) ⇒ SeveralRoutees(routees).send(msg, sender)
|
||||
case msg ⇒ send(logic.select(msg, routees), message, sender)
|
||||
}
|
||||
|
||||
private def send(routee: Routee, msg: Any, sender: ActorRef): Unit = {
|
||||
if (routee == NoRoutee && sender.isInstanceOf[InternalActorRef])
|
||||
sender.asInstanceOf[InternalActorRef].provider.deadLetters.tell(unwrap(msg), sender)
|
||||
else
|
||||
routee.send(unwrap(msg), sender)
|
||||
}
|
||||
|
||||
private def unwrap(msg: Any): Any = msg match {
|
||||
case env: RouterEnvelope ⇒ env.message
|
||||
case _ ⇒ msg
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new instance with the specified routees and the same [[RoutingLogic]].
|
||||
*/
|
||||
def withRoutees(rs: immutable.IndexedSeq[Routee]): Router = copy(routees = rs)
|
||||
|
||||
/**
|
||||
* Create a new instance with one more routee and the same [[RoutingLogic]].
|
||||
*/
|
||||
def addRoutee(routee: Routee): Router = copy(routees = routees :+ routee)
|
||||
|
||||
/**
|
||||
* Create a new instance with one more [[ActorRefRoutee]] for the
|
||||
* specified [[akka.actor.ActorRef]] and the same [[RoutingLogic]].
|
||||
*/
|
||||
def addRoutee(ref: ActorRef): Router = addRoutee(ActorRefRoutee(ref))
|
||||
|
||||
/**
|
||||
* Create a new instance with one more [[ActorSelectionRoutee]] for the
|
||||
* specified [[akka.actor.ActorSelection]] and the same [[RoutingLogic]].
|
||||
*/
|
||||
def addRoutee(sel: ActorSelection): Router = addRoutee(ActorSelectionRoutee(sel))
|
||||
|
||||
/**
|
||||
* Create a new instance without the specified routee.
|
||||
*/
|
||||
def removeRoutee(routee: Routee): Router = copy(routees = routees.filterNot(_ == routee))
|
||||
|
||||
/**
|
||||
* Create a new instance without the [[ActorRefRoutee]] for the specified
|
||||
* [[akka.actor.ActorRef]].
|
||||
*/
|
||||
def removeRoutee(ref: ActorRef): Router = removeRoutee(ActorRefRoutee(ref))
|
||||
|
||||
/**
|
||||
* Create a new instance without the [[ActorSelectionRoutee]] for the specified
|
||||
* [[akka.actor.ActorSelection]].
|
||||
*/
|
||||
def removeRoutee(sel: ActorSelection): Router = removeRoutee(ActorSelectionRoutee(sel))
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Used to broadcast a message to all routees in a router; only the
|
||||
* contained message will be forwarded, i.e. the `Broadcast(...)`
|
||||
* envelope will be stripped off.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class Broadcast(message: Any) extends RouterEnvelope
|
||||
|
||||
/**
|
||||
* Only the contained message will be forwarded to the
|
||||
* destination, i.e. the envelope will be stripped off.
|
||||
*/
|
||||
trait RouterEnvelope {
|
||||
def message: Any
|
||||
}
|
||||
|
||||
403
akka-actor/src/main/scala/akka/routing/RouterConfig.scala
Normal file
403
akka-actor/src/main/scala/akka/routing/RouterConfig.scala
Normal file
|
|
@ -0,0 +1,403 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.ConfigurationException
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.ActorPath
|
||||
import akka.actor.AutoReceivedMessage
|
||||
import akka.actor.OneForOneStrategy
|
||||
import akka.actor.Props
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.actor.Terminated
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.actor.ActorSystem
|
||||
import akka.japi.Util.immutableSeq
|
||||
|
||||
/**
|
||||
* This trait represents a router factory: it produces the actual router actor
|
||||
* and creates the routing table (a function which determines the recipients
|
||||
* for each message which is to be dispatched). The resulting RoutedActorRef
|
||||
* optimizes the sending of the message so that it does NOT go through the
|
||||
* router’s mailbox unless the route returns an empty recipient set.
|
||||
*
|
||||
* '''Caution:''' This means
|
||||
* that the route function is evaluated concurrently without protection by
|
||||
* the RoutedActorRef: either provide a reentrant (i.e. pure) implementation or
|
||||
* do the locking yourself!
|
||||
*
|
||||
* '''Caution:''' Please note that the [[akka.routing.Router]] which needs to
|
||||
* be returned by `createActor()` should not send a message to itself in its
|
||||
* constructor or `preStart()` or publish its self reference from there: if
|
||||
* someone tries sending a message to that reference before the constructor of
|
||||
* RoutedActorRef has returned, there will be a `NullPointerException`!
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
trait RouterConfig extends Serializable {
|
||||
|
||||
/**
|
||||
* Create the actual router, responsible for routing messages to routees.
|
||||
* @param system the ActorSystem this router belongs to
|
||||
*/
|
||||
def createRouter(system: ActorSystem): Router
|
||||
|
||||
/**
|
||||
* Dispatcher ID to use for running the “head” actor, which handles
|
||||
* supervision, death watch and router management messages
|
||||
*/
|
||||
def routerDispatcher: String
|
||||
|
||||
/**
|
||||
* Possibility to define an actor for controlling the routing
|
||||
* logic from external stimuli (e.g. monitoring metrics).
|
||||
* This actor will be a child of the router "head" actor.
|
||||
* Managment messages not handled by the "head" actor are
|
||||
* delegated to this controller actor.
|
||||
*/
|
||||
def routingLogicController(routingLogic: RoutingLogic): Option[Props] = None
|
||||
|
||||
/**
|
||||
* Is the message handled by the router head actor or the
|
||||
* [[#routingLogicController]] actor.
|
||||
*/
|
||||
def isManagementMessage(msg: Any): Boolean = msg match {
|
||||
case _: AutoReceivedMessage | _: Terminated | _: RouterManagementMesssage ⇒ true
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
||||
/*
|
||||
* Specify that this router should stop itself when all routees have terminated (been removed).
|
||||
* By Default it is `true`, unless a `resizer` is used.
|
||||
*/
|
||||
def stopRouterWhenAllRouteesRemoved: Boolean = true
|
||||
|
||||
/**
|
||||
* Overridable merge strategy, by default completely prefers `this` (i.e. no merge).
|
||||
*/
|
||||
def withFallback(other: RouterConfig): RouterConfig = this
|
||||
|
||||
/**
|
||||
* Check that everything is there which is needed. Called in constructor of RoutedActorRef to fail early.
|
||||
*/
|
||||
def verifyConfig(path: ActorPath): Unit = ()
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* The router "head" actor.
|
||||
*/
|
||||
private[akka] def createRouterActor(): RouterActor
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Used to override unset configuration in a router.
|
||||
*/
|
||||
private[akka] trait PoolOverrideUnsetConfig[T <: Pool] extends Pool {
|
||||
|
||||
final def overrideUnsetConfig(other: RouterConfig): RouterConfig =
|
||||
if (other == NoRouter) this // NoRouter is the default, hence “neutral”
|
||||
else {
|
||||
|
||||
other match {
|
||||
case p: Pool ⇒
|
||||
val wssConf: PoolOverrideUnsetConfig[T] =
|
||||
if ((this.supervisorStrategy eq Pool.defaultSupervisorStrategy)
|
||||
&& (p.supervisorStrategy ne Pool.defaultSupervisorStrategy))
|
||||
this.withSupervisorStrategy(p.supervisorStrategy).asInstanceOf[PoolOverrideUnsetConfig[T]]
|
||||
else this
|
||||
|
||||
if (wssConf.resizer.isEmpty && p.resizer.isDefined)
|
||||
wssConf.withResizer(p.resizer.get)
|
||||
else
|
||||
wssConf
|
||||
case _ ⇒ this
|
||||
}
|
||||
}
|
||||
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): T
|
||||
|
||||
def withResizer(resizer: Resizer): T
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Base class for custom router [[Group]]
|
||||
*/
|
||||
abstract class GroupBase extends Group {
|
||||
def getPaths: java.lang.Iterable[String]
|
||||
|
||||
override final def paths: immutable.Iterable[String] = immutableSeq(getPaths)
|
||||
}
|
||||
|
||||
/**
|
||||
* `RouterConfig` for router actor with routee actors that are created external to the
|
||||
* router and the router sends messages to the specified path using actor selection,
|
||||
* without watching for termination.
|
||||
*/
|
||||
trait Group extends RouterConfig {
|
||||
|
||||
def paths: immutable.Iterable[String]
|
||||
|
||||
/**
|
||||
* [[akka.actor.Props]] for a group router based on the settings defined by
|
||||
* this instance.
|
||||
*/
|
||||
def props(): Props = Props.empty.withRouter(this)
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def routeeFor(path: String, context: ActorContext): Routee =
|
||||
ActorSelectionRoutee(context.actorSelection(path))
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] override def createRouterActor(): RouterActor = new RouterActor
|
||||
}
|
||||
|
||||
object Pool {
|
||||
val defaultSupervisorStrategy: SupervisorStrategy = OneForOneStrategy() {
|
||||
case _ ⇒ SupervisorStrategy.Escalate
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Base class for custom router [[Pool]]
|
||||
*/
|
||||
abstract class PoolBase extends Pool
|
||||
|
||||
/**
|
||||
* `RouterConfig` for router actor that creates routees as child actors and removes
|
||||
* them from the router if they terminate.
|
||||
*/
|
||||
trait Pool extends RouterConfig {
|
||||
/**
|
||||
* Initial number of routee instances
|
||||
*/
|
||||
def nrOfInstances: Int
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] def newRoutee(routeeProps: Props, context: ActorContext): Routee =
|
||||
ActorRefRoutee(context.actorOf(routeeProps))
|
||||
|
||||
/**
|
||||
* Pool with dynamically resizable number of routees return the [[akka.routing.Resizer]]
|
||||
* to use. The resizer is invoked once when the router is created, before any messages can
|
||||
* be sent to it. Resize is also triggered when messages are sent to the routees, and the
|
||||
* resizer is invoked asynchronously, i.e. not necessarily before the message has been sent.
|
||||
*/
|
||||
def resizer: Option[Resizer]
|
||||
|
||||
/**
|
||||
* SupervisorStrategy for the head actor, i.e. for supervising the routees of the pool.
|
||||
*/
|
||||
def supervisorStrategy: SupervisorStrategy
|
||||
|
||||
/**
|
||||
* [[akka.actor.Props]] for a pool router based on the settings defined by
|
||||
* this instance and the supplied [[akka.actor.Props]] for the routees created by the
|
||||
* router.
|
||||
*/
|
||||
def props(routeeProps: Props): Props = routeeProps.withRouter(this)
|
||||
|
||||
/*
|
||||
* Specify that this router should stop itself when all routees have terminated (been removed).
|
||||
* By Default it is `true`, unless a `resizer` is used.
|
||||
*/
|
||||
override def stopRouterWhenAllRouteesRemoved: Boolean = resizer.isEmpty
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] override def createRouterActor(): RouterActor =
|
||||
resizer match {
|
||||
case None ⇒ new RouterPoolActor(supervisorStrategy)
|
||||
case Some(r) ⇒ new ResizablePoolActor(supervisorStrategy)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* If a custom router implementation is not a [[Group]] nor
|
||||
* a [[Pool]] it may extend this base class.
|
||||
*/
|
||||
abstract class CustomRouterConfig extends RouterConfig {
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] override def createRouterActor(): RouterActor = new RouterActor
|
||||
|
||||
override def routerDispatcher: String = Dispatchers.DefaultDispatcherId
|
||||
}
|
||||
|
||||
/**
|
||||
* Router configuration which has no default, i.e. external configuration is required.
|
||||
*/
|
||||
case object FromConfig extends FromConfig {
|
||||
/**
|
||||
* Java API: get the singleton instance
|
||||
*/
|
||||
def getInstance = this
|
||||
@inline final def apply(
|
||||
resizer: Option[Resizer] = None,
|
||||
supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
routerDispatcher: String = Dispatchers.DefaultDispatcherId) =
|
||||
new FromConfig(resizer, supervisorStrategy, routerDispatcher)
|
||||
|
||||
@inline final def unapply(fc: FromConfig): Option[String] = Some(fc.routerDispatcher)
|
||||
}
|
||||
|
||||
/**
|
||||
* Java API: Router configuration which has no default, i.e. external configuration is required.
|
||||
*
|
||||
* This can be used when the dispatcher to be used for the head Router needs to be configured
|
||||
* (defaults to default-dispatcher).
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
class FromConfig(override val resizer: Option[Resizer],
|
||||
override val supervisorStrategy: SupervisorStrategy,
|
||||
override val routerDispatcher: String) extends Pool {
|
||||
|
||||
def this() = this(None, Pool.defaultSupervisorStrategy, Dispatchers.DefaultDispatcherId)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router =
|
||||
throw new UnsupportedOperationException("FromConfig must not create Router")
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
override private[akka] def createRouterActor(): RouterActor =
|
||||
throw new UnsupportedOperationException("FromConfig must not create RouterActor")
|
||||
|
||||
override def verifyConfig(path: ActorPath): Unit =
|
||||
throw new ConfigurationException(s"Configuration missing for router [$path] in 'akka.actor.deployment' section.")
|
||||
|
||||
/**
|
||||
* Setting the supervisor strategy to be used for the “head” Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): FromConfig =
|
||||
new FromConfig(resizer, strategy, routerDispatcher)
|
||||
|
||||
/**
|
||||
* Setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): FromConfig =
|
||||
new FromConfig(Some(resizer), supervisorStrategy, routerDispatcher)
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* supervision, death watch and router management messages.
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): FromConfig =
|
||||
new FromConfig(resizer, supervisorStrategy, dispatcherId)
|
||||
|
||||
override val nrOfInstances: Int = 0
|
||||
|
||||
/**
|
||||
* [[akka.actor.Props]] for a group router based on the settings defined by
|
||||
* this instance.
|
||||
*/
|
||||
def props(): Props = Props.empty.withRouter(this)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* Routing configuration that indicates no routing; this is also the default
|
||||
* value which hence overrides the merge strategy in order to accept values
|
||||
* from lower-precedence sources. The decision whether or not to create a
|
||||
* router is taken in the LocalActorRefProvider based on Props.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
abstract class NoRouter extends RouterConfig
|
||||
|
||||
case object NoRouter extends NoRouter {
|
||||
override def createRouter(system: ActorSystem): Router = throw new UnsupportedOperationException("NoRouter has no Router")
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
override private[akka] def createRouterActor(): RouterActor =
|
||||
throw new UnsupportedOperationException("NoRouter must not create RouterActor")
|
||||
override def routerDispatcher: String = throw new UnsupportedOperationException("NoRouter has no dispatcher")
|
||||
override def withFallback(other: akka.routing.RouterConfig): akka.routing.RouterConfig = other
|
||||
|
||||
/**
|
||||
* Java API: get the singleton instance
|
||||
*/
|
||||
def getInstance = this
|
||||
|
||||
def props(routeeProps: Props): Props = routeeProps.withRouter(this)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@SerialVersionUID(1L) private[akka] trait RouterManagementMesssage
|
||||
|
||||
/**
|
||||
* Sending this message to a router will make it send back its currently used routees.
|
||||
* A [[Routees]] message is sent asynchronously to the "requester" containing information
|
||||
* about what routees the router is routing over.
|
||||
*/
|
||||
@SerialVersionUID(1L) abstract class GetRoutees extends RouterManagementMesssage
|
||||
|
||||
@SerialVersionUID(1L) case object GetRoutees extends GetRoutees {
|
||||
/**
|
||||
* Java API: get the singleton instance
|
||||
*/
|
||||
def getInstance = this
|
||||
}
|
||||
|
||||
/**
|
||||
* Message used to carry information about what routees the router is currently using.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class Routees(routees: immutable.IndexedSeq[Routee]) {
|
||||
/**
|
||||
* Java API
|
||||
*/
|
||||
def getRoutees: java.util.List[Routee] = {
|
||||
import scala.collection.JavaConverters._
|
||||
routees.asJava
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a routee by sending this message to the router.
|
||||
* It may be handled after other messages.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class AddRoutee(routee: Routee) extends RouterManagementMesssage
|
||||
|
||||
/**
|
||||
* Remove a specific routee by sending this message to the router.
|
||||
* It may be handled after other messages.
|
||||
*
|
||||
* For a pool, with child routees, the routee is stopped by sending a [[akka.actor.PoisonPill]]
|
||||
* to the routee. Precautions are taken reduce the risk of dropping messages that are concurrently
|
||||
* being routed to the removed routee, but there are no guarantees.
|
||||
*
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class RemoveRoutee(routee: Routee) extends RouterManagementMesssage
|
||||
|
||||
/**
|
||||
* Increase or decrease the number of routees in a [[Pool]].
|
||||
* It may be handled after other messages.
|
||||
*
|
||||
* Positive `change` will add that number of routees to the [[Pool]].
|
||||
* Negative `change` will remove that number of routees from the [[Pool]].
|
||||
* Routees are stopped by sending a [[akka.actor.PoisonPill]] to the routee.
|
||||
* Precautions are taken reduce the risk of dropping messages that are concurrently
|
||||
* being routed to the removed routee, but it is not guaranteed that messages are not
|
||||
* lost.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class AdjustPoolSize(change: Int) extends RouterManagementMesssage
|
||||
File diff suppressed because it is too large
Load diff
|
|
@ -0,0 +1,188 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.Props
|
||||
import akka.dispatch.Dispatchers
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.actor.ActorRef
|
||||
import scala.concurrent.Promise
|
||||
import akka.pattern.ask
|
||||
import akka.pattern.pipe
|
||||
import akka.dispatch.ExecutionContexts
|
||||
import scala.concurrent.duration.FiniteDuration
|
||||
import scala.concurrent.duration._
|
||||
import akka.util.Timeout
|
||||
import java.util.concurrent.TimeUnit
|
||||
import akka.actor.ActorSystem
|
||||
|
||||
/**
|
||||
* Broadcasts the message to all routees, and replies with the first response.
|
||||
*
|
||||
* @param within expecting at least one reply within this duration, otherwise
|
||||
* it will reply with [[akka.pattern.AskTimeoutException]] in a [[akka.actor.Status.Failure]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class ScatterGatherFirstCompletedRoutingLogic(within: FiniteDuration) extends RoutingLogic {
|
||||
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
|
||||
if (routees.isEmpty) NoRoutee
|
||||
else ScatterGatherFirstCompletedRoutees(routees, within)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
private[akka] case class ScatterGatherFirstCompletedRoutees(
|
||||
routees: immutable.IndexedSeq[Routee], within: FiniteDuration) extends Routee {
|
||||
|
||||
override def send(message: Any, sender: ActorRef): Unit = {
|
||||
implicit val ec = ExecutionContexts.sameThreadExecutionContext
|
||||
implicit val timeout = Timeout(within)
|
||||
val promise = Promise[Any]()
|
||||
routees.foreach {
|
||||
case ActorRefRoutee(ref) ⇒
|
||||
promise.tryCompleteWith(ref.ask(message))
|
||||
case ActorSelectionRoutee(sel) ⇒
|
||||
promise.tryCompleteWith(sel.ask(message))
|
||||
case _ ⇒
|
||||
}
|
||||
|
||||
promise.future.pipeTo(sender)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A router pool that broadcasts the message to all routees, and replies with the first response.
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `nrOfInstances` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param nrOfInstances initial number of routees in the pool
|
||||
*
|
||||
* @param resizer optional resizer that dynamically adjust the pool size
|
||||
*
|
||||
* @param within expecting at least one reply within this duration, otherwise
|
||||
* it will reply with [[akka.pattern.AskTimeoutException]] in a [[akka.actor.Status.Failure]]
|
||||
*
|
||||
* @param supervisorStrategy strategy for supervising the routees, see 'Supervision Setup'
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* supervision, death watch and router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class ScatterGatherFirstCompletedPool(
|
||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||
within: FiniteDuration,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Pool with PoolOverrideUnsetConfig[ScatterGatherFirstCompletedPool] {
|
||||
|
||||
def this(config: Config) =
|
||||
this(
|
||||
nrOfInstances = config.getInt("nr-of-instances"),
|
||||
within = Duration(config.getMilliseconds("within"), TimeUnit.MILLISECONDS),
|
||||
resizer = DefaultResizer.fromConfig(config))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param nr initial number of routees in the pool
|
||||
* @param within expecting at least one reply within this duration, otherwise
|
||||
* it will reply with [[akka.pattern.AskTimeoutException]] in a [[akka.actor.Status.Failure]]
|
||||
*/
|
||||
def this(nr: Int, within: FiniteDuration) = this(nrOfInstances = nr, within = within)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(ScatterGatherFirstCompletedRoutingLogic(within))
|
||||
|
||||
/**
|
||||
* Setting the supervisor strategy to be used for the “head” Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): ScatterGatherFirstCompletedPool = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): ScatterGatherFirstCompletedPool = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* supervision, death watch and router management messages.
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): ScatterGatherFirstCompletedPool = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* A router group that broadcasts the message to all routees, and replies with the first response.
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `paths` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* @param paths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*
|
||||
* @param within expecting at least one reply within this duration, otherwise
|
||||
* it will reply with [[akka.pattern.AskTimeoutException]] in a [[akka.actor.Status.Failure]]
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class ScatterGatherFirstCompletedGroup(
|
||||
override val paths: immutable.Iterable[String],
|
||||
within: FiniteDuration,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Group {
|
||||
|
||||
def this(config: Config) =
|
||||
this(
|
||||
paths = immutableSeq(config.getStringList("routees.paths")),
|
||||
within = Duration(config.getMilliseconds("within"), TimeUnit.MILLISECONDS))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param routeePaths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
* @param within expecting at least one reply within this duration, otherwise
|
||||
* it will reply with [[akka.pattern.AskTimeoutException]] in a [[akka.actor.Status.Failure]]
|
||||
*/
|
||||
def this(routeePaths: java.lang.Iterable[String], within: FiniteDuration) =
|
||||
this(paths = immutableSeq(routeePaths), within = within)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(ScatterGatherFirstCompletedRoutingLogic(within))
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): ScatterGatherFirstCompletedGroup = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
}
|
||||
218
akka-actor/src/main/scala/akka/routing/SmallestMailbox.scala
Normal file
218
akka-actor/src/main/scala/akka/routing/SmallestMailbox.scala
Normal file
|
|
@ -0,0 +1,218 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.routing
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import com.typesafe.config.Config
|
||||
import akka.actor.ActorCell
|
||||
import akka.actor.ActorRefWithCell
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.Props
|
||||
|
||||
object SmallestMailboxRoutingLogic {
|
||||
def apply(): SmallestMailboxRoutingLogic = new SmallestMailboxRoutingLogic
|
||||
}
|
||||
|
||||
/**
|
||||
* Tries to send to the non-suspended routee with fewest messages in mailbox.
|
||||
* The selection is done in this order:
|
||||
* <ul>
|
||||
* <li>pick any idle routee (not processing message) with empty mailbox</li>
|
||||
* <li>pick any routee with empty mailbox</li>
|
||||
* <li>pick routee with fewest pending messages in mailbox</li>
|
||||
* <li>pick any remote routee, remote actors are consider lowest priority,
|
||||
* since their mailbox size is unknown</li>
|
||||
* </ul>
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
class SmallestMailboxRoutingLogic extends RoutingLogic {
|
||||
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
|
||||
if (routees.isEmpty) NoRoutee
|
||||
else selectNext(routees)
|
||||
|
||||
// Worst-case a 2-pass inspection with mailbox size checking done on second pass, and only until no one empty is found.
|
||||
// Lowest score wins, score 0 is autowin
|
||||
// If no actor with score 0 is found, it will return that, or if it is terminated, a random of the entire set.
|
||||
// Why? Well, in case we had 0 viable actors and all we got was the default, which is the DeadLetters, anything else is better.
|
||||
// Order of interest, in ascending priority:
|
||||
// 1. The NoRoutee
|
||||
// 2. A Suspended ActorRef
|
||||
// 3. An ActorRef with unknown mailbox size but with one message being processed
|
||||
// 4. An ActorRef with unknown mailbox size that isn't processing anything
|
||||
// 5. An ActorRef with a known mailbox size
|
||||
// 6. An ActorRef without any messages
|
||||
@tailrec private def selectNext(targets: immutable.IndexedSeq[Routee],
|
||||
proposedTarget: Routee = NoRoutee,
|
||||
currentScore: Long = Long.MaxValue,
|
||||
at: Int = 0,
|
||||
deep: Boolean = false): Routee = {
|
||||
if (targets.isEmpty)
|
||||
NoRoutee
|
||||
else if (at >= targets.size) {
|
||||
if (deep) {
|
||||
if (isTerminated(proposedTarget)) targets(ThreadLocalRandom.current.nextInt(targets.size)) else proposedTarget
|
||||
} else selectNext(targets, proposedTarget, currentScore, 0, deep = true)
|
||||
} else {
|
||||
val target = targets(at)
|
||||
val newScore: Long =
|
||||
if (isSuspended(target)) Long.MaxValue - 1 else { //Just about better than the DeadLetters
|
||||
(if (isProcessingMessage(target)) 1l else 0l) +
|
||||
(if (!hasMessages(target)) 0l else { //Race between hasMessages and numberOfMessages here, unfortunate the numberOfMessages returns 0 if unknown
|
||||
val noOfMsgs: Long = if (deep) numberOfMessages(target) else 0
|
||||
if (noOfMsgs > 0) noOfMsgs else Long.MaxValue - 3 //Just better than a suspended actorref
|
||||
})
|
||||
}
|
||||
|
||||
if (newScore == 0) target
|
||||
else if (newScore < 0 || newScore >= currentScore) selectNext(targets, proposedTarget, currentScore, at + 1, deep)
|
||||
else selectNext(targets, target, newScore, at + 1, deep)
|
||||
}
|
||||
}
|
||||
|
||||
protected def isTerminated(a: Routee): Boolean = a match {
|
||||
case ActorRefRoutee(ref) ⇒ ref.isTerminated
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the actor is currently processing a message.
|
||||
* It will always return false for remote actors.
|
||||
* Method is exposed to subclasses to be able to implement custom
|
||||
* routers based on mailbox and actor internal state.
|
||||
*/
|
||||
protected def isProcessingMessage(a: Routee): Boolean = a match {
|
||||
case ActorRefRoutee(x: ActorRefWithCell) ⇒
|
||||
x.underlying match {
|
||||
case cell: ActorCell ⇒ cell.mailbox.isScheduled && cell.currentMessage != null
|
||||
case _ ⇒ false
|
||||
}
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the actor currently has any pending messages
|
||||
* in the mailbox, i.e. the mailbox is not empty.
|
||||
* It will always return false for remote actors.
|
||||
* Method is exposed to subclasses to be able to implement custom
|
||||
* routers based on mailbox and actor internal state.
|
||||
*/
|
||||
protected def hasMessages(a: Routee): Boolean = a match {
|
||||
case ActorRefRoutee(x: ActorRefWithCell) ⇒ x.underlying.hasMessages
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns true if the actor is currently suspended.
|
||||
* It will always return false for remote actors.
|
||||
* Method is exposed to subclasses to be able to implement custom
|
||||
* routers based on mailbox and actor internal state.
|
||||
*/
|
||||
protected def isSuspended(a: Routee): Boolean = a match {
|
||||
case ActorRefRoutee(x: ActorRefWithCell) ⇒
|
||||
x.underlying match {
|
||||
case cell: ActorCell ⇒ cell.mailbox.isSuspended
|
||||
case _ ⇒ true
|
||||
}
|
||||
case _ ⇒ false
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the number of pending messages in the mailbox of the actor.
|
||||
* It will always return 0 for remote actors.
|
||||
* Method is exposed to subclasses to be able to implement custom
|
||||
* routers based on mailbox and actor internal state.
|
||||
*/
|
||||
protected def numberOfMessages(a: Routee): Int = a match {
|
||||
case ActorRefRoutee(x: ActorRefWithCell) ⇒ x.underlying.numberOfMessages
|
||||
case _ ⇒ 0
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A router pool that tries to send to the non-suspended routee with fewest messages in mailbox.
|
||||
* The selection is done in this order:
|
||||
* <ul>
|
||||
* <li>pick any idle routee (not processing message) with empty mailbox</li>
|
||||
* <li>pick any routee with empty mailbox</li>
|
||||
* <li>pick routee with fewest pending messages in mailbox</li>
|
||||
* <li>pick any remote routee, remote actors are consider lowest priority,
|
||||
* since their mailbox size is unknown</li>
|
||||
* </ul>
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `nrOfInstances` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param nrOfInstances initial number of routees in the pool
|
||||
*
|
||||
* @param resizer optional resizer that dynamically adjust the pool size
|
||||
*
|
||||
* @param supervisorStrategy strategy for supervising the routees, see 'Supervision Setup'
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* supervision, death watch and router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
final case class SmallestMailboxPool(
|
||||
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Pool with PoolOverrideUnsetConfig[SmallestMailboxPool] {
|
||||
|
||||
def this(config: Config) =
|
||||
this(
|
||||
nrOfInstances = config.getInt("nr-of-instances"),
|
||||
resizer = DefaultResizer.fromConfig(config))
|
||||
|
||||
/**
|
||||
* Java API
|
||||
* @param nr initial number of routees in the pool
|
||||
*/
|
||||
def this(nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = new Router(SmallestMailboxRoutingLogic())
|
||||
|
||||
/**
|
||||
* Setting the supervisor strategy to be used for the “head” Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): SmallestMailboxPool = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): SmallestMailboxPool = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* supervision, death watch and router management messages.
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): SmallestMailboxPool = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = this.overrideUnsetConfig(other)
|
||||
|
||||
}
|
||||
|
|
@ -1,15 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka
|
||||
|
||||
import scala.collection.immutable
|
||||
|
||||
package object routing {
|
||||
/**
|
||||
* Routing logic, partial function from (sender, message) to a
|
||||
* set of destinations.
|
||||
*/
|
||||
type Route = PartialFunction[(akka.actor.ActorRef, Any), immutable.Iterable[Destination]]
|
||||
}
|
||||
|
|
@ -237,6 +237,11 @@ akka {
|
|||
serialization-bindings {
|
||||
"akka.cluster.ClusterMessage" = akka-cluster
|
||||
}
|
||||
|
||||
router.type-mapping {
|
||||
adaptive-pool = "akka.cluster.routing.AdaptiveLoadBalancingPool"
|
||||
adaptive-group = "akka.cluster.routing.AdaptiveLoadBalancingGroup"
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -22,8 +22,6 @@ import akka.remote.RemoteDeployer
|
|||
import akka.remote.routing.RemoteRouterConfig
|
||||
import akka.routing.RouterConfig
|
||||
import akka.routing.DefaultResizer
|
||||
import akka.cluster.routing.ClusterRouterConfig
|
||||
import akka.cluster.routing.ClusterRouterSettings
|
||||
import akka.cluster.routing.AdaptiveLoadBalancingRouter
|
||||
import akka.cluster.routing.MixMetricsSelector
|
||||
import akka.cluster.routing.HeapMetricsSelector
|
||||
|
|
@ -33,6 +31,14 @@ import akka.cluster.routing.MetricsSelector
|
|||
import akka.dispatch.sysmsg.SystemMessage
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Props
|
||||
import akka.routing.Pool
|
||||
import akka.routing.Group
|
||||
import akka.cluster.routing.ClusterRouterPool
|
||||
import akka.cluster.routing.ClusterRouterGroup
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.routing.DeprecatedRouterConfig
|
||||
import akka.cluster.routing.ClusterRouterPoolSettings
|
||||
import akka.cluster.routing.ClusterRouterGroupSettings
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
|
|
@ -83,7 +89,15 @@ private[akka] class ClusterActorRefProvider(
|
|||
*/
|
||||
private[akka] class ClusterDeployer(_settings: ActorSystem.Settings, _pm: DynamicAccess) extends RemoteDeployer(_settings, _pm) {
|
||||
override def parseConfig(path: String, config: Config): Option[Deploy] = {
|
||||
super.parseConfig(path, config) match {
|
||||
|
||||
// For backwards compatibility we must add this fake routees.paths so that the deployer creates a Group
|
||||
// even though routees.paths is not defined. This will be cleaned up by ticket #3627
|
||||
val config2 =
|
||||
if (config.hasPath("cluster.routees-path"))
|
||||
config.withFallback(ConfigFactory.parseString(s"""routees.paths=["${config.getString("cluster.routees-path")}"]"""))
|
||||
else config
|
||||
|
||||
super.parseConfig(path, config2) match {
|
||||
case d @ Some(deploy) ⇒
|
||||
if (deploy.config.getBoolean("cluster.enabled")) {
|
||||
if (deploy.scope != NoScopeGiven)
|
||||
|
|
@ -91,50 +105,28 @@ private[akka] class ClusterDeployer(_settings: ActorSystem.Settings, _pm: Dynami
|
|||
if (deploy.routerConfig.isInstanceOf[RemoteRouterConfig])
|
||||
throw new ConfigurationException("Cluster deployment can't be combined with [%s]".format(deploy.routerConfig))
|
||||
|
||||
import ClusterRouterSettings.useRoleOption
|
||||
val clusterRouterSettings = ClusterRouterSettings(
|
||||
totalInstances = deploy.config.getInt("nr-of-instances"),
|
||||
maxInstancesPerNode = deploy.config.getInt("cluster.max-nr-of-instances-per-node"),
|
||||
allowLocalRoutees = deploy.config.getBoolean("cluster.allow-local-routees"),
|
||||
routeesPath = deploy.config.getString("cluster.routees-path"),
|
||||
useRole = useRoleOption(deploy.config.getString("cluster.use-role")))
|
||||
|
||||
Some(deploy.copy(
|
||||
routerConfig = ClusterRouterConfig(deploy.routerConfig, clusterRouterSettings), scope = ClusterScope))
|
||||
deploy.routerConfig match {
|
||||
case r: DeprecatedRouterConfig ⇒
|
||||
if (config.hasPath("cluster.routees-path"))
|
||||
Some(deploy.copy(
|
||||
routerConfig = ClusterRouterGroup(r, ClusterRouterGroupSettings.fromConfig(deploy.config)), scope = ClusterScope))
|
||||
else
|
||||
Some(deploy.copy(
|
||||
routerConfig = ClusterRouterPool(r, ClusterRouterPoolSettings.fromConfig(deploy.config)), scope = ClusterScope))
|
||||
case r: Pool ⇒
|
||||
Some(deploy.copy(
|
||||
routerConfig = ClusterRouterPool(r, ClusterRouterPoolSettings.fromConfig(deploy.config)), scope = ClusterScope))
|
||||
case r: Group ⇒
|
||||
Some(deploy.copy(
|
||||
routerConfig = ClusterRouterGroup(r, ClusterRouterGroupSettings.fromConfig(deploy.config)), scope = ClusterScope))
|
||||
case other ⇒
|
||||
throw new IllegalArgumentException(s"Cluster aware router can only wrap Pool or Group, got [${other.getClass.getName}]")
|
||||
}
|
||||
} else d
|
||||
case None ⇒ None
|
||||
}
|
||||
}
|
||||
|
||||
override protected def createRouterConfig(routerType: String, key: String, config: Config, deployment: Config): RouterConfig = {
|
||||
val routees = immutableSeq(deployment.getStringList("routees.paths"))
|
||||
val nrOfInstances = deployment.getInt("nr-of-instances")
|
||||
val resizer = if (config.hasPath("resizer")) Some(DefaultResizer(deployment.getConfig("resizer"))) else None
|
||||
|
||||
routerType match {
|
||||
case "adaptive" ⇒
|
||||
val metricsSelector = deployment.getString("metrics-selector") match {
|
||||
case "mix" ⇒ MixMetricsSelector
|
||||
case "heap" ⇒ HeapMetricsSelector
|
||||
case "cpu" ⇒ CpuMetricsSelector
|
||||
case "load" ⇒ SystemLoadAverageMetricsSelector
|
||||
case fqn ⇒
|
||||
val args = List(classOf[Config] -> deployment)
|
||||
dynamicAccess.createInstanceFor[MetricsSelector](fqn, args).recover({
|
||||
case exception ⇒ throw new IllegalArgumentException(
|
||||
("Cannot instantiate metrics-selector [%s], defined in [%s], " +
|
||||
"make sure it extends [akka.cluster.routing.MetricsSelector] and " +
|
||||
"has constructor with [com.typesafe.config.Config] parameter")
|
||||
.format(fqn, key), exception)
|
||||
}).get
|
||||
}
|
||||
|
||||
AdaptiveLoadBalancingRouter(metricsSelector, nrOfInstances, routees, resizer)
|
||||
|
||||
case _ ⇒ super.createRouterConfig(routerType, key, config, deployment)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
|
|
|
|||
|
|
@ -1,50 +1,105 @@
|
|||
/*
|
||||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.routing
|
||||
|
||||
import java.util.Arrays
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import java.util.concurrent.atomic.AtomicReference
|
||||
|
||||
import scala.annotation.tailrec
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
|
||||
import com.typesafe.config.Config
|
||||
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.ActorSystem
|
||||
import akka.actor.Address
|
||||
import akka.actor.OneForOneStrategy
|
||||
import akka.actor.DynamicAccess
|
||||
import akka.actor.NoSerializationVerificationNeeded
|
||||
import akka.actor.Props
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.ClusterEvent.ClusterMetricsChanged
|
||||
import akka.cluster.ClusterEvent.CurrentClusterState
|
||||
import akka.cluster.NodeMetrics
|
||||
import akka.cluster.StandardMetrics.Cpu
|
||||
import akka.cluster.StandardMetrics.HeapMemory
|
||||
import akka.event.Logging
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.japi.Util.immutableSeq
|
||||
import akka.routing._
|
||||
import akka.actor.Deploy
|
||||
|
||||
object AdaptiveLoadBalancingRouter {
|
||||
private val escalateStrategy: SupervisorStrategy = OneForOneStrategy() {
|
||||
case _ ⇒ SupervisorStrategy.Escalate
|
||||
/**
|
||||
* Load balancing of messages to cluster nodes based on cluster metric data.
|
||||
*
|
||||
* It uses random selection of routees based on probabilities derived from
|
||||
* the remaining capacity of corresponding node.
|
||||
*
|
||||
* @param system the actor system hosting this router
|
||||
*
|
||||
* @param metricsSelector decides what probability to use for selecting a routee, based
|
||||
* on remaining capacity as indicated by the node metrics
|
||||
*/
|
||||
final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsSelector: MetricsSelector = MixMetricsSelector)
|
||||
extends RoutingLogic with NoSerializationVerificationNeeded {
|
||||
|
||||
private val cluster = Cluster(system)
|
||||
|
||||
// The current weighted routees, if any. Weights are produced by the metricsSelector
|
||||
// via the metricsListener Actor. It's only updated by the actor, but accessed from
|
||||
// the threads of the senders.
|
||||
private val weightedRouteesRef =
|
||||
new AtomicReference[(immutable.IndexedSeq[Routee], Set[NodeMetrics], Option[WeightedRoutees])](
|
||||
(Vector.empty, Set.empty, None))
|
||||
|
||||
@tailrec final def metricsChanged(event: ClusterMetricsChanged): Unit = {
|
||||
val oldValue = weightedRouteesRef.get
|
||||
val (routees, _, _) = oldValue
|
||||
val weightedRoutees = Some(new WeightedRoutees(routees, cluster.selfAddress,
|
||||
metricsSelector.weights(event.nodeMetrics)))
|
||||
// retry when CAS failure
|
||||
if (!weightedRouteesRef.compareAndSet(oldValue, (routees, event.nodeMetrics, weightedRoutees)))
|
||||
metricsChanged(event)
|
||||
}
|
||||
|
||||
override def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee =
|
||||
if (routees.isEmpty) NoRoutee
|
||||
else {
|
||||
|
||||
def updateWeightedRoutees(): Option[WeightedRoutees] = {
|
||||
val oldValue = weightedRouteesRef.get
|
||||
val (oldRoutees, oldMetrics, oldWeightedRoutees) = oldValue
|
||||
|
||||
if (routees ne oldRoutees) {
|
||||
val weightedRoutees = Some(new WeightedRoutees(routees, cluster.selfAddress,
|
||||
metricsSelector.weights(oldMetrics)))
|
||||
// ignore, don't update, in case of CAS failure
|
||||
weightedRouteesRef.compareAndSet(oldValue, (routees, oldMetrics, weightedRoutees))
|
||||
weightedRoutees
|
||||
} else oldWeightedRoutees
|
||||
}
|
||||
|
||||
updateWeightedRoutees() match {
|
||||
case Some(weighted) ⇒
|
||||
if (weighted.isEmpty) NoRoutee
|
||||
else weighted(ThreadLocalRandom.current.nextInt(weighted.total) + 1)
|
||||
case None ⇒
|
||||
routees(ThreadLocalRandom.current.nextInt(routees.size))
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A Router that performs load balancing of messages to cluster nodes based on
|
||||
* A router pool that performs load balancing of messages to cluster nodes based on
|
||||
* cluster metric data.
|
||||
*
|
||||
* It uses random selection of routees based on probabilities derived from
|
||||
* the remaining capacity of corresponding node.
|
||||
*
|
||||
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical
|
||||
* sense as this means that the router should both create new actors and use the 'routees'
|
||||
* actor(s). In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
|
||||
* <br>
|
||||
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
|
||||
* be ignored if the router is defined in the configuration file for the actor being used.
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `nrOfInstances` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
|
|
@ -62,146 +117,127 @@ object AdaptiveLoadBalancingRouter {
|
|||
*
|
||||
* @param metricsSelector decides what probability to use for selecting a routee, based
|
||||
* on remaining capacity as indicated by the node metrics
|
||||
* @param routees string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*
|
||||
* @param nrOfInstances initial number of routees in the pool
|
||||
*
|
||||
* @param supervisorStrategy strategy for supervising the routees, see 'Supervision Setup'
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* supervision, death watch and router management messages
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
case class AdaptiveLoadBalancingRouter(
|
||||
final case class AdaptiveLoadBalancingPool(
|
||||
metricsSelector: MetricsSelector = MixMetricsSelector,
|
||||
nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = AdaptiveLoadBalancingRouter.escalateStrategy)
|
||||
extends RouterConfig with AdaptiveLoadBalancingRouterLike with OverrideUnsetConfig[AdaptiveLoadBalancingRouter] {
|
||||
override val nrOfInstances: Int = 0,
|
||||
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Pool {
|
||||
|
||||
def this(config: Config, dynamicAccess: DynamicAccess) =
|
||||
this(nrOfInstances = config.getInt("nr-of-instances"),
|
||||
metricsSelector = MetricsSelector.fromConfig(config, dynamicAccess))
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets nrOfInstances to be created.
|
||||
*
|
||||
* @param selector the selector is responsible for producing weighted mix of routees from the node metrics
|
||||
* @param nr number of routees to create
|
||||
* Java API
|
||||
* @param metricsSelector decides what probability to use for selecting a routee, based
|
||||
* on remaining capacity as indicated by the node metrics
|
||||
* @param nr initial number of routees in the pool
|
||||
*/
|
||||
def this(selector: MetricsSelector, nr: Int) = this(metricsSelector = selector, nrOfInstances = nr)
|
||||
def this(metricsSelector: MetricsSelector, nr: Int) = this(nrOfInstances = nr)
|
||||
|
||||
override def resizer: Option[Resizer] = None
|
||||
|
||||
override def createRouter(system: ActorSystem): Router =
|
||||
new Router(AdaptiveLoadBalancingRoutingLogic(system, metricsSelector))
|
||||
|
||||
override def routingLogicController(routingLogic: RoutingLogic): Option[Props] =
|
||||
Some(Props(classOf[AdaptiveLoadBalancingMetricsListener],
|
||||
routingLogic.asInstanceOf[AdaptiveLoadBalancingRoutingLogic]))
|
||||
|
||||
/**
|
||||
* 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 routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
* Setting the supervisor strategy to be used for the “head” Router actor.
|
||||
*/
|
||||
def this(selector: MetricsSelector, routeePaths: java.lang.Iterable[String]) =
|
||||
this(metricsSelector = selector, routees = immutableSeq(routeePaths))
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): AdaptiveLoadBalancingPool = copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the resizer to be used.
|
||||
*
|
||||
* @param selector the selector is responsible for producing weighted mix of routees from the node metrics
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* supervision, death watch and router management messages.
|
||||
*/
|
||||
def this(selector: MetricsSelector, resizer: Resizer) =
|
||||
this(metricsSelector = selector, resizer = Some(resizer))
|
||||
def withDispatcher(dispatcherId: String): AdaptiveLoadBalancingPool = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
* Uses the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): AdaptiveLoadBalancingRouter =
|
||||
copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): AdaptiveLoadBalancingRouter =
|
||||
copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API for setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): AdaptiveLoadBalancingRouter = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||
case _: FromConfig | _: NoRouter | _: AdaptiveLoadBalancingRouter ⇒ this.overrideUnsetConfig(other)
|
||||
case _ ⇒ throw new IllegalArgumentException("Expected AdaptiveLoadBalancingRouter, got [%s]".format(other))
|
||||
}
|
||||
override def withFallback(other: RouterConfig): RouterConfig =
|
||||
if (this.supervisorStrategy ne Pool.defaultSupervisorStrategy) this
|
||||
else other match {
|
||||
case _: FromConfig | _: NoRouter ⇒ this // NoRouter is the default, hence “neutral”
|
||||
case otherRouter: AdaptiveLoadBalancingPool ⇒
|
||||
if (otherRouter.supervisorStrategy eq Pool.defaultSupervisorStrategy) this
|
||||
else this.withSupervisorStrategy(otherRouter.supervisorStrategy)
|
||||
case otherRouter: AdaptiveLoadBalancingRouter ⇒
|
||||
if (otherRouter.supervisorStrategy eq Pool.defaultSupervisorStrategy) this
|
||||
else this.withSupervisorStrategy(otherRouter.supervisorStrategy)
|
||||
case _ ⇒ throw new IllegalArgumentException("Expected AdaptiveLoadBalancingPool, got [%s]".format(other))
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API.
|
||||
* A router group that performs load balancing of messages to cluster nodes based on
|
||||
* cluster metric data.
|
||||
*
|
||||
* This strategy is a metrics-aware router which performs load balancing of messages to
|
||||
* cluster nodes based on cluster metric data. It consumes [[akka.cluster.ClusterEvent.ClusterMetricsChanged]]
|
||||
* events and the [[akka.cluster.routing.MetricsSelector]] creates an mix of
|
||||
* weighted routees based on the node metrics. Messages are routed randomly to the
|
||||
* weighted routees, i.e. nodes with lower load are more likely to be used than nodes with
|
||||
* higher load
|
||||
* It uses random selection of routees based on probabilities derived from
|
||||
* the remaining capacity of corresponding node.
|
||||
*
|
||||
* The configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide `paths` during instantiation they will be ignored if
|
||||
* the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* @param metricsSelector decides what probability to use for selecting a routee, based
|
||||
* on remaining capacity as indicated by the node metrics
|
||||
*
|
||||
* @param paths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*
|
||||
* @param routerDispatcher dispatcher to use for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
trait AdaptiveLoadBalancingRouterLike { this: RouterConfig ⇒
|
||||
@SerialVersionUID(1L)
|
||||
final case class AdaptiveLoadBalancingGroup(
|
||||
metricsSelector: MetricsSelector = MixMetricsSelector,
|
||||
paths: immutable.Iterable[String] = Nil,
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||
extends Group {
|
||||
|
||||
def metricsSelector: MetricsSelector
|
||||
def this(config: Config, dynamicAccess: DynamicAccess) =
|
||||
this(metricsSelector = MetricsSelector.fromConfig(config, dynamicAccess),
|
||||
paths = immutableSeq(config.getStringList("routees.paths")))
|
||||
|
||||
def nrOfInstances: Int
|
||||
/**
|
||||
* Java API
|
||||
* @param metricsSelector decides what probability to use for selecting a routee, based
|
||||
* on remaining capacity as indicated by the node metrics
|
||||
* @param routeePaths string representation of the actor paths of the routees, messages are
|
||||
* sent with [[akka.actor.ActorSelection]] to these paths
|
||||
*/
|
||||
def this(metricsSelector: MetricsSelector,
|
||||
routeePaths: java.lang.Iterable[String]) = this(paths = immutableSeq(routeePaths))
|
||||
|
||||
def routees: immutable.Iterable[String]
|
||||
override def createRouter(system: ActorSystem): Router =
|
||||
new Router(AdaptiveLoadBalancingRoutingLogic(system, metricsSelector))
|
||||
|
||||
def routerDispatcher: String
|
||||
override def routingLogicController(routingLogic: RoutingLogic): Option[Props] =
|
||||
Some(Props(classOf[AdaptiveLoadBalancingMetricsListener],
|
||||
routingLogic.asInstanceOf[AdaptiveLoadBalancingRoutingLogic]))
|
||||
|
||||
override def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
if (resizer.isEmpty) {
|
||||
if (routees.isEmpty) routeeProvider.createRoutees(nrOfInstances)
|
||||
else routeeProvider.registerRouteesFor(routees)
|
||||
}
|
||||
/**
|
||||
* Setting the dispatcher to be used for the router head actor, which handles
|
||||
* router management messages
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): AdaptiveLoadBalancingGroup = copy(routerDispatcher = dispatcherId)
|
||||
|
||||
val log = Logging(routeeProvider.context.system, routeeProvider.context.self)
|
||||
|
||||
// The current weighted routees, if any. Weights are produced by the metricsSelector
|
||||
// via the metricsListener Actor. It's only updated by the actor, but accessed from
|
||||
// the threads of the senders.
|
||||
@volatile var weightedRoutees: Option[WeightedRoutees] = None
|
||||
|
||||
// subscribe to ClusterMetricsChanged and update weightedRoutees
|
||||
val metricsListener = routeeProvider.context.actorOf(Props(new Actor {
|
||||
|
||||
val cluster = Cluster(context.system)
|
||||
|
||||
override def preStart(): Unit = cluster.subscribe(self, classOf[ClusterMetricsChanged])
|
||||
override def postStop(): Unit = cluster.unsubscribe(self)
|
||||
|
||||
def receive = {
|
||||
case ClusterMetricsChanged(metrics) ⇒ receiveMetrics(metrics)
|
||||
case _: CurrentClusterState ⇒ // ignore
|
||||
}
|
||||
|
||||
def receiveMetrics(metrics: Set[NodeMetrics]): Unit = {
|
||||
// this is the only place from where weightedRoutees is updated
|
||||
weightedRoutees = Some(new WeightedRoutees(routeeProvider.routees, cluster.selfAddress,
|
||||
metricsSelector.weights(metrics)))
|
||||
}
|
||||
|
||||
}).withDispatcher(routerDispatcher).withDeploy(Deploy.local), name = "metricsListener")
|
||||
|
||||
def getNext(): ActorRef = weightedRoutees match {
|
||||
case Some(weighted) ⇒
|
||||
if (weighted.isEmpty) routeeProvider.context.system.deadLetters
|
||||
else weighted(ThreadLocalRandom.current.nextInt(weighted.total) + 1)
|
||||
case None ⇒
|
||||
val currentRoutees = routeeProvider.routees
|
||||
if (currentRoutees.isEmpty) routeeProvider.context.system.deadLetters
|
||||
else currentRoutees(ThreadLocalRandom.current.nextInt(currentRoutees.size))
|
||||
}
|
||||
|
||||
{
|
||||
case (sender, message) ⇒
|
||||
message match {
|
||||
case Broadcast(msg) ⇒ toAll(sender, routeeProvider.routees)
|
||||
case msg ⇒ List(Destination(sender, getNext()))
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -321,6 +357,24 @@ abstract class MixMetricsSelectorBase(selectors: immutable.IndexedSeq[CapacityMe
|
|||
|
||||
}
|
||||
|
||||
object MetricsSelector {
|
||||
def fromConfig(config: Config, dynamicAccess: DynamicAccess) =
|
||||
config.getString("metrics-selector") match {
|
||||
case "mix" ⇒ MixMetricsSelector
|
||||
case "heap" ⇒ HeapMetricsSelector
|
||||
case "cpu" ⇒ CpuMetricsSelector
|
||||
case "load" ⇒ SystemLoadAverageMetricsSelector
|
||||
case fqn ⇒
|
||||
val args = List(classOf[Config] -> config)
|
||||
dynamicAccess.createInstanceFor[MetricsSelector](fqn, args).recover({
|
||||
case exception ⇒ throw new IllegalArgumentException(
|
||||
(s"Cannot instantiate metrics-selector [$fqn], " +
|
||||
"make sure it extends [akka.cluster.routing.MetricsSelector] and " +
|
||||
"has constructor with [com.typesafe.config.Config] parameter"), exception)
|
||||
}).get
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* A MetricsSelector is responsible for producing weights from the node metrics.
|
||||
*/
|
||||
|
|
@ -336,6 +390,7 @@ trait MetricsSelector extends Serializable {
|
|||
* A MetricsSelector producing weights from remaining capacity.
|
||||
* The weights are typically proportional to the remaining capacity.
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
abstract class CapacityMetricsSelector extends MetricsSelector {
|
||||
|
||||
/**
|
||||
|
|
@ -376,23 +431,29 @@ abstract class CapacityMetricsSelector extends MetricsSelector {
|
|||
*
|
||||
* Pick routee based on its weight. Higher weight, higher probability.
|
||||
*/
|
||||
private[cluster] class WeightedRoutees(refs: immutable.IndexedSeq[ActorRef], selfAddress: Address, weights: Map[Address, Int]) {
|
||||
private[cluster] class WeightedRoutees(routees: immutable.IndexedSeq[Routee], selfAddress: Address, weights: Map[Address, Int]) {
|
||||
|
||||
// fill an array of same size as the refs with accumulated weights,
|
||||
// binarySearch is used to pick the right bucket from a requested value
|
||||
// from 1 to the total sum of the used weights.
|
||||
private val buckets: Array[Int] = {
|
||||
def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
|
||||
case Address(_, _, None, None) ⇒ selfAddress
|
||||
case a ⇒ a
|
||||
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) ⇒ selfAddress
|
||||
case a ⇒ a
|
||||
}
|
||||
}
|
||||
val buckets = Array.ofDim[Int](refs.size)
|
||||
val buckets = Array.ofDim[Int](routees.size)
|
||||
val meanWeight = if (weights.isEmpty) 1 else weights.values.sum / weights.size
|
||||
val w = weights.withDefaultValue(meanWeight) // we don’t necessarily have metrics for all addresses
|
||||
var i = 0
|
||||
var sum = 0
|
||||
refs foreach { ref ⇒
|
||||
sum += w(fullAddress(ref))
|
||||
routees foreach { r ⇒
|
||||
sum += w(fullAddress(r))
|
||||
buckets(i) = sum
|
||||
i += 1
|
||||
}
|
||||
|
|
@ -409,9 +470,9 @@ private[cluster] class WeightedRoutees(refs: immutable.IndexedSeq[ActorRef], sel
|
|||
/**
|
||||
* Pick the routee matching a value, from 1 to total.
|
||||
*/
|
||||
def apply(value: Int): ActorRef = {
|
||||
def apply(value: Int): Routee = {
|
||||
require(1 <= value && value <= total, "value must be between [1 - %s]" format total)
|
||||
refs(idx(Arrays.binarySearch(buckets, value)))
|
||||
routees(idx(Arrays.binarySearch(buckets, value)))
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -428,3 +489,132 @@ private[cluster] class WeightedRoutees(refs: immutable.IndexedSeq[ActorRef], sel
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* subscribe to ClusterMetricsChanged and update routing logic
|
||||
*/
|
||||
private[akka] class AdaptiveLoadBalancingMetricsListener(routingLogic: AdaptiveLoadBalancingRoutingLogic)
|
||||
extends Actor {
|
||||
|
||||
val cluster = Cluster(context.system)
|
||||
|
||||
override def preStart(): Unit = cluster.subscribe(self, classOf[ClusterMetricsChanged])
|
||||
|
||||
override def postStop(): Unit = cluster.unsubscribe(self)
|
||||
|
||||
def receive = {
|
||||
case event: ClusterMetricsChanged ⇒ routingLogic.metricsChanged(event)
|
||||
case _: CurrentClusterState ⇒ // ignore
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* A Router that performs load balancing of messages to cluster nodes based on
|
||||
* cluster metric data.
|
||||
*
|
||||
* It uses random selection of routees based on probabilities derived from
|
||||
* the remaining capacity of corresponding node.
|
||||
*
|
||||
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical
|
||||
* sense as this means that the router should both create new actors and use the 'routees'
|
||||
* actor(s). In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
|
||||
* <br>
|
||||
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
||||
* if you provide either 'nrOfInstances' or 'routees' during instantiation they will
|
||||
* be ignored if the router is defined in the configuration file for the actor being used.
|
||||
*
|
||||
* <h1>Supervision Setup</h1>
|
||||
*
|
||||
* Any routees that are created by a router will be created as the router's children.
|
||||
* The router is therefore also the children's supervisor.
|
||||
*
|
||||
* The supervision strategy of the router actor can be configured with
|
||||
* [[#withSupervisorStrategy]]. If no strategy is provided, routers default to
|
||||
* a strategy of “always escalate”. This means that errors are passed up to the
|
||||
* router's supervisor for handling.
|
||||
*
|
||||
* The router's supervisor will treat the error as an error with the router itself.
|
||||
* Therefore a directive to stop or restart will cause the router itself to stop or
|
||||
* restart. The router, in turn, will cause its children to stop and restart.
|
||||
*
|
||||
* @param metricsSelector decides what probability to use for selecting a routee, based
|
||||
* on remaining capacity as indicated by the node metrics
|
||||
* @param routees string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
@SerialVersionUID(1L)
|
||||
@deprecated("Use AdaptiveLoadBalancingPool or AdaptiveLoadBalancingGroup", "2.3")
|
||||
case class AdaptiveLoadBalancingRouter(
|
||||
metricsSelector: MetricsSelector = MixMetricsSelector,
|
||||
nrOfInstances: Int = 0, routees: immutable.Iterable[String] = Nil,
|
||||
override val resizer: Option[Resizer] = None,
|
||||
val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
|
||||
val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy)
|
||||
extends DeprecatedRouterConfig with PoolOverrideUnsetConfig[AdaptiveLoadBalancingRouter] {
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets nrOfInstances to be created.
|
||||
*
|
||||
* @param selector the selector is responsible for producing weighted mix of routees from the node metrics
|
||||
* @param nr number of routees to create
|
||||
*/
|
||||
def this(selector: MetricsSelector, nr: Int) = this(metricsSelector = selector, nrOfInstances = nr)
|
||||
|
||||
/**
|
||||
* 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 routeePaths string representation of the actor paths of the routees that will be looked up
|
||||
* using `actorFor` in [[akka.actor.ActorRefProvider]]
|
||||
*/
|
||||
def this(selector: MetricsSelector, routeePaths: java.lang.Iterable[String]) =
|
||||
this(metricsSelector = selector, routees = immutableSeq(routeePaths))
|
||||
|
||||
/**
|
||||
* Java API: Constructor that sets the resizer to be used.
|
||||
*
|
||||
* @param selector the selector is responsible for producing weighted mix of routees from the node metrics
|
||||
*/
|
||||
def this(selector: MetricsSelector, resizer: Resizer) =
|
||||
this(metricsSelector = selector, resizer = Some(resizer))
|
||||
|
||||
override def paths: immutable.Iterable[String] = routees
|
||||
|
||||
/**
|
||||
* Java API for setting routerDispatcher
|
||||
*/
|
||||
def withDispatcher(dispatcherId: String): AdaptiveLoadBalancingRouter =
|
||||
copy(routerDispatcher = dispatcherId)
|
||||
|
||||
/**
|
||||
* Java API for setting the supervisor strategy to be used for the “head”
|
||||
* Router actor.
|
||||
*/
|
||||
def withSupervisorStrategy(strategy: SupervisorStrategy): AdaptiveLoadBalancingRouter =
|
||||
copy(supervisorStrategy = strategy)
|
||||
|
||||
/**
|
||||
* Java API for setting the resizer to be used.
|
||||
*/
|
||||
def withResizer(resizer: Resizer): AdaptiveLoadBalancingRouter = copy(resizer = Some(resizer))
|
||||
|
||||
/**
|
||||
* Uses the resizer and/or the supervisor strategy of the given Routerconfig
|
||||
* if this RouterConfig doesn't have one, i.e. the resizer defined in code is used if
|
||||
* resizer was not defined in config.
|
||||
*/
|
||||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||
case _: FromConfig | _: NoRouter | _: AdaptiveLoadBalancingRouter ⇒ this.overrideUnsetConfig(other)
|
||||
case _ ⇒ throw new IllegalArgumentException("Expected AdaptiveLoadBalancingRouter, got [%s]".format(other))
|
||||
}
|
||||
|
||||
override def createRouter(system: ActorSystem): Router =
|
||||
new Router(AdaptiveLoadBalancingRoutingLogic(system, metricsSelector))
|
||||
|
||||
override def routingLogicController(routingLogic: RoutingLogic): Option[Props] =
|
||||
Some(Props(classOf[AdaptiveLoadBalancingMetricsListener],
|
||||
routingLogic.asInstanceOf[AdaptiveLoadBalancingRoutingLogic]))
|
||||
|
||||
}
|
||||
|
|
@ -3,140 +3,74 @@
|
|||
*/
|
||||
package akka.cluster.routing
|
||||
|
||||
import java.lang.IllegalStateException
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import scala.collection.immutable
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.ConfigurationException
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Address
|
||||
import akka.actor.Deploy
|
||||
import akka.routing.RouterConfig
|
||||
import akka.routing.Router
|
||||
import akka.actor.Props
|
||||
import akka.actor.ActorContext
|
||||
import akka.routing.Routee
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.actor.Address
|
||||
import akka.actor.ActorCell
|
||||
import akka.actor.Deploy
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.routing.ActorRefRoutee
|
||||
import akka.remote.RemoteScope
|
||||
import akka.actor.Actor
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.routing.Resizer
|
||||
import akka.routing.RouterConfig
|
||||
import akka.routing.Pool
|
||||
import akka.routing.Group
|
||||
import akka.remote.routing.RemoteRouterConfig
|
||||
import akka.routing.RouterActor
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.ClusterEvent._
|
||||
import akka.actor.ActorRef
|
||||
import akka.cluster.Member
|
||||
import akka.cluster.MemberStatus
|
||||
import akka.remote.RemoteScope
|
||||
import akka.routing.Destination
|
||||
import akka.routing.Resizer
|
||||
import akka.routing.Route
|
||||
import akka.routing.RouteeProvider
|
||||
import akka.routing.Router
|
||||
import akka.routing.RouterConfig
|
||||
import akka.remote.routing.RemoteRouterConfig
|
||||
import akka.actor.RootActorPath
|
||||
import akka.actor.ActorCell
|
||||
import akka.actor.RelativeActorPath
|
||||
import scala.annotation.tailrec
|
||||
import akka.actor.RootActorPath
|
||||
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
|
||||
import akka.actor.RelativeActorPath
|
||||
import com.typesafe.config.Config
|
||||
import akka.routing.DeprecatedRouterConfig
|
||||
|
||||
/**
|
||||
* [[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 ClusterRouterConfig(local: RouterConfig, settings: ClusterRouterSettings) extends RouterConfig {
|
||||
|
||||
override def createRouteeProvider(context: ActorContext, routeeProps: Props) =
|
||||
new ClusterRouteeProvider(context, routeeProps, resizer, settings)
|
||||
|
||||
override def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
val localRoute = local.createRoute(routeeProvider)
|
||||
|
||||
// Intercept ClusterDomainEvent and route them to the ClusterRouterActor
|
||||
({
|
||||
case (sender, message: ClusterDomainEvent) ⇒ List(Destination(sender, routeeProvider.context.self))
|
||||
}: Route) orElse localRoute
|
||||
}
|
||||
|
||||
override def createActor(): Router = new ClusterRouterActor(local.supervisorStrategy)
|
||||
|
||||
override def supervisorStrategy: SupervisorStrategy = local.supervisorStrategy
|
||||
|
||||
override def routerDispatcher: String = local.routerDispatcher
|
||||
|
||||
override def resizer: Option[Resizer] = local.resizer
|
||||
|
||||
override def stopRouterWhenAllRouteesRemoved: Boolean = false
|
||||
|
||||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||
case ClusterRouterConfig(_: RemoteRouterConfig, _) ⇒ throw new IllegalStateException(
|
||||
"ClusterRouterConfig is not allowed to wrap a RemoteRouterConfig")
|
||||
case ClusterRouterConfig(_: ClusterRouterConfig, _) ⇒ throw new IllegalStateException(
|
||||
"ClusterRouterConfig is not allowed to wrap a ClusterRouterConfig")
|
||||
case ClusterRouterConfig(local, _) ⇒ copy(local = this.local.withFallback(local))
|
||||
case _ ⇒ copy(local = this.local.withFallback(other))
|
||||
}
|
||||
}
|
||||
|
||||
object ClusterRouterSettings {
|
||||
/**
|
||||
* Settings for create and deploy of the routees
|
||||
*/
|
||||
def apply(totalInstances: Int, maxInstancesPerNode: Int, allowLocalRoutees: Boolean, useRole: Option[String]): ClusterRouterSettings =
|
||||
new ClusterRouterSettings(totalInstances, maxInstancesPerNode, routeesPath = "", allowLocalRoutees, useRole)
|
||||
|
||||
/**
|
||||
* Settings for remote deployment of the routees, allowed to use routees on own node
|
||||
*/
|
||||
def apply(totalInstances: Int, maxInstancesPerNode: Int, useRole: Option[String]): ClusterRouterSettings =
|
||||
apply(totalInstances, maxInstancesPerNode, allowLocalRoutees = true, useRole)
|
||||
|
||||
/**
|
||||
* Settings for lookup of the routees
|
||||
*/
|
||||
def apply(totalInstances: Int, routeesPath: String, allowLocalRoutees: Boolean, useRole: Option[String]): ClusterRouterSettings =
|
||||
new ClusterRouterSettings(totalInstances, maxInstancesPerNode = 1, routeesPath, allowLocalRoutees, useRole)
|
||||
|
||||
/**
|
||||
* Settings for lookup of the routees, allowed to use routees on own node
|
||||
*/
|
||||
def apply(totalInstances: Int, routeesPath: String, useRole: Option[String]): ClusterRouterSettings =
|
||||
apply(totalInstances, routeesPath, allowLocalRoutees = true, useRole)
|
||||
|
||||
def useRoleOption(role: String): Option[String] = role match {
|
||||
case null | "" ⇒ None
|
||||
case _ ⇒ Some(role)
|
||||
}
|
||||
object ClusterRouterGroupSettings {
|
||||
def fromConfig(config: Config): ClusterRouterGroupSettings =
|
||||
ClusterRouterGroupSettings(
|
||||
totalInstances = config.getInt("nr-of-instances"),
|
||||
routeesPath = config.getString("cluster.routees-path"),
|
||||
allowLocalRoutees = config.getBoolean("cluster.allow-local-routees"),
|
||||
useRole = ClusterRouterSettingsBase.useRoleOption(config.getString("cluster.use-role")))
|
||||
}
|
||||
|
||||
/**
|
||||
* `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 ClusterRouterSettings private[akka] (
|
||||
case class ClusterRouterGroupSettings(
|
||||
totalInstances: Int,
|
||||
maxInstancesPerNode: Int,
|
||||
routeesPath: String,
|
||||
allowLocalRoutees: Boolean,
|
||||
useRole: Option[String]) {
|
||||
useRole: Option[String]) extends ClusterRouterSettingsBase {
|
||||
|
||||
/**
|
||||
* Java API: Settings for create and deploy of the routees
|
||||
*/
|
||||
def this(totalInstances: Int, maxInstancesPerNode: Int, allowLocalRoutees: Boolean, useRole: String) =
|
||||
this(totalInstances, maxInstancesPerNode, routeesPath = "", allowLocalRoutees,
|
||||
ClusterRouterSettings.useRoleOption(useRole))
|
||||
|
||||
/**
|
||||
* Java API: Settings for lookup of the routees
|
||||
* Java API
|
||||
*/
|
||||
def this(totalInstances: Int, routeesPath: String, allowLocalRoutees: Boolean, useRole: String) =
|
||||
this(totalInstances, maxInstancesPerNode = 1, routeesPath, allowLocalRoutees,
|
||||
ClusterRouterSettings.useRoleOption(useRole))
|
||||
this(totalInstances, routeesPath, allowLocalRoutees, ClusterRouterSettingsBase.useRoleOption(useRole))
|
||||
|
||||
if (totalInstances <= 0) throw new IllegalArgumentException("totalInstances of cluster router must be > 0")
|
||||
if (maxInstancesPerNode <= 0) throw new IllegalArgumentException("maxInstancesPerNode of cluster router must be > 0")
|
||||
if (isRouteesPathDefined && maxInstancesPerNode != 1)
|
||||
throw new IllegalArgumentException("maxInstancesPerNode of cluster router must be 1 when routeesPath is defined")
|
||||
if (!isRouteesPathDefined) throw new IllegalArgumentException("routeesPath must be defined")
|
||||
|
||||
val routeesPathElements: immutable.Iterable[String] = routeesPath match {
|
||||
case RelativeActorPath(elements) ⇒ elements
|
||||
routeesPath match {
|
||||
case RelativeActorPath(elements) ⇒ // good
|
||||
case _ ⇒
|
||||
throw new IllegalArgumentException("routeesPath [%s] is not a valid relative actor path" format routeesPath)
|
||||
}
|
||||
|
|
@ -145,70 +79,299 @@ case class ClusterRouterSettings private[akka] (
|
|||
|
||||
}
|
||||
|
||||
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")))
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*
|
||||
* Factory and registry for routees of the router.
|
||||
* Deploys new routees on the cluster nodes.
|
||||
* `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
|
||||
*/
|
||||
private[akka] class ClusterRouteeProvider(
|
||||
_context: ActorContext,
|
||||
_routeeProps: Props,
|
||||
_resizer: Option[Resizer],
|
||||
settings: ClusterRouterSettings)
|
||||
extends RouteeProvider(_context, _routeeProps, _resizer) {
|
||||
|
||||
// need this counter as instance variable since Resizer may call createRoutees several times
|
||||
private val childNameCounter = new AtomicInteger
|
||||
|
||||
override def registerRouteesFor(paths: immutable.Iterable[String]): Unit =
|
||||
throw new ConfigurationException("Cluster deployment can not be combined with routees for [%s]"
|
||||
format context.self.path.toString)
|
||||
@SerialVersionUID(1L)
|
||||
case class ClusterRouterPoolSettings(
|
||||
totalInstances: Int,
|
||||
maxInstancesPerNode: Int,
|
||||
allowLocalRoutees: Boolean,
|
||||
useRole: Option[String]) extends ClusterRouterSettingsBase {
|
||||
|
||||
/**
|
||||
* Note that nrOfInstances is ignored for cluster routers, instead
|
||||
* the `totalInstances` parameter is used. That is the same when
|
||||
* using config to define `nr-of-instances`, but when defining the
|
||||
* router programatically or using [[akka.routing.Resizer]] they
|
||||
* might be different. `totalInstances` is the relevant parameter
|
||||
* to use for cluster routers.
|
||||
* Java API
|
||||
*/
|
||||
override def createRoutees(nrOfInstances: Int): Unit = {
|
||||
def this(totalInstances: Int, maxInstancesPerNode: Int, allowLocalRoutees: Boolean, useRole: String) =
|
||||
this(totalInstances, maxInstancesPerNode, allowLocalRoutees, ClusterRouterSettingsBase.useRoleOption(useRole))
|
||||
|
||||
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)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] trait ClusterRouterSettingsBase {
|
||||
def totalInstances: Int
|
||||
def allowLocalRoutees: Boolean
|
||||
def useRole: Option[String]
|
||||
|
||||
if (totalInstances <= 0) throw new IllegalArgumentException("totalInstances of cluster router must be > 0")
|
||||
}
|
||||
|
||||
/**
|
||||
* [[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 {
|
||||
|
||||
require(settings.routeesPath.nonEmpty, "routeesPath must be defined")
|
||||
|
||||
override def paths: immutable.Iterable[String] = if (settings.allowLocalRoutees) List(settings.routeesPath) else Nil
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
override private[akka] def createRouterActor(): RouterActor = new ClusterRouterGroupActor(settings)
|
||||
|
||||
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])
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* [[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 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
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
override private[akka] def newRoutee(routeeProps: Props, context: ActorContext): Routee = {
|
||||
val name = "c" + childNameCounter.incrementAndGet
|
||||
val ref = context.asInstanceOf[ActorCell].attachChild(routeeProps, name, systemService = false)
|
||||
ActorRefRoutee(ref)
|
||||
}
|
||||
|
||||
/**
|
||||
* Initial number of routee instances
|
||||
*/
|
||||
override def nrOfInstances: Int = if (settings.allowLocalRoutees) settings.maxInstancesPerNode else 0
|
||||
|
||||
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
|
||||
|
||||
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])
|
||||
case _ ⇒
|
||||
copy(local = this.local.withFallback(other).asInstanceOf[Pool])
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
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)
|
||||
|
||||
// Intercept ClusterDomainEvent and route them to the ClusterRouterActor
|
||||
override def isManagementMessage(msg: Any): Boolean =
|
||||
(msg.isInstanceOf[ClusterDomainEvent]) || super.isManagementMessage(msg)
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ClusterRouterPoolActor(
|
||||
supervisorStrategy: SupervisorStrategy, val settings: ClusterRouterPoolSettings)
|
||||
extends RouterPoolActor(supervisorStrategy) with ClusterRouterActor {
|
||||
|
||||
override def receive = clusterReceive orElse super.receive
|
||||
|
||||
/**
|
||||
* Adds routees based on totalInstances and maxInstancesPerNode settings
|
||||
*/
|
||||
override def addRoutees(): Unit = {
|
||||
@tailrec
|
||||
def doCreateRoutees(): Unit = selectDeploymentTarget match {
|
||||
def doAddRoutees(): Unit = selectDeploymentTarget match {
|
||||
case None ⇒ // done
|
||||
case Some(target) ⇒
|
||||
val ref =
|
||||
if (settings.isRouteesPathDefined) {
|
||||
context.actorFor(RootActorPath(target) / settings.routeesPathElements)
|
||||
} else {
|
||||
val name = "c" + childNameCounter.incrementAndGet
|
||||
val deploy = Deploy(config = ConfigFactory.empty(), routerConfig = routeeProps.routerConfig,
|
||||
scope = RemoteScope(target))
|
||||
context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false)
|
||||
}
|
||||
val routeeProps = cell.routeeProps
|
||||
val deploy = Deploy(config = ConfigFactory.empty(), routerConfig = routeeProps.routerConfig,
|
||||
scope = RemoteScope(target))
|
||||
val routee = pool.newRoutee(routeeProps.withDeploy(deploy), context)
|
||||
// must register each one, since registered routees are used in selectDeploymentTarget
|
||||
registerRoutees(List(ref))
|
||||
cell.addRoutee(routee)
|
||||
|
||||
// recursion until all created
|
||||
doCreateRoutees()
|
||||
doAddRoutees()
|
||||
}
|
||||
|
||||
doCreateRoutees()
|
||||
doAddRoutees()
|
||||
}
|
||||
|
||||
private[routing] def createRoutees(): Unit = createRoutees(settings.totalInstances)
|
||||
override def maxInstancesPerNode: Int = settings.maxInstancesPerNode
|
||||
|
||||
override def unregisterRoutees(routees: immutable.Iterable[ActorRef]): Unit = {
|
||||
super.unregisterRoutees(routees)
|
||||
if (!settings.isRouteesPathDefined) {
|
||||
// stop remote deployed routees
|
||||
routees foreach context.stop
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] class ClusterRouterGroupActor(val settings: ClusterRouterGroupSettings)
|
||||
extends RouterActor with ClusterRouterActor {
|
||||
|
||||
val group = cell.routerConfig match {
|
||||
case x: Group ⇒ x
|
||||
case other ⇒
|
||||
throw ActorInitializationException("ClusterRouterGroupActor can only be used with Nozle, not " + other.getClass)
|
||||
}
|
||||
|
||||
override def receive = clusterReceive orElse super.receive
|
||||
|
||||
/**
|
||||
* Adds routees based on totalInstances and maxInstancesPerNode settings
|
||||
*/
|
||||
override def addRoutees(): Unit = {
|
||||
@tailrec
|
||||
def doAddRoutees(): Unit = selectDeploymentTarget match {
|
||||
case None ⇒ // done
|
||||
case Some(target) ⇒
|
||||
val routee = group.routeeFor(target + settings.routeesPath, context)
|
||||
// must register each one, since registered routees are used in selectDeploymentTarget
|
||||
cell.addRoutee(routee)
|
||||
|
||||
// recursion until all created
|
||||
doAddRoutees()
|
||||
}
|
||||
|
||||
doAddRoutees()
|
||||
}
|
||||
|
||||
override def maxInstancesPerNode: Int = 1
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
override def preStart(): Unit = {
|
||||
cluster.subscribe(self, classOf[MemberEvent])
|
||||
cluster.subscribe(self, classOf[ReachabilityEvent])
|
||||
}
|
||||
override def postStop(): Unit = cluster.unsubscribe(self)
|
||||
|
||||
var nodes: immutable.SortedSet[Address] = {
|
||||
import Member.addressOrdering
|
||||
cluster.readView.members.collect {
|
||||
case m if isAvailable(m) ⇒ m.address
|
||||
}
|
||||
}
|
||||
|
||||
private def selectDeploymentTarget: Option[Address] = {
|
||||
val currentRoutees = routees
|
||||
def isAvailable(m: Member): Boolean =
|
||||
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)
|
||||
}
|
||||
|
||||
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
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Adds routees based on totalInstances and maxInstancesPerNode settings
|
||||
*/
|
||||
def addRoutees(): Unit
|
||||
|
||||
def maxInstancesPerNode: Int
|
||||
|
||||
def selectDeploymentTarget: Option[Address] = {
|
||||
val currentRoutees = cell.router.routees
|
||||
val currentNodes = availableNodes
|
||||
if (currentNodes.isEmpty || currentRoutees.size >= settings.totalInstances) {
|
||||
None
|
||||
|
|
@ -221,113 +384,47 @@ private[akka] class ClusterRouteeProvider(
|
|||
}
|
||||
|
||||
val (address, count) = numberOfRouteesPerNode.minBy(_._2)
|
||||
if (count < settings.maxInstancesPerNode) Some(address) else None
|
||||
if (count < maxInstancesPerNode) Some(address) else None
|
||||
}
|
||||
}
|
||||
|
||||
private[routing] def cluster: Cluster = Cluster(context.system)
|
||||
|
||||
/**
|
||||
* Fills in self address for local ActorRef
|
||||
*/
|
||||
private[routing] def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
|
||||
case Address(_, _, None, None) ⇒ cluster.selfAddress
|
||||
case a ⇒ a
|
||||
def addMember(member: Member) = {
|
||||
nodes += member.address
|
||||
addRoutees()
|
||||
}
|
||||
|
||||
private[routing] 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
|
||||
}
|
||||
|
||||
@volatile
|
||||
private[routing] var nodes: immutable.SortedSet[Address] = {
|
||||
import Member.addressOrdering
|
||||
cluster.readView.members.collect {
|
||||
case m if isAvailable(m) ⇒ m.address
|
||||
}
|
||||
}
|
||||
|
||||
private[routing] def isAvailable(m: Member): Boolean =
|
||||
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)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
* The router actor, subscribes to cluster events.
|
||||
*/
|
||||
private[akka] class ClusterRouterActor(override val supervisorStrategy: SupervisorStrategy) extends Router {
|
||||
|
||||
// re-subscribe when restart
|
||||
override def preStart(): Unit = {
|
||||
cluster.subscribe(self, classOf[MemberEvent])
|
||||
cluster.subscribe(self, classOf[ReachabilityEvent])
|
||||
}
|
||||
override def postStop(): Unit = cluster.unsubscribe(self)
|
||||
|
||||
// lazy to not interfere with RoutedActorCell initialization
|
||||
lazy val routeeProvider: ClusterRouteeProvider = ref.routeeProvider match {
|
||||
case x: ClusterRouteeProvider ⇒ x
|
||||
case _ ⇒ throw new IllegalStateException(
|
||||
"ClusterRouteeProvider must be used together with [%s]".format(getClass))
|
||||
}
|
||||
|
||||
def cluster: Cluster = routeeProvider.cluster
|
||||
|
||||
def fullAddress(actorRef: ActorRef): Address = routeeProvider.fullAddress(actorRef)
|
||||
|
||||
def registerRoutees(member: Member) = {
|
||||
routeeProvider.nodes += member.address
|
||||
// createRoutees will create routees based on
|
||||
// totalInstances and maxInstancesPerNode
|
||||
routeeProvider.createRoutees()
|
||||
}
|
||||
|
||||
def unregisterRoutees(member: Member) = {
|
||||
def removeMember(member: Member) = {
|
||||
val address = member.address
|
||||
routeeProvider.nodes -= address
|
||||
nodes -= address
|
||||
|
||||
// unregister routees that live on that node
|
||||
val affectedRoutees = routeeProvider.routees.filter(fullAddress(_) == address)
|
||||
routeeProvider.unregisterRoutees(affectedRoutees)
|
||||
val affectedRoutees = cell.router.routees.filter(fullAddress(_) == address)
|
||||
cell.removeRoutees(affectedRoutees, stopChild = true)
|
||||
|
||||
// createRoutees will not create more than createRoutees and maxInstancesPerNode
|
||||
// addRoutees will not create more than createRoutees and maxInstancesPerNode
|
||||
// this is useful when totalInstances < upNodes.size
|
||||
routeeProvider.createRoutees()
|
||||
addRoutees()
|
||||
}
|
||||
|
||||
override def routerReceive: Receive = {
|
||||
def clusterReceive: Receive = {
|
||||
case s: CurrentClusterState ⇒
|
||||
import Member.addressOrdering
|
||||
routeeProvider.nodes = s.members.collect { case m if routeeProvider.isAvailable(m) ⇒ m.address }
|
||||
routeeProvider.createRoutees()
|
||||
nodes = s.members.collect { case m if isAvailable(m) ⇒ m.address }
|
||||
addRoutees()
|
||||
|
||||
case m: MemberEvent if routeeProvider.isAvailable(m.member) ⇒
|
||||
registerRoutees(m.member)
|
||||
case m: MemberEvent if isAvailable(m.member) ⇒
|
||||
addMember(m.member)
|
||||
|
||||
case other: MemberEvent ⇒
|
||||
// other events means that it is no longer interesting, such as
|
||||
// MemberExited, MemberRemoved
|
||||
unregisterRoutees(other.member)
|
||||
removeMember(other.member)
|
||||
|
||||
case UnreachableMember(m) ⇒
|
||||
unregisterRoutees(m)
|
||||
removeMember(m)
|
||||
|
||||
case ReachableMember(m) ⇒
|
||||
if (routeeProvider.isAvailable(m))
|
||||
registerRoutees(m)
|
||||
if (isAvailable(m)) addMember(m)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,167 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster.routing
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.routing.RouterConfig
|
||||
import akka.routing.Router
|
||||
import akka.actor.Props
|
||||
import akka.actor.ActorContext
|
||||
import akka.routing.Routee
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.actor.Address
|
||||
import akka.actor.ActorCell
|
||||
import akka.actor.Deploy
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.routing.ActorRefRoutee
|
||||
import akka.remote.RemoteScope
|
||||
import akka.actor.Actor
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.routing.Resizer
|
||||
import akka.routing.RouterConfig
|
||||
import akka.routing.Pool
|
||||
import akka.routing.Group
|
||||
import akka.remote.routing.RemoteRouterConfig
|
||||
import akka.routing.RouterActor
|
||||
import akka.cluster.Cluster
|
||||
import akka.cluster.ClusterEvent._
|
||||
import akka.actor.ActorRef
|
||||
import akka.cluster.Member
|
||||
import scala.annotation.tailrec
|
||||
import akka.actor.RootActorPath
|
||||
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
|
||||
import akka.actor.RelativeActorPath
|
||||
import com.typesafe.config.Config
|
||||
import akka.routing.DeprecatedRouterConfig
|
||||
|
||||
@deprecated("Use ClusterRouterPoolSettings or ClusterRouterGroupSettings", "2.3")
|
||||
object ClusterRouterSettings {
|
||||
/**
|
||||
* Settings for create and deploy of the routees
|
||||
*/
|
||||
def apply(totalInstances: Int, maxInstancesPerNode: Int, allowLocalRoutees: Boolean, useRole: Option[String]): ClusterRouterSettings =
|
||||
new ClusterRouterSettings(totalInstances, maxInstancesPerNode, routeesPath = "", allowLocalRoutees, useRole)
|
||||
|
||||
/**
|
||||
* Settings for remote deployment of the routees, allowed to use routees on own node
|
||||
*/
|
||||
def apply(totalInstances: Int, maxInstancesPerNode: Int, useRole: Option[String]): ClusterRouterSettings =
|
||||
apply(totalInstances, maxInstancesPerNode, allowLocalRoutees = true, useRole)
|
||||
|
||||
/**
|
||||
* Settings for lookup of the routees
|
||||
*/
|
||||
def apply(totalInstances: Int, routeesPath: String, allowLocalRoutees: Boolean, useRole: Option[String]): ClusterRouterSettings =
|
||||
new ClusterRouterSettings(totalInstances, maxInstancesPerNode = 1, routeesPath, allowLocalRoutees, useRole)
|
||||
|
||||
/**
|
||||
* Settings for lookup of the routees, allowed to use routees on own node
|
||||
*/
|
||||
def apply(totalInstances: Int, routeesPath: String, useRole: Option[String]): ClusterRouterSettings =
|
||||
apply(totalInstances, routeesPath, allowLocalRoutees = true, useRole)
|
||||
|
||||
def useRoleOption(role: String): Option[String] = role match {
|
||||
case null | "" ⇒ None
|
||||
case _ ⇒ Some(role)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
* `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)
|
||||
@deprecated("Use ClusterRouterPoolSettings or ClusterRouterGroupSettings", "2.3")
|
||||
case class ClusterRouterSettings private[akka] (
|
||||
totalInstances: Int,
|
||||
maxInstancesPerNode: Int,
|
||||
routeesPath: String,
|
||||
allowLocalRoutees: Boolean,
|
||||
useRole: Option[String]) extends ClusterRouterSettingsBase {
|
||||
|
||||
/**
|
||||
* Java API: Settings for create and deploy of the routees
|
||||
*/
|
||||
def this(totalInstances: Int, maxInstancesPerNode: Int, allowLocalRoutees: Boolean, useRole: String) =
|
||||
this(totalInstances, maxInstancesPerNode, routeesPath = "", allowLocalRoutees,
|
||||
ClusterRouterSettings.useRoleOption(useRole))
|
||||
|
||||
/**
|
||||
* Java API: Settings for lookup of the routees
|
||||
*/
|
||||
def this(totalInstances: Int, routeesPath: String, allowLocalRoutees: Boolean, useRole: String) =
|
||||
this(totalInstances, maxInstancesPerNode = 1, routeesPath, allowLocalRoutees,
|
||||
ClusterRouterSettings.useRoleOption(useRole))
|
||||
|
||||
if (totalInstances <= 0) throw new IllegalArgumentException("totalInstances of cluster router must be > 0")
|
||||
if (maxInstancesPerNode <= 0) throw new IllegalArgumentException("maxInstancesPerNode of cluster router must be > 0")
|
||||
if (isRouteesPathDefined && maxInstancesPerNode != 1)
|
||||
throw new IllegalArgumentException("maxInstancesPerNode of cluster router must be 1 when routeesPath is defined")
|
||||
|
||||
routeesPath match {
|
||||
case RelativeActorPath(elements) ⇒ // good
|
||||
case _ ⇒
|
||||
throw new IllegalArgumentException("routeesPath [%s] is not a valid relative actor path" format routeesPath)
|
||||
}
|
||||
|
||||
def isRouteesPathDefined: Boolean = (routeesPath ne null) && routeesPath != ""
|
||||
|
||||
}
|
||||
|
||||
@deprecated("Use ClusterRouterPool or ClusterRouterGroup", "2.3")
|
||||
@SerialVersionUID(1L)
|
||||
final case class ClusterRouterConfig(local: DeprecatedRouterConfig, settings: ClusterRouterSettings) extends DeprecatedRouterConfig with ClusterRouterConfigBase {
|
||||
|
||||
require(local.resizer.isEmpty, "Resizer can't be used together with cluster router")
|
||||
|
||||
@transient private val childNameCounter = new AtomicInteger
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
override private[akka] def newRoutee(routeeProps: Props, context: ActorContext): Routee = {
|
||||
val name = "c" + childNameCounter.incrementAndGet
|
||||
val ref = context.asInstanceOf[ActorCell].attachChild(routeeProps, name, systemService = false)
|
||||
ActorRefRoutee(ref)
|
||||
}
|
||||
|
||||
override def nrOfInstances: Int = if (settings.allowLocalRoutees) settings.maxInstancesPerNode else 0
|
||||
|
||||
override def paths: immutable.Iterable[String] =
|
||||
if (settings.allowLocalRoutees && settings.routeesPath.nonEmpty) List(settings.routeesPath) else Nil
|
||||
|
||||
override def resizer: Option[Resizer] = local.resizer
|
||||
|
||||
/**
|
||||
* INTERNAL API
|
||||
*/
|
||||
override private[akka] def createRouterActor(): RouterActor =
|
||||
if (settings.routeesPath.isEmpty)
|
||||
new ClusterRouterPoolActor(local.supervisorStrategy, ClusterRouterPoolSettings(settings.totalInstances,
|
||||
settings.maxInstancesPerNode, settings.allowLocalRoutees, settings.useRole))
|
||||
else
|
||||
new ClusterRouterGroupActor(ClusterRouterGroupSettings(settings.totalInstances, settings.routeesPath,
|
||||
settings.allowLocalRoutees, settings.useRole))
|
||||
|
||||
override def supervisorStrategy: SupervisorStrategy = local.supervisorStrategy
|
||||
|
||||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||
case ClusterRouterConfig(_: ClusterRouterConfig, _) ⇒ throw new IllegalStateException(
|
||||
"ClusterRouterConfig is not allowed to wrap a ClusterRouterConfig")
|
||||
case ClusterRouterConfig(local, _) ⇒
|
||||
copy(local = this.local.withFallback(local).asInstanceOf[DeprecatedRouterConfig])
|
||||
case _ ⇒
|
||||
copy(local = this.local.withFallback(other).asInstanceOf[DeprecatedRouterConfig])
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -136,30 +136,30 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
akka.actor.deployment {
|
||||
/master-node-1/workers {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 100
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 1
|
||||
allow-local-routees = off
|
||||
allow-local-routees = on
|
||||
}
|
||||
}
|
||||
/master-node-2/workers {
|
||||
router = round-robin
|
||||
router = round-robin-group
|
||||
nr-of-instances = 100
|
||||
cluster {
|
||||
enabled = on
|
||||
routees-path = "/user/worker"
|
||||
allow-local-routees = off
|
||||
allow-local-routees = on
|
||||
}
|
||||
}
|
||||
/master-node-3/workers = {
|
||||
router = adaptive
|
||||
router = adaptive-pool
|
||||
nr-of-instances = 100
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 1
|
||||
allow-local-routees = off
|
||||
allow-local-routees = on
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -486,7 +486,7 @@ private[cluster] object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
* itself.
|
||||
*/
|
||||
class Master(settings: StressMultiJvmSpec.Settings, batchInterval: FiniteDuration, tree: Boolean) extends Actor {
|
||||
val workers = context.actorOf(Props[Worker].withRouter(FromConfig), "workers")
|
||||
val workers = context.actorOf(FromConfig.props(Props[Worker]), "workers")
|
||||
val payload = Array.fill(settings.payloadSize)(ThreadLocalRandom.current.nextInt(127).toByte)
|
||||
val retryTimeout = 5.seconds.dilated(context.system)
|
||||
val idCounter = Iterator from 0
|
||||
|
|
@ -1314,6 +1314,6 @@ abstract class StressSpec
|
|||
}
|
||||
enterBarrier("after-" + step)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -28,6 +28,7 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
|
|||
akka.loggers = ["akka.testkit.TestEventListener"]
|
||||
akka.loglevel = INFO
|
||||
akka.remote.log-remote-lifecycle-events = off
|
||||
akka.cluster.failure-detector.monitored-by-nr-of-members = 3
|
||||
"""))
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -0,0 +1,212 @@
|
|||
/*
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.cluster.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
import java.lang.management.ManagementFactory
|
||||
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.FromConfig
|
||||
import akka.routing.RouterRoutees
|
||||
import akka.testkit.{ LongRunningTest, DefaultTimeout, ImplicitSender }
|
||||
import akka.cluster.routing._
|
||||
|
||||
object AdaptiveLoadBalancingRouterMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class Routee extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! Reply(Cluster(context.system).selfAddress)
|
||||
}
|
||||
}
|
||||
|
||||
class Memory extends Actor with ActorLogging {
|
||||
var usedMemory: Array[Array[Int]] = _
|
||||
def receive = {
|
||||
case AllocateMemory ⇒
|
||||
val heap = ManagementFactory.getMemoryMXBean.getHeapMemoryUsage
|
||||
// getMax can be undefined (-1)
|
||||
val max = math.max(heap.getMax, heap.getCommitted)
|
||||
val used = heap.getUsed
|
||||
log.debug("used heap before: [{}] bytes, of max [{}]", used, heap.getMax)
|
||||
// allocate 70% of free space
|
||||
val allocateBytes = (0.7 * (max - used)).toInt
|
||||
val numberOfArrays = allocateBytes / 1024
|
||||
usedMemory = Array.ofDim(numberOfArrays, 248) // each 248 element Int array will use ~ 1 kB
|
||||
log.debug("used heap after: [{}] bytes", ManagementFactory.getMemoryMXBean.getHeapMemoryUsage.getUsed)
|
||||
sender ! "done"
|
||||
}
|
||||
}
|
||||
|
||||
case object AllocateMemory
|
||||
case class Reply(address: Address)
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
|
||||
commonConfig(debugConfig(on = false).withFallback(ConfigFactory.parseString("""
|
||||
akka.cluster.metrics.collect-interval = 1s
|
||||
akka.cluster.metrics.gossip-interval = 1s
|
||||
akka.cluster.metrics.moving-average-half-life = 2s
|
||||
akka.actor.deployment {
|
||||
/router3 = {
|
||||
router = adaptive
|
||||
metrics-selector = cpu
|
||||
nr-of-instances = 9
|
||||
}
|
||||
/router4 = {
|
||||
router = adaptive
|
||||
metrics-selector = "akka.cluster.routing.TestCustomMetricsSelector"
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 2
|
||||
}
|
||||
}
|
||||
}
|
||||
""")).withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
}
|
||||
|
||||
class TestCustomMetricsSelector(config: Config) extends MetricsSelector {
|
||||
override def weights(nodeMetrics: Set[NodeMetrics]): Map[Address, Int] = Map.empty
|
||||
}
|
||||
|
||||
class AdaptiveLoadBalancingRouterMultiJvmNode1 extends AdaptiveLoadBalancingRouterSpec
|
||||
class AdaptiveLoadBalancingRouterMultiJvmNode2 extends AdaptiveLoadBalancingRouterSpec
|
||||
class AdaptiveLoadBalancingRouterMultiJvmNode3 extends AdaptiveLoadBalancingRouterSpec
|
||||
|
||||
abstract class AdaptiveLoadBalancingRouterSpec extends MultiNodeSpec(AdaptiveLoadBalancingRouterMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with DefaultTimeout {
|
||||
import AdaptiveLoadBalancingRouterMultiJvmSpec._
|
||||
|
||||
def currentRoutees(router: ActorRef) =
|
||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees
|
||||
|
||||
def receiveReplies(expectedReplies: Int): Map[Address, Int] = {
|
||||
val zero = Map.empty[Address, Int] ++ roles.map(address(_) -> 0)
|
||||
(receiveWhile(5 seconds, messages = expectedReplies) {
|
||||
case Reply(address) ⇒ address
|
||||
}).foldLeft(zero) {
|
||||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Fills in self address for local ActorRef
|
||||
*/
|
||||
def fullAddress(actorRef: ActorRef): Address = actorRef.path.address match {
|
||||
case Address(_, _, None, None) ⇒ cluster.selfAddress
|
||||
case a ⇒ a
|
||||
}
|
||||
|
||||
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)
|
||||
// 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)
|
||||
router
|
||||
}
|
||||
|
||||
"A cluster with a AdaptiveLoadBalancingRouter" must {
|
||||
"start cluster nodes" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(roles: _*)
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"use all nodes in the cluster when not overloaded" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router1 = startRouter("router1")
|
||||
|
||||
// collect some metrics before we start
|
||||
Thread.sleep(cluster.settings.MetricsInterval.toMillis * 10)
|
||||
|
||||
val iterationCount = 100
|
||||
1 to iterationCount foreach { _ ⇒
|
||||
router1 ! "hit"
|
||||
// wait a while between each message, since metrics is collected periodically
|
||||
Thread.sleep(10)
|
||||
}
|
||||
|
||||
val replies = receiveReplies(iterationCount)
|
||||
|
||||
replies(first) must be > (0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be > (0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
|
||||
}
|
||||
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"prefer node with more free heap capacity" taggedAs LongRunningTest in {
|
||||
System.gc()
|
||||
enterBarrier("gc")
|
||||
|
||||
runOn(second) {
|
||||
within(20.seconds) {
|
||||
system.actorOf(Props[Memory], "memory") ! AllocateMemory
|
||||
expectMsg("done")
|
||||
}
|
||||
}
|
||||
enterBarrier("heap-allocated")
|
||||
|
||||
runOn(first) {
|
||||
val router2 = startRouter("router2")
|
||||
|
||||
// collect some metrics before we start
|
||||
Thread.sleep(cluster.settings.MetricsInterval.toMillis * 10)
|
||||
|
||||
val iterationCount = 3000
|
||||
1 to iterationCount foreach { _ ⇒
|
||||
router2 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(iterationCount)
|
||||
|
||||
replies(third) must be > (replies(second))
|
||||
replies.values.sum must be(iterationCount)
|
||||
|
||||
}
|
||||
|
||||
enterBarrier("after-3")
|
||||
}
|
||||
|
||||
"create routees from configuration" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router3 = system.actorOf(Props[Memory].withRouter(FromConfig()), "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)))
|
||||
}
|
||||
enterBarrier("after-4")
|
||||
}
|
||||
|
||||
"create routees from cluster.enabled configuration" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
val router4 = system.actorOf(Props[Memory].withRouter(FromConfig()), "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(
|
||||
address(first), address(second), address(third)))
|
||||
}
|
||||
enterBarrier("after-5")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,175 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster.oldrouting
|
||||
|
||||
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
|
||||
import akka.actor.Props
|
||||
import akka.cluster.MultiNodeClusterSpec
|
||||
import akka.pattern.ask
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
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.FromConfig
|
||||
import akka.routing.RouterRoutees
|
||||
import akka.testkit._
|
||||
import akka.cluster.routing._
|
||||
|
||||
object ClusterConsistentHashingRouterMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("""
|
||||
common-router-settings = {
|
||||
router = consistent-hashing
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 2
|
||||
}
|
||||
}
|
||||
|
||||
akka.actor.deployment {
|
||||
/router1 = ${common-router-settings}
|
||||
/router3 = ${common-router-settings}
|
||||
/router4 = ${common-router-settings}
|
||||
}
|
||||
""")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
}
|
||||
|
||||
class ClusterConsistentHashingRouterMultiJvmNode1 extends ClusterConsistentHashingRouterSpec
|
||||
class ClusterConsistentHashingRouterMultiJvmNode2 extends ClusterConsistentHashingRouterSpec
|
||||
class ClusterConsistentHashingRouterMultiJvmNode3 extends ClusterConsistentHashingRouterSpec
|
||||
|
||||
abstract class ClusterConsistentHashingRouterSpec extends MultiNodeSpec(ClusterConsistentHashingRouterMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with DefaultTimeout {
|
||||
import ClusterConsistentHashingRouterMultiJvmSpec._
|
||||
|
||||
lazy val router1 = system.actorOf(Props[Echo].withRouter(FromConfig()), "router1")
|
||||
|
||||
def currentRoutees(router: ActorRef) =
|
||||
Await.result(router ? CurrentRoutees, remaining).asInstanceOf[RouterRoutees].routees
|
||||
|
||||
/**
|
||||
* 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
|
||||
}
|
||||
|
||||
"A cluster router with a consistent hashing router" must {
|
||||
"start cluster with 2 nodes" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(first, second)
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"create routees from configuration" in {
|
||||
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)))
|
||||
}
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"select destination based on hashKey" in {
|
||||
runOn(first) {
|
||||
router1 ! ConsistentHashableEnvelope(message = "A", hashKey = "a")
|
||||
val destinationA = expectMsgType[ActorRef]
|
||||
router1 ! ConsistentHashableEnvelope(message = "AA", hashKey = "a")
|
||||
expectMsg(destinationA)
|
||||
}
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"deploy routees to new member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
awaitClusterUp(first, second, third)
|
||||
|
||||
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)
|
||||
}
|
||||
|
||||
enterBarrier("after-3")
|
||||
}
|
||||
|
||||
"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")
|
||||
// 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)
|
||||
}
|
||||
|
||||
enterBarrier("after-4")
|
||||
}
|
||||
|
||||
"handle combination of configured router and programatically defined hashMapping" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
def hashMapping: ConsistentHashMapping = {
|
||||
case s: String ⇒ s
|
||||
}
|
||||
|
||||
val router3 = system.actorOf(Props[Echo].withRouter(ConsistentHashingRouter(hashMapping = hashMapping)), "router3")
|
||||
|
||||
assertHashMapping(router3)
|
||||
}
|
||||
|
||||
enterBarrier("after-5")
|
||||
}
|
||||
|
||||
"handle combination of configured router and programatically defined hashMapping and ClusterRouterConfig" taggedAs LongRunningTest in {
|
||||
runOn(first) {
|
||||
def hashMapping: ConsistentHashMapping = {
|
||||
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")
|
||||
|
||||
assertHashMapping(router4)
|
||||
}
|
||||
|
||||
enterBarrier("after-6")
|
||||
}
|
||||
|
||||
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)
|
||||
|
||||
router ! "a"
|
||||
val destinationA = expectMsgType[ActorRef]
|
||||
router ! "a"
|
||||
expectMsg(destinationA)
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -1,7 +1,7 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster.routing
|
||||
package akka.cluster.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
import scala.concurrent.Await
|
||||
|
|
@ -22,6 +22,7 @@ import akka.routing.RoutedActorRef
|
|||
import akka.routing.RouterRoutees
|
||||
import akka.testkit._
|
||||
import akka.remote.transport.ThrottlerTransportAdapter.Direction
|
||||
import akka.cluster.routing._
|
||||
|
||||
object ClusterRoundRobinRoutedActorMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
|
|
@ -232,7 +233,7 @@ abstract class ClusterRoundRobinRoutedActorSpec extends MultiNodeSpec(ClusterRou
|
|||
enterBarrier("after-5")
|
||||
}
|
||||
|
||||
"deploy routees to only remote nodes when allow-local-routees = off" taggedAs LongRunningTest in {
|
||||
"deploy routees to only remote nodes when allow-local-routees = off" taggedAs LongRunningTest in within(15.seconds) {
|
||||
|
||||
runOn(first) {
|
||||
// max-nr-of-instances-per-node=1 times 3 nodes
|
||||
|
|
@ -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")
|
||||
|
|
|
|||
|
|
@ -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]
|
||||
|
|
|
|||
|
|
@ -0,0 +1,373 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster.routing
|
||||
|
||||
import language.postfixOps
|
||||
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
|
||||
import akka.actor.Props
|
||||
import akka.actor.Terminated
|
||||
import akka.cluster.MultiNodeClusterSpec
|
||||
import akka.pattern.ask
|
||||
import akka.remote.testkit.MultiNodeConfig
|
||||
import akka.remote.testkit.MultiNodeSpec
|
||||
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 ClusterRoundRobinMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class SomeActor(routeeType: RouteeType) extends Actor {
|
||||
def this() = this(DeployRoutee)
|
||||
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! Reply(routeeType, self)
|
||||
}
|
||||
}
|
||||
|
||||
case class Reply(routeeType: RouteeType, ref: ActorRef)
|
||||
|
||||
sealed trait RouteeType extends Serializable
|
||||
object DeployRoutee extends RouteeType
|
||||
object LookupRoutee extends RouteeType
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = false).
|
||||
withFallback(ConfigFactory.parseString("""
|
||||
akka.actor.deployment {
|
||||
/router1 {
|
||||
router = round-robin
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 2
|
||||
}
|
||||
}
|
||||
/router3 {
|
||||
router = round-robin
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
max-nr-of-instances-per-node = 1
|
||||
allow-local-routees = off
|
||||
}
|
||||
}
|
||||
/router4 {
|
||||
router = round-robin
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
routees-path = "/user/myservice"
|
||||
}
|
||||
}
|
||||
/router5 {
|
||||
router = round-robin
|
||||
nr-of-instances = 10
|
||||
cluster {
|
||||
enabled = on
|
||||
use-role = a
|
||||
}
|
||||
}
|
||||
}
|
||||
""")).
|
||||
withFallback(MultiNodeClusterSpec.clusterConfig))
|
||||
|
||||
nodeConfig(first, second)(ConfigFactory.parseString("""akka.cluster.roles =["a", "c"]"""))
|
||||
nodeConfig(third)(ConfigFactory.parseString("""akka.cluster.roles =["b", "c"]"""))
|
||||
|
||||
testTransport(on = true)
|
||||
|
||||
}
|
||||
|
||||
class ClusterRoundRobinMultiJvmNode1 extends ClusterRoundRobinSpec
|
||||
class ClusterRoundRobinMultiJvmNode2 extends ClusterRoundRobinSpec
|
||||
class ClusterRoundRobinMultiJvmNode3 extends ClusterRoundRobinSpec
|
||||
class ClusterRoundRobinMultiJvmNode4 extends ClusterRoundRobinSpec
|
||||
|
||||
abstract class ClusterRoundRobinSpec extends MultiNodeSpec(ClusterRoundRobinMultiJvmSpec)
|
||||
with MultiNodeClusterSpec
|
||||
with ImplicitSender with DefaultTimeout {
|
||||
import ClusterRoundRobinMultiJvmSpec._
|
||||
|
||||
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)
|
||||
(receiveWhile(5 seconds, messages = expectedReplies) {
|
||||
case Reply(`routeeType`, ref) ⇒ fullAddress(ref)
|
||||
}).foldLeft(zero) {
|
||||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* 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
|
||||
}
|
||||
|
||||
def currentRoutees(router: ActorRef) =
|
||||
Await.result(router ? GetRoutees, remaining).asInstanceOf[Routees].routees
|
||||
|
||||
"A cluster router with a RoundRobin router" must {
|
||||
"start cluster with 2 nodes" taggedAs LongRunningTest in {
|
||||
awaitClusterUp(first, second)
|
||||
enterBarrier("after-1")
|
||||
}
|
||||
|
||||
"deploy routees to the member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
router1.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
// max-nr-of-instances-per-node=2 times 2 nodes
|
||||
awaitAssert(currentRoutees(router1).size must be(4))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router1 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
replies(first) must be > (0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be(0)
|
||||
replies(fourth) must be(0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-2")
|
||||
}
|
||||
|
||||
"lookup routees on the member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
// cluster consists of first and second
|
||||
|
||||
system.actorOf(Props(classOf[SomeActor], LookupRoutee), "myservice")
|
||||
enterBarrier("myservice-started")
|
||||
|
||||
runOn(first) {
|
||||
// 2 nodes, 1 routee on each node
|
||||
awaitAssert(currentRoutees(router4).size must be(2))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router4 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(LookupRoutee, iterationCount)
|
||||
|
||||
replies(first) must be > (0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be(0)
|
||||
replies(fourth) must be(0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-3")
|
||||
}
|
||||
|
||||
"deploy routees to new nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
// add third and fourth
|
||||
awaitClusterUp(first, second, third, fourth)
|
||||
|
||||
runOn(first) {
|
||||
// max-nr-of-instances-per-node=2 times 4 nodes
|
||||
awaitAssert(currentRoutees(router1).size must be(8))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router1 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
replies.values.foreach { _ must be > (0) }
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-4")
|
||||
}
|
||||
|
||||
"lookup routees on new nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
// cluster consists of first, second, third and fourth
|
||||
|
||||
runOn(first) {
|
||||
// 4 nodes, 1 routee on each node
|
||||
awaitAssert(currentRoutees(router4).size must be(4))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router4 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(LookupRoutee, iterationCount)
|
||||
|
||||
replies.values.foreach { _ must be > (0) }
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-5")
|
||||
}
|
||||
|
||||
"deploy routees to only remote nodes when allow-local-routees = off" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
// max-nr-of-instances-per-node=1 times 3 nodes
|
||||
awaitAssert(currentRoutees(router3).size must be(3))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router3 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
replies(first) must be(0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be > (0)
|
||||
replies(fourth) must be > (0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-6")
|
||||
}
|
||||
|
||||
"deploy routees to specified node role" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
awaitAssert(currentRoutees(router5).size must be(2))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router5 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
replies(first) must be > (0)
|
||||
replies(second) must be > (0)
|
||||
replies(third) must be(0)
|
||||
replies(fourth) must be(0)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-7")
|
||||
}
|
||||
|
||||
"deploy programatically defined routees to the member nodes in the cluster" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first) {
|
||||
router2.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
// totalInstances = 3, maxInstancesPerNode = 1
|
||||
awaitAssert(currentRoutees(router2).size must be(3))
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router2 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
// note that router2 has totalInstances = 3, maxInstancesPerNode = 1
|
||||
val routees = currentRoutees(router2)
|
||||
val routeeAddresses = routees map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }
|
||||
|
||||
routeeAddresses.size must be(3)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-8")
|
||||
}
|
||||
|
||||
"remove routees for unreachable nodes, and add when reachable again" taggedAs LongRunningTest in within(30.seconds) {
|
||||
|
||||
// myservice is already running
|
||||
|
||||
def routees = currentRoutees(router4)
|
||||
def routeeAddresses = (routees map { case ActorSelectionRoutee(sel) ⇒ fullAddress(sel.anchor) }).toSet
|
||||
|
||||
runOn(first) {
|
||||
// 4 nodes, 1 routee on each node
|
||||
awaitAssert(currentRoutees(router4).size must be(4))
|
||||
|
||||
testConductor.blackhole(first, second, Direction.Both).await
|
||||
|
||||
awaitAssert(routees.size must be(3))
|
||||
routeeAddresses must not contain (address(second))
|
||||
|
||||
testConductor.passThrough(first, second, Direction.Both).await
|
||||
awaitAssert(routees.size must be(4))
|
||||
routeeAddresses must contain(address(second))
|
||||
|
||||
}
|
||||
|
||||
enterBarrier("after-9")
|
||||
}
|
||||
|
||||
"deploy programatically defined routees to other node when a node becomes down" taggedAs LongRunningTest in {
|
||||
muteMarkingAsUnreachable()
|
||||
|
||||
runOn(first) {
|
||||
def routees = currentRoutees(router2)
|
||||
def routeeAddresses = (routees map { case ActorRefRoutee(ref) ⇒ fullAddress(ref) }).toSet
|
||||
|
||||
routees foreach { case ActorRefRoutee(ref) ⇒ watch(ref) }
|
||||
val notUsedAddress = ((roles map address).toSet -- routeeAddresses).head
|
||||
val downAddress = routeeAddresses.find(_ != address(first)).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(downRouteeRef)
|
||||
awaitAssert {
|
||||
routeeAddresses must contain(notUsedAddress)
|
||||
routeeAddresses must not contain (downAddress)
|
||||
}
|
||||
|
||||
val iterationCount = 10
|
||||
for (i ← 0 until iterationCount) {
|
||||
router2 ! "hit"
|
||||
}
|
||||
|
||||
val replies = receiveReplies(DeployRoutee, iterationCount)
|
||||
|
||||
routeeAddresses.size must be(3)
|
||||
replies.values.sum must be(iterationCount)
|
||||
}
|
||||
|
||||
enterBarrier("after-10")
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -7,8 +7,10 @@ import akka.testkit._
|
|||
import akka.actor._
|
||||
import akka.routing._
|
||||
import com.typesafe.config._
|
||||
import akka.cluster.routing.ClusterRouterConfig
|
||||
import akka.cluster.routing.ClusterRouterSettings
|
||||
import akka.cluster.routing.ClusterRouterPool
|
||||
import akka.cluster.routing.ClusterRouterGroup
|
||||
import akka.cluster.routing.ClusterRouterPoolSettings
|
||||
import akka.cluster.routing.ClusterRouterGroupSettings
|
||||
|
||||
object ClusterDeployerSpec {
|
||||
val deployerConf = ConfigFactory.parseString("""
|
||||
|
|
@ -45,7 +47,7 @@ class ClusterDeployerSpec extends AkkaSpec(ClusterDeployerSpec.deployerConf) {
|
|||
|
||||
"A RemoteDeployer" must {
|
||||
|
||||
"be able to parse 'akka.actor.deployment._' with specified cluster lookup routee settings" in {
|
||||
"be able to parse 'akka.actor.deployment._' with specified cluster pool" in {
|
||||
val service = "/user/service1"
|
||||
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1))
|
||||
deployment must not be (None)
|
||||
|
|
@ -54,14 +56,14 @@ class ClusterDeployerSpec extends AkkaSpec(ClusterDeployerSpec.deployerConf) {
|
|||
Deploy(
|
||||
service,
|
||||
deployment.get.config,
|
||||
ClusterRouterConfig(RoundRobinRouter(20), ClusterRouterSettings(
|
||||
ClusterRouterPool(RoundRobinPool(20), ClusterRouterPoolSettings(
|
||||
totalInstances = 20, maxInstancesPerNode = 3, allowLocalRoutees = false, useRole = None)),
|
||||
ClusterScope,
|
||||
Deploy.NoDispatcherGiven,
|
||||
Deploy.NoMailboxGiven)))
|
||||
}
|
||||
|
||||
"be able to parse 'akka.actor.deployment._' with specified cluster deploy routee settings" in {
|
||||
"be able to parse 'akka.actor.deployment._' with specified cluster group" in {
|
||||
val service = "/user/service2"
|
||||
val deployment = system.asInstanceOf[ActorSystemImpl].provider.deployer.lookup(service.split("/").drop(1))
|
||||
deployment must not be (None)
|
||||
|
|
@ -70,13 +72,19 @@ class ClusterDeployerSpec extends AkkaSpec(ClusterDeployerSpec.deployerConf) {
|
|||
Deploy(
|
||||
service,
|
||||
deployment.get.config,
|
||||
ClusterRouterConfig(RoundRobinRouter(20), ClusterRouterSettings(
|
||||
ClusterRouterGroup(RoundRobinGroup(List("/user/myservice")), ClusterRouterGroupSettings(
|
||||
totalInstances = 20, routeesPath = "/user/myservice", allowLocalRoutees = false, useRole = None)),
|
||||
ClusterScope,
|
||||
"mydispatcher",
|
||||
"mymailbox")))
|
||||
}
|
||||
|
||||
"have correct router mappings" in {
|
||||
val mapping = system.asInstanceOf[ActorSystemImpl].provider.deployer.routerTypeMapping
|
||||
mapping("adaptive-pool") must be(classOf[akka.cluster.routing.AdaptiveLoadBalancingPool].getName)
|
||||
mapping("adaptive-group") must be(classOf[akka.cluster.routing.AdaptiveLoadBalancingGroup].getName)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,52 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.cluster.oldrouting
|
||||
|
||||
import akka.testkit._
|
||||
import akka.actor._
|
||||
import akka.routing.RoundRobinRouter
|
||||
import akka.actor.OneForOneStrategy
|
||||
import akka.cluster.routing._
|
||||
|
||||
object ClusterRouterSupervisorSpec {
|
||||
|
||||
class KillableActor(testActor: ActorRef) extends Actor {
|
||||
|
||||
def receive = {
|
||||
case "go away" ⇒
|
||||
throw new IllegalArgumentException("Goodbye then!")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class ClusterRouterSupervisorSpec extends AkkaSpec("""
|
||||
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
|
||||
akka.remote.netty.tcp.port = 0
|
||||
""") {
|
||||
|
||||
import ClusterRouterSupervisorSpec._
|
||||
|
||||
"Cluster aware routers" must {
|
||||
|
||||
"use provided supervisor strategy" in {
|
||||
val router = system.actorOf(Props(classOf[KillableActor], testActor).withRouter(
|
||||
ClusterRouterConfig(RoundRobinRouter(supervisorStrategy = OneForOneStrategy() {
|
||||
case _ ⇒
|
||||
testActor ! "supervised"
|
||||
SupervisorStrategy.Stop
|
||||
}), ClusterRouterSettings(
|
||||
totalInstances = 1,
|
||||
maxInstancesPerNode = 1,
|
||||
allowLocalRoutees = true,
|
||||
useRole = None))), name = "therouter")
|
||||
|
||||
router ! "go away"
|
||||
expectMsg("supervised")
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -5,7 +5,7 @@ package akka.cluster.routing
|
|||
|
||||
import akka.testkit._
|
||||
import akka.actor._
|
||||
import akka.routing.RoundRobinRouter
|
||||
import akka.routing.RoundRobinPool
|
||||
import akka.actor.OneForOneStrategy
|
||||
|
||||
object ClusterRouterSupervisorSpec {
|
||||
|
|
@ -21,6 +21,7 @@ object ClusterRouterSupervisorSpec {
|
|||
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ClusterRouterSupervisorSpec extends AkkaSpec("""
|
||||
akka.actor.provider = "akka.cluster.ClusterActorRefProvider"
|
||||
akka.remote.netty.tcp.port = 0
|
||||
|
|
@ -31,16 +32,36 @@ class ClusterRouterSupervisorSpec extends AkkaSpec("""
|
|||
"Cluster aware routers" must {
|
||||
|
||||
"use provided supervisor strategy" in {
|
||||
val router = system.actorOf(Props(classOf[KillableActor], testActor).withRouter(
|
||||
ClusterRouterConfig(RoundRobinRouter(supervisorStrategy = OneForOneStrategy() {
|
||||
case _ ⇒
|
||||
testActor ! "supervised"
|
||||
SupervisorStrategy.Stop
|
||||
}), ClusterRouterSettings(
|
||||
val router = system.actorOf(
|
||||
ClusterRouterPool(RoundRobinPool(nrOfInstances = 1, supervisorStrategy =
|
||||
OneForOneStrategy(loggingEnabled = false) {
|
||||
case _ ⇒
|
||||
testActor ! "supervised"
|
||||
SupervisorStrategy.Stop
|
||||
}), ClusterRouterPoolSettings(
|
||||
totalInstances = 1,
|
||||
maxInstancesPerNode = 1,
|
||||
allowLocalRoutees = true,
|
||||
useRole = None))), name = "therouter")
|
||||
useRole = None)).
|
||||
props(Props(classOf[KillableActor], testActor)), name = "therouter")
|
||||
|
||||
router ! "go away"
|
||||
expectMsg("supervised")
|
||||
}
|
||||
|
||||
"use provided supervisor strategy of deprecated router" in {
|
||||
val router = system.actorOf(
|
||||
ClusterRouterPool(RoundRobinPool(nrOfInstances = 1, supervisorStrategy =
|
||||
OneForOneStrategy(loggingEnabled = false) {
|
||||
case _ ⇒
|
||||
testActor ! "supervised"
|
||||
SupervisorStrategy.Stop
|
||||
}), ClusterRouterPoolSettings(
|
||||
totalInstances = 1,
|
||||
maxInstancesPerNode = 1,
|
||||
allowLocalRoutees = true,
|
||||
useRole = None)).
|
||||
props(Props(classOf[KillableActor], testActor)), name = "theoldrouter")
|
||||
|
||||
router ! "go away"
|
||||
expectMsg("supervised")
|
||||
|
|
|
|||
|
|
@ -5,10 +5,11 @@
|
|||
package akka.cluster.routing
|
||||
|
||||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
import akka.actor.Address
|
||||
import akka.actor.RootActorPath
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.routing.ActorSelectionRoutee
|
||||
import akka.routing.ActorRefRoutee
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
||||
|
|
@ -21,20 +22,21 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
val c1 = Address("akka.tcp", "sys", "c1", 2551)
|
||||
val d1 = Address("akka.tcp", "sys", "d1", 2551)
|
||||
|
||||
val refA = system.actorFor(RootActorPath(a1) / "user" / "a")
|
||||
val refB = system.actorFor(RootActorPath(b1) / "user" / "b")
|
||||
val refC = system.actorFor(RootActorPath(c1) / "user" / "c")
|
||||
val routeeA = ActorSelectionRoutee(system.actorSelection(RootActorPath(a1) / "user" / "a"))
|
||||
val routeeB = ActorSelectionRoutee(system.actorSelection(RootActorPath(b1) / "user" / "b"))
|
||||
val routeeC = ActorSelectionRoutee(system.actorSelection(RootActorPath(c1) / "user" / "c"))
|
||||
val routees = Vector(routeeA, routeeB, routeeC)
|
||||
val testActorRoutee = ActorRefRoutee(testActor)
|
||||
|
||||
"WeightedRoutees" must {
|
||||
|
||||
"allocate weighted refs" in {
|
||||
"allocate weighted routees" in {
|
||||
val weights = Map(a1 -> 1, b1 -> 3, c1 -> 10)
|
||||
val refs = Vector(refA, refB, refC)
|
||||
val weighted = new WeightedRoutees(refs, a1, weights)
|
||||
val weighted = new WeightedRoutees(routees, a1, weights)
|
||||
|
||||
weighted(1) must be(refA)
|
||||
2 to 4 foreach { weighted(_) must be(refB) }
|
||||
5 to 14 foreach { weighted(_) must be(refC) }
|
||||
weighted(1) must be(routeeA)
|
||||
2 to 4 foreach { weighted(_) must be(routeeB) }
|
||||
5 to 14 foreach { weighted(_) must be(routeeC) }
|
||||
weighted.total must be(14)
|
||||
}
|
||||
|
||||
|
|
@ -45,7 +47,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
empty.total
|
||||
}
|
||||
|
||||
val empty2 = new WeightedRoutees(Vector(refA), a1, Map(a1 -> 0))
|
||||
val empty2 = new WeightedRoutees(Vector(routeeA), a1, Map(a1 -> 0))
|
||||
empty2.isEmpty must be(true)
|
||||
intercept[IllegalArgumentException] {
|
||||
empty2.total
|
||||
|
|
@ -54,7 +56,7 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
empty2(0)
|
||||
}
|
||||
|
||||
val weighted = new WeightedRoutees(Vector(refA, refB, refC), a1, Map.empty)
|
||||
val weighted = new WeightedRoutees(routees, a1, Map.empty)
|
||||
weighted.total must be(3)
|
||||
intercept[IllegalArgumentException] {
|
||||
weighted(0)
|
||||
|
|
@ -64,33 +66,31 @@ class WeightedRouteesSpec extends AkkaSpec(ConfigFactory.parseString("""
|
|||
}
|
||||
}
|
||||
|
||||
"allocate refs for undefined weight" in {
|
||||
"allocate routees for undefined weight" in {
|
||||
val weights = Map(a1 -> 1, b1 -> 7)
|
||||
val refs = Vector(refA, refB, refC)
|
||||
val weighted = new WeightedRoutees(refs, a1, weights)
|
||||
val weighted = new WeightedRoutees(routees, a1, weights)
|
||||
|
||||
weighted(1) must be(refA)
|
||||
2 to 8 foreach { weighted(_) must be(refB) }
|
||||
weighted(1) must be(routeeA)
|
||||
2 to 8 foreach { weighted(_) must be(routeeB) }
|
||||
// undefined, uses the mean of the weights, i.e. 4
|
||||
9 to 12 foreach { weighted(_) must be(refC) }
|
||||
9 to 12 foreach { weighted(_) must be(routeeC) }
|
||||
weighted.total must be(12)
|
||||
}
|
||||
|
||||
"allocate weighted local refs" in {
|
||||
"allocate weighted local routees" in {
|
||||
val weights = Map(a1 -> 2, b1 -> 1, c1 -> 10)
|
||||
val refs = Vector(testActor, refB, refC)
|
||||
val weighted = new WeightedRoutees(refs, a1, weights)
|
||||
val routees2 = Vector(testActorRoutee, routeeB, routeeC)
|
||||
val weighted = new WeightedRoutees(routees2, a1, weights)
|
||||
|
||||
1 to 2 foreach { weighted(_) must be(testActor) }
|
||||
3 to weighted.total foreach { weighted(_) must not be (testActor) }
|
||||
1 to 2 foreach { weighted(_) must be(testActorRoutee) }
|
||||
3 to weighted.total foreach { weighted(_) must not be (testActorRoutee) }
|
||||
}
|
||||
|
||||
"not allocate ref with weight zero" in {
|
||||
val weights = Map(a1 -> 0, b1 -> 2, c1 -> 10)
|
||||
val refs = Vector(refA, refB, refC)
|
||||
val weighted = new WeightedRoutees(refs, a1, weights)
|
||||
val weighted = new WeightedRoutees(routees, a1, weights)
|
||||
|
||||
1 to weighted.total foreach { weighted(_) must not be (refA) }
|
||||
1 to weighted.total foreach { weighted(_) must not be (routeeA) }
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -429,12 +429,12 @@ when a node becomes reachable again, after having been unreachable.
|
|||
|
||||
There are two distinct types of routers.
|
||||
|
||||
* **Router that lookup existing actors and use them as routees.** The routees can be shared between
|
||||
routers running on different nodes in the cluster. One example of a use case for this
|
||||
type of router is a service running on some backend nodes in the cluster and
|
||||
used by routers running on front-end nodes in the cluster.
|
||||
* **Group - router that sends messages to the specified path using actor selection**
|
||||
The routees can be shared between routers running on different nodes in the cluster.
|
||||
One example of a use case for this type of router is a service running on some backend
|
||||
nodes in the cluster and used by routers running on front-end nodes in the cluster.
|
||||
|
||||
* **Router that creates new routees as child actors and deploy them on remote nodes.**
|
||||
* **Pool - router that creates routees as child actors and deploys them on remote nodes.**
|
||||
Each router will have its own routee instances. For example, if you start a router
|
||||
on 3 nodes in a 10 nodes cluster you will have 30 routee actors in total if the router is
|
||||
configured to use one inctance per node. The routees created by the the different routers
|
||||
|
|
@ -442,11 +442,11 @@ There are two distinct types of routers.
|
|||
is a single master that coordinate jobs and delegates the actual work to routees running
|
||||
on other nodes in the cluster.
|
||||
|
||||
Router with Lookup of Routees
|
||||
-----------------------------
|
||||
Router with Group of Routees
|
||||
----------------------------
|
||||
|
||||
When using a router with routees looked up on the cluster member nodes, i.e. the routees
|
||||
are already running, the configuration for a router looks like this:
|
||||
When using a ``Group`` you must start the routee actors on the cluster member nodes.
|
||||
That is not done by the router. The configuration for a group looks like this:
|
||||
|
||||
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala#router-lookup-config
|
||||
|
||||
|
|
@ -472,10 +472,11 @@ The same type of router could also have been defined in code:
|
|||
|
||||
See :ref:`cluster_configuration_java` section for further descriptions of the settings.
|
||||
|
||||
Router Example with Lookup of Routees
|
||||
-------------------------------------
|
||||
Router Example with Group of Routees
|
||||
------------------------------------
|
||||
|
||||
Let's take a look at how to use a cluster aware router with lookup of routees.
|
||||
Let's take a look at how to use a cluster aware router with a group of routees,
|
||||
i.e. router sending to the paths of the routees.
|
||||
|
||||
The example application provides a service to calculate statistics for a text.
|
||||
When some text is sent to the service it splits it into words, and delegates the task
|
||||
|
|
@ -533,10 +534,10 @@ service nodes and 1 client::
|
|||
mvn exec:java \
|
||||
-Dexec.mainClass="sample.cluster.stats.japi.StatsSampleMain"
|
||||
|
||||
Router with Remote Deployed Routees
|
||||
-----------------------------------
|
||||
Router with Pool of Remote Deployed Routees
|
||||
-------------------------------------------
|
||||
|
||||
When using a router with routees created and deployed on the cluster member nodes
|
||||
When using a ``Pool`` with routees created and deployed on the cluster member nodes
|
||||
the configuration for a router looks like this:
|
||||
|
||||
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala#router-deploy-config
|
||||
|
|
@ -555,8 +556,8 @@ The same type of router could also have been defined in code:
|
|||
|
||||
See :ref:`cluster_configuration_java` section for further descriptions of the settings.
|
||||
|
||||
Router Example with Remote Deployed Routees
|
||||
-------------------------------------------
|
||||
Router Example with Pool of Remote Deployed Routees
|
||||
---------------------------------------------------
|
||||
|
||||
Let's take a look at how to use a cluster aware router on single master node that creates
|
||||
and deploys workers. To keep track of a single master we use the :ref:`cluster-singleton`
|
||||
|
|
@ -626,7 +627,7 @@ Download the native Sigar libraries from `Maven Central <http://repo1.maven.org/
|
|||
Adaptive Load Balancing
|
||||
-----------------------
|
||||
|
||||
The ``AdaptiveLoadBalancingRouter`` performs load balancing of messages to cluster nodes based on the cluster metrics data.
|
||||
The ``AdaptiveLoadBalancingPool`` / ``AdaptiveLoadBalancingGroup`` performs load balancing of messages to cluster nodes based on the cluster metrics data.
|
||||
It uses random selection of routees with probabilities derived from the remaining capacity of the corresponding node.
|
||||
It can be configured to use a specific MetricsSelector to produce the probabilities, a.k.a. weights:
|
||||
|
||||
|
|
|
|||
|
|
@ -119,8 +119,9 @@ public class ConsistentHashingRouterDocTest {
|
|||
}
|
||||
};
|
||||
|
||||
ActorRef cache = system.actorOf(Props.create(Cache.class).withRouter(
|
||||
new ConsistentHashingRouter(10).withHashMapper(hashMapper)),
|
||||
ActorRef cache = system.actorOf(
|
||||
new ConsistentHashingRouter(10).withHashMapper(hashMapper).props(
|
||||
Props.create(Cache.class)),
|
||||
"cache");
|
||||
|
||||
cache.tell(new ConsistentHashableEnvelope(
|
||||
|
|
|
|||
|
|
@ -3,201 +3,164 @@
|
|||
*/
|
||||
package docs.jrouting;
|
||||
|
||||
import static akka.pattern.Patterns.ask;
|
||||
import static docs.jrouting.CustomRouterDocTest.Message.DemocratCountResult;
|
||||
import static docs.jrouting.CustomRouterDocTest.Message.DemocratVote;
|
||||
import static docs.jrouting.CustomRouterDocTest.Message.RepublicanCountResult;
|
||||
import static docs.jrouting.CustomRouterDocTest.Message.RepublicanVote;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import akka.routing.FromConfig;
|
||||
import akka.routing.RoundRobinRoutingLogic;
|
||||
import akka.routing.Routee;
|
||||
import akka.routing.RoutingLogic;
|
||||
import akka.routing.SeveralRoutees;
|
||||
import akka.testkit.AkkaJUnitActorSystemResource;
|
||||
import org.junit.*;
|
||||
|
||||
import scala.concurrent.Await;
|
||||
import scala.concurrent.Future;
|
||||
import scala.concurrent.duration.Duration;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
|
||||
import scala.collection.immutable.IndexedSeq;
|
||||
import static akka.japi.Util.immutableIndexedSeq;
|
||||
import docs.jrouting.RouterDocTest.Parent;
|
||||
import docs.jrouting.RouterDocTest.Workers;
|
||||
import docs.routing.CustomRouterDocSpec;
|
||||
import akka.testkit.JavaTestKit;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.OneForOneStrategy;
|
||||
import akka.actor.Props;
|
||||
import akka.actor.SupervisorStrategy;
|
||||
//#imports1
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.dispatch.Dispatchers;
|
||||
import akka.routing.CustomRoute;
|
||||
import akka.routing.CustomRouterConfig;
|
||||
import akka.routing.Destination;
|
||||
import akka.routing.RoundRobinRouter;
|
||||
import akka.routing.RouteeProvider;
|
||||
import akka.testkit.AkkaSpec;
|
||||
import akka.util.Timeout;
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
//#imports1
|
||||
|
||||
|
||||
public class CustomRouterDocTest {
|
||||
|
||||
@ClassRule
|
||||
public static AkkaJUnitActorSystemResource actorSystemResource =
|
||||
new AkkaJUnitActorSystemResource(
|
||||
"CustomRouterDocTest", ConfigFactory.load(ConfigFactory.parseString(
|
||||
"head{}\nworkers{}").withFallback(AkkaSpec.testConf())));
|
||||
new AkkaJUnitActorSystemResource("CustomRouterDocTest",
|
||||
ConfigFactory.parseString(CustomRouterDocSpec.jconfig()));
|
||||
|
||||
private final ActorSystem system = actorSystemResource.getSystem();
|
||||
|
||||
public static class MyActor extends UntypedActor {
|
||||
@Override public void onReceive(Object o) {}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateDispatchers() {
|
||||
//#dispatchers
|
||||
final ActorRef router = system.actorOf(Props.create(MyActor.class)
|
||||
// “head” router will run on "head" dispatcher
|
||||
.withRouter(new RoundRobinRouter(5).withDispatcher("head"))
|
||||
// MyActor “workers” will run on "workers" dispatcher
|
||||
.withDispatcher("workers"));
|
||||
//#dispatchers
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateSupervisor() {
|
||||
//#supervision
|
||||
final SupervisorStrategy strategy =
|
||||
new OneForOneStrategy(5, Duration.create("1 minute"),
|
||||
Collections.<Class<? extends Throwable>>singletonList(Exception.class));
|
||||
final ActorRef router = system.actorOf(Props.create(MyActor.class)
|
||||
.withRouter(new RoundRobinRouter(5).withSupervisorStrategy(strategy)));
|
||||
//#supervision
|
||||
}
|
||||
|
||||
//#crTest
|
||||
@Test
|
||||
public void countVotesAsIntendedNotAsInFlorida() throws Exception {
|
||||
ActorRef routedActor = system.actorOf(
|
||||
Props.empty().withRouter(new VoteCountRouter()));
|
||||
routedActor.tell(DemocratVote, ActorRef.noSender());
|
||||
routedActor.tell(DemocratVote, ActorRef.noSender());
|
||||
routedActor.tell(RepublicanVote, ActorRef.noSender());
|
||||
routedActor.tell(DemocratVote, ActorRef.noSender());
|
||||
routedActor.tell(RepublicanVote, ActorRef.noSender());
|
||||
Timeout timeout = new Timeout(Duration.create(1, "seconds"));
|
||||
Future<Object> democratsResult =
|
||||
ask(routedActor, DemocratCountResult, timeout);
|
||||
Future<Object> republicansResult =
|
||||
ask(routedActor, RepublicanCountResult, timeout);
|
||||
|
||||
assertEquals(3, Await.result(democratsResult, timeout.duration()));
|
||||
assertEquals(2, Await.result(republicansResult, timeout.duration()));
|
||||
}
|
||||
|
||||
//#crTest
|
||||
|
||||
//#CustomRouter
|
||||
//#crMessages
|
||||
enum Message {
|
||||
DemocratVote, DemocratCountResult, RepublicanVote, RepublicanCountResult
|
||||
}
|
||||
|
||||
//#crMessages
|
||||
//#CustomRouter
|
||||
static
|
||||
//#CustomRouter
|
||||
//#crActors
|
||||
public class DemocratActor extends UntypedActor {
|
||||
int counter = 0;
|
||||
|
||||
public void onReceive(Object msg) {
|
||||
switch ((Message) msg) {
|
||||
case DemocratVote:
|
||||
counter++;
|
||||
break;
|
||||
case DemocratCountResult:
|
||||
getSender().tell(counter, getSelf());
|
||||
break;
|
||||
default:
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#crActors
|
||||
//#CustomRouter
|
||||
static
|
||||
//#CustomRouter
|
||||
//#crActors
|
||||
public class RepublicanActor extends UntypedActor {
|
||||
int counter = 0;
|
||||
|
||||
public void onReceive(Object msg) {
|
||||
switch ((Message) msg) {
|
||||
case RepublicanVote:
|
||||
counter++;
|
||||
break;
|
||||
case RepublicanCountResult:
|
||||
getSender().tell(counter, getSelf());
|
||||
break;
|
||||
default:
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#crActors
|
||||
//#CustomRouter
|
||||
static
|
||||
//#CustomRouter
|
||||
//#crRouter
|
||||
public class VoteCountRouter extends CustomRouterConfig {
|
||||
//#routing-logic
|
||||
public class RedundancyRoutingLogic implements RoutingLogic {
|
||||
private final int nbrCopies;
|
||||
|
||||
@Override public String routerDispatcher() {
|
||||
return Dispatchers.DefaultDispatcherId();
|
||||
public RedundancyRoutingLogic(int nbrCopies) {
|
||||
this.nbrCopies = nbrCopies;
|
||||
}
|
||||
RoundRobinRoutingLogic roundRobin = new RoundRobinRoutingLogic();
|
||||
|
||||
@Override public SupervisorStrategy supervisorStrategy() {
|
||||
return SupervisorStrategy.defaultStrategy();
|
||||
}
|
||||
|
||||
//#crRoute
|
||||
@Override
|
||||
public CustomRoute createCustomRoute(RouteeProvider routeeProvider) {
|
||||
final ActorRef democratActor =
|
||||
routeeProvider.context().actorOf(Props.create(DemocratActor.class), "d");
|
||||
final ActorRef republicanActor =
|
||||
routeeProvider.context().actorOf(Props.create(RepublicanActor.class), "r");
|
||||
List<ActorRef> routees =
|
||||
Arrays.asList(new ActorRef[] { democratActor, republicanActor });
|
||||
|
||||
//#crRegisterRoutees
|
||||
routeeProvider.registerRoutees(routees);
|
||||
//#crRegisterRoutees
|
||||
|
||||
//#crRoutingLogic
|
||||
return new CustomRoute() {
|
||||
@Override
|
||||
public scala.collection.immutable.Seq<Destination> destinationsFor(
|
||||
ActorRef sender, Object msg) {
|
||||
switch ((Message) msg) {
|
||||
case DemocratVote:
|
||||
case DemocratCountResult:
|
||||
return akka.japi.Util.immutableSingletonSeq(
|
||||
new Destination(sender, democratActor));
|
||||
case RepublicanVote:
|
||||
case RepublicanCountResult:
|
||||
return akka.japi.Util.immutableSingletonSeq(
|
||||
new Destination(sender, republicanActor));
|
||||
default:
|
||||
throw new IllegalArgumentException("Unknown message: " + msg);
|
||||
}
|
||||
}
|
||||
};
|
||||
//#crRoutingLogic
|
||||
public Routee select(Object message, IndexedSeq<Routee> routees) {
|
||||
List<Routee> targets = new ArrayList<Routee>();
|
||||
for (int i = 0; i < nbrCopies; i++) {
|
||||
targets.add(roundRobin.select(message, routees));
|
||||
}
|
||||
return new SeveralRoutees(targets);
|
||||
}
|
||||
//#crRoute
|
||||
}
|
||||
//#routing-logic
|
||||
|
||||
static
|
||||
//#unit-test-logic
|
||||
public final class TestRoutee implements Routee {
|
||||
public final int n;
|
||||
|
||||
public TestRoutee(int n) {
|
||||
this.n = n;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void send(Object message, ActorRef sender) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return n;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object obj) {
|
||||
return (obj instanceof TestRoutee) &&
|
||||
n == ((TestRoutee) obj).n;
|
||||
}
|
||||
}
|
||||
|
||||
//#unit-test-logic
|
||||
|
||||
static public class Storage extends UntypedActor {
|
||||
public void onReceive(Object msg) {
|
||||
getSender().tell(msg, getSelf());
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void unitTestRoutingLogic() {
|
||||
//#unit-test-logic
|
||||
RedundancyRoutingLogic logic = new RedundancyRoutingLogic(3);
|
||||
|
||||
List<Routee> routeeList = new ArrayList<Routee>();
|
||||
for (int n = 1; n <= 7; n++) {
|
||||
routeeList.add(new TestRoutee(n));
|
||||
}
|
||||
IndexedSeq<Routee> routees = immutableIndexedSeq(routeeList);
|
||||
|
||||
SeveralRoutees r1 = (SeveralRoutees) logic.select("msg", routees);
|
||||
assertEquals(r1.getRoutees().get(0), routeeList.get(0));
|
||||
assertEquals(r1.getRoutees().get(1), routeeList.get(1));
|
||||
assertEquals(r1.getRoutees().get(2), routeeList.get(2));
|
||||
|
||||
SeveralRoutees r2 = (SeveralRoutees) logic.select("msg", routees);
|
||||
assertEquals(r2.getRoutees().get(0), routeeList.get(3));
|
||||
assertEquals(r2.getRoutees().get(1), routeeList.get(4));
|
||||
assertEquals(r2.getRoutees().get(2), routeeList.get(5));
|
||||
|
||||
SeveralRoutees r3 = (SeveralRoutees) logic.select("msg", routees);
|
||||
assertEquals(r3.getRoutees().get(0), routeeList.get(6));
|
||||
assertEquals(r3.getRoutees().get(1), routeeList.get(0));
|
||||
assertEquals(r3.getRoutees().get(2), routeeList.get(1));
|
||||
|
||||
//#unit-test-logic
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateUsageOfCustomRouter() {
|
||||
new JavaTestKit(system) {{
|
||||
//#usage-1
|
||||
for (int n = 1; n <= 10; n++) {
|
||||
system.actorOf(Props.create(Storage.class), "s" + n);
|
||||
}
|
||||
|
||||
List<String> paths = new ArrayList<String>();
|
||||
for (int n = 1; n <= 10; n++) {
|
||||
paths.add("/user/s" + n);
|
||||
}
|
||||
|
||||
ActorRef redundancy1 =
|
||||
system.actorOf(new RedundancyGroup(paths, 3).props(),
|
||||
"redundancy1");
|
||||
redundancy1.tell("important", getTestActor());
|
||||
//#usage-1
|
||||
|
||||
for (int i = 0; i < 3; i++) {
|
||||
expectMsgEquals("important");
|
||||
}
|
||||
|
||||
//#usage-2
|
||||
ActorRef redundancy2 = system.actorOf(FromConfig.getInstance().props(),
|
||||
"redundancy2");
|
||||
redundancy2.tell("very important", getTestActor());
|
||||
//#usage-2
|
||||
|
||||
for (int i = 0; i < 5; i++) {
|
||||
expectMsgEquals("very important");
|
||||
}
|
||||
}};
|
||||
}
|
||||
|
||||
//#crRouter
|
||||
//#CustomRouter
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,48 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.jrouting;
|
||||
|
||||
import java.io.Serializable;
|
||||
|
||||
import akka.actor.UntypedActor;
|
||||
|
||||
//#fibonacciActor
|
||||
public class FibonacciActor extends UntypedActor {
|
||||
public void onReceive(Object msg) {
|
||||
if (msg instanceof FibonacciNumber) {
|
||||
FibonacciNumber fibonacciNumber = (FibonacciNumber) msg;
|
||||
getSender().tell(fibonacci(fibonacciNumber.getNbr()), getSelf());
|
||||
} else {
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
|
||||
private int fibonacci(int n) {
|
||||
return fib(n, 1, 0);
|
||||
}
|
||||
|
||||
private int fib(int n, int b, int a) {
|
||||
if (n == 0)
|
||||
return a;
|
||||
// recursion
|
||||
return fib(n - 1, a + b, b);
|
||||
}
|
||||
|
||||
public static class FibonacciNumber implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
private final int nbr;
|
||||
|
||||
public FibonacciNumber(int nbr) {
|
||||
this.nbr = nbr;
|
||||
}
|
||||
|
||||
public int getNbr() {
|
||||
return nbr;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
//#fibonacciActor
|
||||
|
||||
|
|
@ -1,75 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.jrouting;
|
||||
|
||||
import akka.routing.ScatterGatherFirstCompletedRouter;
|
||||
import akka.routing.BroadcastRouter;
|
||||
import akka.routing.RandomRouter;
|
||||
import akka.routing.RoundRobinRouter;
|
||||
import akka.routing.SmallestMailboxRouter;
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.Props;
|
||||
import scala.concurrent.duration.Duration;
|
||||
import akka.util.Timeout;
|
||||
import scala.concurrent.Future;
|
||||
import scala.concurrent.Await;
|
||||
|
||||
//#parentActor
|
||||
public class ParentActor extends UntypedActor {
|
||||
public void onReceive(Object msg) throws Exception {
|
||||
if (msg.equals("rrr")) {
|
||||
//#roundRobinRouter
|
||||
ActorRef roundRobinRouter = getContext().actorOf(
|
||||
Props.create(PrintlnActor.class).withRouter(new RoundRobinRouter(5)),
|
||||
"router");
|
||||
for (int i = 1; i <= 10; i++) {
|
||||
roundRobinRouter.tell(i, getSelf());
|
||||
}
|
||||
//#roundRobinRouter
|
||||
} else if (msg.equals("rr")) {
|
||||
//#randomRouter
|
||||
ActorRef randomRouter = getContext().actorOf(
|
||||
Props.create(PrintlnActor.class).withRouter(new RandomRouter(5)),
|
||||
"router");
|
||||
for (int i = 1; i <= 10; i++) {
|
||||
randomRouter.tell(i, getSelf());
|
||||
}
|
||||
//#randomRouter
|
||||
} else if (msg.equals("smr")) {
|
||||
//#smallestMailboxRouter
|
||||
ActorRef smallestMailboxRouter = getContext().actorOf(
|
||||
Props.create(PrintlnActor.class).withRouter(new SmallestMailboxRouter(5)),
|
||||
"router");
|
||||
for (int i = 1; i <= 10; i++) {
|
||||
smallestMailboxRouter.tell(i, getSelf());
|
||||
}
|
||||
//#smallestMailboxRouter
|
||||
} else if (msg.equals("br")) {
|
||||
//#broadcastRouter
|
||||
ActorRef broadcastRouter = getContext().actorOf(
|
||||
Props.create(PrintlnActor.class).withRouter(new BroadcastRouter(5)), "router");
|
||||
broadcastRouter.tell("this is a broadcast message", getSelf());
|
||||
//#broadcastRouter
|
||||
} else if (msg.equals("sgfcr")) {
|
||||
//#scatterGatherFirstCompletedRouter
|
||||
ActorRef scatterGatherFirstCompletedRouter = getContext().actorOf(
|
||||
Props.create(FibonacciActor.class).withRouter(
|
||||
new ScatterGatherFirstCompletedRouter(5, Duration.create(2, "seconds"))),
|
||||
"router");
|
||||
Timeout timeout = new Timeout(Duration.create(5, "seconds"));
|
||||
Future<Object> futureResult = akka.pattern.Patterns.ask(
|
||||
scatterGatherFirstCompletedRouter, new FibonacciActor.FibonacciNumber(10),
|
||||
timeout);
|
||||
int result = (Integer) Await.result(futureResult, timeout.duration());
|
||||
//#scatterGatherFirstCompletedRouter
|
||||
System.out.println(
|
||||
String.format("The result of calculating Fibonacci for 10 is %d", result));
|
||||
} else {
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#parentActor
|
||||
|
|
@ -1,16 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.jrouting;
|
||||
|
||||
import akka.actor.UntypedActor;
|
||||
|
||||
//#printlnActor
|
||||
public class PrintlnActor extends UntypedActor {
|
||||
public void onReceive(Object msg) {
|
||||
System.out.println(String.format("Received message '%s' in actor %s",
|
||||
msg, getSelf().path().name()));
|
||||
}
|
||||
}
|
||||
|
||||
//#printlnActor
|
||||
58
akka-docs/rst/java/code/docs/jrouting/RedundancyGroup.java
Normal file
58
akka-docs/rst/java/code/docs/jrouting/RedundancyGroup.java
Normal file
|
|
@ -0,0 +1,58 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.jrouting;
|
||||
|
||||
//#group
|
||||
import java.util.List;
|
||||
|
||||
import scala.Option;
|
||||
import scala.collection.immutable.Iterable;
|
||||
import akka.actor.ActorContext;
|
||||
import akka.actor.ActorPath;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Props;
|
||||
import akka.dispatch.Dispatchers;
|
||||
import akka.routing.Group;
|
||||
import akka.routing.Routee;
|
||||
import akka.routing.Router;
|
||||
import akka.routing.RouterActor;
|
||||
import akka.routing.RouterConfig;
|
||||
import akka.routing.RoutingLogic;
|
||||
|
||||
import com.typesafe.config.Config;
|
||||
|
||||
import akka.routing.GroupBase;
|
||||
import static docs.jrouting.CustomRouterDocTest.RedundancyRoutingLogic;
|
||||
|
||||
public class RedundancyGroup extends GroupBase {
|
||||
private final List<String> paths;
|
||||
private final int nbrCopies;
|
||||
|
||||
public RedundancyGroup(List<String> paths, int nbrCopies) {
|
||||
this.paths = paths;
|
||||
this.nbrCopies = nbrCopies;
|
||||
}
|
||||
|
||||
public RedundancyGroup(Config config) {
|
||||
this(config.getStringList("routees.paths"),
|
||||
config.getInt("nbr-copies"));
|
||||
}
|
||||
|
||||
@Override
|
||||
public java.lang.Iterable<String> getPaths() {
|
||||
return paths;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Router createRouter(ActorSystem system) {
|
||||
return new Router(new RedundancyRoutingLogic(nbrCopies));
|
||||
}
|
||||
|
||||
@Override
|
||||
public String routerDispatcher() {
|
||||
return Dispatchers.DefaultDispatcherId();
|
||||
}
|
||||
|
||||
}
|
||||
//#group
|
||||
412
akka-docs/rst/java/code/docs/jrouting/RouterDocTest.java
Normal file
412
akka-docs/rst/java/code/docs/jrouting/RouterDocTest.java
Normal file
|
|
@ -0,0 +1,412 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.jrouting;
|
||||
|
||||
import akka.testkit.AkkaJUnitActorSystemResource;
|
||||
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
|
||||
import static org.junit.Assert.*;
|
||||
import scala.concurrent.duration.Duration;
|
||||
import scala.concurrent.duration.FiniteDuration;
|
||||
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import akka.testkit.JavaTestKit;
|
||||
import akka.actor.ActorSystem;
|
||||
|
||||
|
||||
//#imports1
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.Props;
|
||||
import akka.actor.Terminated;
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.routing.ActorRefRoutee;
|
||||
import akka.routing.Routee;
|
||||
import akka.routing.Router;
|
||||
|
||||
//#imports1
|
||||
|
||||
|
||||
//#imports2
|
||||
import akka.actor.Address;
|
||||
import akka.actor.AddressFromURIString;
|
||||
import akka.actor.Kill;
|
||||
import akka.actor.PoisonPill;
|
||||
import akka.actor.SupervisorStrategy;
|
||||
import akka.actor.OneForOneStrategy;
|
||||
import akka.remote.routing.RemoteRouterConfig;
|
||||
import akka.routing.Broadcast;
|
||||
import akka.routing.BroadcastGroup;
|
||||
import akka.routing.BroadcastPool;
|
||||
import akka.routing.ConsistentHashingGroup;
|
||||
import akka.routing.ConsistentHashingPool;
|
||||
import akka.routing.DefaultResizer;
|
||||
import akka.routing.FromConfig;
|
||||
import akka.routing.RandomGroup;
|
||||
import akka.routing.RandomPool;
|
||||
import akka.routing.RoundRobinGroup;
|
||||
import akka.routing.RoundRobinPool;
|
||||
import akka.routing.RoundRobinRoutingLogic;
|
||||
import akka.routing.ScatterGatherFirstCompletedGroup;
|
||||
import akka.routing.ScatterGatherFirstCompletedPool;
|
||||
import akka.routing.SmallestMailboxPool;
|
||||
|
||||
//#imports2
|
||||
|
||||
public class RouterDocTest {
|
||||
|
||||
@ClassRule
|
||||
public static AkkaJUnitActorSystemResource actorSystemResource =
|
||||
new AkkaJUnitActorSystemResource("RouterDocTest",
|
||||
ConfigFactory.parseString(docs.routing.RouterDocSpec.config()));
|
||||
|
||||
private final ActorSystem system = actorSystemResource.getSystem();
|
||||
|
||||
|
||||
static
|
||||
//#router-in-actor
|
||||
public final class Work implements Serializable {
|
||||
private static final long serialVersionUID = 1L;
|
||||
public final String payload;
|
||||
public Work(String payload) {
|
||||
this.payload = payload;
|
||||
}
|
||||
}
|
||||
|
||||
//#router-in-actor
|
||||
static
|
||||
//#router-in-actor
|
||||
public class Master extends UntypedActor {
|
||||
|
||||
Router router;
|
||||
{
|
||||
List<Routee> routees = new ArrayList<Routee>();
|
||||
for (int i = 0; i < 5; i++) {
|
||||
ActorRef r = getContext().actorOf(Props.create(Worker.class));
|
||||
getContext().watch(r);
|
||||
routees.add(new ActorRefRoutee(r));
|
||||
}
|
||||
router = new Router(new RoundRobinRoutingLogic(), routees);
|
||||
}
|
||||
|
||||
public void onReceive(Object msg) {
|
||||
if (msg instanceof Work) {
|
||||
router.route(msg, getSender());
|
||||
} else if (msg instanceof Terminated) {
|
||||
router = router.removeRoutee(((Terminated) msg).actor());
|
||||
ActorRef r = getContext().actorOf(Props.create(Worker.class));
|
||||
getContext().watch(r);
|
||||
router = router.addRoutee(new ActorRefRoutee(r));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
//#router-in-actor
|
||||
|
||||
static public class Worker extends UntypedActor {
|
||||
public void onReceive(Object msg) {}
|
||||
}
|
||||
|
||||
static public class Echo extends UntypedActor {
|
||||
public void onReceive(Object msg) {
|
||||
getSender().tell(msg, getSelf());
|
||||
}
|
||||
}
|
||||
|
||||
static public class Replier extends UntypedActor {
|
||||
public void onReceive(Object msg) {
|
||||
//#reply-with-self
|
||||
getSender().tell("reply", getSelf());
|
||||
//#reply-with-self
|
||||
|
||||
//#reply-with-parent
|
||||
getSender().tell("reply", getContext().parent());
|
||||
//#reply-with-parent
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
||||
static
|
||||
//#create-worker-actors
|
||||
public class Workers extends UntypedActor {
|
||||
@Override public void preStart() {
|
||||
getContext().actorOf(Props.create(Worker.class), "w1");
|
||||
getContext().actorOf(Props.create(Worker.class), "w2");
|
||||
getContext().actorOf(Props.create(Worker.class), "w3");
|
||||
}
|
||||
// ...
|
||||
//#create-worker-actors
|
||||
|
||||
|
||||
public void onReceive(Object msg) {}
|
||||
}
|
||||
|
||||
static public class Parent extends UntypedActor {
|
||||
|
||||
//#paths
|
||||
List<String> paths = Arrays.asList("/user/workers/w1", "/user/workers/w2",
|
||||
"/user/workers/w3");
|
||||
//#paths
|
||||
|
||||
//#round-robin-pool-1
|
||||
ActorRef router1 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(Props.create(Worker.class)),
|
||||
"router1");
|
||||
//#round-robin-pool-1
|
||||
|
||||
//#round-robin-pool-2
|
||||
ActorRef router2 =
|
||||
getContext().actorOf(new RoundRobinPool(5).props(Props.create(Worker.class)),
|
||||
"router2");
|
||||
//#round-robin-pool-2
|
||||
|
||||
//#round-robin-group-1
|
||||
ActorRef router3 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(), "router3");
|
||||
//#round-robin-group-1
|
||||
|
||||
//#round-robin-group-2
|
||||
ActorRef router4 =
|
||||
getContext().actorOf(new RoundRobinGroup(paths).props(), "router4");
|
||||
//#round-robin-group-2
|
||||
|
||||
//#random-pool-1
|
||||
ActorRef router5 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(
|
||||
Props.create(Worker.class)), "router5");
|
||||
//#random-pool-1
|
||||
|
||||
//#random-pool-2
|
||||
ActorRef router6 =
|
||||
getContext().actorOf(new RandomPool(5).props(Props.create(Worker.class)),
|
||||
"router6");
|
||||
//#random-pool-2
|
||||
|
||||
//#random-group-1
|
||||
ActorRef router7 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(), "router7");
|
||||
//#random-group-1
|
||||
|
||||
//#random-group-2
|
||||
ActorRef router8 =
|
||||
getContext().actorOf(new RandomGroup(paths).props(), "router8");
|
||||
//#random-group-2
|
||||
|
||||
//#smallest-mailbox-pool-1
|
||||
ActorRef router9 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(
|
||||
Props.create(Worker.class)), "router9");
|
||||
//#smallest-mailbox-pool-1
|
||||
|
||||
//#smallest-mailbox-pool-2
|
||||
ActorRef router10 =
|
||||
getContext().actorOf(new SmallestMailboxPool(5).props(
|
||||
Props.create(Worker.class)), "router10");
|
||||
//#smallest-mailbox-pool-2
|
||||
|
||||
//#broadcast-pool-1
|
||||
ActorRef router11 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(
|
||||
Props.create(Worker.class)), "router11");
|
||||
//#broadcast-pool-1
|
||||
|
||||
//#broadcast-pool-2
|
||||
ActorRef router12 =
|
||||
getContext().actorOf(new BroadcastPool(5).props(Props.create(Worker.class)),
|
||||
"router12");
|
||||
//#broadcast-pool-2
|
||||
|
||||
//#broadcast-group-1
|
||||
ActorRef router13 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(), "router13");
|
||||
//#broadcast-group-1
|
||||
|
||||
//#broadcast-group-2
|
||||
ActorRef router14 =
|
||||
getContext().actorOf(new BroadcastGroup(paths).props(), "router14");
|
||||
//#broadcast-group-2
|
||||
|
||||
//#scatter-gather-pool-1
|
||||
ActorRef router15 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(
|
||||
Props.create(Worker.class)), "router15");
|
||||
//#scatter-gather-pool-1
|
||||
|
||||
//#scatter-gather-pool-2
|
||||
FiniteDuration within = FiniteDuration.create(10, TimeUnit.SECONDS);
|
||||
ActorRef router16 =
|
||||
getContext().actorOf(new ScatterGatherFirstCompletedPool(5, within).props(
|
||||
Props.create(Worker.class)), "router16");
|
||||
//#scatter-gather-pool-2
|
||||
|
||||
//#scatter-gather-group-1
|
||||
ActorRef router17 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(), "router17");
|
||||
//#scatter-gather-group-1
|
||||
|
||||
//#scatter-gather-group-2
|
||||
FiniteDuration within2 = FiniteDuration.create(10, TimeUnit.SECONDS);
|
||||
ActorRef router18 =
|
||||
getContext().actorOf(new ScatterGatherFirstCompletedGroup(paths, within2).props(),
|
||||
"router18");
|
||||
//#scatter-gather-group-2
|
||||
|
||||
//#consistent-hashing-pool-1
|
||||
ActorRef router19 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(Props.create(Worker.class)),
|
||||
"router19");
|
||||
//#consistent-hashing-pool-1
|
||||
|
||||
//#consistent-hashing-pool-2
|
||||
ActorRef router20 =
|
||||
getContext().actorOf(new ConsistentHashingPool(5).props(
|
||||
Props.create(Worker.class)), "router20");
|
||||
//#consistent-hashing-pool-2
|
||||
|
||||
//#consistent-hashing-group-1
|
||||
ActorRef router21 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(), "router21");
|
||||
//#consistent-hashing-group-1
|
||||
|
||||
//#consistent-hashing-group-2
|
||||
ActorRef router22 =
|
||||
getContext().actorOf(new ConsistentHashingGroup(paths).props(), "router22");
|
||||
//#consistent-hashing-group-2
|
||||
|
||||
//#resize-pool-1
|
||||
ActorRef router23 =
|
||||
getContext().actorOf(FromConfig.getInstance().props(
|
||||
Props.create(Worker.class)), "router23");
|
||||
//#resize-pool-1
|
||||
|
||||
//#resize-pool-2
|
||||
DefaultResizer resizer = new DefaultResizer(2, 15);
|
||||
ActorRef router24 =
|
||||
getContext().actorOf(new RoundRobinPool(5).withResizer(resizer).props(
|
||||
Props.create(Worker.class)), "router24");
|
||||
//#resize-pool-2
|
||||
|
||||
public void onReceive(Object msg) {}
|
||||
}
|
||||
|
||||
|
||||
|
||||
@Test
|
||||
public void createActors() {
|
||||
//#create-workers
|
||||
system.actorOf(Props.create(Workers.class), "workers");
|
||||
//#create-workers
|
||||
|
||||
//#create-parent
|
||||
system.actorOf(Props.create(Parent.class), "parent");
|
||||
//#create-parent
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateDispatcher() {
|
||||
//#dispatchers
|
||||
Props props =
|
||||
// “head” will run on "router-dispatcher" dispatcher
|
||||
new RoundRobinPool(5).withDispatcher("router-dispatcher").props(
|
||||
Props.create(Worker.class))
|
||||
// Worker routees will run on "workers-dispatcher" dispatcher
|
||||
.withDispatcher("workers-dispatcher");
|
||||
ActorRef router = system.actorOf(props);
|
||||
//#dispatchers
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateBroadcast() {
|
||||
new JavaTestKit(system) {{
|
||||
ActorRef router = system.actorOf(new RoundRobinPool(5).props(
|
||||
Props.create(Echo.class)));
|
||||
//#broadcastDavyJonesWarning
|
||||
router.tell(new Broadcast("Watch out for Davy Jones' locker"), getTestActor());
|
||||
//#broadcastDavyJonesWarning
|
||||
assertEquals(5, receiveN(5).length);
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstratePoisonPill() {
|
||||
new JavaTestKit(system) {{
|
||||
ActorRef router = watch(system.actorOf(new RoundRobinPool(5).props(
|
||||
Props.create(Echo.class))));
|
||||
//#poisonPill
|
||||
router.tell(PoisonPill.getInstance(), getTestActor());
|
||||
//#poisonPill
|
||||
expectTerminated(router);
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateBroadcastPoisonPill() {
|
||||
new JavaTestKit(system) {{
|
||||
ActorRef router = watch(system.actorOf(new RoundRobinPool(5).props(
|
||||
Props.create(Echo.class))));
|
||||
//#broadcastPoisonPill
|
||||
router.tell(new Broadcast(PoisonPill.getInstance()), getTestActor());
|
||||
//#broadcastPoisonPill
|
||||
expectTerminated(router);
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateKill() {
|
||||
new JavaTestKit(system) {{
|
||||
ActorRef router = watch(system.actorOf(new RoundRobinPool(5).props(
|
||||
Props.create(Echo.class))));
|
||||
//#kill
|
||||
router.tell(Kill.getInstance(), getTestActor());
|
||||
//#kill
|
||||
expectTerminated(router);
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateBroadcastKill() {
|
||||
new JavaTestKit(system) {{
|
||||
ActorRef router = watch(system.actorOf(new RoundRobinPool(5).props(
|
||||
Props.create(Echo.class))));
|
||||
//#broadcastKill
|
||||
router.tell(new Broadcast(Kill.getInstance()), getTestActor());
|
||||
//#broadcastKill
|
||||
expectTerminated(router);
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateRemoteDeploy() {
|
||||
//#remoteRoutees
|
||||
Address[] addresses = {
|
||||
new Address("akka", "remotesys", "otherhost", 1234),
|
||||
AddressFromURIString.parse("akka://othersys@anotherhost:1234")};
|
||||
ActorRef routerRemote = system.actorOf(
|
||||
new RemoteRouterConfig(new RoundRobinPool(5), addresses).props(
|
||||
Props.create(Echo.class)));
|
||||
//#remoteRoutees
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateSupervisor() {
|
||||
//#supervision
|
||||
final SupervisorStrategy strategy =
|
||||
new OneForOneStrategy(5, Duration.create(1, TimeUnit.MINUTES),
|
||||
Collections.<Class<? extends Throwable>>singletonList(Exception.class));
|
||||
final ActorRef router = system.actorOf(new RoundRobinPool(5).
|
||||
withSupervisorStrategy(strategy).props(Props.create(Echo.class)));
|
||||
//#supervision
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,61 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.jrouting;
|
||||
|
||||
import akka.routing.FromConfig;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.Props;
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.actor.ActorSystem;
|
||||
import com.typesafe.config.ConfigFactory;
|
||||
import com.typesafe.config.Config;
|
||||
|
||||
public class RouterViaConfigExample {
|
||||
|
||||
public static class ExampleActor extends UntypedActor {
|
||||
public void onReceive(Object msg) {
|
||||
if (msg instanceof Message) {
|
||||
Message message = (Message) msg;
|
||||
System.out.println(String.format("Received %s in router %s",
|
||||
message.getNbr(), getSelf().path().name()));
|
||||
} else {
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Message {
|
||||
private final int nbr;
|
||||
|
||||
public Message(int nbr) {
|
||||
this.nbr = nbr;
|
||||
}
|
||||
|
||||
public int getNbr() {
|
||||
return nbr;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
public static void main(String... args) {
|
||||
Config config = ConfigFactory.parseString("akka.actor.deployment {\n" + " /router {\n"
|
||||
+ " router = round-robin\n" + " nr-of-instances = 5\n" + " }\n" + "}\n");
|
||||
ActorSystem system = ActorSystem.create("Example", config);
|
||||
//#configurableRouting
|
||||
ActorRef router = system.actorOf(
|
||||
Props.create(ExampleActor.class).withRouter(new FromConfig()), "myrouter1");
|
||||
//#configurableRouting
|
||||
for (int i = 1; i <= 10; i++) {
|
||||
router.tell(new ExampleActor.Message(i), ActorRef.noSender());
|
||||
}
|
||||
|
||||
//#configurableRoutingWithResizer
|
||||
ActorRef router2 = system.actorOf(
|
||||
Props.create(ExampleActor.class).withRouter(new FromConfig()), "myrouter2");
|
||||
//#configurableRoutingWithResizer
|
||||
for (int i = 1; i <= 10; i++) {
|
||||
router2.tell(new ExampleActor.Message(i), ActorRef.noSender());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -1,122 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.jrouting;
|
||||
|
||||
import java.util.Arrays;
|
||||
|
||||
import akka.testkit.AkkaJUnitActorSystemResource;
|
||||
import org.junit.ClassRule;
|
||||
import org.junit.Test;
|
||||
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Kill;
|
||||
import akka.actor.PoisonPill;
|
||||
import akka.actor.Props;
|
||||
import akka.actor.Terminated;
|
||||
import akka.routing.Broadcast;
|
||||
import akka.routing.RoundRobinRouter;
|
||||
import akka.testkit.JavaTestKit;
|
||||
import docs.jrouting.RouterViaProgramExample.ExampleActor;
|
||||
import docs.routing.RouterViaProgramDocSpec.Echo;
|
||||
|
||||
public class RouterViaProgramDocTest {
|
||||
|
||||
@ClassRule
|
||||
public static AkkaJUnitActorSystemResource actorSystemResource =
|
||||
new AkkaJUnitActorSystemResource("RouterViaProgramDocTest");
|
||||
|
||||
private final ActorSystem system = actorSystemResource.getSystem();
|
||||
|
||||
private static class JavaTestKitWithSelf extends JavaTestKit {
|
||||
public JavaTestKitWithSelf(ActorSystem system) {
|
||||
super(system);
|
||||
}
|
||||
/**
|
||||
* Wrap `getRef()` so our examples look like they're within a normal actor.
|
||||
*/
|
||||
public ActorRef getSelf() {
|
||||
return getRef();
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
@Test
|
||||
public void demonstrateRouteesFromPaths() {
|
||||
new JavaTestKit(system) {{
|
||||
//#programmaticRoutingRouteePaths
|
||||
ActorRef actor1 = system.actorOf(Props.create(ExampleActor.class), "actor1");
|
||||
ActorRef actor2 = system.actorOf(Props.create(ExampleActor.class), "actor2");
|
||||
ActorRef actor3 = system.actorOf(Props.create(ExampleActor.class), "actor3");
|
||||
Iterable<String> routees = Arrays.asList(
|
||||
new String[] { "/user/actor1", "/user/actor2", "/user/actor3" });
|
||||
ActorRef router = system.actorOf(
|
||||
Props.empty().withRouter(new RoundRobinRouter(routees)));
|
||||
//#programmaticRoutingRouteePaths
|
||||
for (int i = 1; i <= 6; i++) {
|
||||
router.tell(new ExampleActor.Message(i), ActorRef.noSender());
|
||||
}
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateBroadcast() {
|
||||
new JavaTestKitWithSelf(system) {{
|
||||
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||
//#broadcastDavyJonesWarning
|
||||
router.tell(new Broadcast("Watch out for Davy Jones' locker"), getSelf());
|
||||
//#broadcastDavyJonesWarning
|
||||
receiveN(5, duration("5 seconds"));
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstratePoisonPill() {
|
||||
new JavaTestKitWithSelf(system) {{
|
||||
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||
watch(router);
|
||||
//#poisonPill
|
||||
router.tell(PoisonPill.getInstance(), getSelf());
|
||||
//#poisonPill
|
||||
expectMsgClass(Terminated.class);
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateBroadcastOfPoisonPill() {
|
||||
new JavaTestKitWithSelf(system) {{
|
||||
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||
watch(router);
|
||||
//#broadcastPoisonPill
|
||||
router.tell(new Broadcast(PoisonPill.getInstance()), getSelf());
|
||||
//#broadcastPoisonPill
|
||||
expectMsgClass(Terminated.class);
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateKill() {
|
||||
new JavaTestKitWithSelf(system) {{
|
||||
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||
watch(router);
|
||||
//#kill
|
||||
router.tell(Kill.getInstance(), getSelf());
|
||||
//#kill
|
||||
expectMsgClass(Terminated.class);
|
||||
}};
|
||||
}
|
||||
|
||||
@Test
|
||||
public void demonstrateBroadcastOfKill() {
|
||||
new JavaTestKitWithSelf(system) {{
|
||||
ActorRef router = system.actorOf(Props.create(Echo.class).withRouter(new RoundRobinRouter(5)));
|
||||
watch(router);
|
||||
//#broadcastKill
|
||||
router.tell(new Broadcast(Kill.getInstance()), getSelf());
|
||||
//#broadcastKill
|
||||
expectMsgClass(Terminated.class);
|
||||
}};
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,103 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.jrouting;
|
||||
|
||||
import akka.routing.RoundRobinRouter;
|
||||
import akka.routing.DefaultResizer;
|
||||
import akka.remote.routing.RemoteRouterConfig;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.Props;
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Address;
|
||||
import akka.actor.AddressFromURIString;
|
||||
import java.util.Arrays;
|
||||
|
||||
public class RouterViaProgramExample {
|
||||
|
||||
public static class ExampleActor extends UntypedActor {
|
||||
public void onReceive(Object msg) {
|
||||
if (msg instanceof Message) {
|
||||
Message message = (Message) msg;
|
||||
System.out.println(String.format("Received %s in router %s",
|
||||
message.getNbr(), getSelf().path().name()));
|
||||
} else {
|
||||
unhandled(msg);
|
||||
}
|
||||
}
|
||||
|
||||
public static class Message {
|
||||
private final int nbr;
|
||||
|
||||
public Message(int nbr) {
|
||||
this.nbr = nbr;
|
||||
}
|
||||
|
||||
public int getNbr() {
|
||||
return nbr;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
public static void main(String... args) {
|
||||
ActorSystem system = ActorSystem.create("RPE");
|
||||
//#programmaticRoutingNrOfInstances
|
||||
int nrOfInstances = 5;
|
||||
ActorRef router1 = system.actorOf(
|
||||
Props.create(ExampleActor.class).withRouter(new RoundRobinRouter(nrOfInstances)));
|
||||
//#programmaticRoutingNrOfInstances
|
||||
for (int i = 1; i <= 6; i++) {
|
||||
router1.tell(new ExampleActor.Message(i), ActorRef.noSender());
|
||||
}
|
||||
|
||||
//#programmaticRoutingRoutees
|
||||
ActorRef actor1 = system.actorOf(Props.create(ExampleActor.class));
|
||||
ActorRef actor2 = system.actorOf(Props.create(ExampleActor.class));
|
||||
ActorRef actor3 = system.actorOf(Props.create(ExampleActor.class));
|
||||
Iterable<ActorRef> routees = Arrays.asList(
|
||||
new ActorRef[] { actor1, actor2, actor3 });
|
||||
ActorRef router2 = system.actorOf(
|
||||
Props.empty().withRouter(RoundRobinRouter.create(routees)));
|
||||
//#programmaticRoutingRoutees
|
||||
for (int i = 1; i <= 6; i++) {
|
||||
router2.tell(new ExampleActor.Message(i), ActorRef.noSender());
|
||||
}
|
||||
|
||||
//#programmaticRoutingWithResizer
|
||||
int lowerBound = 2;
|
||||
int upperBound = 15;
|
||||
DefaultResizer resizer = new DefaultResizer(lowerBound, upperBound);
|
||||
ActorRef router3 = system.actorOf(
|
||||
Props.create(ExampleActor.class).withRouter(new RoundRobinRouter(resizer)));
|
||||
//#programmaticRoutingWithResizer
|
||||
for (int i = 1; i <= 6; i++) {
|
||||
router3.tell(new ExampleActor.Message(i), ActorRef.noSender());
|
||||
}
|
||||
|
||||
//#remoteRoutees
|
||||
Address addr1 = new Address("akka", "remotesys", "otherhost", 1234);
|
||||
Address addr2 = AddressFromURIString.parse("akka://othersys@anotherhost:1234");
|
||||
Address[] addresses = new Address[] { addr1, addr2 };
|
||||
ActorRef routerRemote = system.actorOf(Props.create(ExampleActor.class)
|
||||
.withRouter(new RemoteRouterConfig(new RoundRobinRouter(5), addresses)));
|
||||
//#remoteRoutees
|
||||
}
|
||||
|
||||
@SuppressWarnings("unused")
|
||||
private class CompileCheckJavaDocsForRouting extends UntypedActor {
|
||||
|
||||
@Override
|
||||
public void onReceive(Object o) {
|
||||
//#reply-with-parent
|
||||
getSender().tell("reply", getContext().parent()); // replies go to router
|
||||
//#reply-with-parent
|
||||
//#reply-with-self
|
||||
getSender().tell("reply", getSelf()); // replies go to this actor
|
||||
//#reply-with-self
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
@ -233,7 +233,7 @@ This is also done via configuration::
|
|||
actor {
|
||||
deployment {
|
||||
/serviceA/aggregation {
|
||||
router = "round-robin"
|
||||
router = "round-robin-pool"
|
||||
nr-of-instances = 10
|
||||
target {
|
||||
nodes = ["akka.tcp://app@10.0.0.2:2552", "akka://app@10.0.0.3:2552"]
|
||||
|
|
|
|||
|
|
@ -2,125 +2,132 @@
|
|||
.. _routing-java:
|
||||
|
||||
Routing
|
||||
==============
|
||||
=======
|
||||
|
||||
A Router is an actor that receives messages and efficiently routes them to other actors, known as
|
||||
its *routees*.
|
||||
Messages can be sent via a router to efficiently route them to destination actors, known as
|
||||
its *routees*. A ``Router`` can be used inside or outside of an actor, and you can manage the
|
||||
routees yourselves or use a self contained router actor with configuration capabilities.
|
||||
|
||||
Different routing strategies can be used, according to your application's needs. Akka comes with
|
||||
several useful routing strategies right out of the box. But, as you will see in this chapter, it is
|
||||
also possible to :ref:`create your own <custom-router-java>`.
|
||||
|
||||
The routers shipped with Akka are:
|
||||
.. _simple-router-java:
|
||||
|
||||
* ``akka.routing.RoundRobinRouter``
|
||||
* ``akka.routing.RandomRouter``
|
||||
* ``akka.routing.SmallestMailboxRouter``
|
||||
* ``akka.routing.BroadcastRouter``
|
||||
* ``akka.routing.ScatterGatherFirstCompletedRouter``
|
||||
* ``akka.routing.ConsistentHashingRouter``
|
||||
A Simple Router
|
||||
^^^^^^^^^^^^^^^
|
||||
|
||||
Routers in Action
|
||||
^^^^^^^^^^^^^^^^^
|
||||
The following example illustrates how to use a ``Router`` and manage the routees from within an actor.
|
||||
|
||||
Sending a message to a router is easy.
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#router-in-actor
|
||||
|
||||
.. code-block:: java
|
||||
We create a ``Router`` and specify that it should use ``RoundRobinRoutingLogic`` when routing the
|
||||
messages to the routees.
|
||||
|
||||
router.tell(new MyMsg(), sender);
|
||||
The routing logic shipped with Akka are:
|
||||
|
||||
A router actor forwards messages to its routees according to its routing policy.
|
||||
* ``akka.routing.RoundRobinRoutingLogic``
|
||||
* ``akka.routing.RandomRoutingLogic``
|
||||
* ``akka.routing.SmallestMailboxRoutingLogic``
|
||||
* ``akka.routing.BroadcastRoutingLogic``
|
||||
* ``akka.routing.ScatterGatherFirstCompletedRoutingLogic``
|
||||
* ``akka.routing.ConsistentHashingRoutingLogic``
|
||||
|
||||
We create the routees as ordinary child actors wrapped in ``ActorRefRoutee``. We watch
|
||||
the routees to be able to replace them if they are terminated.
|
||||
|
||||
Sending messages via the router is done with the ``route`` method, as is done for the ``Work`` messages
|
||||
in the example above.
|
||||
|
||||
The ``Router`` is immutable and the ``RoutingLogic`` is thread safe; meaning that they can also be used
|
||||
outside of actors.
|
||||
|
||||
.. note::
|
||||
|
||||
In general, any message sent to a router will be sent onwards to its routees. But there are a
|
||||
In general, any message sent to a router will be sent onwards to its routees, but there is one exception.
|
||||
The special :ref:`broadcast-messages-java` will send to *all* of a router's routees
|
||||
|
||||
A Router Actor
|
||||
^^^^^^^^^^^^^^
|
||||
|
||||
A router can also be created as a self contained actor that manages the routees itself and
|
||||
loads routing logic and other settings from configuration.
|
||||
|
||||
This type of router actor comes in two distinct flavors:
|
||||
|
||||
* Pool - The router creates routees as child actors and removes them from the router if they
|
||||
terminate.
|
||||
|
||||
* Group - The routee actors are created externally to the router and the router sends
|
||||
messages to the specified path using actor selection, without watching for termination.
|
||||
|
||||
The settings for a router actor can be defined in configuration or programmatically.
|
||||
Although router actors can be defined in the configuration file, they must still be created
|
||||
programmatically, i.e. you cannot make a router through external configuration alone.
|
||||
If you define the router actor in the configuration file then these settings will be used
|
||||
instead of any programmatically provided parameters.
|
||||
|
||||
You send messages to the routees via the router actor in the same way as for ordinary actors,
|
||||
i.e. via its ``ActorRef``. The router actor forwards messages onto its routees without changing
|
||||
the original sender. When a routee replies to a routed message, the reply will be sent to the
|
||||
original sender, not to the router actor.
|
||||
|
||||
.. note::
|
||||
|
||||
In general, any message sent to a router will be sent onwards to its routees, but there are a
|
||||
few exceptions. These are documented in the :ref:`router-special-messages-java` section below.
|
||||
|
||||
Creating a Router
|
||||
*****************
|
||||
Pool
|
||||
----
|
||||
|
||||
Routers and routees are closely intertwined. Router actors are created by specifying the desired
|
||||
*routee* :class:`Props` then attaching the router's :class:`RouterConfig`. When you create a router
|
||||
actor it will create routees, as needed, as its children.
|
||||
|
||||
For example, the following code and configuration snippets show how to create a :ref:`round-robin
|
||||
<round-robin-router-java>` router that forwards messages to five ``ExampleActor`` routees. The
|
||||
The following code and configuration snippets show how to create a :ref:`round-robin
|
||||
<round-robin-router-java>` router that forwards messages to five ``Worker`` routees. The
|
||||
routees will be created as the router's children.
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-round-robin
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-round-robin-pool
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaConfigExample.java#configurableRouting
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#round-robin-pool-1
|
||||
|
||||
Here is the same example, but with the router configuration provided programmatically instead of
|
||||
from configuration.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#programmaticRoutingNrOfInstances
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#round-robin-pool-2
|
||||
|
||||
Sometimes, rather than having the router create its routees, it is desirable to create routees
|
||||
separately and provide them to the router for its use. You can do this by passing an
|
||||
:class:`Iterable` of routees to the router's configuration.
|
||||
Remote Deployed Routees
|
||||
***********************
|
||||
|
||||
The example below shows how to create a router by providing it with the :class:`ActorRef`\s of three
|
||||
routee actors.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#programmaticRoutingRoutees
|
||||
|
||||
Routees can also be specified by providing their path strings instead of their :class:`ActorRef`\s.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramDocTest.java#programmaticRoutingRouteePaths
|
||||
|
||||
In addition to being able to supply looked-up remote actors as routees, you can ask the router to
|
||||
In addition to being able to create local actors as routees, you can instruct the router to
|
||||
deploy its created children on a set of remote hosts. Routees will be deployed in round-robin
|
||||
fashion. In order to deploy routees remotely, wrap the router configuration in a
|
||||
:class:`RemoteRouterConfig`, attaching the remote addresses of the nodes to deploy to. Remote
|
||||
``RemoteRouterConfig``, attaching the remote addresses of the nodes to deploy to. Remote
|
||||
deployment requires the ``akka-remote`` module to be included in the classpath.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#remoteRoutees
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#remoteRoutees
|
||||
|
||||
There are a few gotchas to be aware of when creating routers:
|
||||
Senders
|
||||
*******
|
||||
|
||||
* If you define the ``router`` in the configuration file then this value will be used instead of any
|
||||
programmatically provided parameters.
|
||||
* Although routers can be configured in the configuration file, they must still be created
|
||||
programmatically, i.e. you cannot make a router through external configuration alone.
|
||||
* If you provide the ``routees`` in the router configuration then
|
||||
the value of ``nrOfInstances``, if provided, will be disregarded.
|
||||
* When you provide routees programmatically the router will generally ignore the routee
|
||||
:class:`Props`, as it does not need to create routees. However, if you use a :ref:`resizable
|
||||
router <resizable-routers-java>` then the routee :class:`Props` will be used whenever the
|
||||
resizer creates new routees.
|
||||
|
||||
Routers, Routees and Senders
|
||||
****************************
|
||||
When a routee sends a message, it can :ref:`set itself as the sender
|
||||
<actors-tell-sender-java>`.
|
||||
|
||||
The router forwards messages onto its routees without changing the original sender. When a routee
|
||||
replies to a routed message, the reply will be sent to the original sender, not to the router.
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#reply-with-self
|
||||
|
||||
When a router creates routees, they are created as the routers children. This gives each routee its
|
||||
own identity in the actor system.
|
||||
However, it is often useful for routees to set the *router* as a sender. For example, you might want
|
||||
to set the router as the sender if you want to hide the details of the routees behind the router.
|
||||
The following code snippet shows how to set the parent router as sender.
|
||||
|
||||
When a routee replies it can :ref:`set itself <actors-tell-sender-java>` as the sender of the reply.
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#reply-with-parent
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#reply-with-self
|
||||
|
||||
However, it is often useful for routees to set the *router* as the sender. For example, you might
|
||||
want to set the router as the sender if you want to hide the details of the routees behind the
|
||||
router. The following code snippet shows how to set the parent router as sender.
|
||||
Supervision
|
||||
***********
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#reply-with-parent
|
||||
|
||||
Note that different code would be needed if the routees were not children of the router, i.e. if
|
||||
they were provided when the router was created.
|
||||
|
||||
Routers and Supervision
|
||||
^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
Routees can be created by a router or provided to the router when it is created. Any routees that
|
||||
are created by a router will be created as the router's children. The router is therefore also the
|
||||
children's supervisor.
|
||||
Routees that are created by a pool router will be created as the router's children. The router is
|
||||
therefore also the children's supervisor.
|
||||
|
||||
The supervision strategy of the router actor can be configured with the
|
||||
:meth:`RouterConfig.supervisorStrategy` property. If no configuration is provided, routers default
|
||||
``supervisorStrategy`` property of the Pool. If no configuration is provided, routers default
|
||||
to a strategy of “always escalate”. This means that errors are passed up to the router's supervisor
|
||||
for handling. The router's supervisor will decide what to do about any errors.
|
||||
|
||||
|
|
@ -140,97 +147,116 @@ by specifying the strategy when defining the router.
|
|||
|
||||
Setting the strategy is easily done:
|
||||
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java
|
||||
:include: supervision
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#supervision
|
||||
|
||||
.. _note-router-terminated-children-java:
|
||||
|
||||
.. note::
|
||||
|
||||
If the child of a router terminates, the router will not automatically spawn
|
||||
a new child. In the event that all children of a router have terminated the
|
||||
router will terminate itself, unless it is a dynamic router, e.g. using
|
||||
If the child of a pool router terminates, the pool router will not automatically spawn
|
||||
a new child. In the event that all children of a pool router have terminated the
|
||||
router will terminate itself unless it is a dynamic router, e.g. using
|
||||
a resizer.
|
||||
|
||||
Group
|
||||
-----
|
||||
|
||||
Sometimes, rather than having the router actor create its routees, it is desirable to create routees
|
||||
separately and provide them to the router for its use. You can do this by passing an
|
||||
paths of the routees to the router's configuration. Messages will be sent with ``ActorSelection``
|
||||
to these paths.
|
||||
|
||||
The example below shows how to create a router by providing it with the path strings of three
|
||||
routee actors.
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-round-robin-group
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#round-robin-group-1
|
||||
|
||||
Here is the same example, but with the router configuration provided programmatically instead of
|
||||
from configuration.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#round-robin-group-2
|
||||
|
||||
The routee actors are created externally from the router:
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#create-workers
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#create-worker-actors
|
||||
|
||||
The paths may contain protocol and address information for actors running on remote hosts.
|
||||
Remoting requires the ``akka-remote`` module to be included in the classpath.
|
||||
|
||||
Router usage
|
||||
^^^^^^^^^^^^
|
||||
|
||||
In this section we will describe how to use the different router types.
|
||||
First we need to create some actors that will be used in the examples:
|
||||
In this section we will describe how to create the different types of router actors.
|
||||
|
||||
.. includecode:: code/docs/jrouting/PrintlnActor.java#printlnActor
|
||||
The router actors in this section are created from within a top level actor named ``parent``.
|
||||
Note that deployment paths in the configuration starts with ``/parent/`` followed by the name
|
||||
of the router actor.
|
||||
|
||||
and
|
||||
|
||||
.. includecode:: code/docs/jrouting/FibonacciActor.java#fibonacciActor
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#create-parent
|
||||
|
||||
.. _round-robin-router-java:
|
||||
|
||||
RoundRobinRouter
|
||||
****************
|
||||
RoundRobinPool and RoundRobinGroup
|
||||
----------------------------------
|
||||
|
||||
Routes in a `round-robin <http://en.wikipedia.org/wiki/Round-robin>`_ fashion to its routees.
|
||||
Code example:
|
||||
|
||||
.. includecode:: code/docs/jrouting/ParentActor.java#roundRobinRouter
|
||||
RoundRobinPool defined in configuration:
|
||||
|
||||
When run you should see a similar output to this:
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-round-robin-pool
|
||||
|
||||
.. code-block:: scala
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#round-robin-pool-1
|
||||
|
||||
Received message '1' in actor $b
|
||||
Received message '2' in actor $c
|
||||
Received message '3' in actor $d
|
||||
Received message '6' in actor $b
|
||||
Received message '4' in actor $e
|
||||
Received message '8' in actor $d
|
||||
Received message '5' in actor $f
|
||||
Received message '9' in actor $e
|
||||
Received message '10' in actor $f
|
||||
Received message '7' in actor $c
|
||||
RoundRobinPool defined in code:
|
||||
|
||||
If you look closely to the output you can see that each of the routees received two messages which
|
||||
is exactly what you would expect from a round-robin router to happen.
|
||||
(The name of an actor is automatically created in the format ``$letter`` unless you specify it -
|
||||
hence the names printed above.)
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#round-robin-pool-2
|
||||
|
||||
This is an example of how to define a round-robin router in configuration:
|
||||
RoundRobinGroup defined in configuration:
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-round-robin
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-round-robin-group
|
||||
|
||||
RandomRouter
|
||||
************
|
||||
As the name implies this router type selects one of its routees randomly and forwards
|
||||
the message it receives to this routee.
|
||||
This procedure will happen each time it receives a message.
|
||||
Code example:
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#round-robin-group-1
|
||||
|
||||
.. includecode:: code/docs/jrouting/ParentActor.java#randomRouter
|
||||
RoundRobinGroup defined in code:
|
||||
|
||||
When run you should see a similar output to this:
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java
|
||||
:include: paths,round-robin-group-2
|
||||
|
||||
.. code-block:: scala
|
||||
RandomPool and RandomGroup
|
||||
--------------------------
|
||||
|
||||
Received message '1' in actor $e
|
||||
Received message '2' in actor $c
|
||||
Received message '4' in actor $b
|
||||
Received message '5' in actor $d
|
||||
Received message '3' in actor $e
|
||||
Received message '6' in actor $c
|
||||
Received message '7' in actor $d
|
||||
Received message '8' in actor $e
|
||||
Received message '9' in actor $d
|
||||
Received message '10' in actor $d
|
||||
This router type selects one of its routees randomly for each message.
|
||||
|
||||
The result from running the random router should be different, or at least random, every time you run it.
|
||||
Try to run it a couple of times to verify its behavior if you don't trust us.
|
||||
RandomPool defined in configuration:
|
||||
|
||||
This is an example of how to define a random router in configuration:
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-random-pool
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-random
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#random-pool-1
|
||||
|
||||
SmallestMailboxRouter
|
||||
*********************
|
||||
A Router that tries to send to the non-suspended routee with fewest messages in mailbox.
|
||||
RandomPool defined in code:
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#random-pool-2
|
||||
|
||||
RandomGroup defined in configuration:
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-random-group
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#random-group-1
|
||||
|
||||
RandomGroup defined in code:
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java
|
||||
:include: paths,random-group-2
|
||||
|
||||
|
||||
SmallestMailboxPool
|
||||
-------------------
|
||||
|
||||
A Router that tries to send to the non-suspended child routee with fewest messages in mailbox.
|
||||
The selection is done in this order:
|
||||
|
||||
* pick any idle routee (not processing message) with empty mailbox
|
||||
|
|
@ -239,36 +265,45 @@ The selection is done in this order:
|
|||
* pick any remote routee, remote actors are consider lowest priority,
|
||||
since their mailbox size is unknown
|
||||
|
||||
Code example:
|
||||
SmallestMailboxPool defined in configuration:
|
||||
|
||||
.. includecode:: code/docs/jrouting/ParentActor.java#smallestMailboxRouter
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-smallest-mailbox-pool
|
||||
|
||||
This is an example of how to define a smallest-mailbox router in configuration:
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#smallest-mailbox-pool-1
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-smallest-mailbox
|
||||
SmallestMailboxPool defined in code:
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#smallest-mailbox-pool-2
|
||||
|
||||
There is no Group variant of the SmallestMailboxPool because the size of the mailbox
|
||||
and the internal dispatching state of the actor is not practically available from the paths
|
||||
of the routees.
|
||||
|
||||
BroadcastPool and BroadcastGroup
|
||||
--------------------------------
|
||||
|
||||
BroadcastRouter
|
||||
***************
|
||||
A broadcast router forwards the message it receives to *all* its routees.
|
||||
Code example:
|
||||
|
||||
.. includecode:: code/docs/jrouting/ParentActor.java#broadcastRouter
|
||||
BroadcastPool defined in configuration:
|
||||
|
||||
When run you should see a similar output to this:
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-broadcast-pool
|
||||
|
||||
.. code-block:: scala
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#broadcast-pool-1
|
||||
|
||||
Received message 'this is a broadcast message' in actor $f
|
||||
Received message 'this is a broadcast message' in actor $d
|
||||
Received message 'this is a broadcast message' in actor $e
|
||||
Received message 'this is a broadcast message' in actor $c
|
||||
Received message 'this is a broadcast message' in actor $b
|
||||
BroadcastPool defined in code:
|
||||
|
||||
As you can see here above each of the routees, five in total, received the broadcast message.
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#broadcast-pool-2
|
||||
|
||||
This is an example of how to define a broadcast router in configuration:
|
||||
BroadcastGroup defined in configuration:
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-broadcast
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-broadcast-group
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#broadcast-group-1
|
||||
|
||||
BroadcastGroup defined in code:
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java
|
||||
:include: paths,broadcast-group-2
|
||||
|
||||
.. note::
|
||||
|
||||
|
|
@ -276,39 +311,50 @@ This is an example of how to define a broadcast router in configuration:
|
|||
broadcast every message, then you can use a non-broadcasting router and use
|
||||
:ref:`broadcast-messages-java` as needed.
|
||||
|
||||
ScatterGatherFirstCompletedRouter
|
||||
*********************************
|
||||
The ScatterGatherFirstCompletedRouter will send the message on to all its routees as a future.
|
||||
It then waits for first result it gets back. This result will be sent back to original sender.
|
||||
Code example:
|
||||
|
||||
.. includecode:: code/docs/jrouting/ParentActor.java#scatterGatherFirstCompletedRouter
|
||||
ScatterGatherFirstCompletedPool and ScatterGatherFirstCompletedGroup
|
||||
--------------------------------------------------------------------
|
||||
|
||||
When run you should see this:
|
||||
The ScatterGatherFirstCompletedRouter will send the message on to all its routees.
|
||||
It then waits for first reply it gets back. This result will be sent back to original sender.
|
||||
Other replies are discarded.
|
||||
|
||||
.. code-block:: scala
|
||||
It is expecting at least one reply within a configured duration, otherwise it will reply with
|
||||
``akka.pattern.AskTimeoutException`` in a ``akka.actor.Status.Failure``.
|
||||
|
||||
The result of calculating Fibonacci for 10 is 55
|
||||
ScatterGatherFirstCompletedPool defined in configuration:
|
||||
|
||||
From the output above you can't really see that all the routees performed the calculation, but they did!
|
||||
The result you see is from the first routee that returned its calculation to the router.
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-scatter-gather-pool
|
||||
|
||||
This is an example of how to define a scatter-gather router in configuration:
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#scatter-gather-pool-1
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-scatter-gather
|
||||
ScatterGatherFirstCompletedPool defined in code:
|
||||
|
||||
ConsistentHashingRouter
|
||||
***********************
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#scatter-gather-pool-2
|
||||
|
||||
The ConsistentHashingRouter uses `consistent hashing <http://en.wikipedia.org/wiki/Consistent_hashing>`_
|
||||
to select a connection based on the sent message. This
|
||||
ScatterGatherFirstCompletedGroup defined in configuration:
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-scatter-gather-group
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#scatter-gather-group-1
|
||||
|
||||
ScatterGatherFirstCompletedGroup defined in code:
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java
|
||||
:include: paths,scatter-gather-group-2
|
||||
|
||||
ConsistentHashingPool and ConsistentHashingGroup
|
||||
------------------------------------------------
|
||||
|
||||
The ConsistentHashingPool uses `consistent hashing <http://en.wikipedia.org/wiki/Consistent_hashing>`_
|
||||
to select a routee based on the sent message. This
|
||||
`article <http://weblogs.java.net/blog/tomwhite/archive/2007/11/consistent_hash.html>`_ gives good
|
||||
insight into how consistent hashing is implemented.
|
||||
|
||||
There is 3 ways to define what data to use for the consistent hash key.
|
||||
|
||||
* You can define ``withHashMapper`` of the router to map incoming
|
||||
messages to their consistent hash key. This makes the the decision
|
||||
messages to their consistent hash key. This makes the decision
|
||||
transparent for the sender.
|
||||
|
||||
* The messages may implement ``akka.routing.ConsistentHashingRouter.ConsistentHashable``.
|
||||
|
|
@ -322,40 +368,58 @@ There is 3 ways to define what data to use for the consistent hash key.
|
|||
These ways to define the consistent hash key can be use together and at
|
||||
the same time for one router. The ``withHashMapper`` is tried first.
|
||||
|
||||
|
||||
Code example:
|
||||
|
||||
.. includecode:: code/docs/jrouting/ConsistentHashingRouterDocTest.java
|
||||
:include: imports1
|
||||
.. includecode:: code/docs/jrouting/ConsistentHashingRouterDocTest.java#cache-actor
|
||||
|
||||
.. includecode:: code/docs/jrouting/ConsistentHashingRouterDocTest.java
|
||||
:include: cache-actor
|
||||
|
||||
.. includecode:: code/docs/jrouting/ConsistentHashingRouterDocTest.java
|
||||
:include: imports2
|
||||
|
||||
.. includecode:: code/docs/jrouting/ConsistentHashingRouterDocTest.java
|
||||
:include: consistent-hashing-router
|
||||
.. includecode:: code/docs/jrouting/ConsistentHashingRouterDocTest.java#consistent-hashing-router
|
||||
|
||||
In the above example you see that the ``Get`` message implements ``ConsistentHashable`` itself,
|
||||
while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. The ``Evict``
|
||||
message is handled by the ``withHashMapper``.
|
||||
message is handled by the ``hashMapping`` partial function.
|
||||
|
||||
This is an example of how to define a consistent-hashing router in configuration:
|
||||
ConsistentHashingPool defined in configuration:
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-consistent-hashing
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-consistent-hashing-pool
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#consistent-hashing-pool-1
|
||||
|
||||
ConsistentHashingPool defined in code:
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#consistent-hashing-pool-2
|
||||
|
||||
ConsistentHashingGroup defined in configuration:
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-consistent-hashing-group
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#consistent-hashing-group-1
|
||||
|
||||
ConsistentHashingGroup defined in code:
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java
|
||||
:include: paths,consistent-hashing-group-2
|
||||
|
||||
|
||||
``virtual-nodes-factor`` is the number of virtual nodes per routee that is used in the
|
||||
consistent hash node ring to make the distribution more uniform.
|
||||
|
||||
.. _router-special-messages-java:
|
||||
|
||||
Handling for Special Messages
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
Specially Handled Messages
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
Most messages sent to routers will be forwarded according to the routers' usual routing rules.
|
||||
Most messages sent to router actors will be forwarded according to the routers' routing logic.
|
||||
However there are a few types of messages that have special behavior.
|
||||
|
||||
Note that these special messages, except for the ``Broadcast`` message, are only handled by
|
||||
self contained router actors and not by the ``akka.routing.Router`` component described
|
||||
in :ref:`simple-router-java`.
|
||||
|
||||
.. _broadcast-messages-java:
|
||||
|
||||
Broadcast Messages
|
||||
******************
|
||||
------------------
|
||||
|
||||
A ``Broadcast`` message can be used to send a message to *all* of a router's routees. When a router
|
||||
receives a ``Broadcast`` message, it will broadcast that message's *payload* to all routees, no
|
||||
|
|
@ -364,20 +428,20 @@ matter how that router would normally route its messages.
|
|||
The example below shows how you would use a ``Broadcast`` message to send a very important message
|
||||
to every routee of a router.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramDocTest.java#broadcastDavyJonesWarning
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#broadcastDavyJonesWarning
|
||||
|
||||
In this example the router receives the ``Broadcast`` message, extracts its payload
|
||||
(``"Watch out for Davy Jones' locker"``), and then sends the payload on to all of the router's
|
||||
routees. It is up to each each routee actor to handle the received payload message.
|
||||
|
||||
PoisonPill Messages
|
||||
*******************
|
||||
-------------------
|
||||
|
||||
A ``PoisonPill`` message has special handling for all actors, including for routers. When any actor
|
||||
receives a ``PoisonPill`` message, that actor will be stopped. See the :ref:`poison-pill-java`
|
||||
documentation for details.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramDocTest.java#poisonPill
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#poisonPill
|
||||
|
||||
For a router, which normally passes on messages to routees, it is important to realise that
|
||||
``PoisonPill`` messages are processed by the router only. ``PoisonPill`` messages sent to a router
|
||||
|
|
@ -386,16 +450,16 @@ will *not* be sent on to routees.
|
|||
However, a ``PoisonPill`` message sent to a router may still affect its routees, because it will
|
||||
stop the router and when the router stops it also stops its children. Stopping children is normal
|
||||
actor behavior. The router will stop routees that it has created as children. Each child will
|
||||
process its current message and then tstop. This may lead to some messages being unprocessed. See
|
||||
the documentation on :ref:`stopping-actors-java` for more information.
|
||||
process its current message and then stop. This may lead to some messages being unprocessed.
|
||||
See the documentation on :ref:`stopping-actors-java` for more information.
|
||||
|
||||
If you wish to stop a router and its routees, but you would like the routees to first process all
|
||||
the messages currently in their mailboxes, then you should not send a ``PoisonPill`` message to the
|
||||
router. Instead you should wrap a ``PoisonPill`` message inside a broadcast message so that each
|
||||
routee will the ``PoisonPill`` message directly. Note that this will stop all routees, even if the
|
||||
router. Instead you should wrap a ``PoisonPill`` message inside a ``Broadcast`` message so that each
|
||||
routee will receive the ``PoisonPill`` message. Note that this will stop all routees, even if the
|
||||
routees aren't children of the router, i.e. even routees programmatically provided to the router.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramDocTest.java#broadcastPoisonPill
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#broadcastPoisonPill
|
||||
|
||||
With the code shown above, each routee will receive a ``PoisonPill`` message. Each routee will
|
||||
continue to process its messages as normal, eventually processing the ``PoisonPill``. This will
|
||||
|
|
@ -410,48 +474,63 @@ a resizer.
|
|||
discusses in more detail how ``PoisonPill`` messages can be used to shut down routers and routees.
|
||||
|
||||
Kill Messages
|
||||
*************
|
||||
-------------
|
||||
|
||||
``Kill`` messages are another type of message that has special handling. See
|
||||
:ref:`killing-actors-java` for general information about how actors handle ``Kill`` messages.
|
||||
|
||||
When a ``Kill`` message is sent to a router the router processes the message internally, and does
|
||||
*not* send it on to its routees. The router will throw an :class:`ActorKilledException` and fail. It
|
||||
*not* send it on to its routees. The router will throw an ``ActorKilledException`` and fail. It
|
||||
will then be either resumed, restarted or terminated, depending how it is supervised.
|
||||
|
||||
Routees that are children of the router will also be suspended, and will be affected by the
|
||||
supervision directive that is applied to the router. Routees that are not the routers children, i.e.
|
||||
those that were created externally to the router, will not be affected.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramDocTest.java#kill
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#kill
|
||||
|
||||
As with the ``PoisonPill`` message, there is a distinction between killing a router, which
|
||||
indirectly kills its children (who happen to be routees), and killing routees directly (some of whom
|
||||
may not be children.) To kill routees directly the router should be sent a ``Kill`` message wrapped
|
||||
in a ``Broadcast`` message.
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramDocTest.java#broadcastKill
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#broadcastKill
|
||||
|
||||
Managagement Messages
|
||||
---------------------
|
||||
|
||||
* Sending ``akka.routing.GetRoutees`` to a router actor will make it send back its currently used routees
|
||||
in a ``akka.routing.Routees`` message.
|
||||
* Sending ``akka.routing.AddRoutee`` to a router actor will add that routee to its collection of routees.
|
||||
* Sending ``akka.routing.RemoveRoutee`` to a router actor will remove that routee to its collection of routees.
|
||||
* Sending ``akka.routing.AdjustPoolSize`` to a pool router actor will add or remove that number of routees to
|
||||
its collection of routees.
|
||||
|
||||
These management messages may be handled after other messages, so if you send ``AddRoutee`` immediately followed
|
||||
an ordinary message you are not guaranteed that the routees have been changed when the ordinary message
|
||||
is routed. If you need to know when the change has been applied you can send ``AddRoutee`` followed by ``GetRoutees``
|
||||
and when you receive the ``Routees`` reply you know that the preceeding change has been applied.
|
||||
|
||||
.. _resizable-routers-java:
|
||||
|
||||
Dynamically Resizable Routers
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
Dynamically Resizable Pool
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
All routers can be used with a fixed number of routees or with a resize strategy to adjust the number
|
||||
All pools can be used with a fixed number of routees or with a resize strategy to adjust the number
|
||||
of routees dynamically.
|
||||
|
||||
This is an example of how to create a resizable router that is defined in configuration:
|
||||
Pool with resizer defined in configuration:
|
||||
|
||||
.. includecode:: ../scala/code/docs/routing/RouterViaConfigDocSpec.scala#config-resize
|
||||
.. includecode:: ../scala/code/docs/routing/RouterDocSpec.scala#config-resize-pool
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaConfigExample.java#configurableRoutingWithResizer
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#resize-pool-1
|
||||
|
||||
Several more configuration options are available and described in ``akka.actor.deployment.default.resizer``
|
||||
section of the reference :ref:`configuration`.
|
||||
|
||||
This is an example of how to programmatically create a resizable router:
|
||||
Pool with resizer defined in code:
|
||||
|
||||
.. includecode:: code/docs/jrouting/RouterViaProgramExample.java#programmaticRoutingWithResizer
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#resize-pool-2
|
||||
|
||||
*It is also worth pointing out that if you define the ``router`` in the configuration file then this value
|
||||
will be used instead of any programmatically sent parameters.*
|
||||
|
|
@ -460,7 +539,7 @@ will be used instead of any programmatically sent parameters.*
|
|||
|
||||
Resizing is triggered by sending messages to the actor pool, but it is not
|
||||
completed synchronously; instead a message is sent to the “head”
|
||||
:class:`Router` to perform the size change. Thus you cannot rely on resizing
|
||||
``RouterActor`` to perform the size change. Thus you cannot rely on resizing
|
||||
to instantaneously create new workers when all others are busy, because the
|
||||
message just sent will be queued to the mailbox of a busy actor. To remedy
|
||||
this, configure the pool to use a balancing dispatcher, see `Configuring
|
||||
|
|
@ -478,8 +557,8 @@ routees.
|
|||
A normal actor can be used for routing messages, but an actor's single-threaded processing can
|
||||
become a bottleneck. Routers can achieve much higher throughput with an optimization to the usual
|
||||
message-processing pipeline that allows concurrent routing. This is achieved by embedding routers'
|
||||
routing logic directly in their :class:`ActorRef` rather than in the router actor. Messages sent to
|
||||
a router's :class:`ActorRef` can be immediately routed to the routee, bypassing the single-threaded
|
||||
routing logic directly in their ``ActorRef`` rather than in the router actor. Messages sent to
|
||||
a router's ``ActorRef`` can be immediately routed to the routee, bypassing the single-threaded
|
||||
router actor entirely.
|
||||
|
||||
The cost to this is, of course, that the internals of routing code are more complicated than if
|
||||
|
|
@ -503,82 +582,54 @@ lower maximum throughput is acceptable in your application you may wish to stick
|
|||
actors. This section, however, assumes that you wish to get maximum performance and so demonstrates
|
||||
how you can create your own router.
|
||||
|
||||
The router created in this example is a simple vote counter. It will route the votes to specific vote counter actors.
|
||||
In this case we only have two parties the Republicans and the Democrats. We would like a router that forwards all
|
||||
democrat related messages to the Democrat actor and all republican related messages to the Republican actor.
|
||||
The router created in this example is replicating each message to a few destinations.
|
||||
|
||||
We begin with defining the class:
|
||||
Start with the routing logic:
|
||||
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#crRouter
|
||||
:exclude: crRoute
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#routing-logic
|
||||
|
||||
The next step is to implement the ``createCustomRoute`` method in the class just defined:
|
||||
``select`` will be called for each message and in this example pick a few destinations by round-robin,
|
||||
by reusing the existing ``RoundRobinRoutingLogic`` and wrap the result in a ``SeveralRoutees``
|
||||
instance. ``SeveralRoutees`` will send the message to all of the supplied routues.
|
||||
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#crRoute
|
||||
The implementation of the routing logic must be thread safe, since it might be used outside of actors.
|
||||
|
||||
As you can see above we start off by creating the routees and put them in a collection.
|
||||
A unit test of the routing logic:
|
||||
|
||||
Make sure that you don't miss to implement the line below as it is *really* important.
|
||||
It registers the routees internally and failing to call this method will
|
||||
cause a ``ActorInitializationException`` to be thrown when the router is used.
|
||||
Therefore always make sure to do the following in your custom router:
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#unit-test-logic
|
||||
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#crRegisterRoutees
|
||||
You could stop here and use the ``RedundancyRoutingLogic`` with a ``akka.routing.Router``
|
||||
as described in :ref:`simple-router-java`.
|
||||
|
||||
The routing logic is where your magic sauce is applied. In our example it inspects the message types
|
||||
and forwards to the correct routee based on this:
|
||||
Let us continue and make this into a self contained, configurable, router actor.
|
||||
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#crRoutingLogic
|
||||
Create a class that extends ``PoolBase``, ``GroupBase`` or ``CustomRouterConfig``. That class is a factory
|
||||
for the routing logic and holds the configuration for the router. Here we make it a ``Group``.
|
||||
|
||||
As you can see above what's returned in the ``CustomRoute`` function, which defines the mapping
|
||||
from incoming sender/message to a ``List`` of ``Destination(sender, routee)``.
|
||||
The sender is what "parent" the routee should see - changing this could be useful if you for example want
|
||||
another actor than the original sender to intermediate the result of the routee (if there is a result).
|
||||
For more information about how to alter the original sender we refer to the source code of
|
||||
`ScatterGatherFirstCompletedRouter <https://github.com/akka/akka/blob/master/akka-actor/src/main/scala/akka/routing/Routing.scala#L375>`_
|
||||
.. includecode:: code/docs/jrouting/RedundancyGroup.java#group
|
||||
|
||||
All in all the custom router looks like this:
|
||||
This can be used exactly as the router actors provided by Akka.
|
||||
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#CustomRouter
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#usage-1
|
||||
|
||||
If you are interested in how to use the VoteCountRouter it looks like this:
|
||||
Note that we added a constructor in ``RedundancyGroup`` that takes a ``Config`` parameter.
|
||||
That makes it possible to define it in configuration.
|
||||
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#crTest
|
||||
.. includecode:: ../scala/code/docs/routing/CustomRouterDocSpec.scala#jconfig
|
||||
|
||||
.. caution::
|
||||
|
||||
When creating a cutom router the resulting RoutedActorRef optimizes the
|
||||
sending of the message so that it does NOT go through the router’s mailbox
|
||||
unless the route returns an empty recipient set.
|
||||
|
||||
This means that the ``route`` function defined in the ``RouterConfig``
|
||||
or the function returned from ``CreateCustomRoute`` in
|
||||
``CustomRouterConfig`` is evaluated concurrently without protection by
|
||||
the RoutedActorRef: either provide a reentrant (i.e. pure) implementation
|
||||
or do the locking yourself!
|
||||
|
||||
|
||||
Configured Custom Router
|
||||
************************
|
||||
|
||||
It is possible to define configuration properties for custom routers. In the ``router`` property of the deployment
|
||||
configuration you define the fully qualified class name of the router class. The router class must extend
|
||||
``akka.routing.CustomRouterConfig`` and have constructor with one ``com.typesafe.config.Config`` parameter.
|
||||
Note the fully qualified class name in the ``router`` property. The router class must extend
|
||||
``akka.routing.RouterConfig`` (``Pool``, ``Group`` or ``CustomRouterConfig``) and have
|
||||
constructor with one ``com.typesafe.config.Config`` parameter.
|
||||
The deployment section of the configuration is passed to the constructor.
|
||||
|
||||
Custom Resizer
|
||||
**************
|
||||
|
||||
A router with dynamically resizable number of routees is implemented by providing a ``akka.routing.Resizer``
|
||||
in ``resizer`` method of the ``RouterConfig``. See ``akka.routing.DefaultResizer`` for inspiration
|
||||
of how to write your own resize strategy.
|
||||
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#usage-2
|
||||
|
||||
Configuring Dispatchers
|
||||
^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
The dispatcher for created children of the router will be taken from
|
||||
:class:`Props` as described in :ref:`dispatchers-java`. For a dynamic pool it
|
||||
makes sense to configure the :class:`BalancingDispatcher` if the precise
|
||||
``Props`` as described in :ref:`dispatchers-java`. For a pool it
|
||||
makes sense to configure the ``BalancingDispatcher`` if the precise
|
||||
routing is not so important (i.e. no consistent hashing or round-robin is
|
||||
required); this enables newly created routees to pick up work immediately by
|
||||
stealing it from their siblings.
|
||||
|
|
@ -591,13 +642,13 @@ stealing it from their siblings.
|
|||
|
||||
The “head” router cannot always run on the same dispatcher, because it
|
||||
does not process the same type of messages, hence this special actor does
|
||||
not use the dispatcher configured in :class:`Props`, but takes the
|
||||
not use the dispatcher configured in ``Props``, but takes the
|
||||
``routerDispatcher`` from the :class:`RouterConfig` instead, which defaults to
|
||||
the actor system’s default dispatcher. All standard routers allow setting this
|
||||
property in their constructor or factory method, custom routers have to
|
||||
implement the method in a suitable way.
|
||||
|
||||
.. includecode:: code/docs/jrouting/CustomRouterDocTest.java#dispatchers
|
||||
.. includecode:: code/docs/jrouting/RouterDocTest.java#dispatchers
|
||||
|
||||
.. note::
|
||||
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@
|
|||
Migration Guide 2.2.x to 2.3.x
|
||||
################################
|
||||
|
||||
The 2.2 release contains some structural changes that require some
|
||||
The 2.3 release contains some structural changes that require some
|
||||
simple, mechanical source-level changes in client code.
|
||||
|
||||
When migrating from earlier versions you should first follow the instructions for
|
||||
|
|
@ -29,4 +29,54 @@ configured time of unreachability. This feature is disabled by default, as it al
|
|||
|
||||
During the deprecation phase ``akka.cluster.auto-down=on`` is interpreted at as instant auto-down.
|
||||
|
||||
=======
|
||||
Routers
|
||||
=======
|
||||
|
||||
The routers have been cleaned up and enhanced. The routing logic has been extracted to be usable within
|
||||
normal actors as well. Some usability problems have been have been solved, such as properly reject invalid
|
||||
configuration combinations. Routees can be dynamically added and removed by sending special management messages
|
||||
to the router.
|
||||
|
||||
The two types of routers have been named ``Pool`` and ``Group`` to make them more distinguishable and reduce confusion
|
||||
of their subtle differences:
|
||||
|
||||
* Pool - The router creates routees as child actors and removes them from the router if they
|
||||
terminate.
|
||||
|
||||
* Group - The routee actors are created externally to the router and the router sends
|
||||
messages to the specified path using actor selection, without watching for termination.
|
||||
|
||||
Configuration of routers is compatible with 2.2.x, but the ``router`` type should preferably be specified
|
||||
with ``-pool`` or ``-group`` suffix.
|
||||
|
||||
Some classes used for programmatic definition of routers have been renamed, but the old classes remain as
|
||||
deprecated. The compiler will guide you with deprecation warning. For example ``RoundRobinRouter`` has
|
||||
been renamed to ``RoundRobinPool`` or ``RoundRobinGroup`` depending on which type you are actually using.
|
||||
|
||||
There is no replacement for ``SmallestMailboxRouter`` combined with routee paths, i.e. a group, because that
|
||||
combination is not useful.
|
||||
|
||||
An optional API enhancement that makes the code read better is to use the ``props`` method instead of ``withRouter``.
|
||||
``withRouter`` has not been deprecated and you can continue to use that if you prefer that way of defining a router.
|
||||
|
||||
Example in Scala::
|
||||
|
||||
context.actorOf(FromConfig.props(Props[Worker]), "router1")
|
||||
context.actorOf(RoundRobinPool(5).props(Props[Worker]), "router2")
|
||||
|
||||
Example in Java::
|
||||
|
||||
getContext().actorOf(FromConfig.getInstance().props(Props.create(Worker.class)),
|
||||
"router1");
|
||||
|
||||
getContext().actorOf(new RoundRobinPool(5).props(Props.create(Worker.class)),
|
||||
"router2");
|
||||
|
||||
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
|
||||
without much trouble.
|
||||
|
||||
Read more about the new routers in the :ref:`documentation for Scala <routing-scala>` and
|
||||
:ref:`documentation for Java <routing-java>`.
|
||||
|
||||
|
|
|
|||
|
|
@ -420,12 +420,12 @@ when a node becomes reachable again, after having been unreachable.
|
|||
|
||||
There are two distinct types of routers.
|
||||
|
||||
* **Router that lookup existing actors and use them as routees.** The routees can be shared between
|
||||
routers running on different nodes in the cluster. One example of a use case for this
|
||||
type of router is a service running on some backend nodes in the cluster and
|
||||
used by routers running on front-end nodes in the cluster.
|
||||
* **Group - router that sends messages to the specified path using actor selection**
|
||||
The routees can be shared between routers running on different nodes in the cluster.
|
||||
One example of a use case for this type of router is a service running on some backend
|
||||
nodes in the cluster and used by routers running on front-end nodes in the cluster.
|
||||
|
||||
* **Router that creates new routees as child actors and deploy them on remote nodes.**
|
||||
* **Pool - router that creates routees as child actors and deploys them on remote nodes.**
|
||||
Each router will have its own routee instances. For example, if you start a router
|
||||
on 3 nodes in a 10 nodes cluster you will have 30 routee actors in total if the router is
|
||||
configured to use one inctance per node. The routees created by the the different routers
|
||||
|
|
@ -433,11 +433,11 @@ There are two distinct types of routers.
|
|||
is a single master that coordinate jobs and delegates the actual work to routees running
|
||||
on other nodes in the cluster.
|
||||
|
||||
Router with Lookup of Routees
|
||||
-----------------------------
|
||||
Router with Group of Routees
|
||||
----------------------------
|
||||
|
||||
When using a router with routees looked up on the cluster member nodes, i.e. the routees
|
||||
are already running, the configuration for a router looks like this:
|
||||
When using a ``Group`` you must start the routee actors on the cluster member nodes.
|
||||
That is not done by the router. The configuration for a group looks like this:
|
||||
|
||||
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSpec.scala#router-lookup-config
|
||||
|
||||
|
|
@ -463,10 +463,11 @@ The same type of router could also have been defined in code:
|
|||
|
||||
See :ref:`cluster_configuration_scala` section for further descriptions of the settings.
|
||||
|
||||
Router Example with Lookup of Routees
|
||||
-------------------------------------
|
||||
Router Example with Group of Routees
|
||||
------------------------------------
|
||||
|
||||
Let's take a look at how to use a cluster aware router with lookup of routees.
|
||||
Let's take a look at how to use a cluster aware router with a group of routees,
|
||||
i.e. router sending to the paths of the routees.
|
||||
|
||||
The example application provides a service to calculate statistics for a text.
|
||||
When some text is sent to the service it splits it into words, and delegates the task
|
||||
|
|
@ -518,10 +519,10 @@ service nodes and 1 client::
|
|||
|
||||
run-main sample.cluster.stats.StatsSample
|
||||
|
||||
Router with Remote Deployed Routees
|
||||
-----------------------------------
|
||||
Router with Pool of Remote Deployed Routees
|
||||
-------------------------------------------
|
||||
|
||||
When using a router with routees created and deployed on the cluster member nodes
|
||||
When using a ``Pool`` with routees created and deployed on the cluster member nodes
|
||||
the configuration for a router looks like this:
|
||||
|
||||
.. includecode:: ../../../akka-samples/akka-sample-cluster/src/multi-jvm/scala/sample/cluster/stats/StatsSampleSingleMasterSpec.scala#router-deploy-config
|
||||
|
|
@ -540,8 +541,8 @@ The same type of router could also have been defined in code:
|
|||
|
||||
See :ref:`cluster_configuration_scala` section for further descriptions of the settings.
|
||||
|
||||
Router Example with Remote Deployed Routees
|
||||
-------------------------------------------
|
||||
Router Example with Pool of Remote Deployed Routees
|
||||
---------------------------------------------------
|
||||
|
||||
Let's take a look at how to use a cluster aware router on single master node that creates
|
||||
and deploys workers. To keep track of a single master we use the :ref:`cluster-singleton`
|
||||
|
|
@ -598,7 +599,7 @@ Download the native Sigar libraries from `Maven Central <http://repo1.maven.org/
|
|||
Adaptive Load Balancing
|
||||
-----------------------
|
||||
|
||||
The ``AdaptiveLoadBalancingRouter`` performs load balancing of messages to cluster nodes based on the cluster metrics data.
|
||||
The ``AdaptiveLoadBalancingPool`` / ``AdaptiveLoadBalancingGroup`` performs load balancing of messages to cluster nodes based on the cluster metrics data.
|
||||
It uses random selection of routees with probabilities derived from the remaining capacity of the corresponding node.
|
||||
It can be configured to use a specific MetricsSelector to produce the probabilities, a.k.a. weights:
|
||||
|
||||
|
|
|
|||
|
|
@ -5,6 +5,8 @@ package docs.routing
|
|||
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.testkit.ImplicitSender
|
||||
import akka.routing.FromConfig
|
||||
import akka.actor.ActorRef
|
||||
|
||||
object ConsistentHashingRouterDocSpec {
|
||||
|
||||
|
|
@ -39,9 +41,11 @@ class ConsistentHashingRouterDocSpec extends AkkaSpec with ImplicitSender {
|
|||
|
||||
"demonstrate usage of ConsistentHashableRouter" in {
|
||||
|
||||
def context = system
|
||||
|
||||
//#consistent-hashing-router
|
||||
import akka.actor.Props
|
||||
import akka.routing.ConsistentHashingRouter
|
||||
import akka.routing.ConsistentHashingPool
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashMapping
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope
|
||||
|
||||
|
|
@ -49,8 +53,9 @@ class ConsistentHashingRouterDocSpec extends AkkaSpec with ImplicitSender {
|
|||
case Evict(key) ⇒ key
|
||||
}
|
||||
|
||||
val cache = system.actorOf(Props[Cache].withRouter(ConsistentHashingRouter(10,
|
||||
hashMapping = hashMapping)), name = "cache")
|
||||
val cache: ActorRef =
|
||||
context.actorOf(ConsistentHashingPool(10, hashMapping = hashMapping).
|
||||
props(Props[Cache]), name = "cache")
|
||||
|
||||
cache ! ConsistentHashableEnvelope(
|
||||
message = Entry("hello", "HELLO"), hashKey = "hello")
|
||||
|
|
@ -68,6 +73,7 @@ class ConsistentHashingRouterDocSpec extends AkkaSpec with ImplicitSender {
|
|||
expectMsg(None)
|
||||
|
||||
//#consistent-hashing-router
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
142
akka-docs/rst/scala/code/docs/routing/CustomRouterDocSpec.scala
Normal file
142
akka-docs/rst/scala/code/docs/routing/CustomRouterDocSpec.scala
Normal file
|
|
@ -0,0 +1,142 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.routing
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.testkit.ImplicitSender
|
||||
import akka.actor.Actor
|
||||
import akka.actor.Props
|
||||
import CustomRouterDocSpec.RedundancyRoutingLogic
|
||||
import scala.collection.immutable
|
||||
import akka.actor.ActorSystem
|
||||
import akka.routing.FromConfig
|
||||
import akka.actor.ActorRef
|
||||
|
||||
object CustomRouterDocSpec {
|
||||
|
||||
val config = """
|
||||
#//#config
|
||||
akka.actor.deployment {
|
||||
/redundancy2 {
|
||||
router = "docs.routing.RedundancyGroup"
|
||||
routees.paths = ["/user/s1", "/user/s2", "/user/s3"]
|
||||
nbr-copies = 5
|
||||
}
|
||||
}
|
||||
#//#config
|
||||
"""
|
||||
|
||||
val jconfig = """
|
||||
#//#jconfig
|
||||
akka.actor.deployment {
|
||||
/redundancy2 {
|
||||
router = "docs.jrouting.RedundancyGroup"
|
||||
routees.paths = ["/user/s1", "/user/s2", "/user/s3"]
|
||||
nbr-copies = 5
|
||||
}
|
||||
}
|
||||
#//#jconfig
|
||||
"""
|
||||
|
||||
//#routing-logic
|
||||
import scala.collection.immutable
|
||||
import scala.concurrent.forkjoin.ThreadLocalRandom
|
||||
import akka.routing.RoundRobinRoutingLogic
|
||||
import akka.routing.RoutingLogic
|
||||
import akka.routing.Routee
|
||||
import akka.routing.SeveralRoutees
|
||||
|
||||
class RedundancyRoutingLogic(nbrCopies: Int) extends RoutingLogic {
|
||||
val roundRobin = RoundRobinRoutingLogic()
|
||||
def select(message: Any, routees: immutable.IndexedSeq[Routee]): Routee = {
|
||||
val targets = (1 to nbrCopies).map(_ ⇒ roundRobin.select(message, routees))
|
||||
SeveralRoutees(targets)
|
||||
}
|
||||
}
|
||||
//#routing-logic
|
||||
|
||||
class Storage extends Actor {
|
||||
def receive = {
|
||||
case x ⇒ sender ! x
|
||||
}
|
||||
}
|
||||
|
||||
//#unit-test-logic
|
||||
case class TestRoutee(n: Int) extends Routee {
|
||||
override def send(message: Any, sender: ActorRef): Unit = ()
|
||||
}
|
||||
|
||||
//#unit-test-logic
|
||||
}
|
||||
|
||||
//#group
|
||||
import akka.dispatch.Dispatchers
|
||||
import akka.routing.Group
|
||||
import akka.routing.Router
|
||||
import akka.japi.Util.immutableSeq
|
||||
import com.typesafe.config.Config
|
||||
|
||||
case class RedundancyGroup(override val paths: immutable.Iterable[String], nbrCopies: Int) extends Group {
|
||||
|
||||
def this(config: Config) = this(
|
||||
paths = immutableSeq(config.getStringList("routees.paths")),
|
||||
nbrCopies = config.getInt("nbr-copies"))
|
||||
|
||||
override def createRouter(system: ActorSystem): Router =
|
||||
new Router(new RedundancyRoutingLogic(nbrCopies))
|
||||
|
||||
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId
|
||||
}
|
||||
//#group
|
||||
|
||||
class CustomRouterDocSpec extends AkkaSpec(CustomRouterDocSpec.config) with ImplicitSender {
|
||||
|
||||
import CustomRouterDocSpec._
|
||||
import akka.routing.SeveralRoutees
|
||||
|
||||
"unit test routing logic" in {
|
||||
//#unit-test-logic
|
||||
val logic = new RedundancyRoutingLogic(nbrCopies = 3)
|
||||
|
||||
val routees = for (n ← 1 to 7) yield TestRoutee(n)
|
||||
|
||||
val r1 = logic.select("msg", routees)
|
||||
r1.asInstanceOf[SeveralRoutees].routees must be(
|
||||
Vector(TestRoutee(1), TestRoutee(2), TestRoutee(3)))
|
||||
|
||||
val r2 = logic.select("msg", routees)
|
||||
r2.asInstanceOf[SeveralRoutees].routees must be(
|
||||
Vector(TestRoutee(4), TestRoutee(5), TestRoutee(6)))
|
||||
|
||||
val r3 = logic.select("msg", routees)
|
||||
r3.asInstanceOf[SeveralRoutees].routees must be(
|
||||
Vector(TestRoutee(7), TestRoutee(1), TestRoutee(2)))
|
||||
//#unit-test-logic
|
||||
|
||||
}
|
||||
|
||||
"demonstrate usage of custom router" in {
|
||||
//#usage-1
|
||||
for (n ← 1 to 10) system.actorOf(Props[Storage], "s" + n)
|
||||
|
||||
val paths = for (n ← 1 to 10) yield ("/user/s" + n)
|
||||
val redundancy1: ActorRef =
|
||||
system.actorOf(RedundancyGroup(paths, nbrCopies = 3).props(),
|
||||
name = "redundancy1")
|
||||
redundancy1 ! "important"
|
||||
//#usage-1
|
||||
|
||||
for (_ ← 1 to 3) expectMsg("important")
|
||||
|
||||
//#usage-2
|
||||
val redundancy2: ActorRef = system.actorOf(FromConfig.props(),
|
||||
name = "redundancy2")
|
||||
redundancy2 ! "very important"
|
||||
//#usage-2
|
||||
|
||||
for (_ ← 1 to 5) expectMsg("very important")
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -3,32 +3,417 @@
|
|||
*/
|
||||
package docs.routing
|
||||
|
||||
import RouterDocSpec.MyActor
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.routing.RoundRobinRouter
|
||||
import scala.concurrent.duration._
|
||||
import akka.testkit._
|
||||
import akka.actor.{ ActorRef, Props, Actor }
|
||||
import akka.actor.Terminated
|
||||
import akka.routing.FromConfig
|
||||
import akka.routing.RoundRobinPool
|
||||
import akka.routing.RandomPool
|
||||
import akka.routing.RoundRobinGroup
|
||||
import akka.routing.SmallestMailboxPool
|
||||
import akka.routing.BroadcastPool
|
||||
import akka.routing.BroadcastGroup
|
||||
import akka.routing.ConsistentHashingGroup
|
||||
import akka.routing.ConsistentHashingPool
|
||||
import akka.routing.DefaultResizer
|
||||
import akka.routing.ScatterGatherFirstCompletedGroup
|
||||
import akka.routing.RandomGroup
|
||||
import akka.routing.ScatterGatherFirstCompletedPool
|
||||
|
||||
object RouterDocSpec {
|
||||
class MyActor extends Actor {
|
||||
|
||||
val config = """
|
||||
#//#config-round-robin-pool
|
||||
akka.actor.deployment {
|
||||
/parent/router1 {
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
#//#config-round-robin-pool
|
||||
|
||||
#//#config-round-robin-group
|
||||
akka.actor.deployment {
|
||||
/parent/router3 {
|
||||
router = round-robin-group
|
||||
routees.paths = ["/user/workers/w1", "/user/workers/w2", "/user/workers/w3"]
|
||||
}
|
||||
}
|
||||
#//#config-round-robin-group
|
||||
|
||||
#//#config-random-pool
|
||||
akka.actor.deployment {
|
||||
/parent/router5 {
|
||||
router = random-pool
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
#//#config-random-pool
|
||||
|
||||
#//#config-random-group
|
||||
akka.actor.deployment {
|
||||
/parent/router7 {
|
||||
router = random-group
|
||||
routees.paths = ["/user/workers/w1", "/user/workers/w2", "/user/workers/w3"]
|
||||
}
|
||||
}
|
||||
#//#config-random-group
|
||||
|
||||
#//#config-smallest-mailbox-pool
|
||||
akka.actor.deployment {
|
||||
/parent/router9 {
|
||||
router = smallest-mailbox-pool
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
#//#config-smallest-mailbox-pool
|
||||
|
||||
#//#config-broadcast-pool
|
||||
akka.actor.deployment {
|
||||
/parent/router11 {
|
||||
router = broadcast-pool
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
#//#config-broadcast-pool
|
||||
|
||||
#//#config-broadcast-group
|
||||
akka.actor.deployment {
|
||||
/parent/router13 {
|
||||
router = broadcast-group
|
||||
routees.paths = ["/user/workers/w1", "/user/workers/w2", "/user/workers/w3"]
|
||||
}
|
||||
}
|
||||
#//#config-broadcast-group
|
||||
|
||||
#//#config-scatter-gather-pool
|
||||
akka.actor.deployment {
|
||||
/parent/router15 {
|
||||
router = scatter-gather-pool
|
||||
nr-of-instances = 5
|
||||
within = 10 seconds
|
||||
}
|
||||
}
|
||||
#//#config-scatter-gather-pool
|
||||
|
||||
#//#config-scatter-gather-group
|
||||
akka.actor.deployment {
|
||||
/parent/router17 {
|
||||
router = scatter-gather-group
|
||||
routees.paths = ["/user/workers/w1", "/user/workers/w2", "/user/workers/w3"]
|
||||
within = 10 seconds
|
||||
}
|
||||
}
|
||||
#//#config-scatter-gather-group
|
||||
|
||||
#//#config-consistent-hashing-pool
|
||||
akka.actor.deployment {
|
||||
/parent/router19 {
|
||||
router = consistent-hashing-pool
|
||||
nr-of-instances = 5
|
||||
virtual-nodes-factor = 10
|
||||
}
|
||||
}
|
||||
#//#config-consistent-hashing-pool
|
||||
|
||||
#//#config-consistent-hashing-group
|
||||
akka.actor.deployment {
|
||||
/parent/router21 {
|
||||
router = consistent-hashing-group
|
||||
routees.paths = ["/user/workers/w1", "/user/workers/w2", "/user/workers/w3"]
|
||||
virtual-nodes-factor = 10
|
||||
}
|
||||
}
|
||||
#//#config-consistent-hashing-group
|
||||
|
||||
#//#config-resize-pool
|
||||
akka.actor.deployment {
|
||||
/parent/router23 {
|
||||
router = round-robin-pool
|
||||
resizer {
|
||||
lower-bound = 2
|
||||
upper-bound = 15
|
||||
messages-per-resize = 100
|
||||
}
|
||||
}
|
||||
}
|
||||
#//#config-resize-pool
|
||||
|
||||
router-dispatcher {}
|
||||
workers-dispatcher {}
|
||||
"""
|
||||
|
||||
case class Work(payload: String)
|
||||
|
||||
//#router-in-actor
|
||||
import akka.routing.ActorRefRoutee
|
||||
import akka.routing.Router
|
||||
import akka.routing.RoundRobinRoutingLogic
|
||||
|
||||
class Master extends Actor {
|
||||
var router = {
|
||||
val routees = Vector.fill(5) {
|
||||
val r = context.actorOf(Props[Worker])
|
||||
context watch r
|
||||
ActorRefRoutee(r)
|
||||
}
|
||||
Router(RoundRobinRoutingLogic(), routees)
|
||||
}
|
||||
|
||||
def receive = {
|
||||
case w: Work ⇒
|
||||
router.route(w, sender)
|
||||
case Terminated(a) ⇒
|
||||
router = router.removeRoutee(a)
|
||||
val r = context.actorOf(Props[Worker])
|
||||
context watch r
|
||||
router = router.addRoutee(r)
|
||||
}
|
||||
}
|
||||
//#router-in-actor
|
||||
|
||||
class Worker extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒
|
||||
}
|
||||
}
|
||||
|
||||
//#create-worker-actors
|
||||
class Workers extends Actor {
|
||||
context.actorOf(Props[Worker], name = "w1")
|
||||
context.actorOf(Props[Worker], name = "w2")
|
||||
context.actorOf(Props[Worker], name = "w3")
|
||||
// ...
|
||||
//#create-worker-actors
|
||||
|
||||
def receive = {
|
||||
case _ ⇒
|
||||
}
|
||||
}
|
||||
|
||||
class Parent extends Actor {
|
||||
|
||||
//#paths
|
||||
val paths = List("/user/workers/w1", "/user/workers/w2", "/user/workers/w3")
|
||||
//#paths
|
||||
|
||||
//#round-robin-pool-1
|
||||
val router1: ActorRef =
|
||||
context.actorOf(FromConfig.props(Props[Worker]), "router1")
|
||||
//#round-robin-pool-1
|
||||
|
||||
//#round-robin-pool-2
|
||||
val router2: ActorRef =
|
||||
context.actorOf(RoundRobinPool(5).props(Props[Worker]), "router2")
|
||||
//#round-robin-pool-2
|
||||
|
||||
//#round-robin-group-1
|
||||
val router3: ActorRef =
|
||||
context.actorOf(FromConfig.props(), "router3")
|
||||
//#round-robin-group-1
|
||||
|
||||
//#round-robin-group-2
|
||||
val router4: ActorRef =
|
||||
context.actorOf(RoundRobinGroup(paths).props(), "router4")
|
||||
//#round-robin-group-2
|
||||
|
||||
//#random-pool-1
|
||||
val router5: ActorRef =
|
||||
context.actorOf(FromConfig.props(Props[Worker]), "router5")
|
||||
//#random-pool-1
|
||||
|
||||
//#random-pool-2
|
||||
val router6: ActorRef =
|
||||
context.actorOf(RandomPool(5).props(Props[Worker]), "router6")
|
||||
//#random-pool-2
|
||||
|
||||
//#random-group-1
|
||||
val router7: ActorRef =
|
||||
context.actorOf(FromConfig.props(), "router7")
|
||||
//#random-group-1
|
||||
|
||||
//#random-group-2
|
||||
val router8: ActorRef =
|
||||
context.actorOf(RandomGroup(paths).props(), "router8")
|
||||
//#random-group-2
|
||||
|
||||
//#smallest-mailbox-pool-1
|
||||
val router9: ActorRef =
|
||||
context.actorOf(FromConfig.props(Props[Worker]), "router9")
|
||||
//#smallest-mailbox-pool-1
|
||||
|
||||
//#smallest-mailbox-pool-2
|
||||
val router10: ActorRef =
|
||||
context.actorOf(SmallestMailboxPool(5).props(Props[Worker]), "router10")
|
||||
//#smallest-mailbox-pool-2
|
||||
|
||||
//#broadcast-pool-1
|
||||
val router11: ActorRef =
|
||||
context.actorOf(FromConfig.props(Props[Worker]), "router11")
|
||||
//#broadcast-pool-1
|
||||
|
||||
//#broadcast-pool-2
|
||||
val router12: ActorRef =
|
||||
context.actorOf(BroadcastPool(5).props(Props[Worker]), "router12")
|
||||
//#broadcast-pool-2
|
||||
|
||||
//#broadcast-group-1
|
||||
val router13: ActorRef =
|
||||
context.actorOf(FromConfig.props(), "router13")
|
||||
//#broadcast-group-1
|
||||
|
||||
//#broadcast-group-2
|
||||
val router14: ActorRef =
|
||||
context.actorOf(BroadcastGroup(paths).props(), "router14")
|
||||
//#broadcast-group-2
|
||||
|
||||
//#scatter-gather-pool-1
|
||||
val router15: ActorRef =
|
||||
context.actorOf(FromConfig.props(Props[Worker]), "router15")
|
||||
//#scatter-gather-pool-1
|
||||
|
||||
//#scatter-gather-pool-2
|
||||
val router16: ActorRef =
|
||||
context.actorOf(ScatterGatherFirstCompletedPool(5, within = 10.seconds).
|
||||
props(Props[Worker]), "router16")
|
||||
//#scatter-gather-pool-2
|
||||
|
||||
//#scatter-gather-group-1
|
||||
val router17: ActorRef =
|
||||
context.actorOf(FromConfig.props(), "router17")
|
||||
//#scatter-gather-group-1
|
||||
|
||||
//#scatter-gather-group-2
|
||||
val router18: ActorRef =
|
||||
context.actorOf(ScatterGatherFirstCompletedGroup(paths,
|
||||
within = 10.seconds).props(), "router18")
|
||||
//#scatter-gather-group-2
|
||||
|
||||
//#consistent-hashing-pool-1
|
||||
val router19: ActorRef =
|
||||
context.actorOf(FromConfig.props(Props[Worker]), "router19")
|
||||
//#consistent-hashing-pool-1
|
||||
|
||||
//#consistent-hashing-pool-2
|
||||
val router20: ActorRef =
|
||||
context.actorOf(ConsistentHashingPool(5).props(Props[Worker]),
|
||||
"router20")
|
||||
//#consistent-hashing-pool-2
|
||||
|
||||
//#consistent-hashing-group-1
|
||||
val router21: ActorRef =
|
||||
context.actorOf(FromConfig.props(), "router21")
|
||||
//#consistent-hashing-group-1
|
||||
|
||||
//#consistent-hashing-group-2
|
||||
val router22: ActorRef =
|
||||
context.actorOf(ConsistentHashingGroup(paths).props(), "router22")
|
||||
//#consistent-hashing-group-2
|
||||
|
||||
//#resize-pool-1
|
||||
val router23: ActorRef =
|
||||
context.actorOf(FromConfig.props(Props[Worker]), "router23")
|
||||
//#resize-pool-1
|
||||
|
||||
//#resize-pool-2
|
||||
val resizer = DefaultResizer(lowerBound = 2, upperBound = 15)
|
||||
val router24: ActorRef =
|
||||
context.actorOf(RoundRobinPool(5, Some(resizer)).props(Props[Worker]),
|
||||
"router24")
|
||||
//#resize-pool-2
|
||||
|
||||
def receive = {
|
||||
case _ ⇒
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case m ⇒ sender ! m
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RouterDocSpec extends AkkaSpec("""
|
||||
router {}
|
||||
workers {}
|
||||
""") {
|
||||
class RouterDocSpec extends AkkaSpec(RouterDocSpec.config) with ImplicitSender {
|
||||
|
||||
import RouterDocSpec._
|
||||
|
||||
//#dispatchers
|
||||
val router: ActorRef = system.actorOf(Props[MyActor]
|
||||
// “head” will run on "router" dispatcher
|
||||
.withRouter(RoundRobinRouter(5, routerDispatcher = "router"))
|
||||
// MyActor workers will run on "workers" dispatcher
|
||||
.withDispatcher("workers"))
|
||||
//#dispatchers
|
||||
//#create-workers
|
||||
system.actorOf(Props[Workers], "workers")
|
||||
//#create-workers
|
||||
|
||||
//#create-parent
|
||||
system.actorOf(Props[Parent], "parent")
|
||||
//#create-parent
|
||||
|
||||
"demonstrate dispatcher" in {
|
||||
//#dispatchers
|
||||
val router: ActorRef = system.actorOf(
|
||||
// “head” will run on "router-dispatcher" dispatcher
|
||||
RoundRobinPool(5, routerDispatcher = "router-dispatcher").props(Props[Worker])
|
||||
// Worker routees will run on "workers-dispatcher" dispatcher
|
||||
.withDispatcher("workers-dispatcher"))
|
||||
//#dispatchers
|
||||
}
|
||||
|
||||
"demonstrate broadcast" in {
|
||||
val router = system.actorOf(RoundRobinPool(nrOfInstances = 5).props(Props[Echo]))
|
||||
//#broadcastDavyJonesWarning
|
||||
import akka.routing.Broadcast
|
||||
router ! Broadcast("Watch out for Davy Jones' locker")
|
||||
//#broadcastDavyJonesWarning
|
||||
receiveN(5, 5.seconds.dilated) must have length (5)
|
||||
}
|
||||
|
||||
"demonstrate PoisonPill" in {
|
||||
val router = watch(system.actorOf(RoundRobinPool(nrOfInstances = 5).props(Props[Echo])))
|
||||
//#poisonPill
|
||||
import akka.actor.PoisonPill
|
||||
router ! PoisonPill
|
||||
//#poisonPill
|
||||
expectTerminated(router)
|
||||
}
|
||||
|
||||
"demonstrate broadcast of PoisonPill" in {
|
||||
val router = watch(system.actorOf(RoundRobinPool(nrOfInstances = 5).props(Props[Echo])))
|
||||
//#broadcastPoisonPill
|
||||
import akka.actor.PoisonPill
|
||||
import akka.routing.Broadcast
|
||||
router ! Broadcast(PoisonPill)
|
||||
//#broadcastPoisonPill
|
||||
expectTerminated(router)
|
||||
}
|
||||
|
||||
"demonstrate Kill" in {
|
||||
val router = watch(system.actorOf(RoundRobinPool(nrOfInstances = 5).props(Props[Echo])))
|
||||
//#kill
|
||||
import akka.actor.Kill
|
||||
router ! Kill
|
||||
//#kill
|
||||
expectTerminated(router)
|
||||
}
|
||||
|
||||
"demonstrate broadcast of Kill" in {
|
||||
val router = watch(system.actorOf(RoundRobinPool(nrOfInstances = 5).props(Props[Echo])))
|
||||
//#broadcastKill
|
||||
import akka.actor.Kill
|
||||
import akka.routing.Broadcast
|
||||
router ! Broadcast(Kill)
|
||||
//#broadcastKill
|
||||
expectTerminated(router)
|
||||
}
|
||||
|
||||
"demonstrate remote deploy" in {
|
||||
//#remoteRoutees
|
||||
import akka.actor.{ Address, AddressFromURIString }
|
||||
import akka.remote.routing.RemoteRouterConfig
|
||||
val addresses = Seq(
|
||||
Address("akka", "remotesys", "otherhost", 1234),
|
||||
AddressFromURIString("akka://othersys@anotherhost:1234"))
|
||||
val routerRemote = system.actorOf(
|
||||
RemoteRouterConfig(RoundRobinPool(5), addresses).props(Props[Echo]))
|
||||
//#remoteRoutees
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -1,94 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.routing
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import akka.routing.{ ScatterGatherFirstCompletedRouter, BroadcastRouter, RandomRouter, RoundRobinRouter }
|
||||
import annotation.tailrec
|
||||
import akka.actor.{ Props, Actor }
|
||||
import scala.concurrent.duration._
|
||||
import akka.util.Timeout
|
||||
import scala.concurrent.Await
|
||||
import akka.pattern.ask
|
||||
import akka.routing.SmallestMailboxRouter
|
||||
|
||||
case class FibonacciNumber(nbr: Int)
|
||||
|
||||
//#printlnActor
|
||||
class PrintlnActor extends Actor {
|
||||
def receive = {
|
||||
case msg ⇒
|
||||
println("Received message '%s' in actor %s".format(msg, self.path.name))
|
||||
}
|
||||
}
|
||||
|
||||
//#printlnActor
|
||||
|
||||
//#fibonacciActor
|
||||
class FibonacciActor extends Actor {
|
||||
def receive = {
|
||||
case FibonacciNumber(nbr) ⇒ sender ! fibonacci(nbr)
|
||||
}
|
||||
|
||||
private def fibonacci(n: Int): Int = {
|
||||
@tailrec
|
||||
def fib(n: Int, b: Int, a: Int): Int = n match {
|
||||
case 0 ⇒ a
|
||||
case _ ⇒ fib(n - 1, a + b, b)
|
||||
}
|
||||
|
||||
fib(n, 1, 0)
|
||||
}
|
||||
}
|
||||
|
||||
//#fibonacciActor
|
||||
|
||||
//#parentActor
|
||||
class ParentActor extends Actor {
|
||||
def receive = {
|
||||
case "rrr" ⇒
|
||||
//#roundRobinRouter
|
||||
val roundRobinRouter =
|
||||
context.actorOf(Props[PrintlnActor].withRouter(RoundRobinRouter(5)), "router")
|
||||
1 to 10 foreach {
|
||||
i ⇒ roundRobinRouter ! i
|
||||
}
|
||||
//#roundRobinRouter
|
||||
case "rr" ⇒
|
||||
//#randomRouter
|
||||
val randomRouter =
|
||||
context.actorOf(Props[PrintlnActor].withRouter(RandomRouter(5)), "router")
|
||||
1 to 10 foreach {
|
||||
i ⇒ randomRouter ! i
|
||||
}
|
||||
//#randomRouter
|
||||
case "smr" ⇒
|
||||
//#smallestMailboxRouter
|
||||
val smallestMailboxRouter = context.actorOf(Props[PrintlnActor].
|
||||
withRouter(SmallestMailboxRouter(5)), "router")
|
||||
1 to 10 foreach {
|
||||
i ⇒ smallestMailboxRouter ! i
|
||||
}
|
||||
//#smallestMailboxRouter
|
||||
case "br" ⇒
|
||||
//#broadcastRouter
|
||||
val broadcastRouter =
|
||||
context.actorOf(Props[PrintlnActor].withRouter(BroadcastRouter(5)), "router")
|
||||
broadcastRouter ! "this is a broadcast message"
|
||||
//#broadcastRouter
|
||||
case "sgfcr" ⇒
|
||||
//#scatterGatherFirstCompletedRouter
|
||||
val scatterGatherFirstCompletedRouter = context.actorOf(
|
||||
Props[FibonacciActor].withRouter(ScatterGatherFirstCompletedRouter(
|
||||
nrOfInstances = 5, within = 2 seconds)), "router")
|
||||
implicit val timeout = Timeout(5 seconds)
|
||||
val futureResult = scatterGatherFirstCompletedRouter ? FibonacciNumber(10)
|
||||
val result = Await.result(futureResult, timeout.duration)
|
||||
//#scatterGatherFirstCompletedRouter
|
||||
println("The result of calculating Fibonacci for 10 is %d".format(result))
|
||||
}
|
||||
}
|
||||
|
||||
//#parentActor
|
||||
|
|
@ -1,158 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.routing
|
||||
|
||||
import akka.actor.{ Actor, Props, ActorSystem, ActorLogging }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.routing.FromConfig
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashable
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.testkit.ImplicitSender
|
||||
|
||||
object RouterWithConfigDocSpec {
|
||||
|
||||
val config = ConfigFactory.parseString("""
|
||||
|
||||
//#config-round-robin
|
||||
akka.actor.deployment {
|
||||
/myrouter1 {
|
||||
router = round-robin
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
//#config-round-robin
|
||||
|
||||
//#config-resize
|
||||
akka.actor.deployment {
|
||||
/myrouter2 {
|
||||
router = round-robin
|
||||
resizer {
|
||||
lower-bound = 2
|
||||
upper-bound = 15
|
||||
}
|
||||
}
|
||||
}
|
||||
//#config-resize
|
||||
|
||||
//#config-random
|
||||
akka.actor.deployment {
|
||||
/myrouter3 {
|
||||
router = random
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
//#config-random
|
||||
|
||||
//#config-smallest-mailbox
|
||||
akka.actor.deployment {
|
||||
/myrouter4 {
|
||||
router = smallest-mailbox
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
//#config-smallest-mailbox
|
||||
|
||||
//#config-broadcast
|
||||
akka.actor.deployment {
|
||||
/myrouter5 {
|
||||
router = broadcast
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
//#config-broadcast
|
||||
|
||||
//#config-scatter-gather
|
||||
akka.actor.deployment {
|
||||
/myrouter6 {
|
||||
router = scatter-gather
|
||||
nr-of-instances = 5
|
||||
within = 10 seconds
|
||||
}
|
||||
}
|
||||
//#config-scatter-gather
|
||||
|
||||
//#config-consistent-hashing
|
||||
akka.actor.deployment {
|
||||
/myrouter7 {
|
||||
router = consistent-hashing
|
||||
nr-of-instances = 5
|
||||
virtual-nodes-factor = 10
|
||||
}
|
||||
}
|
||||
//#config-consistent-hashing
|
||||
|
||||
""")
|
||||
|
||||
case class Message(nbr: Int) extends ConsistentHashable {
|
||||
override def consistentHashKey = nbr
|
||||
}
|
||||
|
||||
class ExampleActor extends Actor with ActorLogging {
|
||||
def receive = {
|
||||
case Message(nbr) ⇒
|
||||
log.debug("Received %s in router %s".format(nbr, self.path.name))
|
||||
sender ! nbr
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
class RouterWithConfigDocSpec extends AkkaSpec(RouterWithConfigDocSpec.config) with ImplicitSender {
|
||||
|
||||
import RouterWithConfigDocSpec._
|
||||
|
||||
"demonstrate configured round-robin router" in {
|
||||
//#configurableRouting
|
||||
val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
|
||||
"myrouter1")
|
||||
//#configurableRouting
|
||||
1 to 10 foreach { i ⇒ router ! Message(i) }
|
||||
receiveN(10)
|
||||
}
|
||||
|
||||
"demonstrate configured random router" in {
|
||||
val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
|
||||
"myrouter3")
|
||||
1 to 10 foreach { i ⇒ router ! Message(i) }
|
||||
receiveN(10)
|
||||
}
|
||||
|
||||
"demonstrate configured smallest-mailbox router" in {
|
||||
val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
|
||||
"myrouter4")
|
||||
1 to 10 foreach { i ⇒ router ! Message(i) }
|
||||
receiveN(10)
|
||||
}
|
||||
|
||||
"demonstrate configured broadcast router" in {
|
||||
val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
|
||||
"myrouter5")
|
||||
1 to 10 foreach { i ⇒ router ! Message(i) }
|
||||
receiveN(5 * 10)
|
||||
}
|
||||
|
||||
"demonstrate configured scatter-gather router" in {
|
||||
val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
|
||||
"myrouter6")
|
||||
1 to 10 foreach { i ⇒ router ! Message(i) }
|
||||
receiveN(10)
|
||||
}
|
||||
|
||||
"demonstrate configured consistent-hashing router" in {
|
||||
val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
|
||||
"myrouter7")
|
||||
1 to 10 foreach { i ⇒ router ! Message(i) }
|
||||
receiveN(10)
|
||||
}
|
||||
|
||||
"demonstrate configured round-robin router with resizer" in {
|
||||
//#configurableRoutingWithResizer
|
||||
val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
|
||||
"myrouter2")
|
||||
//#configurableRoutingWithResizer
|
||||
1 to 10 foreach { i ⇒ router ! Message(i) }
|
||||
receiveN(10)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,52 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.routing
|
||||
|
||||
import akka.actor.{ Actor, Props, ActorSystem }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.routing.FromConfig
|
||||
|
||||
case class Message(nbr: Int)
|
||||
|
||||
class ExampleActor extends Actor {
|
||||
def receive = {
|
||||
case Message(nbr) ⇒ println("Received %s in router %s".format(nbr, self.path.name))
|
||||
}
|
||||
}
|
||||
|
||||
object RouterWithConfigExample extends App {
|
||||
val config = ConfigFactory.parseString("""
|
||||
//#config
|
||||
akka.actor.deployment {
|
||||
/router {
|
||||
router = round-robin
|
||||
nr-of-instances = 5
|
||||
}
|
||||
}
|
||||
//#config
|
||||
//#config-resize
|
||||
akka.actor.deployment {
|
||||
/router2 {
|
||||
router = round-robin
|
||||
resizer {
|
||||
lower-bound = 2
|
||||
upper-bound = 15
|
||||
}
|
||||
}
|
||||
}
|
||||
//#config-resize
|
||||
""")
|
||||
val system = ActorSystem("Example", config)
|
||||
//#configurableRouting
|
||||
val router = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
|
||||
"router")
|
||||
//#configurableRouting
|
||||
1 to 10 foreach { i ⇒ router ! Message(i) }
|
||||
|
||||
//#configurableRoutingWithResizer
|
||||
val router2 = system.actorOf(Props[ExampleActor].withRouter(FromConfig()),
|
||||
"router2")
|
||||
//#configurableRoutingWithResizer
|
||||
1 to 10 foreach { i ⇒ router2 ! Message(i) }
|
||||
}
|
||||
|
|
@ -1,101 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.routing
|
||||
|
||||
import akka.actor._
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashable
|
||||
import akka.routing.FromConfig
|
||||
import akka.routing.RoundRobinRouter
|
||||
import akka.testkit._
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.ActorPath
|
||||
|
||||
object RouterViaProgramDocSpec {
|
||||
case class Message1(nbr: Int)
|
||||
case class Reply1(name: String, m: Message1)
|
||||
|
||||
class ExampleActor1 extends Actor {
|
||||
def receive = {
|
||||
case m @ Message1(nbr) ⇒ sender ! Reply1(self.path.name, m)
|
||||
}
|
||||
}
|
||||
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case m ⇒ sender ! m
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class RouterViaProgramDocSpec extends AkkaSpec with ImplicitSender {
|
||||
import RouterViaProgramDocSpec._
|
||||
|
||||
"demonstrate routees from paths" in {
|
||||
|
||||
//#programmaticRoutingRouteePaths
|
||||
val actor1 = system.actorOf(Props[ExampleActor1], "actor1")
|
||||
val actor2 = system.actorOf(Props[ExampleActor1], "actor2")
|
||||
val actor3 = system.actorOf(Props[ExampleActor1], "actor3")
|
||||
val routees = Vector[String]("/user/actor1", "/user/actor2", "/user/actor3")
|
||||
val router = system.actorOf(
|
||||
Props.empty.withRouter(RoundRobinRouter(routees = routees)))
|
||||
//#programmaticRoutingRouteePaths
|
||||
1 to 6 foreach { i ⇒ router ! Message1(i) }
|
||||
val received = receiveN(6, 5.seconds.dilated)
|
||||
1 to 6 foreach { i ⇒
|
||||
val expectedName = (routees((i - 1) % routees.length)).split("/").last
|
||||
val expectedMsg = Message1(i)
|
||||
received must contain[AnyRef](Reply1(expectedName, expectedMsg))
|
||||
}
|
||||
}
|
||||
|
||||
"demonstrate broadcast" in {
|
||||
val router = system.actorOf(Props[Echo].withRouter(RoundRobinRouter(nrOfInstances = 5)))
|
||||
//#broadcastDavyJonesWarning
|
||||
import akka.routing.Broadcast
|
||||
router ! Broadcast("Watch out for Davy Jones' locker")
|
||||
//#broadcastDavyJonesWarning
|
||||
receiveN(5, 5.seconds.dilated) must have length (5)
|
||||
}
|
||||
|
||||
"demonstrate PoisonPill" in {
|
||||
val router = watch(system.actorOf(Props[Echo].withRouter(RoundRobinRouter(nrOfInstances = 5))))
|
||||
//#poisonPill
|
||||
import akka.actor.PoisonPill
|
||||
router ! PoisonPill
|
||||
//#poisonPill
|
||||
expectMsgPF() { case Terminated(`router`) ⇒ () }
|
||||
}
|
||||
|
||||
"demonstrate broadcast of PoisonPill" in {
|
||||
val router = watch(system.actorOf(Props[Echo].withRouter(RoundRobinRouter(nrOfInstances = 5))))
|
||||
//#broadcastPoisonPill
|
||||
import akka.actor.PoisonPill
|
||||
import akka.routing.Broadcast
|
||||
router ! Broadcast(PoisonPill)
|
||||
//#broadcastPoisonPill
|
||||
expectMsgPF() { case Terminated(`router`) ⇒ () }
|
||||
}
|
||||
|
||||
"demonstrate Kill" in {
|
||||
val router = watch(system.actorOf(Props[Echo].withRouter(RoundRobinRouter(nrOfInstances = 5))))
|
||||
//#kill
|
||||
import akka.actor.Kill
|
||||
router ! Kill
|
||||
//#kill
|
||||
expectMsgPF() { case Terminated(`router`) ⇒ () }
|
||||
}
|
||||
|
||||
"demonstrate broadcast of Kill" in {
|
||||
val router = watch(system.actorOf(Props[Echo].withRouter(RoundRobinRouter(nrOfInstances = 5))))
|
||||
//#broadcastKill
|
||||
import akka.actor.Kill
|
||||
import akka.routing.Broadcast
|
||||
router ! Broadcast(Kill)
|
||||
//#broadcastKill
|
||||
expectMsgPF() { case Terminated(`router`) ⇒ () }
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,53 +0,0 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package docs.routing
|
||||
|
||||
import akka.routing.RoundRobinRouter
|
||||
import akka.actor.{ ActorRef, Props, Actor, ActorSystem }
|
||||
import akka.routing.DefaultResizer
|
||||
import akka.remote.routing.RemoteRouterConfig
|
||||
|
||||
case class Message1(nbr: Int)
|
||||
|
||||
class ExampleActor1 extends Actor {
|
||||
def receive = {
|
||||
case Message1(nbr) ⇒ println("Received %s in router %s".format(nbr, self.path.name))
|
||||
}
|
||||
}
|
||||
|
||||
object RoutingProgrammaticallyExample extends App {
|
||||
val system = ActorSystem("RPE")
|
||||
//#programmaticRoutingNrOfInstances
|
||||
val router1 = system.actorOf(Props[ExampleActor1].withRouter(
|
||||
RoundRobinRouter(nrOfInstances = 5)))
|
||||
//#programmaticRoutingNrOfInstances
|
||||
1 to 6 foreach { i ⇒ router1 ! Message1(i) }
|
||||
|
||||
//#programmaticRoutingRoutees
|
||||
val actor1 = system.actorOf(Props[ExampleActor1])
|
||||
val actor2 = system.actorOf(Props[ExampleActor1])
|
||||
val actor3 = system.actorOf(Props[ExampleActor1])
|
||||
val routees = Vector[ActorRef](actor1, actor2, actor3)
|
||||
val router2 = system.actorOf(Props.empty.withRouter(
|
||||
RoundRobinRouter(routees = routees)))
|
||||
//#programmaticRoutingRoutees
|
||||
1 to 6 foreach { i ⇒ router2 ! Message1(i) }
|
||||
|
||||
//#programmaticRoutingWithResizer
|
||||
val resizer = DefaultResizer(lowerBound = 2, upperBound = 15)
|
||||
val router3 = system.actorOf(Props[ExampleActor1].withRouter(
|
||||
RoundRobinRouter(resizer = Some(resizer))))
|
||||
//#programmaticRoutingWithResizer
|
||||
1 to 6 foreach { i ⇒ router3 ! Message1(i) }
|
||||
|
||||
//#remoteRoutees
|
||||
import akka.actor.{ Address, AddressFromURIString }
|
||||
val addresses = Seq(
|
||||
Address("akka", "remotesys", "otherhost", 1234),
|
||||
AddressFromURIString("akka://othersys@anotherhost:1234"))
|
||||
val routerRemote = system.actorOf(Props[ExampleActor1].withRouter(
|
||||
RemoteRouterConfig(RoundRobinRouter(5), addresses)))
|
||||
//#remoteRoutees
|
||||
|
||||
}
|
||||
|
|
@ -240,7 +240,7 @@ This is also done via configuration::
|
|||
actor {
|
||||
deployment {
|
||||
/serviceA/aggregation {
|
||||
router = "round-robin"
|
||||
router = "round-robin-pool"
|
||||
nr-of-instances = 10
|
||||
target {
|
||||
nodes = ["akka.tcp://app@10.0.0.2:2552", "akka.tcp://app@10.0.0.3:2552"]
|
||||
|
|
|
|||
|
|
@ -2,102 +2,110 @@
|
|||
.. _routing-scala:
|
||||
|
||||
Routing
|
||||
===============
|
||||
=======
|
||||
|
||||
A Router is an actor that receives messages and efficiently routes them to other actors, known as
|
||||
its *routees*.
|
||||
Messages can be sent via a router to efficiently route them to destination actors, known as
|
||||
its *routees*. A ``Router`` can be used inside or outside of an actor, and you can manage the
|
||||
routees yourselves or use a self contained router actor with configuration capabilities.
|
||||
|
||||
Different routing strategies can be used, according to your application's needs. Akka comes with
|
||||
several useful routing strategies right out of the box. But, as you will see in this chapter, it is
|
||||
also possible to :ref:`create your own <custom-router-scala>`.
|
||||
|
||||
The routers shipped with Akka are:
|
||||
.. _simple-router-scala:
|
||||
|
||||
* ``akka.routing.RoundRobinRouter``
|
||||
* ``akka.routing.RandomRouter``
|
||||
* ``akka.routing.SmallestMailboxRouter``
|
||||
* ``akka.routing.BroadcastRouter``
|
||||
* ``akka.routing.ScatterGatherFirstCompletedRouter``
|
||||
* ``akka.routing.ConsistentHashingRouter``
|
||||
A Simple Router
|
||||
^^^^^^^^^^^^^^^
|
||||
|
||||
Routers in Action
|
||||
^^^^^^^^^^^^^^^^^
|
||||
The following example illustrates how to use a ``Router`` and manage the routees from within an actor.
|
||||
|
||||
Sending a message to a router is easy.
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#router-in-actor
|
||||
|
||||
.. code-block:: scala
|
||||
We create a ``Router`` and specify that it should use ``RoundRobinRoutingLogic`` when routing the
|
||||
messages to the routees.
|
||||
|
||||
router ! MyMsg
|
||||
The routing logic shipped with Akka are:
|
||||
|
||||
A router actor forwards messages to its routees according to its routing policy.
|
||||
* ``akka.routing.RoundRobinRoutingLogic``
|
||||
* ``akka.routing.RandomRoutingLogic``
|
||||
* ``akka.routing.SmallestMailboxRoutingLogic``
|
||||
* ``akka.routing.BroadcastRoutingLogic``
|
||||
* ``akka.routing.ScatterGatherFirstCompletedRoutingLogic``
|
||||
* ``akka.routing.ConsistentHashingRoutingLogic``
|
||||
|
||||
We create the routees as ordinary child actors wrapped in ``ActorRefRoutee``. We watch
|
||||
the routees to be able to replace them if they are terminated.
|
||||
|
||||
Sending messages via the router is done with the ``route`` method, as is done for the ``Work`` messages
|
||||
in the example above.
|
||||
|
||||
The ``Router`` is immutable and the ``RoutingLogic`` is thread safe; meaning that they can also be used
|
||||
outside of actors.
|
||||
|
||||
.. note::
|
||||
|
||||
In general, any message sent to a router will be sent onwards to its routees. But there are a
|
||||
In general, any message sent to a router will be sent onwards to its routees, but there is one exception.
|
||||
The special :ref:`broadcast-messages-scala` will send to *all* of a router's routees
|
||||
|
||||
A Router Actor
|
||||
^^^^^^^^^^^^^^
|
||||
|
||||
A router can also be created as a self contained actor that manages the routees itself and
|
||||
loads routing logic and other settings from configuration.
|
||||
|
||||
This type of router actor comes in two distinct flavors:
|
||||
|
||||
* Pool - The router creates routees as child actors and removes them from the router if they
|
||||
terminate.
|
||||
|
||||
* Group - The routee actors are created externally to the router and the router sends
|
||||
messages to the specified path using actor selection, without watching for termination.
|
||||
|
||||
The settings for a router actor can be defined in configuration or programmatically.
|
||||
Although router actors can be defined in the configuration file, they must still be created
|
||||
programmatically, i.e. you cannot make a router through external configuration alone.
|
||||
If you define the router actor in the configuration file then these settings will be used
|
||||
instead of any programmatically provided parameters.
|
||||
|
||||
You send messages to the routees via the router actor in the same way as for ordinary actors,
|
||||
i.e. via its ``ActorRef``. The router actor forwards messages onto its routees without changing
|
||||
the original sender. When a routee replies to a routed message, the reply will be sent to the
|
||||
original sender, not to the router actor.
|
||||
|
||||
.. note::
|
||||
|
||||
In general, any message sent to a router will be sent onwards to its routees, but there are a
|
||||
few exceptions. These are documented in the :ref:`router-special-messages-scala` section below.
|
||||
|
||||
Creating a Router
|
||||
*****************
|
||||
Pool
|
||||
----
|
||||
|
||||
Routers and routees are closely intertwined. Router actors are created by specifying the desired
|
||||
*routee* :class:`Props` then attaching the router's :class:`RouterConfig`. When you create a router
|
||||
actor it will create routees, as needed, as its children.
|
||||
|
||||
For example, the following code and configuration snippets show how to create a :ref:`round-robin
|
||||
<round-robin-router-scala>` router that forwards messages to five ``ExampleActor`` routees. The
|
||||
The following code and configuration snippets show how to create a :ref:`round-robin
|
||||
<round-robin-router-scala>` router that forwards messages to five ``Worker`` routees. The
|
||||
routees will be created as the router's children.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-round-robin
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-round-robin-pool
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#configurableRouting
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#round-robin-pool-1
|
||||
|
||||
Here is the same example, but with the router configuration provided programmatically instead of
|
||||
from configuration.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramExample.scala#programmaticRoutingNrOfInstances
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#round-robin-pool-2
|
||||
|
||||
Sometimes, rather than having the router create its routees, it is desirable to create routees
|
||||
separately and provide them to the router for its use. You can do this by passing an
|
||||
:class:`Iterable` of routees to the router's configuration.
|
||||
Remote Deployed Routees
|
||||
***********************
|
||||
|
||||
The example below shows how to create a router by providing it with the :class:`ActorRef`\s of three
|
||||
routee actors.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramExample.scala#programmaticRoutingRoutees
|
||||
|
||||
Routees can also be specified by providing their path strings instead of their :class:`ActorRef`\s.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramDocSpec.scala#programmaticRoutingRouteePaths
|
||||
|
||||
In addition to being able to supply looked-up remote actors as routees, you can ask the router to
|
||||
In addition to being able to create local actors as routees, you can instruct the router to
|
||||
deploy its created children on a set of remote hosts. Routees will be deployed in round-robin
|
||||
fashion. In order to deploy routees remotely, wrap the router configuration in a
|
||||
:class:`RemoteRouterConfig`, attaching the remote addresses of the nodes to deploy to. Remote
|
||||
``RemoteRouterConfig``, attaching the remote addresses of the nodes to deploy to. Remote
|
||||
deployment requires the ``akka-remote`` module to be included in the classpath.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramExample.scala#remoteRoutees
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#remoteRoutees
|
||||
|
||||
There are a few gotchas to be aware of when creating routers:
|
||||
|
||||
* If you define the ``router`` in the configuration file then this value will be used instead of any
|
||||
programmatically provided parameters.
|
||||
* Although routers can be configured in the configuration file, they must still be created
|
||||
programmatically, i.e. you cannot make a router through external configuration alone.
|
||||
* If you provide the ``routees`` in the router configuration then
|
||||
the value of ``nrOfInstances``, if provided, will be disregarded.
|
||||
* When you provide routees programmatically the router will generally ignore the routee
|
||||
:class:`Props`, as it does not need to create routees. However, if you use a :ref:`resizable
|
||||
router <resizable-routers-scala>` then the routee :class:`Props` will be used whenever the
|
||||
resizer creates new routees.
|
||||
|
||||
Routers, Routees and Senders
|
||||
****************************
|
||||
|
||||
The router forwards messages onto its routees without changing the original sender. When a routee
|
||||
replies to a routed message, the reply will be sent to the original sender, not to the router.
|
||||
|
||||
When a router creates routees, they are created as the routers children. This gives each routee its
|
||||
own identity in the actor system.
|
||||
Senders
|
||||
*******
|
||||
|
||||
By default, when a routee sends a message, it will :ref:`implicitly set itself as the sender
|
||||
<actors-tell-sender-scala>`.
|
||||
|
|
@ -110,18 +118,15 @@ The following code snippet shows how to set the parent router as sender.
|
|||
|
||||
.. includecode:: code/docs/actor/ActorDocSpec.scala#reply-with-sender
|
||||
|
||||
Note that different code would be needed if the routees were not children of the router, i.e. if
|
||||
they were provided when the router was created.
|
||||
|
||||
Routers and Supervision
|
||||
^^^^^^^^^^^^^^^^^^^^^^^
|
||||
Supervision
|
||||
***********
|
||||
|
||||
Routees can be created by a router or provided to the router when it is created. Any routees that
|
||||
are created by a router will be created as the router's children. The router is therefore also the
|
||||
children's supervisor.
|
||||
Routees that are created by a pool router will be created as the router's children. The router is
|
||||
therefore also the children's supervisor.
|
||||
|
||||
The supervision strategy of the router actor can be configured with the
|
||||
:meth:`RouterConfig.supervisorStrategy` property. If no configuration is provided, routers default
|
||||
``supervisorStrategy`` property of the Pool. If no configuration is provided, routers default
|
||||
to a strategy of “always escalate”. This means that errors are passed up to the router's supervisor
|
||||
for handling. The router's supervisor will decide what to do about any errors.
|
||||
|
||||
|
|
@ -142,97 +147,115 @@ by specifying the strategy when defining the router.
|
|||
Setting the strategy is easily done:
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala#supervision
|
||||
:include: supervision
|
||||
:exclude: custom-strategy
|
||||
|
||||
.. _note-router-terminated-children-scala:
|
||||
|
||||
.. note::
|
||||
|
||||
If the child of a router terminates, the router will not automatically spawn
|
||||
a new child. In the event that all children of a router have terminated the
|
||||
If the child of a pool router terminates, the pool router will not automatically spawn
|
||||
a new child. In the event that all children of a pool router have terminated the
|
||||
router will terminate itself unless it is a dynamic router, e.g. using
|
||||
a resizer.
|
||||
|
||||
Group
|
||||
-----
|
||||
|
||||
Sometimes, rather than having the router actor create its routees, it is desirable to create routees
|
||||
separately and provide them to the router for its use. You can do this by passing an
|
||||
paths of the routees to the router's configuration. Messages will be sent with ``ActorSelection``
|
||||
to these paths.
|
||||
|
||||
The example below shows how to create a router by providing it with the path strings of three
|
||||
routee actors.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-round-robin-group
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#round-robin-group-1
|
||||
|
||||
Here is the same example, but with the router configuration provided programmatically instead of
|
||||
from configuration.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#round-robin-group-2
|
||||
|
||||
The routee actors are created externally from the router:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#create-workers
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#create-worker-actors
|
||||
|
||||
The paths may contain protocol and address information for actors running on remote hosts.
|
||||
Remoting requires the ``akka-remote`` module to be included in the classpath.
|
||||
|
||||
Router usage
|
||||
^^^^^^^^^^^^
|
||||
|
||||
In this section we will describe how to use the different router types.
|
||||
First we need to create some actors that will be used in the examples:
|
||||
In this section we will describe how to create the different types of router actors.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterTypeExample.scala#printlnActor
|
||||
The router actors in this section are created from within a top level actor named ``parent``.
|
||||
Note that deployment paths in the configuration starts with ``/parent/`` followed by the name
|
||||
of the router actor.
|
||||
|
||||
and
|
||||
|
||||
.. includecode:: code/docs/routing/RouterTypeExample.scala#fibonacciActor
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#create-parent
|
||||
|
||||
.. _round-robin-router-scala:
|
||||
|
||||
RoundRobinRouter
|
||||
****************
|
||||
RoundRobinPool and RoundRobinGroup
|
||||
----------------------------------
|
||||
|
||||
Routes in a `round-robin <http://en.wikipedia.org/wiki/Round-robin>`_ fashion to its routees.
|
||||
Code example:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterTypeExample.scala#roundRobinRouter
|
||||
RoundRobinPool defined in configuration:
|
||||
|
||||
When run you should see a similar output to this:
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-round-robin-pool
|
||||
|
||||
.. code-block:: scala
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#round-robin-pool-1
|
||||
|
||||
Received message '1' in actor $b
|
||||
Received message '2' in actor $c
|
||||
Received message '3' in actor $d
|
||||
Received message '6' in actor $b
|
||||
Received message '4' in actor $e
|
||||
Received message '8' in actor $d
|
||||
Received message '5' in actor $f
|
||||
Received message '9' in actor $e
|
||||
Received message '10' in actor $f
|
||||
Received message '7' in actor $c
|
||||
RoundRobinPool defined in code:
|
||||
|
||||
If you look closely to the output you can see that each of the routees received two messages which
|
||||
is exactly what you would expect from a round-robin router to happen.
|
||||
(The name of an actor is automatically created in the format ``$letter`` unless you specify it -
|
||||
hence the names printed above.)
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#round-robin-pool-2
|
||||
|
||||
This is an example of how to define a round-robin router in configuration:
|
||||
RoundRobinGroup defined in configuration:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-round-robin
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-round-robin-group
|
||||
|
||||
RandomRouter
|
||||
************
|
||||
As the name implies this router type selects one of its routees randomly and forwards
|
||||
the message it receives to this routee.
|
||||
This procedure will happen each time it receives a message.
|
||||
Code example:
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#round-robin-group-1
|
||||
|
||||
.. includecode:: code/docs/routing/RouterTypeExample.scala#randomRouter
|
||||
RoundRobinGroup defined in code:
|
||||
|
||||
When run you should see a similar output to this:
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala
|
||||
:include: paths,round-robin-group-2
|
||||
|
||||
.. code-block:: scala
|
||||
RandomPool and RandomGroup
|
||||
--------------------------
|
||||
|
||||
Received message '1' in actor $e
|
||||
Received message '2' in actor $c
|
||||
Received message '4' in actor $b
|
||||
Received message '5' in actor $d
|
||||
Received message '3' in actor $e
|
||||
Received message '6' in actor $c
|
||||
Received message '7' in actor $d
|
||||
Received message '8' in actor $e
|
||||
Received message '9' in actor $d
|
||||
Received message '10' in actor $d
|
||||
This router type selects one of its routees randomly for each message.
|
||||
|
||||
The result from running the random router should be different, or at least random, every time you run it.
|
||||
Try to run it a couple of times to verify its behavior if you don't trust us.
|
||||
RandomPool defined in configuration:
|
||||
|
||||
This is an example of how to define a random router in configuration:
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-random-pool
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-random
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#random-pool-1
|
||||
|
||||
SmallestMailboxRouter
|
||||
*********************
|
||||
A Router that tries to send to the non-suspended routee with fewest messages in mailbox.
|
||||
RandomPool defined in code:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#random-pool-2
|
||||
|
||||
RandomGroup defined in configuration:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-random-group
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#random-group-1
|
||||
|
||||
RandomGroup defined in code:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala
|
||||
:include: paths,random-group-2
|
||||
|
||||
|
||||
SmallestMailboxPool
|
||||
-------------------
|
||||
|
||||
A Router that tries to send to the non-suspended child routee with fewest messages in mailbox.
|
||||
The selection is done in this order:
|
||||
|
||||
* pick any idle routee (not processing message) with empty mailbox
|
||||
|
|
@ -241,37 +264,45 @@ The selection is done in this order:
|
|||
* pick any remote routee, remote actors are consider lowest priority,
|
||||
since their mailbox size is unknown
|
||||
|
||||
Code example:
|
||||
SmallestMailboxPool defined in configuration:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterTypeExample.scala#smallestMailboxRouter
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-smallest-mailbox-pool
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#smallest-mailbox-pool-1
|
||||
|
||||
This is an example of how to define a smallest-mailbox router in configuration:
|
||||
SmallestMailboxPool defined in code:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-smallest-mailbox
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#smallest-mailbox-pool-2
|
||||
|
||||
There is no Group variant of the SmallestMailboxPool because the size of the mailbox
|
||||
and the internal dispatching state of the actor is not practically available from the paths
|
||||
of the routees.
|
||||
|
||||
BroadcastPool and BroadcastGroup
|
||||
--------------------------------
|
||||
|
||||
BroadcastRouter
|
||||
***************
|
||||
A broadcast router forwards the message it receives to *all* its routees.
|
||||
Code example:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterTypeExample.scala#broadcastRouter
|
||||
BroadcastPool defined in configuration:
|
||||
|
||||
When run you should see a similar output to this:
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-broadcast-pool
|
||||
|
||||
.. code-block:: scala
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#broadcast-pool-1
|
||||
|
||||
Received message 'this is a broadcast message' in actor $f
|
||||
Received message 'this is a broadcast message' in actor $d
|
||||
Received message 'this is a broadcast message' in actor $e
|
||||
Received message 'this is a broadcast message' in actor $c
|
||||
Received message 'this is a broadcast message' in actor $b
|
||||
BroadcastPool defined in code:
|
||||
|
||||
As you can see here above each of the routees, five in total, received the broadcast message.
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#broadcast-pool-2
|
||||
|
||||
This is an example of how to define a broadcast router in configuration:
|
||||
BroadcastGroup defined in configuration:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-broadcast
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-broadcast-group
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#broadcast-group-1
|
||||
|
||||
BroadcastGroup defined in code:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala
|
||||
:include: paths,broadcast-group-2
|
||||
|
||||
.. note::
|
||||
|
||||
|
|
@ -280,33 +311,42 @@ This is an example of how to define a broadcast router in configuration:
|
|||
:ref:`broadcast-messages-scala` as needed.
|
||||
|
||||
|
||||
ScatterGatherFirstCompletedRouter
|
||||
*********************************
|
||||
The ScatterGatherFirstCompletedRouter will send the message on to all its routees as a future.
|
||||
It then waits for first result it gets back. This result will be sent back to original sender.
|
||||
Code example:
|
||||
ScatterGatherFirstCompletedPool and ScatterGatherFirstCompletedGroup
|
||||
--------------------------------------------------------------------
|
||||
|
||||
.. includecode:: code/docs/routing/RouterTypeExample.scala#scatterGatherFirstCompletedRouter
|
||||
The ScatterGatherFirstCompletedRouter will send the message on to all its routees.
|
||||
It then waits for first reply it gets back. This result will be sent back to original sender.
|
||||
Other replies are discarded.
|
||||
|
||||
When run you should see this:
|
||||
It is expecting at least one reply within a configured duration, otherwise it will reply with
|
||||
``akka.pattern.AskTimeoutException`` in a ``akka.actor.Status.Failure``.
|
||||
|
||||
.. code-block:: scala
|
||||
ScatterGatherFirstCompletedPool defined in configuration:
|
||||
|
||||
The result of calculating Fibonacci for 10 is 55
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-scatter-gather-pool
|
||||
|
||||
From the output above you can't really see that all the routees performed the calculation, but they did!
|
||||
The result you see is from the first routee that returned its calculation to the router.
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#scatter-gather-pool-1
|
||||
|
||||
This is an example of how to define a scatter-gather router in configuration:
|
||||
ScatterGatherFirstCompletedPool defined in code:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-scatter-gather
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#scatter-gather-pool-2
|
||||
|
||||
ScatterGatherFirstCompletedGroup defined in configuration:
|
||||
|
||||
ConsistentHashingRouter
|
||||
***********************
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-scatter-gather-group
|
||||
|
||||
The ConsistentHashingRouter uses `consistent hashing <http://en.wikipedia.org/wiki/Consistent_hashing>`_
|
||||
to select a connection based on the sent message. This
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#scatter-gather-group-1
|
||||
|
||||
ScatterGatherFirstCompletedGroup defined in code:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala
|
||||
:include: paths,scatter-gather-group-2
|
||||
|
||||
ConsistentHashingPool and ConsistentHashingGroup
|
||||
------------------------------------------------
|
||||
|
||||
The ConsistentHashingPool uses `consistent hashing <http://en.wikipedia.org/wiki/Consistent_hashing>`_
|
||||
to select a routee based on the sent message. This
|
||||
`article <http://weblogs.java.net/blog/tomwhite/archive/2007/11/consistent_hash.html>`_ gives good
|
||||
insight into how consistent hashing is implemented.
|
||||
|
||||
|
|
@ -327,6 +367,7 @@ There is 3 ways to define what data to use for the consistent hash key.
|
|||
These ways to define the consistent hash key can be use together and at
|
||||
the same time for one router. The ``hashMapping`` is tried first.
|
||||
|
||||
|
||||
Code example:
|
||||
|
||||
.. includecode:: code/docs/routing/ConsistentHashingRouterDocSpec.scala#cache-actor
|
||||
|
|
@ -337,22 +378,47 @@ In the above example you see that the ``Get`` message implements ``ConsistentHas
|
|||
while the ``Entry`` message is wrapped in a ``ConsistentHashableEnvelope``. The ``Evict``
|
||||
message is handled by the ``hashMapping`` partial function.
|
||||
|
||||
This is an example of how to define a consistent-hashing router in configuration:
|
||||
ConsistentHashingPool defined in configuration:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-consistent-hashing
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-consistent-hashing-pool
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#consistent-hashing-pool-1
|
||||
|
||||
ConsistentHashingPool defined in code:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#consistent-hashing-pool-2
|
||||
|
||||
ConsistentHashingGroup defined in configuration:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-consistent-hashing-group
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#consistent-hashing-group-1
|
||||
|
||||
ConsistentHashingGroup defined in code:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala
|
||||
:include: paths,consistent-hashing-group-2
|
||||
|
||||
|
||||
``virtual-nodes-factor`` is the number of virtual nodes per routee that is used in the
|
||||
consistent hash node ring to make the distribution more uniform.
|
||||
|
||||
.. _router-special-messages-scala:
|
||||
|
||||
Handling for Special Messages
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
Specially Handled Messages
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
Most messages sent to routers will be forwarded according to the routers' usual routing rules.
|
||||
Most messages sent to router actors will be forwarded according to the routers' routing logic.
|
||||
However there are a few types of messages that have special behavior.
|
||||
|
||||
Note that these special messages, except for the ``Broadcast`` message, are only handled by
|
||||
self contained router actors and not by the ``akka.routing.Router`` component described
|
||||
in :ref:`simple-router-scala`.
|
||||
|
||||
.. _broadcast-messages-scala:
|
||||
|
||||
Broadcast Messages
|
||||
******************
|
||||
------------------
|
||||
|
||||
A ``Broadcast`` message can be used to send a message to *all* of a router's routees. When a router
|
||||
receives a ``Broadcast`` message, it will broadcast that message's *payload* to all routees, no
|
||||
|
|
@ -361,38 +427,38 @@ matter how that router would normally route its messages.
|
|||
The example below shows how you would use a ``Broadcast`` message to send a very important message
|
||||
to every routee of a router.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramDocSpec.scala#broadcastDavyJonesWarning
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#broadcastDavyJonesWarning
|
||||
|
||||
In this example the router receives the ``Broadcast`` message, extracts its payload
|
||||
(``"Watch out for Davy Jones' locker"``), and then sends the payload on to all of the router's
|
||||
routees. It is up to each each routee actor to handle the received payload message.
|
||||
|
||||
PoisonPill Messages
|
||||
*******************
|
||||
-------------------
|
||||
|
||||
A ``PoisonPill`` message has special handling for all actors, including for routers. When any actor
|
||||
receives a ``PoisonPill`` message, that actor will be stopped. See the :ref:`poison-pill-scala`
|
||||
documentation for details.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramDocSpec.scala#poisonPill
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#poisonPill
|
||||
|
||||
For a router, which normally passes on messages to routees, it is important to realised that
|
||||
For a router, which normally passes on messages to routees, it is important to realise that
|
||||
``PoisonPill`` messages are processed by the router only. ``PoisonPill`` messages sent to a router
|
||||
will *not* be sent on to routees.
|
||||
|
||||
However, a ``PoisonPill`` message sent to a router may still affect its routees, because it will
|
||||
stop the router and when the router stops it also stops its children. Stopping children is normal
|
||||
actor behavior. The router will stop routees that it has created as children. Each child will
|
||||
process its current message and then tstop. This may lead to some messages being unprocessed.
|
||||
process its current message and then stop. This may lead to some messages being unprocessed.
|
||||
See the documentation on :ref:`stopping-actors-scala` for more information.
|
||||
|
||||
If you wish to stop a router and its routees, but you would like the routees to first process all
|
||||
the messages currently in their mailboxes, then you should not send a ``PoisonPill`` message to the
|
||||
router. Instead you should wrap a ``PoisonPill`` message inside a broadcast message so that each
|
||||
routee will the ``PoisonPill`` message directly. Note that this will stop all routees, even if the
|
||||
router. Instead you should wrap a ``PoisonPill`` message inside a ``Broadcast`` message so that each
|
||||
routee will receive the ``PoisonPill`` message. Note that this will stop all routees, even if the
|
||||
routees aren't children of the router, i.e. even routees programmatically provided to the router.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramDocSpec.scala#broadcastPoisonPill
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#broadcastPoisonPill
|
||||
|
||||
With the code shown above, each routee will receive a ``PoisonPill`` message. Each routee will
|
||||
continue to process its messages as normal, eventually processing the ``PoisonPill``. This will
|
||||
|
|
@ -407,48 +473,63 @@ a resizer.
|
|||
discusses in more detail how ``PoisonPill`` messages can be used to shut down routers and routees.
|
||||
|
||||
Kill Messages
|
||||
*************
|
||||
-------------
|
||||
|
||||
``Kill`` messages are another type of message that has special handling. See
|
||||
:ref:`killing-actors-scala` for general information about how actors handle ``Kill`` messages.
|
||||
|
||||
When a ``Kill`` message is sent to a router the router processes the message internally, and does
|
||||
*not* send it on to its routees. The router will throw an :class:`ActorKilledException` and fail. It
|
||||
*not* send it on to its routees. The router will throw an ``ActorKilledException`` and fail. It
|
||||
will then be either resumed, restarted or terminated, depending how it is supervised.
|
||||
|
||||
Routees that are children of the router will also be suspended, and will be affected by the
|
||||
supervision directive that is applied to the router. Routees that are not the routers children, i.e.
|
||||
those that were created externally to the router, will not be affected.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramDocSpec.scala#kill
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#kill
|
||||
|
||||
As with the ``PoisonPill`` message, there is a distinction between killing a router, which
|
||||
indirectly kills its children (who happen to be routees), and killing routees directly (some of whom
|
||||
may not be children.) To kill routees directly the router should be sent a ``Kill`` message wrapped
|
||||
in a ``Broadcast`` message.
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramDocSpec.scala#broadcastKill
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#broadcastKill
|
||||
|
||||
Managagement Messages
|
||||
---------------------
|
||||
|
||||
* Sending ``akka.routing.GetRoutees`` to a router actor will make it send back its currently used routees
|
||||
in a ``akka.routing.Routees`` message.
|
||||
* Sending ``akka.routing.AddRoutee`` to a router actor will add that routee to its collection of routees.
|
||||
* Sending ``akka.routing.RemoveRoutee`` to a router actor will remove that routee to its collection of routees.
|
||||
* Sending ``akka.routing.AdjustPoolSize`` to a pool router actor will add or remove that number of routees to
|
||||
its collection of routees.
|
||||
|
||||
These management messages may be handled after other messages, so if you send ``AddRoutee`` immediately followed
|
||||
an ordinary message you are not guaranteed that the routees have been changed when the ordinary message
|
||||
is routed. If you need to know when the change has been applied you can send ``AddRoutee`` followed by ``GetRoutees``
|
||||
and when you receive the ``Routees`` reply you know that the preceeding change has been applied.
|
||||
|
||||
.. _resizable-routers-scala:
|
||||
|
||||
Dynamically Resizable Routers
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
Dynamically Resizable Pool
|
||||
^^^^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
All routers can be used with a fixed number of routees or with a resize strategy to adjust the number
|
||||
All pools can be used with a fixed number of routees or with a resize strategy to adjust the number
|
||||
of routees dynamically.
|
||||
|
||||
This is an example of how to create a resizable router that is defined in configuration:
|
||||
Pool with resizer defined in configuration:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#config-resize
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#config-resize-pool
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaConfigDocSpec.scala#configurableRoutingWithResizer
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#resize-pool-1
|
||||
|
||||
Several more configuration options are available and described in ``akka.actor.deployment.default.resizer``
|
||||
section of the reference :ref:`configuration`.
|
||||
|
||||
This is an example of how to programmatically create a resizable router:
|
||||
Pool with resizer defined in code:
|
||||
|
||||
.. includecode:: code/docs/routing/RouterViaProgramExample.scala#programmaticRoutingWithResizer
|
||||
.. includecode:: code/docs/routing/RouterDocSpec.scala#resize-pool-2
|
||||
|
||||
*It is also worth pointing out that if you define the ``router`` in the configuration file then this value
|
||||
will be used instead of any programmatically sent parameters.*
|
||||
|
|
@ -457,7 +538,7 @@ will be used instead of any programmatically sent parameters.*
|
|||
|
||||
Resizing is triggered by sending messages to the actor pool, but it is not
|
||||
completed synchronously; instead a message is sent to the “head”
|
||||
:class:`Router` to perform the size change. Thus you cannot rely on resizing
|
||||
``RouterActor`` to perform the size change. Thus you cannot rely on resizing
|
||||
to instantaneously create new workers when all others are busy, because the
|
||||
message just sent will be queued to the mailbox of a busy actor. To remedy
|
||||
this, configure the pool to use a balancing dispatcher, see `Configuring
|
||||
|
|
@ -475,8 +556,8 @@ routees.
|
|||
A normal actor can be used for routing messages, but an actor's single-threaded processing can
|
||||
become a bottleneck. Routers can achieve much higher throughput with an optimization to the usual
|
||||
message-processing pipeline that allows concurrent routing. This is achieved by embedding routers'
|
||||
routing logic directly in their :class:`ActorRef` rather than in the router actor. Messages sent to
|
||||
a router's :class:`ActorRef` can be immediately routed to the routee, bypassing the single-threaded
|
||||
routing logic directly in their ``ActorRef`` rather than in the router actor. Messages sent to
|
||||
a router's ``ActorRef`` can be immediately routed to the routee, bypassing the single-threaded
|
||||
router actor entirely.
|
||||
|
||||
The cost to this is, of course, that the internals of routing code are more complicated than if
|
||||
|
|
@ -500,80 +581,54 @@ lower maximum throughput is acceptable in your application you may wish to stick
|
|||
actors. This section, however, assumes that you wish to get maximum performance and so demonstrates
|
||||
how you can create your own router.
|
||||
|
||||
The router created in this example is a simple vote counter. It will route the votes to specific vote counter actors.
|
||||
In this case we only have two parties the Republicans and the Democrats. We would like a router that forwards all
|
||||
democrat related messages to the Democrat actor and all republican related messages to the Republican actor.
|
||||
The router created in this example is replicating each message to a few destinations.
|
||||
|
||||
We begin with defining the class:
|
||||
Start with the routing logic:
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala#crRouter
|
||||
:exclude: crRoute
|
||||
.. includecode:: code/docs/routing/CustomRouterDocSpec.scala#routing-logic
|
||||
|
||||
The next step is to implement the ``createRoute`` method in the class just defined:
|
||||
``select`` will be called for each message and in this example pick a few destinations by round-robin,
|
||||
by reusing the existing ``RoundRobinRoutingLogic`` and wrap the result in a ``SeveralRoutees``
|
||||
instance. ``SeveralRoutees`` will send the message to all of the supplied routues.
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala#crRoute
|
||||
The implementation of the routing logic must be thread safe, since it might be used outside of actors.
|
||||
|
||||
As you can see above we start off by creating the routees and put them in a collection.
|
||||
A unit test of the routing logic:
|
||||
|
||||
Make sure that you don't miss to implement the line below as it is *really* important.
|
||||
It registers the routees internally and failing to call this method will
|
||||
cause a ``ActorInitializationException`` to be thrown when the router is used.
|
||||
Therefore always make sure to do the following in your custom router:
|
||||
.. includecode:: code/docs/routing/CustomRouterDocSpec.scala#unit-test-logic
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala#crRegisterRoutees
|
||||
You could stop here and use the ``RedundancyRoutingLogic`` with a ``akka.routing.Router``
|
||||
as described in :ref:`simple-router-scala`.
|
||||
|
||||
The routing logic is where your magic sauce is applied. In our example it inspects the message types
|
||||
and forwards to the correct routee based on this:
|
||||
Let us continue and make this into a self contained, configurable, router actor.
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala#crRoutingLogic
|
||||
Create a class that extends ``Pool``, ``Group`` or ``CustomRouterConfig``. That class is a factory
|
||||
for the routing logic and holds the configuration for the router. Here we make it a ``Group``.
|
||||
|
||||
As you can see above what's returned in the partial function is a ``List`` of ``Destination(sender, routee)``.
|
||||
The sender is what "parent" the routee should see - changing this could be useful if you for example want
|
||||
another actor than the original sender to intermediate the result of the routee (if there is a result).
|
||||
For more information about how to alter the original sender we refer to the source code of
|
||||
`ScatterGatherFirstCompletedRouter <https://github.com/akka/akka/blob/master/akka-actor/src/main/scala/akka/routing/Routing.scala#L375>`_
|
||||
.. includecode:: code/docs/routing/CustomRouterDocSpec.scala#group
|
||||
|
||||
All in all the custom router looks like this:
|
||||
This can be used exactly as the router actors provided by Akka.
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala#CustomRouter
|
||||
.. includecode:: code/docs/routing/CustomRouterDocSpec.scala#usage-1
|
||||
|
||||
If you are interested in how to use the VoteCountRouter you can have a look at the test class
|
||||
`RoutingSpec <https://github.com/akka/akka/blob/master/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala>`_
|
||||
Note that we added a constructor in ``RedundancyGroup`` that takes a ``Config`` parameter.
|
||||
That makes it possible to define it in configuration.
|
||||
|
||||
.. caution::
|
||||
.. includecode:: code/docs/routing/CustomRouterDocSpec.scala#config
|
||||
|
||||
When creating a cutom router the resulting RoutedActorRef optimizes the
|
||||
sending of the message so that it does NOT go through the router’s mailbox
|
||||
unless the route returns an empty recipient set.
|
||||
|
||||
This means that the ``route`` function defined in the ``RouterConfig``
|
||||
or the function returned from ``CreateCustomRoute`` in
|
||||
``CustomRouterConfig`` is evaluated concurrently without protection by
|
||||
the RoutedActorRef: either provide a reentrant (i.e. pure) implementation
|
||||
or do the locking yourself!
|
||||
|
||||
|
||||
Configured Custom Router
|
||||
************************
|
||||
|
||||
It is possible to define configuration properties for custom routers. In the ``router`` property of the deployment
|
||||
configuration you define the fully qualified class name of the router class. The router class must extend
|
||||
``akka.routing.RouterConfig`` and have constructor with one ``com.typesafe.config.Config`` parameter.
|
||||
Note the fully qualified class name in the ``router`` property. The router class must extend
|
||||
``akka.routing.RouterConfig`` (``Pool``, ``Group`` or ``CustomRouterConfig``) and have
|
||||
constructor with one ``com.typesafe.config.Config`` parameter.
|
||||
The deployment section of the configuration is passed to the constructor.
|
||||
|
||||
Custom Resizer
|
||||
**************
|
||||
|
||||
A router with dynamically resizable number of routees is implemented by providing a ``akka.routing.Resizer``
|
||||
in ``resizer`` method of the ``RouterConfig``. See ``akka.routing.DefaultResizer`` for inspiration
|
||||
of how to write your own resize strategy.
|
||||
|
||||
.. includecode:: code/docs/routing/CustomRouterDocSpec.scala#usage-2
|
||||
|
||||
Configuring Dispatchers
|
||||
^^^^^^^^^^^^^^^^^^^^^^^
|
||||
|
||||
The dispatcher for created children of the router will be taken from
|
||||
:class:`Props` as described in :ref:`dispatchers-scala`. For a dynamic pool it
|
||||
makes sense to configure the :class:`BalancingDispatcher` if the precise
|
||||
``Props`` as described in :ref:`dispatchers-scala`. For a pool it
|
||||
makes sense to configure the ``BalancingDispatcher`` if the precise
|
||||
routing is not so important (i.e. no consistent hashing or round-robin is
|
||||
required); this enables newly created routees to pick up work immediately by
|
||||
stealing it from their siblings.
|
||||
|
|
@ -586,7 +641,7 @@ stealing it from their siblings.
|
|||
|
||||
The “head” router cannot always run on the same dispatcher, because it
|
||||
does not process the same type of messages, hence this special actor does
|
||||
not use the dispatcher configured in :class:`Props`, but takes the
|
||||
not use the dispatcher configured in ``Props``, but takes the
|
||||
``routerDispatcher`` from the :class:`RouterConfig` instead, which defaults to
|
||||
the actor system’s default dispatcher. All standard routers allow setting this
|
||||
property in their constructor or factory method, custom routers have to
|
||||
|
|
|
|||
|
|
@ -772,16 +772,3 @@ Some `Specs2 <http://specs2.org>`_ users have contributed examples of how to wor
|
|||
* Specifications are by default executed concurrently, which requires some care
|
||||
when writing the tests or alternatively the ``sequential`` keyword.
|
||||
|
||||
Testing Custom Router Logic
|
||||
===========================
|
||||
|
||||
Given the following custom (dummy) router:
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/routing/CustomRouteSpec.scala#custom-router
|
||||
|
||||
This might be tested by dispatching messages and asserting their reception at
|
||||
the right destinations, but that can be inconvenient. Therefore exists the
|
||||
:obj:`ExtractRoute` extractor, which can be used like so:
|
||||
|
||||
.. includecode:: ../../../akka-actor-tests/src/test/scala/akka/routing/CustomRouteSpec.scala#test-route
|
||||
|
||||
|
|
|
|||
|
|
@ -1,7 +1,7 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.router
|
||||
package akka.remote.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
|
|
@ -1,10 +1,10 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.router
|
||||
package akka.remote.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import scala.collection.immutable
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Props
|
||||
|
|
@ -19,9 +19,9 @@ import akka.routing.RouterRoutees
|
|||
import akka.routing.RoundRobinRouter
|
||||
import akka.routing.RoutedActorRef
|
||||
import akka.routing.Resizer
|
||||
import akka.routing.RouteeProvider
|
||||
import akka.testkit._
|
||||
import scala.concurrent.duration._
|
||||
import akka.routing.Routee
|
||||
|
||||
object RoundRobinRoutedRemoteActorMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
|
|
@ -33,9 +33,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec extends MultiNodeConfig {
|
|||
|
||||
class TestResizer extends Resizer {
|
||||
def isTimeForResize(messageCounter: Long): Boolean = messageCounter <= 10
|
||||
def resize(routeeProvider: RouteeProvider): Unit = {
|
||||
routeeProvider.createRoutees(nrOfInstances = 1)
|
||||
}
|
||||
def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int = 1
|
||||
}
|
||||
|
||||
val first = role("first")
|
||||
|
|
@ -51,6 +49,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec extends MultiNodeConfig {
|
|||
/service-hello.target.nodes = ["@first@", "@second@", "@third@"]
|
||||
|
||||
/service-hello2.router = "round-robin"
|
||||
/service-hello2.nr-of-instances = 0
|
||||
/service-hello2.target.nodes = ["@first@", "@second@", "@third@"]
|
||||
""")
|
||||
}
|
||||
|
|
@ -1,7 +1,7 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.router
|
||||
package akka.remote.oldrouting
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
|
|
@ -0,0 +1,93 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.routing
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Address
|
||||
import akka.actor.PoisonPill
|
||||
import akka.actor.Props
|
||||
import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec }
|
||||
import akka.routing.Broadcast
|
||||
import akka.routing.RandomPool
|
||||
import akka.routing.RoutedActorRef
|
||||
import akka.testkit._
|
||||
|
||||
object RemoteRandomMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class SomeActor extends Actor {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = false))
|
||||
|
||||
deployOnAll("""
|
||||
/service-hello {
|
||||
router = "random-pool"
|
||||
nr-of-instances = 3
|
||||
target.nodes = ["@first@", "@second@", "@third@"]
|
||||
}
|
||||
""")
|
||||
}
|
||||
|
||||
class RemoteRandomMultiJvmNode1 extends RemoteRandomSpec
|
||||
class RemoteRandomMultiJvmNode2 extends RemoteRandomSpec
|
||||
class RemoteRandomMultiJvmNode3 extends RemoteRandomSpec
|
||||
class RemoteRandomMultiJvmNode4 extends RemoteRandomSpec
|
||||
|
||||
class RemoteRandomSpec extends MultiNodeSpec(RemoteRandomMultiJvmSpec)
|
||||
with STMultiNodeSpec with ImplicitSender with DefaultTimeout {
|
||||
import RemoteRandomMultiJvmSpec._
|
||||
|
||||
def initialParticipants = roles.size
|
||||
|
||||
"A remote random pool" must {
|
||||
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first, second, third) {
|
||||
enterBarrier("start", "broadcast-end", "end", "done")
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
enterBarrier("start")
|
||||
val actor = system.actorOf(RandomPool(nrOfInstances = 0).props(Props[SomeActor]), "service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val connectionCount = 3
|
||||
val iterationCount = 100
|
||||
|
||||
for (i ← 0 until iterationCount; k ← 0 until connectionCount) {
|
||||
actor ! "hit"
|
||||
}
|
||||
|
||||
val replies: Map[Address, Int] = (receiveWhile(5.seconds, messages = connectionCount * iterationCount) {
|
||||
case ref: ActorRef ⇒ ref.path.address
|
||||
}).foldLeft(Map(node(first).address -> 0, node(second).address -> 0, node(third).address -> 0)) {
|
||||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
|
||||
enterBarrier("broadcast-end")
|
||||
actor ! Broadcast(PoisonPill)
|
||||
|
||||
enterBarrier("end")
|
||||
// since it's random we can't be too strict in the assert
|
||||
replies.values count (_ > 0) must be > (connectionCount - 2)
|
||||
replies.get(node(fourth).address) must be(None)
|
||||
|
||||
// shut down the actor before we let the other node(s) shut down so we don't try to send
|
||||
// "Terminate" to a shut down node
|
||||
system.stop(actor)
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,199 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.routing
|
||||
|
||||
import language.postfixOps
|
||||
import scala.collection.immutable
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Props
|
||||
import akka.actor.PoisonPill
|
||||
import akka.actor.Address
|
||||
import scala.concurrent.Await
|
||||
import akka.pattern.ask
|
||||
import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec }
|
||||
import akka.routing.Broadcast
|
||||
import akka.routing.GetRoutees
|
||||
import akka.routing.Routees
|
||||
import akka.routing.RoundRobinPool
|
||||
import akka.routing.RoundRobinGroup
|
||||
import akka.routing.RoutedActorRef
|
||||
import akka.routing.Resizer
|
||||
import akka.routing.Routee
|
||||
import akka.routing.FromConfig
|
||||
import akka.testkit._
|
||||
import scala.concurrent.duration._
|
||||
|
||||
object RemoteRoundRobinMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class SomeActor extends Actor {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
class TestResizer extends Resizer {
|
||||
override def isTimeForResize(messageCounter: Long): Boolean = messageCounter <= 10
|
||||
override def resize(currentRoutees: immutable.IndexedSeq[Routee]): Int = 1
|
||||
}
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = false))
|
||||
|
||||
deployOnAll("""
|
||||
/service-hello {
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 3
|
||||
target.nodes = ["@first@", "@second@", "@third@"]
|
||||
}
|
||||
|
||||
/service-hello2 {
|
||||
router = round-robin-pool
|
||||
target.nodes = ["@first@", "@second@", "@third@"]
|
||||
}
|
||||
|
||||
/service-hello3 {
|
||||
router = round-robin-group
|
||||
routees.paths = [
|
||||
"@first@/user/target-first",
|
||||
"@second@/user/target-second",
|
||||
"@third@/user/target-third"]
|
||||
}
|
||||
""")
|
||||
}
|
||||
|
||||
class RemoteRoundRobinMultiJvmNode1 extends RemoteRoundRobinSpec
|
||||
class RemoteRoundRobinMultiJvmNode2 extends RemoteRoundRobinSpec
|
||||
class RemoteRoundRobinMultiJvmNode3 extends RemoteRoundRobinSpec
|
||||
class RemoteRoundRobinMultiJvmNode4 extends RemoteRoundRobinSpec
|
||||
|
||||
class RemoteRoundRobinSpec extends MultiNodeSpec(RemoteRoundRobinMultiJvmSpec)
|
||||
with STMultiNodeSpec with ImplicitSender with DefaultTimeout {
|
||||
import RemoteRoundRobinMultiJvmSpec._
|
||||
|
||||
def initialParticipants = roles.size
|
||||
|
||||
"A remote round robin pool" must {
|
||||
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first, second, third) {
|
||||
enterBarrier("start", "broadcast-end", "end")
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
enterBarrier("start")
|
||||
val actor = system.actorOf(RoundRobinPool(nrOfInstances = 0).props(Props[SomeActor]), "service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val connectionCount = 3
|
||||
val iterationCount = 10
|
||||
|
||||
for (i ← 0 until iterationCount; k ← 0 until connectionCount) {
|
||||
actor ! "hit"
|
||||
}
|
||||
|
||||
val replies: Map[Address, Int] = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) {
|
||||
case ref: ActorRef ⇒ ref.path.address
|
||||
}).foldLeft(Map(node(first).address -> 0, node(second).address -> 0, node(third).address -> 0)) {
|
||||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
|
||||
enterBarrier("broadcast-end")
|
||||
actor ! Broadcast(PoisonPill)
|
||||
|
||||
enterBarrier("end")
|
||||
replies.values foreach { _ must be(iterationCount) }
|
||||
replies.get(node(fourth).address) must be(None)
|
||||
|
||||
// shut down the actor before we let the other node(s) shut down so we don't try to send
|
||||
// "Terminate" to a shut down node
|
||||
system.stop(actor)
|
||||
}
|
||||
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
|
||||
"A remote round robin pool with resizer" must {
|
||||
"be locally instantiated on a remote node after several resize rounds" taggedAs LongRunningTest in within(5 seconds) {
|
||||
|
||||
runOn(first, second, third) {
|
||||
enterBarrier("start", "broadcast-end", "end")
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
enterBarrier("start")
|
||||
val actor = system.actorOf(RoundRobinPool(
|
||||
nrOfInstances = 1,
|
||||
resizer = Some(new TestResizer)).props(Props[SomeActor]), "service-hello2")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
actor ! GetRoutees
|
||||
// initial nrOfInstances 1 + inital resize => 2
|
||||
expectMsgType[Routees].routees.size must be(2)
|
||||
|
||||
val repliesFrom: Set[ActorRef] =
|
||||
(for (n ← 3 to 9) yield {
|
||||
// each message trigger a resize, incrementing number of routees with 1
|
||||
actor ! "hit"
|
||||
Await.result(actor ? GetRoutees, remaining).asInstanceOf[Routees].routees.size must be(n)
|
||||
expectMsgType[ActorRef]
|
||||
}).toSet
|
||||
|
||||
enterBarrier("broadcast-end")
|
||||
actor ! Broadcast(PoisonPill)
|
||||
|
||||
enterBarrier("end")
|
||||
repliesFrom.size must be(7)
|
||||
val repliesFromAddresses = repliesFrom.map(_.path.address)
|
||||
repliesFromAddresses must be === (Set(node(first), node(second), node(third)).map(_.address))
|
||||
|
||||
// shut down the actor before we let the other node(s) shut down so we don't try to send
|
||||
// "Terminate" to a shut down node
|
||||
system.stop(actor)
|
||||
}
|
||||
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
|
||||
"A remote round robin group" must {
|
||||
"send messages with actor selection to remote paths" taggedAs LongRunningTest in {
|
||||
|
||||
runOn(first, second, third) {
|
||||
system.actorOf(Props[SomeActor], name = "target-" + myself.name)
|
||||
enterBarrier("start", "end")
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
enterBarrier("start")
|
||||
val actor = system.actorOf(FromConfig.props(), "service-hello3")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val connectionCount = 3
|
||||
val iterationCount = 10
|
||||
|
||||
for (i ← 0 until iterationCount; k ← 0 until connectionCount) {
|
||||
actor ! "hit"
|
||||
}
|
||||
|
||||
val replies: Map[Address, Int] = (receiveWhile(5 seconds, messages = connectionCount * iterationCount) {
|
||||
case ref: ActorRef ⇒ ref.path.address
|
||||
}).foldLeft(Map(node(first).address -> 0, node(second).address -> 0, node(third).address -> 0)) {
|
||||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
|
||||
enterBarrier("end")
|
||||
replies.values foreach { _ must be(iterationCount) }
|
||||
replies.get(node(fourth).address) must be(None)
|
||||
}
|
||||
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,97 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.routing
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.Actor
|
||||
import akka.actor.ActorRef
|
||||
import akka.actor.Address
|
||||
import akka.actor.PoisonPill
|
||||
import akka.actor.Props
|
||||
import akka.remote.testkit.{ STMultiNodeSpec, MultiNodeConfig, MultiNodeSpec }
|
||||
import akka.routing.Broadcast
|
||||
import akka.routing.ScatterGatherFirstCompletedPool
|
||||
import akka.routing.RoutedActorRef
|
||||
import akka.testkit._
|
||||
import akka.testkit.TestEvent._
|
||||
|
||||
object RemoteScatterGatherMultiJvmSpec extends MultiNodeConfig {
|
||||
|
||||
class SomeActor extends Actor {
|
||||
def receive = {
|
||||
case "hit" ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
|
||||
val first = role("first")
|
||||
val second = role("second")
|
||||
val third = role("third")
|
||||
val fourth = role("fourth")
|
||||
|
||||
commonConfig(debugConfig(on = false))
|
||||
|
||||
deployOnAll("""
|
||||
/service-hello {
|
||||
router = "scatter-gather-pool"
|
||||
nr-of-instances = 3
|
||||
target.nodes = ["@first@", "@second@", "@third@"]
|
||||
}
|
||||
""")
|
||||
}
|
||||
|
||||
class RemoteScatterGatherMultiJvmNode1 extends RemoteScatterGatherSpec
|
||||
class RemoteScatterGatherMultiJvmNode2 extends RemoteScatterGatherSpec
|
||||
class RemoteScatterGatherMultiJvmNode3 extends RemoteScatterGatherSpec
|
||||
class RemoteScatterGatherMultiJvmNode4 extends RemoteScatterGatherSpec
|
||||
|
||||
class RemoteScatterGatherSpec extends MultiNodeSpec(RemoteScatterGatherMultiJvmSpec)
|
||||
with STMultiNodeSpec with ImplicitSender with DefaultTimeout {
|
||||
import RemoteScatterGatherMultiJvmSpec._
|
||||
|
||||
def initialParticipants = roles.size
|
||||
|
||||
"A remote ScatterGatherFirstCompleted pool" must {
|
||||
"be locally instantiated on a remote node and be able to communicate through its RemoteActorRef" taggedAs LongRunningTest in {
|
||||
|
||||
system.eventStream.publish(Mute(EventFilter.warning(pattern = ".*received dead letter from.*")))
|
||||
|
||||
runOn(first, second, third) {
|
||||
enterBarrier("start", "broadcast-end", "end", "done")
|
||||
}
|
||||
|
||||
runOn(fourth) {
|
||||
enterBarrier("start")
|
||||
val actor = system.actorOf(ScatterGatherFirstCompletedPool(nrOfInstances = 1, within = 10.seconds).props(Props[SomeActor]), "service-hello")
|
||||
actor.isInstanceOf[RoutedActorRef] must be(true)
|
||||
|
||||
val connectionCount = 3
|
||||
val iterationCount = 10
|
||||
|
||||
for (i ← 0 until iterationCount; k ← 0 until connectionCount) {
|
||||
actor ! "hit"
|
||||
}
|
||||
|
||||
val replies: Map[Address, Int] = (receiveWhile(5.seconds, messages = connectionCount * iterationCount) {
|
||||
case ref: ActorRef ⇒ ref.path.address
|
||||
}).foldLeft(Map(node(first).address -> 0, node(second).address -> 0, node(third).address -> 0)) {
|
||||
case (replyMap, address) ⇒ replyMap + (address -> (replyMap(address) + 1))
|
||||
}
|
||||
|
||||
enterBarrier("broadcast-end")
|
||||
actor ! Broadcast(PoisonPill)
|
||||
|
||||
enterBarrier("end")
|
||||
replies.values.sum must be === connectionCount * iterationCount
|
||||
replies.get(node(fourth).address) must be(None)
|
||||
|
||||
// shut down the actor before we let the other node(s) shut down so we don't try to send
|
||||
// "Terminate" to a shut down node
|
||||
system.stop(actor)
|
||||
enterBarrier("done")
|
||||
}
|
||||
|
||||
enterBarrier("done")
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -293,7 +293,8 @@ private[akka] class RemoteActorRefProvider(
|
|||
case NonFatal(e) ⇒ throw new IllegalArgumentException(s"remote deployment failed for [$path]", e)
|
||||
}
|
||||
|
||||
case _ ⇒ local.actorOf(system, props, supervisor, path, systemService, deployment.headOption, false, async)
|
||||
case _ ⇒
|
||||
local.actorOf(system, props, supervisor, path, systemService, deployment.headOption, false, async)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -9,6 +9,8 @@ import akka.remote.routing._
|
|||
import akka.ConfigurationException
|
||||
import akka.japi.Util.immutableSeq
|
||||
import com.typesafe.config._
|
||||
import akka.routing.Pool
|
||||
import akka.remote.routing.RemoteRouterConfig
|
||||
|
||||
@SerialVersionUID(1L)
|
||||
case class RemoteScope(node: Address) extends Scope {
|
||||
|
|
@ -30,7 +32,10 @@ private[akka] class RemoteDeployer(_settings: ActorSystem.Settings, _pm: Dynamic
|
|||
case _ ⇒
|
||||
val nodes = immutableSeq(deploy.config.getStringList("target.nodes")).map(AddressFromURIString(_))
|
||||
if (nodes.isEmpty || deploy.routerConfig == NoRouter) d
|
||||
else Some(deploy.copy(routerConfig = RemoteRouterConfig(deploy.routerConfig, nodes)))
|
||||
else deploy.routerConfig match {
|
||||
case r: Pool ⇒ Some(deploy.copy(routerConfig = RemoteRouterConfig(r, nodes)))
|
||||
case _ ⇒ d
|
||||
}
|
||||
}
|
||||
case None ⇒ None
|
||||
}
|
||||
|
|
|
|||
|
|
@ -3,41 +3,62 @@
|
|||
*/
|
||||
package akka.remote.routing
|
||||
|
||||
import akka.routing.{ Route, Router, RouterConfig, RouteeProvider, Resizer }
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.actor.ActorContext
|
||||
import akka.actor.Deploy
|
||||
import akka.routing.Router
|
||||
import akka.actor.Props
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.actor.ActorContext
|
||||
import akka.routing.Routee
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import akka.actor.Address
|
||||
import akka.actor.ActorCell
|
||||
import akka.ConfigurationException
|
||||
import akka.actor.Deploy
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import akka.routing.ActorRefRoutee
|
||||
import akka.remote.RemoteScope
|
||||
import akka.actor.Actor
|
||||
import akka.actor.SupervisorStrategy
|
||||
import akka.routing.Resizer
|
||||
import akka.routing.RouterConfig
|
||||
import akka.routing.Pool
|
||||
import akka.actor.ActorSystem
|
||||
import akka.routing.RouterActor
|
||||
import akka.japi.Util.immutableSeq
|
||||
import scala.collection.immutable
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
import java.lang.IllegalStateException
|
||||
|
||||
/**
|
||||
* [[akka.routing.RouterConfig]] implementation for remote deployment on defined
|
||||
* target nodes. Delegates other duties to the local [[akka.routing.RouterConfig]],
|
||||
* target nodes. Delegates other duties to the local [[akka.routing.Pool]],
|
||||
* 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 RemoteRouterConfig(local: RouterConfig, nodes: Iterable[Address]) extends RouterConfig {
|
||||
final case class RemoteRouterConfig(local: Pool, nodes: Iterable[Address]) extends Pool {
|
||||
|
||||
def this(local: RouterConfig, nodes: java.lang.Iterable[Address]) = this(local, immutableSeq(nodes))
|
||||
def this(local: RouterConfig, nodes: Array[Address]) = this(local, nodes: Iterable[Address])
|
||||
require(nodes.nonEmpty, "Must specify list of remote target.nodes")
|
||||
|
||||
override def createRouteeProvider(context: ActorContext, routeeProps: Props) =
|
||||
new RemoteRouteeProvider(nodes, context, routeeProps, resizer)
|
||||
def this(local: Pool, nodes: java.lang.Iterable[Address]) = this(local, immutableSeq(nodes))
|
||||
def this(local: Pool, nodes: Array[Address]) = this(local, nodes: Iterable[Address])
|
||||
|
||||
override def createRoute(routeeProvider: RouteeProvider): Route = {
|
||||
local.createRoute(routeeProvider)
|
||||
// need this iterator as instance variable since Resizer may call createRoutees several times
|
||||
@transient private val nodeAddressIter: Iterator[Address] = Stream.continually(nodes).flatten.iterator
|
||||
// need this counter as instance variable since Resizer may call createRoutees several times
|
||||
@transient private val childNameCounter = new AtomicInteger
|
||||
|
||||
override def createRouter(system: ActorSystem): Router = local.createRouter(system)
|
||||
|
||||
override def nrOfInstances: Int = local.nrOfInstances
|
||||
|
||||
override def newRoutee(routeeProps: Props, context: ActorContext): Routee = {
|
||||
val name = "c" + childNameCounter.incrementAndGet
|
||||
val deploy = Deploy(config = ConfigFactory.empty(), routerConfig = routeeProps.routerConfig,
|
||||
scope = RemoteScope(nodeAddressIter.next))
|
||||
|
||||
// attachChild means that the provider will treat this call as if possibly done out of the wrong
|
||||
// context and use RepointableActorRef instead of LocalActorRef. Seems like a slightly sub-optimal
|
||||
// choice in a corner case (and hence not worth fixing).
|
||||
val ref = context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false)
|
||||
ActorRefRoutee(ref)
|
||||
}
|
||||
|
||||
override def createActor(): Router = local.createActor()
|
||||
override def createRouterActor(): RouterActor = local.createRouterActor()
|
||||
|
||||
override def supervisorStrategy: SupervisorStrategy = local.supervisorStrategy
|
||||
|
||||
|
|
@ -48,44 +69,9 @@ final case class RemoteRouterConfig(local: RouterConfig, nodes: Iterable[Address
|
|||
override def withFallback(other: RouterConfig): RouterConfig = other match {
|
||||
case RemoteRouterConfig(local: RemoteRouterConfig, nodes) ⇒ throw new IllegalStateException(
|
||||
"RemoteRouterConfig is not allowed to wrap a RemoteRouterConfig")
|
||||
case RemoteRouterConfig(local, nodes) ⇒ copy(local = this.local.withFallback(local))
|
||||
case _ ⇒ copy(local = this.local.withFallback(other))
|
||||
case RemoteRouterConfig(local: Pool, nodes) ⇒
|
||||
copy(local = this.local.withFallback(local).asInstanceOf[Pool])
|
||||
case _ ⇒ copy(local = this.local.withFallback(other).asInstanceOf[Pool])
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Factory and registry for routees of the router.
|
||||
* Deploys new routees on the specified `nodes`, round-robin.
|
||||
*
|
||||
* Routee paths may not be combined with remote target nodes.
|
||||
*/
|
||||
final class RemoteRouteeProvider(nodes: Iterable[Address], _context: ActorContext, _routeeProps: Props, _resizer: Option[Resizer])
|
||||
extends RouteeProvider(_context, _routeeProps, _resizer) {
|
||||
|
||||
if (nodes.isEmpty)
|
||||
throw new ConfigurationException("Must specify list of remote target.nodes for [%s]" format context.self.path.toString)
|
||||
|
||||
// need this iterator as instance variable since Resizer may call createRoutees several times
|
||||
private val nodeAddressIter: Iterator[Address] = Stream.continually(nodes).flatten.iterator
|
||||
// need this counter as instance variable since Resizer may call createRoutees several times
|
||||
private val childNameCounter = new AtomicInteger
|
||||
|
||||
override def registerRouteesFor(paths: immutable.Iterable[String]): Unit =
|
||||
throw new ConfigurationException("Remote target.nodes can not be combined with routees for [%s]"
|
||||
format context.self.path.toString)
|
||||
|
||||
override def createRoutees(nrOfInstances: Int): Unit = {
|
||||
val refs = immutable.IndexedSeq.fill(nrOfInstances) {
|
||||
val name = "c" + childNameCounter.incrementAndGet
|
||||
val deploy = Deploy(config = ConfigFactory.empty(), routerConfig = routeeProps.routerConfig,
|
||||
scope = RemoteScope(nodeAddressIter.next))
|
||||
|
||||
// attachChild means that the provider will treat this call as if possibly done out of the wrong
|
||||
// context and use RepointableActorRef instead of LocalActorRef. Seems like a slightly sub-optimal
|
||||
// choice in a corner case (and hence not worth fixing).
|
||||
context.asInstanceOf[ActorCell].attachChild(routeeProps.withDeploy(deploy), name, systemService = false)
|
||||
}
|
||||
registerRoutees(refs)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -14,7 +14,7 @@ object RemoteDeployerSpec {
|
|||
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
|
||||
akka.actor.deployment {
|
||||
/service2 {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 3
|
||||
remote = "akka://sys@wallace:2552"
|
||||
dispatcher = mydispatcher
|
||||
|
|
@ -42,7 +42,7 @@ class RemoteDeployerSpec extends AkkaSpec(RemoteDeployerSpec.deployerConf) {
|
|||
Deploy(
|
||||
service,
|
||||
deployment.get.config,
|
||||
RoundRobinRouter(3),
|
||||
RoundRobinPool(3),
|
||||
RemoteScope(Address("akka", "sys", "wallace", 2552)),
|
||||
"mydispatcher")))
|
||||
}
|
||||
|
|
|
|||
|
|
@ -26,7 +26,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
}
|
||||
akka.actor.deployment {
|
||||
/remote-override {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 4
|
||||
}
|
||||
}""") {
|
||||
|
|
@ -40,12 +40,12 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
akka {
|
||||
actor.deployment {
|
||||
/blub {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 2
|
||||
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"]
|
||||
}
|
||||
/elastic-blub {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
resizer {
|
||||
lower-bound = 2
|
||||
upper-bound = 3
|
||||
|
|
@ -54,17 +54,17 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
}
|
||||
/remote-blub {
|
||||
remote = "akka.tcp://${sysName}@localhost:${port}"
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 2
|
||||
}
|
||||
/local-blub {
|
||||
remote = "akka://MasterRemoteRouterSpec"
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 2
|
||||
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"]
|
||||
}
|
||||
/local-blub2 {
|
||||
router = round-robin
|
||||
router = round-robin-pool
|
||||
nr-of-instances = 4
|
||||
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"]
|
||||
}
|
||||
|
|
@ -80,7 +80,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
|
||||
"deploy its children on remote host driven by configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2)), "blub")
|
||||
val router = masterSystem.actorOf(RoundRobinPool(2).props(Props[Echo]), "blub")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
|
|
@ -94,8 +94,8 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
|
||||
"deploy its children on remote host driven by programatic definition" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(new RemoteRouterConfig(RoundRobinRouter(2),
|
||||
Seq(Address("akka.tcp", sysName, "localhost", port)))), "blub2")
|
||||
val router = masterSystem.actorOf(new RemoteRouterConfig(RoundRobinPool(2),
|
||||
Seq(Address("akka.tcp", sysName, "localhost", port))).props(Props[Echo]), "blub2")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
|
|
@ -109,7 +109,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
|
||||
"deploy dynamic resizable number of children on remote host driven by configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(FromConfig), "elastic-blub")
|
||||
val router = masterSystem.actorOf(FromConfig.props(Props[Echo]), "elastic-blub")
|
||||
val replies = for (i ← 1 to 5000) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
|
|
@ -123,7 +123,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
|
||||
"deploy remote routers based on configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(FromConfig), "remote-blub")
|
||||
val router = masterSystem.actorOf(FromConfig.props(Props[Echo]), "remote-blub")
|
||||
router.path.address.toString must be(s"akka.tcp://${sysName}@localhost:${port}")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
|
|
@ -140,7 +140,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
|
||||
"deploy remote routers based on explicit deployment" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2))
|
||||
val router = masterSystem.actorOf(RoundRobinPool(2).props(Props[Echo])
|
||||
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "remote-blub2")
|
||||
router.path.address.toString must be(s"akka.tcp://${sysName}@localhost:${port}")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
|
|
@ -158,7 +158,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
|
||||
"let remote deployment be overridden by local configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2))
|
||||
val router = masterSystem.actorOf(RoundRobinPool(2).props(Props[Echo])
|
||||
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "local-blub")
|
||||
router.path.address.toString must be("akka://MasterRemoteRouterSpec")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
|
|
@ -176,7 +176,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
|
||||
"let remote deployment router be overridden by local configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2))
|
||||
val router = masterSystem.actorOf(RoundRobinPool(2).props(Props[Echo])
|
||||
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "local-blub2")
|
||||
router.path.address.toString must be(s"akka.tcp://${sysName}@localhost:${port}")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
|
|
@ -194,7 +194,7 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
|
||||
"let remote deployment be overridden by remote configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2))
|
||||
val router = masterSystem.actorOf(RoundRobinPool(2).props(Props[Echo])
|
||||
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "remote-override")
|
||||
router.path.address.toString must be(s"akka.tcp://${sysName}@localhost:${port}")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
|
|
@ -215,13 +215,13 @@ class RemoteRouterSpec extends AkkaSpec("""
|
|||
val escalator = OneForOneStrategy() {
|
||||
case e ⇒ probe.ref ! e; SupervisorStrategy.Escalate
|
||||
}
|
||||
val router = masterSystem.actorOf(Props.empty.withRouter(new RemoteRouterConfig(
|
||||
RoundRobinRouter(1, supervisorStrategy = escalator),
|
||||
Seq(Address("akka.tcp", sysName, "localhost", port)))), "blub3")
|
||||
val router = masterSystem.actorOf(new RemoteRouterConfig(
|
||||
RoundRobinPool(1, supervisorStrategy = escalator),
|
||||
Seq(Address("akka.tcp", sysName, "localhost", port))).props(Props.empty), "blub3")
|
||||
|
||||
router.tell(CurrentRoutees, probe.ref)
|
||||
router.tell(GetRoutees, probe.ref)
|
||||
EventFilter[ActorKilledException](occurrences = 1).intercept {
|
||||
probe.expectMsgType[RouterRoutees].routees.head ! Kill
|
||||
probe.expectMsgType[Routees].routees.head.send(Kill, testActor)
|
||||
}(masterSystem)
|
||||
probe.expectMsgType[ActorKilledException]
|
||||
}
|
||||
|
|
|
|||
|
|
@ -0,0 +1,232 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.remote.oldrouting
|
||||
|
||||
import akka.testkit._
|
||||
import akka.routing._
|
||||
import akka.actor._
|
||||
import akka.remote.routing._
|
||||
import com.typesafe.config._
|
||||
import akka.remote.RemoteScope
|
||||
|
||||
object RemoteRouterSpec {
|
||||
class Echo extends Actor {
|
||||
def receive = {
|
||||
case _ ⇒ sender ! self
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class RemoteRouterSpec extends AkkaSpec("""
|
||||
akka.actor.provider = "akka.remote.RemoteActorRefProvider"
|
||||
akka.remote.netty.tcp {
|
||||
hostname = localhost
|
||||
port = 0
|
||||
}
|
||||
akka.actor.deployment {
|
||||
/remote-override {
|
||||
router = round-robin
|
||||
nr-of-instances = 4
|
||||
}
|
||||
}""") {
|
||||
|
||||
import RemoteRouterSpec._
|
||||
|
||||
val port = system.asInstanceOf[ExtendedActorSystem].provider.getDefaultAddress.port.get
|
||||
val sysName = system.name
|
||||
val conf = ConfigFactory.parseString(
|
||||
s"""
|
||||
akka {
|
||||
actor.deployment {
|
||||
/blub {
|
||||
router = round-robin
|
||||
nr-of-instances = 2
|
||||
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"]
|
||||
}
|
||||
/elastic-blub {
|
||||
router = round-robin
|
||||
resizer {
|
||||
lower-bound = 2
|
||||
upper-bound = 3
|
||||
}
|
||||
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"]
|
||||
}
|
||||
/remote-blub {
|
||||
remote = "akka.tcp://${sysName}@localhost:${port}"
|
||||
router = round-robin
|
||||
nr-of-instances = 2
|
||||
}
|
||||
/local-blub {
|
||||
remote = "akka://MasterRemoteRouterSpec"
|
||||
router = round-robin
|
||||
nr-of-instances = 2
|
||||
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"]
|
||||
}
|
||||
/local-blub2 {
|
||||
router = round-robin
|
||||
nr-of-instances = 4
|
||||
target.nodes = ["akka.tcp://${sysName}@localhost:${port}"]
|
||||
}
|
||||
}
|
||||
}""").withFallback(system.settings.config)
|
||||
val masterSystem = ActorSystem("Master" + sysName, conf)
|
||||
|
||||
override def afterTermination() {
|
||||
shutdown(masterSystem)
|
||||
}
|
||||
|
||||
"A Remote Router" must {
|
||||
|
||||
"deploy its children on remote host driven by configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2)), "blub")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
}
|
||||
val children = replies.toSet
|
||||
children must have size 2
|
||||
children.map(_.parent) must have size 1
|
||||
children foreach (_.address.toString must be === s"akka.tcp://${sysName}@localhost:${port}")
|
||||
masterSystem.stop(router)
|
||||
}
|
||||
|
||||
"deploy its children on remote host driven by programatic definition" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(new RemoteRouterConfig(RoundRobinRouter(2),
|
||||
Seq(Address("akka.tcp", sysName, "localhost", port)))), "blub2")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
}
|
||||
val children = replies.toSet
|
||||
children must have size 2
|
||||
children.map(_.parent) must have size 1
|
||||
children foreach (_.address.toString must be === s"akka.tcp://${sysName}@localhost:${port}")
|
||||
masterSystem.stop(router)
|
||||
}
|
||||
|
||||
"deploy dynamic resizable number of children on remote host driven by configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(FromConfig), "elastic-blub")
|
||||
val replies = for (i ← 1 to 5000) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
}
|
||||
val children = replies.toSet
|
||||
children.size must be >= 2
|
||||
children.map(_.parent) must have size 1
|
||||
children foreach (_.address.toString must be === s"akka.tcp://${sysName}@localhost:${port}")
|
||||
masterSystem.stop(router)
|
||||
}
|
||||
|
||||
"deploy remote routers based on configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(FromConfig), "remote-blub")
|
||||
router.path.address.toString must be(s"akka.tcp://${sysName}@localhost:${port}")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
}
|
||||
val children = replies.toSet
|
||||
children must have size 2
|
||||
val parents = children.map(_.parent)
|
||||
parents must have size 1
|
||||
parents.head must be(router.path)
|
||||
children foreach (_.address.toString must be === s"akka.tcp://${sysName}@localhost:${port}")
|
||||
masterSystem.stop(router)
|
||||
}
|
||||
|
||||
"deploy remote routers based on explicit deployment" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2))
|
||||
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "remote-blub2")
|
||||
router.path.address.toString must be(s"akka.tcp://${sysName}@localhost:${port}")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
}
|
||||
val children = replies.toSet
|
||||
children must have size 2
|
||||
val parents = children.map(_.parent)
|
||||
parents must have size 1
|
||||
parents.head must be(router.path)
|
||||
children foreach (_.address.toString must be === s"akka.tcp://${sysName}@localhost:${port}")
|
||||
masterSystem.stop(router)
|
||||
}
|
||||
|
||||
"let remote deployment be overridden by local configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2))
|
||||
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "local-blub")
|
||||
router.path.address.toString must be("akka://MasterRemoteRouterSpec")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
}
|
||||
val children = replies.toSet
|
||||
children must have size 2
|
||||
val parents = children.map(_.parent)
|
||||
parents must have size 1
|
||||
parents.head.address must be(Address("akka.tcp", sysName, "localhost", port))
|
||||
children foreach (_.address.toString must be === s"akka.tcp://${sysName}@localhost:${port}")
|
||||
masterSystem.stop(router)
|
||||
}
|
||||
|
||||
"let remote deployment router be overridden by local configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2))
|
||||
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "local-blub2")
|
||||
router.path.address.toString must be(s"akka.tcp://${sysName}@localhost:${port}")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
}
|
||||
val children = replies.toSet
|
||||
children must have size 4
|
||||
val parents = children.map(_.parent)
|
||||
parents must have size 1
|
||||
parents.head must be(router.path)
|
||||
children foreach (_.address.toString must be === s"akka.tcp://${sysName}@localhost:${port}")
|
||||
masterSystem.stop(router)
|
||||
}
|
||||
|
||||
"let remote deployment be overridden by remote configuration" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val router = masterSystem.actorOf(Props[Echo].withRouter(RoundRobinRouter(2))
|
||||
.withDeploy(Deploy(scope = RemoteScope(AddressFromURIString(s"akka.tcp://${sysName}@localhost:${port}")))), "remote-override")
|
||||
router.path.address.toString must be(s"akka.tcp://${sysName}@localhost:${port}")
|
||||
val replies = for (i ← 1 to 5) yield {
|
||||
router.tell("", probe.ref)
|
||||
probe.expectMsgType[ActorRef].path
|
||||
}
|
||||
val children = replies.toSet
|
||||
children must have size 4
|
||||
val parents = children.map(_.parent)
|
||||
parents must have size 1
|
||||
parents.head must be(router.path)
|
||||
children foreach (_.address.toString must be === s"akka.tcp://${sysName}@localhost:${port}")
|
||||
masterSystem.stop(router)
|
||||
}
|
||||
|
||||
"set supplied supervisorStrategy" in {
|
||||
val probe = TestProbe()(masterSystem)
|
||||
val escalator = OneForOneStrategy() {
|
||||
case e ⇒ probe.ref ! e; SupervisorStrategy.Escalate
|
||||
}
|
||||
val router = masterSystem.actorOf(Props.empty.withRouter(new RemoteRouterConfig(
|
||||
RoundRobinRouter(1, supervisorStrategy = escalator),
|
||||
Seq(Address("akka.tcp", sysName, "localhost", port)))), "blub3")
|
||||
|
||||
router.tell(CurrentRoutees, probe.ref)
|
||||
EventFilter[ActorKilledException](occurrences = 1).intercept {
|
||||
probe.expectMsgType[RouterRoutees].routees.head ! Kill
|
||||
}(masterSystem)
|
||||
probe.expectMsgType[ActorKilledException]
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -1,14 +1,19 @@
|
|||
package sample.cluster.factorial.japi;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.Props;
|
||||
import akka.event.Logging;
|
||||
import akka.event.LoggingAdapter;
|
||||
import akka.routing.FromConfig;
|
||||
import akka.cluster.routing.AdaptiveLoadBalancingRouter;
|
||||
import akka.cluster.routing.ClusterRouterConfig;
|
||||
import akka.cluster.routing.ClusterRouterSettings;
|
||||
import akka.cluster.routing.AdaptiveLoadBalancingPool;
|
||||
import akka.cluster.routing.AdaptiveLoadBalancingGroup;
|
||||
import akka.cluster.routing.ClusterRouterPool;
|
||||
import akka.cluster.routing.ClusterRouterGroup;
|
||||
import akka.cluster.routing.ClusterRouterGroupSettings;
|
||||
import akka.cluster.routing.ClusterRouterPoolSettings;
|
||||
import akka.cluster.routing.HeapMetricsSelector;
|
||||
import akka.cluster.routing.SystemLoadAverageMetricsSelector;
|
||||
|
||||
|
|
@ -20,7 +25,7 @@ public class FactorialFrontend extends UntypedActor {
|
|||
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
||||
|
||||
ActorRef backend = getContext().actorOf(
|
||||
Props.empty().withRouter(FromConfig.getInstance()),
|
||||
FromConfig.getInstance().props(),
|
||||
"factorialBackendRouter");
|
||||
|
||||
public FactorialFrontend(int upToN, boolean repeat) {
|
||||
|
|
@ -66,10 +71,10 @@ abstract class FactorialFrontend2 extends UntypedActor {
|
|||
boolean allowLocalRoutees = true;
|
||||
String useRole = "backend";
|
||||
ActorRef backend = getContext().actorOf(
|
||||
Props.empty().withRouter(new ClusterRouterConfig(
|
||||
new AdaptiveLoadBalancingRouter(HeapMetricsSelector.getInstance(), 0),
|
||||
new ClusterRouterSettings(
|
||||
totalInstances, routeesPath, allowLocalRoutees, useRole))),
|
||||
new ClusterRouterGroup(
|
||||
new AdaptiveLoadBalancingGroup(HeapMetricsSelector.getInstance(), Collections.<String>emptyList()),
|
||||
new ClusterRouterGroupSettings(
|
||||
totalInstances, routeesPath, allowLocalRoutees, useRole)).props(),
|
||||
"factorialBackendRouter2");
|
||||
//#router-lookup-in-code
|
||||
}
|
||||
|
|
@ -82,11 +87,12 @@ abstract class FactorialFrontend3 extends UntypedActor {
|
|||
boolean allowLocalRoutees = false;
|
||||
String useRole = "backend";
|
||||
ActorRef backend = getContext().actorOf(
|
||||
Props.create(FactorialBackend.class).withRouter(new ClusterRouterConfig(
|
||||
new AdaptiveLoadBalancingRouter(
|
||||
new ClusterRouterPool(
|
||||
new AdaptiveLoadBalancingPool(
|
||||
SystemLoadAverageMetricsSelector.getInstance(), 0),
|
||||
new ClusterRouterSettings(
|
||||
totalInstances, maxInstancesPerNode, allowLocalRoutees, useRole))),
|
||||
new ClusterRouterPoolSettings(
|
||||
totalInstances, maxInstancesPerNode, allowLocalRoutees, useRole)).
|
||||
props(Props.create(FactorialBackend.class)),
|
||||
"factorialBackendRouter3");
|
||||
//#router-deploy-in-code
|
||||
}
|
||||
|
|
@ -1,13 +1,18 @@
|
|||
package sample.cluster.stats.japi;
|
||||
|
||||
import java.util.Collections;
|
||||
|
||||
import sample.cluster.stats.japi.StatsMessages.StatsJob;
|
||||
//#imports
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.Props;
|
||||
import akka.actor.UntypedActor;
|
||||
import akka.cluster.routing.ClusterRouterConfig;
|
||||
import akka.cluster.routing.ClusterRouterSettings;
|
||||
import akka.routing.ConsistentHashingRouter;
|
||||
import akka.cluster.routing.ClusterRouterGroup;
|
||||
import akka.cluster.routing.ClusterRouterPool;
|
||||
import akka.cluster.routing.ClusterRouterGroupSettings;
|
||||
import akka.cluster.routing.ClusterRouterPoolSettings;
|
||||
import akka.routing.ConsistentHashingGroup;
|
||||
import akka.routing.ConsistentHashingPool;
|
||||
import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope;
|
||||
import akka.routing.FromConfig;
|
||||
//#imports
|
||||
|
|
@ -19,7 +24,7 @@ public class StatsService extends UntypedActor {
|
|||
// have a router with only lookup of routees you can use Props.empty()
|
||||
// instead of Props.create(StatsWorker.class).
|
||||
ActorRef workerRouter = getContext().actorOf(
|
||||
Props.create(StatsWorker.class).withRouter(FromConfig.getInstance()),
|
||||
FromConfig.getInstance().props(Props.create(StatsWorker.class)),
|
||||
"workerRouter");
|
||||
|
||||
@Override
|
||||
|
|
@ -59,9 +64,9 @@ abstract class StatsService2 extends UntypedActor {
|
|||
boolean allowLocalRoutees = true;
|
||||
String useRole = "compute";
|
||||
ActorRef workerRouter = getContext().actorOf(
|
||||
Props.empty().withRouter(new ClusterRouterConfig(
|
||||
new ConsistentHashingRouter(0), new ClusterRouterSettings(
|
||||
totalInstances, routeesPath, allowLocalRoutees, useRole))),
|
||||
new ClusterRouterGroup(
|
||||
new ConsistentHashingGroup(Collections.<String>emptyList()), new ClusterRouterGroupSettings(
|
||||
totalInstances, routeesPath, allowLocalRoutees, useRole)).props(),
|
||||
"workerRouter2");
|
||||
//#router-lookup-in-code
|
||||
}
|
||||
|
|
@ -74,9 +79,10 @@ abstract class StatsService3 extends UntypedActor {
|
|||
boolean allowLocalRoutees = false;
|
||||
String useRole = "compute";
|
||||
ActorRef workerRouter = getContext().actorOf(
|
||||
Props.create(StatsWorker.class).withRouter(new ClusterRouterConfig(
|
||||
new ConsistentHashingRouter(0), new ClusterRouterSettings(
|
||||
totalInstances, maxInstancesPerNode, allowLocalRoutees, useRole))),
|
||||
new ClusterRouterPool(
|
||||
new ConsistentHashingPool(0), new ClusterRouterPoolSettings(
|
||||
totalInstances, maxInstancesPerNode, allowLocalRoutees, useRole)).
|
||||
props(Props.create(StatsWorker.class)),
|
||||
"workerRouter3");
|
||||
//#router-deploy-in-code
|
||||
}
|
||||
|
|
|
|||
|
|
@ -24,7 +24,7 @@ akka {
|
|||
# //#config-router-lookup
|
||||
akka.actor.deployment {
|
||||
/statsService/workerRouter {
|
||||
router = consistent-hashing
|
||||
router = consistent-hashing-group
|
||||
nr-of-instances = 100
|
||||
cluster {
|
||||
enabled = on
|
||||
|
|
@ -39,7 +39,7 @@ akka.actor.deployment {
|
|||
# //#config-router-deploy
|
||||
akka.actor.deployment {
|
||||
/singleton/statsService/workerRouter {
|
||||
router = consistent-hashing
|
||||
router = consistent-hashing-pool
|
||||
nr-of-instances = 100
|
||||
cluster {
|
||||
enabled = on
|
||||
|
|
@ -54,7 +54,7 @@ akka.actor.deployment {
|
|||
# //#adaptive-router
|
||||
akka.actor.deployment {
|
||||
/factorialFrontend/factorialBackendRouter = {
|
||||
router = adaptive
|
||||
router = adaptive-group
|
||||
# metrics-selector = heap
|
||||
# metrics-selector = load
|
||||
# metrics-selector = cpu
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ object FactorialFrontend {
|
|||
//#frontend
|
||||
class FactorialFrontend(upToN: Int, repeat: Boolean) extends Actor with ActorLogging {
|
||||
|
||||
val backend = context.actorOf(Props.empty.withRouter(FromConfig),
|
||||
val backend = context.actorOf(FromConfig.props(),
|
||||
name = "factorialBackendRouter")
|
||||
|
||||
override def preStart(): Unit = sendJobs()
|
||||
|
|
@ -141,16 +141,16 @@ class MetricsListener extends Actor with ActorLogging {
|
|||
// not used, only for documentation
|
||||
abstract class FactorialFrontend2 extends Actor {
|
||||
//#router-lookup-in-code
|
||||
import akka.cluster.routing.ClusterRouterConfig
|
||||
import akka.cluster.routing.ClusterRouterSettings
|
||||
import akka.cluster.routing.AdaptiveLoadBalancingRouter
|
||||
import akka.cluster.routing.ClusterRouterGroup
|
||||
import akka.cluster.routing.ClusterRouterGroupSettings
|
||||
import akka.cluster.routing.AdaptiveLoadBalancingGroup
|
||||
import akka.cluster.routing.HeapMetricsSelector
|
||||
|
||||
val backend = context.actorOf(Props.empty.withRouter(
|
||||
ClusterRouterConfig(AdaptiveLoadBalancingRouter(HeapMetricsSelector),
|
||||
ClusterRouterSettings(
|
||||
val backend = context.actorOf(
|
||||
ClusterRouterGroup(AdaptiveLoadBalancingGroup(HeapMetricsSelector),
|
||||
ClusterRouterGroupSettings(
|
||||
totalInstances = 100, routeesPath = "/user/factorialBackend",
|
||||
allowLocalRoutees = true, useRole = Some("backend")))),
|
||||
allowLocalRoutees = true, useRole = Some("backend"))).props(),
|
||||
name = "factorialBackendRouter2")
|
||||
//#router-lookup-in-code
|
||||
}
|
||||
|
|
@ -158,16 +158,16 @@ abstract class FactorialFrontend2 extends Actor {
|
|||
// not used, only for documentation
|
||||
abstract class FactorialFrontend3 extends Actor {
|
||||
//#router-deploy-in-code
|
||||
import akka.cluster.routing.ClusterRouterConfig
|
||||
import akka.cluster.routing.ClusterRouterSettings
|
||||
import akka.cluster.routing.AdaptiveLoadBalancingRouter
|
||||
import akka.cluster.routing.ClusterRouterPool
|
||||
import akka.cluster.routing.ClusterRouterPoolSettings
|
||||
import akka.cluster.routing.AdaptiveLoadBalancingPool
|
||||
import akka.cluster.routing.SystemLoadAverageMetricsSelector
|
||||
|
||||
val backend = context.actorOf(Props[FactorialBackend].withRouter(
|
||||
ClusterRouterConfig(AdaptiveLoadBalancingRouter(
|
||||
SystemLoadAverageMetricsSelector), ClusterRouterSettings(
|
||||
val backend = context.actorOf(
|
||||
ClusterRouterPool(AdaptiveLoadBalancingPool(
|
||||
SystemLoadAverageMetricsSelector), ClusterRouterPoolSettings(
|
||||
totalInstances = 100, maxInstancesPerNode = 3,
|
||||
allowLocalRoutees = false, useRole = Some("backend")))),
|
||||
allowLocalRoutees = false, useRole = Some("backend"))).props(Props[FactorialBackend]),
|
||||
name = "factorialBackendRouter3")
|
||||
//#router-deploy-in-code
|
||||
}
|
||||
|
|
@ -37,7 +37,7 @@ class StatsService extends Actor {
|
|||
// This router is used both with lookup and deploy of routees. If you
|
||||
// have a router with only lookup of routees you can use Props.empty
|
||||
// instead of Props[StatsWorker.class].
|
||||
val workerRouter = context.actorOf(Props[StatsWorker].withRouter(FromConfig),
|
||||
val workerRouter = context.actorOf(FromConfig.props(Props[StatsWorker]),
|
||||
name = "workerRouter")
|
||||
|
||||
def receive = {
|
||||
|
|
@ -224,14 +224,14 @@ class StatsSampleClient(servicePath: String) extends Actor {
|
|||
// not used, only for documentation
|
||||
abstract class StatsService2 extends Actor {
|
||||
//#router-lookup-in-code
|
||||
import akka.cluster.routing.ClusterRouterConfig
|
||||
import akka.cluster.routing.ClusterRouterSettings
|
||||
import akka.routing.ConsistentHashingRouter
|
||||
import akka.cluster.routing.ClusterRouterGroup
|
||||
import akka.cluster.routing.ClusterRouterGroupSettings
|
||||
import akka.routing.ConsistentHashingGroup
|
||||
|
||||
val workerRouter = context.actorOf(Props.empty.withRouter(
|
||||
ClusterRouterConfig(ConsistentHashingRouter(), ClusterRouterSettings(
|
||||
val workerRouter = context.actorOf(
|
||||
ClusterRouterGroup(ConsistentHashingGroup(Nil), ClusterRouterGroupSettings(
|
||||
totalInstances = 100, routeesPath = "/user/statsWorker",
|
||||
allowLocalRoutees = true, useRole = Some("compute")))),
|
||||
allowLocalRoutees = true, useRole = Some("compute"))).props(),
|
||||
name = "workerRouter2")
|
||||
//#router-lookup-in-code
|
||||
}
|
||||
|
|
@ -239,14 +239,14 @@ abstract class StatsService2 extends Actor {
|
|||
// not used, only for documentation
|
||||
abstract class StatsService3 extends Actor {
|
||||
//#router-deploy-in-code
|
||||
import akka.cluster.routing.ClusterRouterConfig
|
||||
import akka.cluster.routing.ClusterRouterSettings
|
||||
import akka.routing.ConsistentHashingRouter
|
||||
import akka.cluster.routing.ClusterRouterPool
|
||||
import akka.cluster.routing.ClusterRouterPoolSettings
|
||||
import akka.routing.ConsistentHashingPool
|
||||
|
||||
val workerRouter = context.actorOf(Props[StatsWorker].withRouter(
|
||||
ClusterRouterConfig(ConsistentHashingRouter(), ClusterRouterSettings(
|
||||
val workerRouter = context.actorOf(
|
||||
ClusterRouterPool(ConsistentHashingPool(0), ClusterRouterPoolSettings(
|
||||
totalInstances = 100, maxInstancesPerNode = 3,
|
||||
allowLocalRoutees = false, useRole = None))),
|
||||
allowLocalRoutees = false, useRole = None)).props(Props[StatsWorker]),
|
||||
name = "workerRouter3")
|
||||
//#router-deploy-in-code
|
||||
}
|
||||
|
|
|
|||
|
|
@ -37,7 +37,7 @@ object StatsSampleSingleMasterSpecConfig extends MultiNodeConfig {
|
|||
#//#router-deploy-config
|
||||
akka.actor.deployment {
|
||||
/singleton/statsService/workerRouter {
|
||||
router = consistent-hashing
|
||||
router = consistent-hashing-pool
|
||||
nr-of-instances = 100
|
||||
cluster {
|
||||
enabled = on
|
||||
|
|
|
|||
Some files were not shown because too many files have changed in this diff Show more
Loading…
Add table
Add a link
Reference in a new issue