Added BroadcastRouter which broadcasts all messages to all the connections it manages, also added tests.

Signed-off-by: Jonas Bonér <jonas@jonasboner.com>
This commit is contained in:
Jonas Bonér 2011-11-24 16:35:37 +01:00
parent 1bf5abb29a
commit ef6c8370cd
4 changed files with 132 additions and 22 deletions

View file

@ -400,4 +400,79 @@ class RoutingSpec extends AkkaSpec {
}
})
}
"broadcast router" must {
"be started when constructed" in {
val actor1 = actorOf[TestActor]
val props = RoutedProps(routerFactory = () new BroadcastRouter, connectionManager = new LocalConnectionManager(List(actor1)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
actor.isShutdown must be(false)
}
"broadcast message using !" in {
val doneLatch = new CountDownLatch(2)
val counter1 = new AtomicInteger
val connection1 = actorOf(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counter1.addAndGet(msg)
}
})
val counter2 = new AtomicInteger
val connection2 = actorOf(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg)
}
})
val props = RoutedProps(routerFactory = () new BroadcastRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
actor ! 1
actor ! "end"
doneLatch.await(5, TimeUnit.SECONDS) must be(true)
counter1.get must be(1)
counter2.get must be(1)
}
"broadcast message using ?" in {
val doneLatch = new CountDownLatch(2)
val counter1 = new AtomicInteger
val connection1 = actorOf(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int
counter1.addAndGet(msg)
sender ! "ack"
}
})
val counter2 = new AtomicInteger
val connection2 = actorOf(new Actor {
def receive = {
case "end" doneLatch.countDown()
case msg: Int counter2.addAndGet(msg)
}
})
val props = RoutedProps(routerFactory = () new BroadcastRouter, connectionManager = new LocalConnectionManager(List(connection1, connection2)))
val actor = new RoutedActorRef(app, props, app.guardian, "foo")
actor ? 1
actor ! "end"
doneLatch.await(5, TimeUnit.SECONDS) must be(true)
counter1.get must be(1)
counter2.get must be(1)
}
}
}

View file

@ -16,7 +16,7 @@ import akka.actor.Timeout.intToTimeout
import akka.config.ConfigurationException
import akka.dispatch.{ SystemMessage, Supervise, Promise, MessageDispatcher, Future, DefaultPromise }
import akka.event.{ Logging, DeathWatch, ActorClassification }
import akka.routing.{ ScatterGatherFirstCompletedRouter, Routing, RouterType, Router, RoutedProps, RoutedActorRef, RoundRobinRouter, RandomRouter, LocalConnectionManager, DirectRouter }
import akka.routing.{ ScatterGatherFirstCompletedRouter, Routing, RouterType, Router, RoutedProps, RoutedActorRef, RoundRobinRouter, RandomRouter, LocalConnectionManager, DirectRouter, BroadcastRouter }
import akka.util.Helpers
import akka.AkkaException
@ -211,13 +211,14 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
// create a routed actor ref
case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, DeploymentConfig.LocalScope))
implicit val dispatcher = if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher
implicit val timeout = app.AkkaConfig.ActorTimeout
val routerFactory: () Router = DeploymentConfig.routerTypeFor(routerType) match {
case RouterType.Direct () new DirectRouter
case RouterType.Random () new RandomRouter
case RouterType.RoundRobin () new RoundRobinRouter
case RouterType.ScatterGather () new ScatterGatherFirstCompletedRouter()(
if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher, app.AkkaConfig.ActorTimeout)
case RouterType.Direct () new DirectRouter
case RouterType.Random () new RandomRouter
case RouterType.Broadcast () new BroadcastRouter
case RouterType.RoundRobin () new RoundRobinRouter
case RouterType.ScatterGather () new ScatterGatherFirstCompletedRouter
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
case RouterType.LeastMessages sys.error("Router LeastMessages not supported yet")

View file

@ -41,6 +41,11 @@ object RouterType {
*/
object ScatterGather extends RouterType
/**
* A RouterType that broadcasts the messages to all connections.
*/
object Broadcast extends RouterType
/**
* A RouterType that selects the connection based on the least amount of cpu usage
*/
@ -67,15 +72,14 @@ object RouterType {
* Routed ActorRef configuration object, this is thread safe and fully sharable.
*/
case class RoutedProps private[akka] (
routerFactory: () Router = RoutedProps.defaultRouterFactory,
connectionManager: ConnectionManager = new LocalConnectionManager(List()),
routerFactory: () Router,
connectionManager: ConnectionManager,
timeout: Timeout = RoutedProps.defaultTimeout,
localOnly: Boolean = RoutedProps.defaultLocalOnly) {
}
object RoutedProps {
final val defaultTimeout = Timeout(Duration.MinusInf)
final val defaultRouterFactory = () new RoundRobinRouter
final val defaultLocalOnly = false
}
@ -257,12 +261,41 @@ trait BasicRouter extends Router {
private def throwNoConnectionsError = throw new RoutingException("No replica connections for router")
}
/**
* A Router that uses broadcasts a message to all its connections.
*/
class BroadcastRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter with Serializable {
override def route(message: Any)(implicit sender: ActorRef) = {
connectionManager.connections.iterable foreach { connection
try {
connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
} catch {
case e: Exception
connectionManager.remove(connection)
throw e
}
}
}
//protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G] =
override def route[T](message: Any, timeout: Timeout): Future[T] = {
import Future._
implicit val t = timeout
val futures = connectionManager.connections.iterable map { connection
connection.?(message, timeout).asInstanceOf[Future[T]]
}
Future.firstCompletedOf(futures)
}
protected def next: Option[ActorRef] = None
}
/**
* A DirectRouter a Router that only has a single connected actorRef and forwards all request to that actorRef.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class DirectRouter extends BasicRouter {
class DirectRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
private val state = new AtomicReference[DirectRouterState]
@ -304,7 +337,7 @@ class DirectRouter extends BasicRouter {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RandomRouter extends BasicRouter {
class RandomRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
import java.security.SecureRandom
private val state = new AtomicReference[RandomRouterState]
@ -346,7 +379,7 @@ class RandomRouter extends BasicRouter {
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RoundRobinRouter extends BasicRouter {
class RoundRobinRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
private val state = new AtomicReference[RoundRobinState]
@ -437,7 +470,7 @@ trait ScatterGatherRouter extends BasicRouter with Serializable {
* (wrapped into {@link Routing.Broadcast} and sent with "?" method). For the messages sent in a fire-forget
* mode, the router would behave as {@link RoundRobinRouter}
*/
class ScatterGatherFirstCompletedRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends RoundRobinRouter with ScatterGatherRouter {
class ScatterGatherFirstCompletedRouter(implicit dispatcher: MessageDispatcher, timeout: Timeout) extends RoundRobinRouter with ScatterGatherRouter {
protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G] = Future.firstCompletedOf(results)
}

View file

@ -68,14 +68,6 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
deployer.lookupDeploymentFor(path.toString) match {
case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, DeploymentConfig.RemoteScope(remoteAddresses)))
// FIXME move to AccrualFailureDetector as soon as we have the Gossiper up and running and remove the option to select impl in the akka.conf file since we only have one
// val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match {
// case FailureDetectorType.NoOp new NoOpFailureDetector
// case FailureDetectorType.RemoveConnectionOnFirstFailure new RemoveConnectionOnFirstFailureFailureDetector
// case FailureDetectorType.BannagePeriod(timeToBan) new BannagePeriodFailureDetector(timeToBan)
// case FailureDetectorType.Custom(implClass) FailureDetector.createCustomFailureDetector(implClass)
// }
def isReplicaNode: Boolean = remoteAddresses exists { _ == app.address }
//app.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(app.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
@ -85,6 +77,9 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
local.actorOf(props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create
} else {
implicit val dispatcher = if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher
implicit val timeout = app.AkkaConfig.ActorTimeout
// we are on the single "reference" node uses the remote actors on the replica nodes
val routerFactory: () Router = DeploymentConfig.routerTypeFor(routerType) match {
case RouterType.Direct
@ -93,6 +88,12 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
.format(name, remoteAddresses.mkString(", ")))
() new DirectRouter
case RouterType.Broadcast
if (remoteAddresses.size != 1) throw new ConfigurationException(
"Actor [%s] configured with Broadcast router must have exactly 1 remote node configured. Found [%s]"
.format(name, remoteAddresses.mkString(", ")))
() new BroadcastRouter
case RouterType.Random
if (remoteAddresses.size < 1) throw new ConfigurationException(
"Actor [%s] configured with Random router must have at least 1 remote node configured. Found [%s]"