Added Routing.Broadcast message and handling to be able to broadcast a message to all the actors a load-balancer represents

This commit is contained in:
Jonas Bonér 2011-04-01 15:33:46 +02:00
parent 384332da39
commit d97b8fbd9c
5 changed files with 49 additions and 44 deletions

1
.gitignore vendored
View file

@ -45,3 +45,4 @@ run-codefellow
multiverse.log multiverse.log
.eprj .eprj
.*.swp .*.swp
akka-tutorials/akka-tutorial-pi-sbt/project/boot/

View file

@ -10,12 +10,14 @@ import scala.collection.JavaConversions._
/** /**
* An Iterator that is either always empty or yields an infinite number of Ts. * An Iterator that is either always empty or yields an infinite number of Ts.
*/ */
trait InfiniteIterator[T] extends Iterator[T] trait InfiniteIterator[T] extends Iterator[T] {
val items: Seq[T]
}
/** /**
* CyclicIterator is a round-robin style InfiniteIterator that cycles the supplied List. * CyclicIterator is a round-robin style InfiniteIterator that cycles the supplied List.
*/ */
case class CyclicIterator[T](items: Seq[T]) extends InfiniteIterator[T] { case class CyclicIterator[T](val items: Seq[T]) extends InfiniteIterator[T] {
def this(items: java.util.List[T]) = this(items.toSeq) def this(items: java.util.List[T]) = this(items.toSeq)
@volatile private[this] var current: Seq[T] = items @volatile private[this] var current: Seq[T] = items
@ -29,14 +31,13 @@ case class CyclicIterator[T](items: Seq[T]) extends InfiniteIterator[T] {
} }
override def exists(f: T => Boolean): Boolean = items.exists(f) override def exists(f: T => Boolean): Boolean = items.exists(f)
} }
/** /**
* This InfiniteIterator always returns the Actor that has the currently smallest mailbox * This InfiniteIterator always returns the Actor that has the currently smallest mailbox
* useful for work-stealing. * useful for work-stealing.
*/ */
case class SmallestMailboxFirstIterator(items : Seq[ActorRef]) extends InfiniteIterator[ActorRef] { case class SmallestMailboxFirstIterator(val items : Seq[ActorRef]) extends InfiniteIterator[ActorRef] {
def this(items: java.util.List[ActorRef]) = this(items.toSeq) def this(items: java.util.List[ActorRef]) = this(items.toSeq)
def hasNext = items != Nil def hasNext = items != Nil

View file

@ -15,7 +15,11 @@ trait Dispatcher { this: Actor =>
protected def routes: PartialFunction[Any, ActorRef] protected def routes: PartialFunction[Any, ActorRef]
protected def broadcast(message: Any) {}
protected def dispatch: Receive = { protected def dispatch: Receive = {
case Routing.Broadcast(message) =>
broadcast(message)
case a if routes.isDefinedAt(a) => case a if routes.isDefinedAt(a) =>
if (isSenderDefined) routes(a).forward(transform(a))(someSelf) if (isSenderDefined) routes(a).forward(transform(a))(someSelf)
else routes(a).!(transform(a))(None) else routes(a).!(transform(a))(None)
@ -34,14 +38,19 @@ abstract class UntypedDispatcher extends UntypedActor {
protected def route(msg: Any): ActorRef protected def route(msg: Any): ActorRef
protected def broadcast(message: Any) {}
private def isSenderDefined = self.senderFuture.isDefined || self.sender.isDefined private def isSenderDefined = self.senderFuture.isDefined || self.sender.isDefined
@throws(classOf[Exception]) @throws(classOf[Exception])
def onReceive(msg: Any): Unit = { def onReceive(msg: Any): Unit = {
val r = route(msg) if (msg.isInstanceOf[Routing.Broadcast]) broadcast(msg.asInstanceOf[Routing.Broadcast].message)
if (r eq null) throw new IllegalStateException("No route for " + msg + " defined!") else {
if (isSenderDefined) r.forward(transform(msg))(someSelf) val r = route(msg)
else r.!(transform(msg))(None) if (r eq null) throw new IllegalStateException("No route for " + msg + " defined!")
if (isSenderDefined) r.forward(transform(msg))(someSelf)
else r.!(transform(msg))(None)
}
} }
} }
@ -52,7 +61,11 @@ abstract class UntypedDispatcher extends UntypedActor {
trait LoadBalancer extends Dispatcher { self: Actor => trait LoadBalancer extends Dispatcher { self: Actor =>
protected def seq: InfiniteIterator[ActorRef] protected def seq: InfiniteIterator[ActorRef]
protected def routes = { case x if seq.hasNext => seq.next } protected def routes = {
case x if seq.hasNext => seq.next
}
override def broadcast(message: Any) = seq.items.foreach(_ ! message)
override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) ) override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) )
} }
@ -68,5 +81,7 @@ abstract class UntypedLoadBalancer extends UntypedDispatcher {
if (seq.hasNext) seq.next if (seq.hasNext) seq.next
else null else null
override def broadcast(message: Any) = seq.items.foreach(_ ! message)
override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) ) override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) )
} }

View file

@ -9,6 +9,9 @@ import akka.actor.Actor._
object Routing { object Routing {
sealed trait RoutingMessage
case class Broadcast(message: Any) extends RoutingMessage
type PF[A, B] = PartialFunction[A, B] type PF[A, B] = PartialFunction[A, B]
/** /**
@ -31,7 +34,7 @@ object Routing {
/** /**
* Creates a LoadBalancer from the thunk-supplied InfiniteIterator. * Creates a LoadBalancer from the thunk-supplied InfiniteIterator.
*/ */
def loadBalancerActor(actors: => InfiniteIterator[ActorRef]): ActorRef = def loadBalancerActor(actors: => InfiniteIterator[ActorRef]): ActorRef =
actorOf(new Actor with LoadBalancer { actorOf(new Actor with LoadBalancer {
val seq = actors val seq = actors
}).start }).start
@ -39,7 +42,7 @@ object Routing {
/** /**
* Creates a Dispatcher given a routing and a message-transforming function. * Creates a Dispatcher given a routing and a message-transforming function.
*/ */
def dispatcherActor(routing: PF[Any, ActorRef], msgTransformer: (Any) => Any): ActorRef = def dispatcherActor(routing: PF[Any, ActorRef], msgTransformer: (Any) => Any): ActorRef =
actorOf(new Actor with Dispatcher { actorOf(new Actor with Dispatcher {
override def transform(msg: Any) = msgTransformer(msg) override def transform(msg: Any) = msgTransformer(msg)
def routes = routing def routes = routing
@ -48,7 +51,7 @@ object Routing {
/** /**
* Creates a Dispatcher given a routing. * Creates a Dispatcher given a routing.
*/ */
def dispatcherActor(routing: PF[Any, ActorRef]): ActorRef = actorOf(new Actor with Dispatcher { def dispatcherActor(routing: PF[Any, ActorRef]): ActorRef = actorOf(new Actor with Dispatcher {
def routes = routing def routes = routing
}).start }).start

View file

@ -4,9 +4,10 @@
package akka.tutorial.sbt.pi package akka.tutorial.sbt.pi
import akka.actor.{Actor, ActorRef} import akka.actor.{Actor, ActorRef, PoisonPill}
import Actor._ import Actor._
import akka.routing.{Routing, CyclicIterator} import akka.routing.{Routing, CyclicIterator}
import Routing._
import akka.event.EventHandler import akka.event.EventHandler
import akka.dispatch.Dispatchers import akka.dispatch.Dispatchers
@ -17,38 +18,21 @@ object Main extends App {
Pi.calculate Pi.calculate
} }
/*
Pi estimate: 3.1415926435897883
=== 8 workers (with custom dispatcher 4/4)
Calculation time: 5163 millis
=== 8 workers (with default dispatcher)
Calculation time: 6789 millis
=== 4 workers
Calculation time: 5438 millis
=== 2 workers
Calculation time: 6002 millis
=== 1 workers
Calculation time: 8173 millis
*/
object Pi { object Pi {
val nrOfWorkers = 4 val nrOfWorkers = 4
val nrOfMessages = 10000 val nrOfMessages = 10000
val nrOfElements = 10000 val nrOfElements = 10000
// ===== Messages ===== // ===== Messages =====
sealed trait PiMessage sealed trait PiMessage
case class Calculate(nrOfMessages: Int, nrOfElements: Int) extends PiMessage
case class Work(arg: Int, fun: (Int) => Double) extends PiMessage case class Work(arg: Int, fun: (Int) => Double) extends PiMessage
case class Result(value: Double) extends PiMessage case class Result(value: Double) extends PiMessage
// ===== Worker ===== // ===== Worker =====
class Worker extends Actor { class Worker extends Actor {
def receive = { def receive = {
case Work(arg, fun) => self.reply(Result(fun(arg))) case Work(arg, fun) => self reply Result(fun(arg))
} }
} }
@ -68,19 +52,20 @@ object Pi {
// wrap them with a load-balancing router // wrap them with a load-balancing router
val router = Routing.loadBalancerActor(CyclicIterator(workers)).start val router = Routing.loadBalancerActor(CyclicIterator(workers)).start
// define the work
val algorithm = (i: Int) => {
val range = (i * nrOfElements) to ((i + 1) * nrOfElements - 1)
val results = for (j <- range) yield (4 * math.pow(-1, j) / (2 * j + 1))
results.sum
}
def receive = { def receive = {
case Calculate(nrOfMessages, nrOfElements) => case Calculate(nrOfMessages, nrOfElements) =>
// define the work
val fun = (i: Int) => {
val range = (i * nrOfElements) to ((i + 1) * nrOfElements - 1)
val results = for (j <- range) yield (4 * math.pow(-1, j) / (2 * j + 1))
results.sum
}
// schedule work // schedule work
for (arg <- 0 until nrOfMessages) router ! Work(arg, fun) for (arg <- 0 until nrOfMessages) router ! Work(arg, algorithm)
// send a PoisonPill to all workers telling them to shut down themselves // send a PoisonPill to all workers telling them to shut down themselves
router broadcast PoisonPill router ! Broadcast(PoisonPill)
case Result(value) => case Result(value) =>
pi += value pi += value
@ -92,12 +77,12 @@ object Pi {
override def postStop = { override def postStop = {
EventHandler.info(this, "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis".format(pi, (now - start))) EventHandler.info(this, "\n\tPi estimate: \t\t%s\n\tCalculation time: \t%s millis".format(pi, (now - start)))
latch.nrOfResultsDown latch.countDown
} }
} }
def calculate = { def calculate = {
val latch = new nrOfResultsDownLatch(1) val latch = new CountDownLatch(1)
// create the master // create the master
val master = actorOf(new Master(latch)).start val master = actorOf(new Master(latch)).start