2010-05-03 19:32:40 +02:00
|
|
|
/**
|
2011-07-14 16:03:08 +02:00
|
|
|
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
2010-05-03 19:32:40 +02:00
|
|
|
*/
|
|
|
|
|
|
2010-10-26 12:49:25 +02:00
|
|
|
package akka.routing
|
2010-02-13 21:45:35 +01:00
|
|
|
|
2011-07-28 15:48:03 +03:00
|
|
|
import akka.actor._
|
2011-08-27 08:10:25 +03:00
|
|
|
|
2011-11-25 14:49:09 +01:00
|
|
|
import akka.japi.Creator
|
2011-12-08 14:30:57 +01:00
|
|
|
import akka.util.ReflectiveAccess
|
|
|
|
|
import java.lang.reflect.InvocationTargetException
|
|
|
|
|
import akka.config.ConfigurationException
|
|
|
|
|
import akka.routing.Routing.Broadcast
|
|
|
|
|
import akka.actor.DeploymentConfig.Deploy
|
|
|
|
|
import java.util.concurrent.atomic.AtomicInteger
|
2011-08-27 08:10:25 +03:00
|
|
|
|
2011-11-11 20:05:53 +01:00
|
|
|
sealed trait RouterType
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Used for declarative configuration of Routing.
|
|
|
|
|
*
|
|
|
|
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
|
|
|
*/
|
|
|
|
|
object RouterType {
|
|
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
/**
|
|
|
|
|
* A RouterType that indicates no routing - i.e. direct message.
|
|
|
|
|
*/
|
|
|
|
|
object NoRouting extends RouterType
|
2011-11-11 20:05:53 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A RouterType that randomly selects a connection to send a message to.
|
|
|
|
|
*/
|
|
|
|
|
object Random extends RouterType
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A RouterType that selects the connection by using round robin.
|
|
|
|
|
*/
|
|
|
|
|
object RoundRobin extends RouterType
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A RouterType that selects the connection by using scatter gather.
|
|
|
|
|
*/
|
|
|
|
|
object ScatterGather extends RouterType
|
|
|
|
|
|
2011-11-24 16:35:37 +01:00
|
|
|
/**
|
|
|
|
|
* A RouterType that broadcasts the messages to all connections.
|
|
|
|
|
*/
|
|
|
|
|
object Broadcast extends RouterType
|
|
|
|
|
|
2011-11-11 20:05:53 +01:00
|
|
|
/**
|
|
|
|
|
* A RouterType that selects the connection based on the least amount of cpu usage
|
|
|
|
|
*/
|
|
|
|
|
object LeastCPU extends RouterType
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A RouterType that select the connection based on the least amount of ram used.
|
|
|
|
|
*/
|
|
|
|
|
object LeastRAM extends RouterType
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A RouterType that select the connection where the actor has the least amount of messages in its mailbox.
|
|
|
|
|
*/
|
|
|
|
|
object LeastMessages extends RouterType
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A user-defined custom RouterType.
|
|
|
|
|
*/
|
|
|
|
|
case class Custom(implClass: String) extends RouterType
|
2011-12-08 14:30:57 +01:00
|
|
|
|
2011-11-11 20:05:53 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Contains the configuration to create local and clustered routed actor references.
|
|
|
|
|
* Routed ActorRef configuration object, this is thread safe and fully sharable.
|
|
|
|
|
*/
|
|
|
|
|
case class RoutedProps private[akka] (
|
2011-11-24 16:35:37 +01:00
|
|
|
routerFactory: () ⇒ Router,
|
2011-12-08 14:30:57 +01:00
|
|
|
connectionManager: ConnectionManager) {
|
2011-11-25 14:49:09 +01:00
|
|
|
|
|
|
|
|
// Java API
|
2011-12-08 14:30:57 +01:00
|
|
|
def this(creator: Creator[Router], connectionManager: ConnectionManager) {
|
|
|
|
|
this(() ⇒ creator.create(), connectionManager)
|
2011-11-25 14:49:09 +01:00
|
|
|
}
|
2011-12-08 14:30:57 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
///**
|
|
|
|
|
// * The Router is responsible for sending a message to one (or more) of its connections. Connections are stored in the
|
|
|
|
|
// * {@link FailureDetector} and each Router should be linked to only one {@link FailureDetector}.
|
|
|
|
|
// *
|
|
|
|
|
// * @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
|
|
|
// */
|
|
|
|
|
//trait Router {
|
|
|
|
|
//
|
|
|
|
|
// /**
|
|
|
|
|
// * Initializes this Router with a given set of Connections. The Router can use this datastructure to ask for
|
|
|
|
|
// * the current connections, signal that there were problems with one of the connections and see if there have
|
|
|
|
|
// * been changes in the connections.
|
|
|
|
|
// *
|
|
|
|
|
// * This method is not threadsafe, and should only be called once
|
|
|
|
|
// *
|
|
|
|
|
// * JMM Guarantees:
|
|
|
|
|
// * This method guarantees that all changes made in this method, are visible before one of the routing methods is called.
|
|
|
|
|
// */
|
|
|
|
|
// def init(connectionManager: ConnectionManager)
|
|
|
|
|
//
|
|
|
|
|
// /**
|
|
|
|
|
// * Routes the message to one of the connections.
|
|
|
|
|
// *
|
|
|
|
|
// * @throws RoutingException if something goes wrong while routing the message
|
|
|
|
|
// */
|
|
|
|
|
// def route(message: Any)(implicit sender: ActorRef)
|
|
|
|
|
//
|
|
|
|
|
// /**
|
|
|
|
|
// * Routes the message using a timeout to one of the connections and returns a Future to synchronize on the
|
|
|
|
|
// * completion of the processing of the message.
|
|
|
|
|
// *
|
|
|
|
|
// * @throws RoutingExceptionif something goes wrong while routing the message.
|
|
|
|
|
// */
|
|
|
|
|
// def route[T](message: Any, timeout: Timeout): Future[T]
|
|
|
|
|
//}
|
|
|
|
|
//
|
|
|
|
|
///**
|
|
|
|
|
// * An {@link AkkaException} thrown when something goes wrong while routing a message
|
|
|
|
|
// */
|
|
|
|
|
//class RoutingException(message: String) extends AkkaException(message)
|
|
|
|
|
//
|
2011-11-25 14:49:09 +01:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
/**
|
|
|
|
|
* A RoutedActorRef is an ActorRef that has a set of connected ActorRef and it uses a Router to send a message to
|
|
|
|
|
* on (or more) of these actors.
|
|
|
|
|
*/
|
|
|
|
|
private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath)
|
|
|
|
|
extends LocalActorRef(
|
|
|
|
|
_system,
|
|
|
|
|
_props.copy(creator = _props.routerConfig),
|
|
|
|
|
_supervisor,
|
|
|
|
|
_path) {
|
|
|
|
|
|
|
|
|
|
val route: Routing.Route = _props.routerConfig.createRoute(_props.creator, actorContext)
|
|
|
|
|
|
|
|
|
|
override def !(message: Any)(implicit sender: ActorRef = null) {
|
|
|
|
|
route(message) match {
|
|
|
|
|
case null ⇒ super.!(message)(sender)
|
|
|
|
|
case ref: ActorRef ⇒ ref.!(message)(sender)
|
|
|
|
|
case refs: Traversable[ActorRef] ⇒ refs foreach (_.!(message)(sender))
|
|
|
|
|
}
|
|
|
|
|
}
|
2011-11-11 20:05:53 +01:00
|
|
|
}
|
|
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
trait RouterConfig extends Function0[Actor] {
|
|
|
|
|
def adaptFromDeploy(deploy: Option[Deploy]): RouterConfig
|
|
|
|
|
def createRoute(creator: () ⇒ Actor, actorContext: ActorContext): Routing.Route
|
2011-11-11 20:05:53 +01:00
|
|
|
}
|
|
|
|
|
|
2011-08-12 10:03:33 +03:00
|
|
|
/**
|
2011-12-08 14:30:57 +01:00
|
|
|
* Routing configuration that indicates no routing.
|
|
|
|
|
* Oxymoron style.
|
2011-08-12 10:03:33 +03:00
|
|
|
*/
|
2011-12-08 14:30:57 +01:00
|
|
|
case object NoRouting extends RouterConfig {
|
2011-08-12 10:03:33 +03:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
def adaptFromDeploy(deploy: Option[Deploy]) = null
|
2011-08-12 10:03:33 +03:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
def createRoute(creator: () ⇒ Actor, actorContext: ActorContext) = null
|
2011-08-12 10:03:33 +03:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
def apply(): Actor = null
|
2011-08-12 10:03:33 +03:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2011-12-08 14:30:57 +01:00
|
|
|
* The Router is responsible for sending a message to one (or more) of its connections. Connections are stored in the
|
|
|
|
|
* {@link FailureDetector} and each Router should be linked to only one {@link FailureDetector}.
|
|
|
|
|
*
|
|
|
|
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
2011-08-27 08:10:25 +03:00
|
|
|
*/
|
2011-12-08 14:30:57 +01:00
|
|
|
trait Router {
|
|
|
|
|
// TODO (HE): implement failure detection
|
|
|
|
|
}
|
2011-08-27 08:10:25 +03:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* A Helper class to create actor references that use routing.
|
|
|
|
|
*/
|
2011-07-28 15:48:03 +03:00
|
|
|
object Routing {
|
2011-10-07 15:22:36 +02:00
|
|
|
|
2011-07-28 15:48:03 +03:00
|
|
|
sealed trait RoutingMessage
|
2011-05-24 19:04:25 +02:00
|
|
|
|
2011-08-27 08:10:25 +03:00
|
|
|
/**
|
2011-10-07 19:42:10 +02:00
|
|
|
* Used to broadcast a message to all connections in a router. E.g. every connection gets the message
|
|
|
|
|
* regardless of their routing algorithm.
|
2011-08-27 08:10:25 +03:00
|
|
|
*/
|
2011-07-28 15:48:03 +03:00
|
|
|
case class Broadcast(message: Any) extends RoutingMessage
|
2011-08-17 10:21:27 +03:00
|
|
|
|
2011-10-11 11:18:47 +02:00
|
|
|
def createCustomRouter(implClass: String): Router = {
|
2011-12-08 14:30:57 +01:00
|
|
|
ReflectiveAccess.createInstance(implClass, Array[Class[_]](), Array[AnyRef]()) match {
|
|
|
|
|
case Right(router) ⇒ router.asInstanceOf[Router]
|
|
|
|
|
case Left(exception) ⇒
|
|
|
|
|
val cause = exception match {
|
|
|
|
|
case i: InvocationTargetException ⇒ i.getTargetException
|
|
|
|
|
case _ ⇒ exception
|
|
|
|
|
}
|
2011-05-24 19:04:25 +02:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
throw new ConfigurationException("Could not instantiate custom Router of [" +
|
|
|
|
|
implClass + "] due to: " + cause, cause)
|
2011-07-28 15:48:03 +03:00
|
|
|
}
|
|
|
|
|
}
|
2011-09-08 11:02:17 +02:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
type Route = (Any) ⇒ AnyRef
|
2011-07-28 15:48:03 +03:00
|
|
|
}
|
2010-03-04 19:02:23 +01:00
|
|
|
|
2011-07-28 15:48:03 +03:00
|
|
|
/**
|
2011-12-08 14:30:57 +01:00
|
|
|
* 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 'targets' does not make logical sense as this means
|
|
|
|
|
* that the round robin should both create new actors and use the 'targets' actor(s).
|
|
|
|
|
* In this case the 'nrOfInstances' will be ignored and the 'targets' will be used.
|
|
|
|
|
* <br>
|
|
|
|
|
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
|
|
|
|
|
* if you provide either 'nrOfInstances' or 'targets' to during instantiation they will
|
|
|
|
|
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
|
2011-07-28 15:48:03 +03:00
|
|
|
*/
|
2011-12-08 14:30:57 +01:00
|
|
|
case class RoundRobinRouter(nrOfInstances: Int = 0, targets: Iterable[ActorRef] = Nil)
|
|
|
|
|
extends Router with RouterConfig {
|
2011-07-28 15:48:03 +03:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
def adaptFromDeploy(deploy: Option[Deploy]): RouterConfig = {
|
|
|
|
|
deploy match {
|
|
|
|
|
case Some(d) ⇒ copy(nrOfInstances = d.nrOfInstances.factor)
|
|
|
|
|
case _ ⇒ this
|
2011-11-24 16:35:37 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
def apply(): Actor = new Actor {
|
|
|
|
|
def receive = {
|
|
|
|
|
case _ ⇒
|
2011-11-24 16:35:37 +01:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
def createRoute(creator: () ⇒ Actor, context: ActorContext): Routing.Route = {
|
|
|
|
|
val routees: Vector[ActorRef] = (nrOfInstances, targets) match {
|
|
|
|
|
case (0, Nil) ⇒ throw new IllegalArgumentException("Insufficient information - missing configuration.")
|
|
|
|
|
case (x, Nil) ⇒
|
|
|
|
|
println("----> 0, Nil")
|
|
|
|
|
(1 to x).map(_ ⇒ context.actorOf(context.props.copy(creator = creator, routerConfig = NoRouting)))(scala.collection.breakOut)
|
|
|
|
|
case (x, xs) ⇒
|
|
|
|
|
println("----> x, xs")
|
|
|
|
|
Vector.empty[ActorRef] ++ xs
|
2011-08-12 10:03:33 +03:00
|
|
|
}
|
2011-07-28 15:48:03 +03:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
val next = new AtomicInteger(0)
|
2011-08-12 10:03:33 +03:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
def getNext(): ActorRef = {
|
|
|
|
|
routees(next.getAndIncrement % routees.size)
|
|
|
|
|
}
|
2011-08-18 11:35:14 +02:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
{
|
|
|
|
|
case _: AutoReceivedMessage ⇒ null //TODO: handle system specific messages
|
|
|
|
|
case Broadcast(msg) ⇒ routees
|
|
|
|
|
case msg ⇒ getNext()
|
2011-07-28 15:48:03 +03:00
|
|
|
}
|
2011-08-12 10:03:33 +03:00
|
|
|
}
|
|
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
/*
|
2011-08-29 15:50:40 +02:00
|
|
|
private val state = new AtomicReference[RoundRobinState]
|
2011-05-17 21:15:27 +02:00
|
|
|
|
2011-10-07 15:42:55 +02:00
|
|
|
def next: Option[ActorRef] = currentState.next
|
2011-05-17 21:15:27 +02:00
|
|
|
|
2011-08-12 10:03:33 +03:00
|
|
|
@tailrec
|
2011-10-07 15:42:55 +02:00
|
|
|
private def currentState: RoundRobinState = {
|
|
|
|
|
val current = state.get
|
2011-05-17 21:15:27 +02:00
|
|
|
|
2011-10-07 15:42:55 +02:00
|
|
|
if (current != null && current.version == connectionManager.version) {
|
2011-08-12 10:03:33 +03:00
|
|
|
//we are lucky, since there has not been any change in the connections. So therefor we can use the existing state.
|
2011-10-07 15:42:55 +02:00
|
|
|
current
|
2011-08-12 10:03:33 +03:00
|
|
|
} else {
|
|
|
|
|
//there has been a change in connections, or it was the first try, so we need to update the internal state
|
2011-08-18 11:35:14 +02:00
|
|
|
|
2011-10-07 15:42:55 +02:00
|
|
|
val connections = connectionManager.connections
|
|
|
|
|
val newState = new RoundRobinState(connections.iterable.toIndexedSeq[ActorRef], connections.version)
|
|
|
|
|
if (state.compareAndSet(current, newState))
|
2011-12-08 14:30:57 +01:00
|
|
|
//we are lucky since we just updated the state, so we can send it back as the state to use
|
2011-08-12 10:03:33 +03:00
|
|
|
newState
|
2011-08-26 17:25:18 +02:00
|
|
|
else //we failed to update the state, lets try again... better luck next time.
|
2011-10-07 15:42:55 +02:00
|
|
|
currentState
|
2011-05-21 15:37:09 +02:00
|
|
|
}
|
2011-08-12 10:03:33 +03:00
|
|
|
}
|
|
|
|
|
|
2011-08-26 17:25:18 +02:00
|
|
|
private case class RoundRobinState(array: IndexedSeq[ActorRef], version: Long) {
|
2011-08-12 10:03:33 +03:00
|
|
|
|
|
|
|
|
private val index = new AtomicInteger(0)
|
|
|
|
|
|
2011-08-29 15:50:40 +02:00
|
|
|
def next: Option[ActorRef] = if (array.isEmpty) None else Some(array(nextIndex))
|
2011-05-21 15:37:09 +02:00
|
|
|
|
2011-08-12 10:03:33 +03:00
|
|
|
@tailrec
|
2011-08-29 15:50:40 +02:00
|
|
|
private def nextIndex: Int = {
|
|
|
|
|
val oldIndex = index.get
|
2011-08-12 10:03:33 +03:00
|
|
|
var newIndex = if (oldIndex == array.length - 1) 0 else oldIndex + 1
|
|
|
|
|
|
2011-08-29 15:50:40 +02:00
|
|
|
if (!index.compareAndSet(oldIndex, newIndex)) nextIndex
|
2011-08-12 10:03:33 +03:00
|
|
|
else oldIndex
|
|
|
|
|
}
|
2011-05-17 21:15:27 +02:00
|
|
|
}
|
2011-12-08 14:30:57 +01:00
|
|
|
*/
|
2011-05-17 21:15:27 +02:00
|
|
|
}
|
2011-08-17 10:21:27 +03:00
|
|
|
|
2011-12-08 14:30:57 +01:00
|
|
|
///**
|
|
|
|
|
// * An Abstract Router implementation that already provides the basic infrastructure so that a concrete
|
|
|
|
|
// * Router only needs to implement the next method.
|
|
|
|
|
// */
|
|
|
|
|
//trait BasicRouter extends Router {
|
|
|
|
|
//
|
|
|
|
|
// @volatile
|
|
|
|
|
// protected var connectionManager: ConnectionManager = _
|
|
|
|
|
//
|
|
|
|
|
// def init(connectionManager: ConnectionManager) = {
|
|
|
|
|
// this.connectionManager = connectionManager
|
|
|
|
|
// }
|
|
|
|
|
//
|
|
|
|
|
// def route(message: Any)(implicit sender: ActorRef) = message match {
|
|
|
|
|
// case Routing.Broadcast(message) ⇒
|
|
|
|
|
//
|
|
|
|
|
// //it is a broadcast message, we are going to send to message to all connections.
|
|
|
|
|
// 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
|
|
|
|
|
// }
|
|
|
|
|
// }
|
|
|
|
|
// case _ ⇒
|
|
|
|
|
// //it no broadcast message, we are going to select an actor from the connections and send the message to him.
|
|
|
|
|
// next match {
|
|
|
|
|
// case Some(connection) ⇒
|
|
|
|
|
// try {
|
|
|
|
|
// connection.!(message)(sender) // we use original sender, so this is essentially a 'forward'
|
|
|
|
|
// } catch {
|
|
|
|
|
// case e: Exception ⇒
|
|
|
|
|
// connectionManager.remove(connection)
|
|
|
|
|
// throw e
|
|
|
|
|
// }
|
|
|
|
|
// case None ⇒
|
|
|
|
|
// throwNoConnectionsError
|
|
|
|
|
// }
|
|
|
|
|
// }
|
|
|
|
|
//
|
|
|
|
|
// def route[T](message: Any, timeout: Timeout): Future[T] = message match {
|
|
|
|
|
// case Routing.Broadcast(message) ⇒
|
|
|
|
|
// throw new RoutingException("Broadcasting using '?'/'ask' is for the time being is not supported. Use ScatterGatherRouter.")
|
|
|
|
|
// case _ ⇒
|
|
|
|
|
// //it no broadcast message, we are going to select an actor from the connections and send the message to him.
|
|
|
|
|
// next match {
|
|
|
|
|
// case Some(connection) ⇒
|
|
|
|
|
// try {
|
|
|
|
|
// connection.?(message, timeout).asInstanceOf[Future[T]] //FIXME this does not preserve the original sender, shouldn't it??
|
|
|
|
|
// } catch {
|
|
|
|
|
// case e: Exception ⇒
|
|
|
|
|
// connectionManager.remove(connection)
|
|
|
|
|
// throw e
|
|
|
|
|
// }
|
|
|
|
|
// case None ⇒
|
|
|
|
|
// throwNoConnectionsError
|
|
|
|
|
// }
|
|
|
|
|
// }
|
|
|
|
|
//
|
|
|
|
|
// protected def next: Option[ActorRef]
|
|
|
|
|
//
|
|
|
|
|
// 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ér</a>
|
|
|
|
|
// */
|
|
|
|
|
//class DirectRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
|
|
|
|
|
//
|
|
|
|
|
// private val state = new AtomicReference[DirectRouterState]
|
|
|
|
|
//
|
|
|
|
|
// lazy val next: Option[ActorRef] = {
|
|
|
|
|
// val current = currentState
|
|
|
|
|
// if (current.ref == null) None else Some(current.ref)
|
|
|
|
|
// }
|
|
|
|
|
//
|
|
|
|
|
// @tailrec
|
|
|
|
|
// private def currentState: DirectRouterState = {
|
|
|
|
|
// val current = state.get
|
|
|
|
|
//
|
|
|
|
|
// if (current != null && connectionManager.version == current.version) {
|
|
|
|
|
// //we are lucky since nothing has changed in the connections.
|
|
|
|
|
// current
|
|
|
|
|
// } else {
|
|
|
|
|
// //there has been a change in the connections, or this is the first time this method is called. So we are going to do some updating.
|
|
|
|
|
//
|
|
|
|
|
// val connections = connectionManager.connections
|
|
|
|
|
//
|
|
|
|
|
// val connectionCount = connections.iterable.size
|
|
|
|
|
// if (connectionCount > 1)
|
|
|
|
|
// throw new RoutingException("A DirectRouter can't have more than 1 connected Actor, but found [%s]".format(connectionCount))
|
|
|
|
|
//
|
|
|
|
|
// val newState = new DirectRouterState(connections.iterable.head, connections.version)
|
|
|
|
|
// if (state.compareAndSet(current, newState))
|
|
|
|
|
// //we are lucky since we just updated the state, so we can send it back as the state to use
|
|
|
|
|
// newState
|
|
|
|
|
// else //we failed to update the state, lets try again... better luck next time.
|
|
|
|
|
// currentState // recur
|
|
|
|
|
// }
|
|
|
|
|
// }
|
|
|
|
|
//
|
|
|
|
|
// private case class DirectRouterState(ref: ActorRef, version: Long)
|
|
|
|
|
//
|
|
|
|
|
//}
|
|
|
|
|
//
|
|
|
|
|
///**
|
|
|
|
|
// * A Router that randomly selects one of the target connections to send a message to.
|
|
|
|
|
// *
|
|
|
|
|
// * @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
|
|
|
// */
|
|
|
|
|
//class RandomRouter(implicit val dispatcher: MessageDispatcher, timeout: Timeout) extends BasicRouter {
|
|
|
|
|
//
|
|
|
|
|
// import java.security.SecureRandom
|
|
|
|
|
//
|
|
|
|
|
// private val state = new AtomicReference[RandomRouterState]
|
|
|
|
|
//
|
|
|
|
|
// private val random = new ThreadLocal[SecureRandom] {
|
|
|
|
|
// override def initialValue = SecureRandom.getInstance("SHA1PRNG")
|
|
|
|
|
// }
|
|
|
|
|
//
|
|
|
|
|
// def next: Option[ActorRef] = currentState.array match {
|
|
|
|
|
// case a if a.isEmpty ⇒ None
|
|
|
|
|
// case a ⇒ Some(a(random.get.nextInt(a.length)))
|
|
|
|
|
// }
|
|
|
|
|
//
|
|
|
|
|
// @tailrec
|
|
|
|
|
// private def currentState: RandomRouterState = {
|
|
|
|
|
// val current = state.get
|
|
|
|
|
//
|
|
|
|
|
// if (current != null && current.version == connectionManager.version) {
|
|
|
|
|
// //we are lucky, since there has not been any change in the connections. So therefor we can use the existing state.
|
|
|
|
|
// current
|
|
|
|
|
// } else {
|
|
|
|
|
// //there has been a change in connections, or it was the first try, so we need to update the internal state
|
|
|
|
|
//
|
|
|
|
|
// val connections = connectionManager.connections
|
|
|
|
|
// val newState = new RandomRouterState(connections.iterable.toIndexedSeq, connections.version)
|
|
|
|
|
// if (state.compareAndSet(current, newState))
|
|
|
|
|
// //we are lucky since we just updated the state, so we can send it back as the state to use
|
|
|
|
|
// newState
|
|
|
|
|
// else //we failed to update the state, lets try again... better luck next time.
|
|
|
|
|
// currentState
|
|
|
|
|
// }
|
|
|
|
|
// }
|
|
|
|
|
//
|
|
|
|
|
// private case class RandomRouterState(array: IndexedSeq[ActorRef], version: Long)
|
|
|
|
|
//}
|
|
|
|
|
|
2011-08-29 15:50:40 +02:00
|
|
|
/**
|
2011-08-17 10:21:27 +03:00
|
|
|
* ScatterGatherRouter broadcasts the message to all connections and gathers results according to the
|
|
|
|
|
* specified strategy (specific router needs to implement `gather` method).
|
2011-08-26 08:24:26 +02:00
|
|
|
* Scatter-gather pattern will be applied only to the messages broadcasted using Future
|
2011-08-17 10:21:27 +03:00
|
|
|
* (wrapped into {@link Routing.Broadcast} and sent with "?" method). For the messages, sent in a fire-forget
|
2011-08-26 08:24:26 +02:00
|
|
|
* mode, the router would behave as {@link BasicRouter}, unless it's mixed in with other router type
|
|
|
|
|
*
|
2011-08-17 10:21:27 +03:00
|
|
|
* FIXME: This also is the location where a failover is done in the future if an ActorRef fails and a different one needs to be selected.
|
|
|
|
|
* FIXME: this is also the location where message buffering should be done in case of failure.
|
|
|
|
|
*/
|
2011-12-08 14:30:57 +01:00
|
|
|
/*
|
2011-08-17 10:21:27 +03:00
|
|
|
trait ScatterGatherRouter extends BasicRouter with Serializable {
|
|
|
|
|
|
2011-08-29 15:50:40 +02:00
|
|
|
/**
|
2011-10-07 15:42:55 +02:00
|
|
|
* Aggregates the responses into a single Future.
|
|
|
|
|
*
|
2011-08-29 15:50:40 +02:00
|
|
|
* @param results Futures of the responses from connections
|
|
|
|
|
*/
|
2011-08-17 10:21:27 +03:00
|
|
|
protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G]
|
|
|
|
|
|
2011-10-27 18:04:12 +02:00
|
|
|
private def scatterGather[S, G >: S](message: Any, timeout: Timeout): Future[G] = {
|
2011-12-08 14:30:57 +01:00
|
|
|
val responses = connectionManager.connections.iterable.flatMap {
|
|
|
|
|
actor ⇒
|
|
|
|
|
try {
|
|
|
|
|
if (actor.isTerminated) throw ActorInitializationException(actor, "For compatability - check death first", new Exception) // for stack trace
|
|
|
|
|
Some(actor.?(message, timeout).asInstanceOf[Future[S]])
|
|
|
|
|
} catch {
|
|
|
|
|
case e: Exception ⇒
|
|
|
|
|
connectionManager.remove(actor)
|
|
|
|
|
None
|
|
|
|
|
}
|
2011-08-17 10:21:27 +03:00
|
|
|
}
|
|
|
|
|
|
2011-08-26 17:25:18 +02:00
|
|
|
if (responses.isEmpty)
|
2011-08-17 10:21:27 +03:00
|
|
|
throw new RoutingException("No connections can process the message [%s] sent to scatter-gather router" format (message))
|
2011-08-29 15:50:40 +02:00
|
|
|
else gather(responses)
|
2011-08-17 10:21:27 +03:00
|
|
|
}
|
|
|
|
|
|
2011-10-27 18:04:12 +02:00
|
|
|
override def route[T](message: Any, timeout: Timeout): Future[T] = message match {
|
2011-08-17 10:21:27 +03:00
|
|
|
case Routing.Broadcast(message) ⇒ scatterGather(message, timeout)
|
2011-12-08 14:30:57 +01:00
|
|
|
case message ⇒ super.route(message, timeout)
|
2011-08-17 10:21:27 +03:00
|
|
|
}
|
|
|
|
|
}
|
2011-12-08 14:30:57 +01:00
|
|
|
*/
|
2011-08-17 10:21:27 +03:00
|
|
|
|
2011-08-29 15:50:40 +02:00
|
|
|
/**
|
2011-08-17 10:21:27 +03:00
|
|
|
* Simple router that broadcasts the message to all connections, and replies with the first response
|
2011-08-26 08:24:26 +02:00
|
|
|
* Scatter-gather pattern will be applied only to the messages broadcasted using Future
|
2011-08-17 10:21:27 +03:00
|
|
|
* (wrapped into {@link Routing.Broadcast} and sent with "?" method). For the messages sent in a fire-forget
|
2011-08-26 08:24:26 +02:00
|
|
|
* mode, the router would behave as {@link RoundRobinRouter}
|
2011-08-17 10:21:27 +03:00
|
|
|
*/
|
2011-12-08 14:30:57 +01:00
|
|
|
/*
|
2011-11-24 16:35:37 +01:00
|
|
|
class ScatterGatherFirstCompletedRouter(implicit dispatcher: MessageDispatcher, timeout: Timeout) extends RoundRobinRouter with ScatterGatherRouter {
|
2011-09-08 15:54:06 +02:00
|
|
|
protected def gather[S, G >: S](results: Iterable[Future[S]]): Future[G] = Future.firstCompletedOf(results)
|
2011-08-17 10:21:27 +03:00
|
|
|
}
|
2011-12-08 14:30:57 +01:00
|
|
|
*/
|