!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:
Patrik Nordwall 2013-09-19 08:00:05 +02:00
parent 81ca6fe8c8
commit ebadd567b2
104 changed files with 9671 additions and 5006 deletions

View file

@ -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
}
}

View 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")
}
}

View file

@ -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
}
}

View file

@ -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

View file

@ -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)
}
}

View file

@ -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) }
}

View file

@ -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`) () }
}
}

View file

@ -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
}