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:
parent
384332da39
commit
d97b8fbd9c
5 changed files with 49 additions and 44 deletions
1
.gitignore
vendored
1
.gitignore
vendored
|
|
@ -45,3 +45,4 @@ run-codefellow
|
||||||
multiverse.log
|
multiverse.log
|
||||||
.eprj
|
.eprj
|
||||||
.*.swp
|
.*.swp
|
||||||
|
akka-tutorials/akka-tutorial-pi-sbt/project/boot/
|
||||||
|
|
@ -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
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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,15 +38,20 @@ 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 = {
|
||||||
|
if (msg.isInstanceOf[Routing.Broadcast]) broadcast(msg.asInstanceOf[Routing.Broadcast].message)
|
||||||
|
else {
|
||||||
val r = route(msg)
|
val r = route(msg)
|
||||||
if (r eq null) throw new IllegalStateException("No route for " + msg + " defined!")
|
if (r eq null) throw new IllegalStateException("No route for " + msg + " defined!")
|
||||||
if (isSenderDefined) r.forward(transform(msg))(someSelf)
|
if (isSenderDefined) r.forward(transform(msg))(someSelf)
|
||||||
else r.!(transform(msg))(None)
|
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) )
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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]
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -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,24 +18,6 @@ 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
|
||||||
|
|
@ -42,13 +25,14 @@ object Pi {
|
||||||
|
|
||||||
// ===== 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
|
||||||
|
|
||||||
def receive = {
|
|
||||||
case Calculate(nrOfMessages, nrOfElements) =>
|
|
||||||
// define the work
|
// define the work
|
||||||
val fun = (i: Int) => {
|
val algorithm = (i: Int) => {
|
||||||
val range = (i * nrOfElements) to ((i + 1) * nrOfElements - 1)
|
val range = (i * nrOfElements) to ((i + 1) * nrOfElements - 1)
|
||||||
val results = for (j <- range) yield (4 * math.pow(-1, j) / (2 * j + 1))
|
val results = for (j <- range) yield (4 * math.pow(-1, j) / (2 * j + 1))
|
||||||
results.sum
|
results.sum
|
||||||
}
|
}
|
||||||
|
|
||||||
|
def receive = {
|
||||||
|
case Calculate(nrOfMessages, nrOfElements) =>
|
||||||
// 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
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue