pekko/akka-actor/src/main/scala/akka/routing/Routing.scala

777 lines
27 KiB
Scala
Raw Normal View History

/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.routing
2010-02-13 21:45:35 +01:00
2011-07-28 15:48:03 +03:00
import akka.actor._
import akka.dispatch.Future
import java.util.concurrent.atomic.AtomicLong
import java.util.concurrent.atomic.AtomicBoolean
import akka.util.{ Duration, Timeout }
import akka.util.duration._
import akka.config.ConfigurationException
import scala.collection.JavaConversions.iterableAsScalaIterable
/**
* 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.createActor()),
_supervisor,
_path) {
private val routeeProps = _props.copy(routerConfig = NoRouter)
private val resizeProgress = new AtomicBoolean
private val resizeCounter = new AtomicLong
2011-12-13 16:05:56 +01:00
@volatile
private var _routees: IndexedSeq[ActorRef] = IndexedSeq.empty[ActorRef] // this MUST be initialized during createRoute
2011-12-13 16:05:56 +01:00
def routees = _routees
def addRoutees(newRoutees: IndexedSeq[ActorRef]) {
_routees = _routees ++ newRoutees
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
newRoutees foreach underlying.watch
}
def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]) {
_routees = _routees diff abandonedRoutees
abandonedRoutees foreach underlying.unwatch
}
val route = _props.routerConfig.createRoute(routeeProps, actorContext)
// initial resize, before message send
resize()
2011-12-13 16:05:56 +01:00
def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match {
case _: AutoReceivedMessage Nil
case Terminated(_) Nil
case CurrentRoutees
sender ! RouterRoutees(_routees)
Nil
2011-12-13 16:05:56 +01:00
case _
if (route.isDefinedAt(sender, message)) route(sender, message)
else Nil
}
if (_props.routerConfig.resizer.isEmpty && _routees.isEmpty)
throw new ActorInitializationException("router " + _props.routerConfig + " did not register routees!")
2011-12-13 16:05:56 +01:00
_routees match {
case x _routees = x // volatile write to publish the route before sending messages
}
override def !(message: Any)(implicit sender: ActorRef = null): Unit = {
resize()
val s = if (sender eq null) underlying.system.deadLetters else sender
val msg = message match {
case Broadcast(m) m
case m m
}
2011-12-13 16:05:56 +01:00
applyRoute(s, message) match {
case Nil super.!(message)(s)
case refs refs foreach (p p.recipient.!(msg)(p.sender))
}
}
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = {
resize()
super.?(message)(timeout)
}
def resize() {
for (r _props.routerConfig.resizer) {
if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeProgress.compareAndSet(false, true)) {
try {
r.resize(routeeProps, actorContext, routees, _props.routerConfig)
} finally {
resizeProgress.set(false)
}
}
}
}
}
/**
* This trait represents a router factory: it produces the actual router actor
* and creates the routing table (a function which determines the recipients
* for each message which is to be dispatched). The resulting RoutedActorRef
* optimizes the sending of the message so that it does NOT go through the
* routers mailbox unless the route returns an empty recipient set.
*
* '''Caution:''' This means
* that the route function is evaluated concurrently without protection by
* the RoutedActorRef: either provide a reentrant (i.e. pure) implementation or
* do the locking yourself!
*
* '''Caution:''' Please note that the [[akka.routing.Router]] which needs to
* be returned by `createActor()` should not send a message to itself in its
* constructor or `preStart()` or publish its self reference from there: if
* someone tries sending a message to that reference before the constructor of
* RoutedActorRef has returned, there will be a `NullPointerException`!
*/
trait RouterConfig {
def createRoute(routeeProps: Props, actorContext: ActorContext): Route
def createActor(): Router = new Router {}
def adaptFromDeploy(deploy: Option[Deploy]): RouterConfig = {
deploy match {
case Some(Deploy(_, _, _, NoRouter, _)) this
case Some(Deploy(_, _, _, r, _)) r
case _ this
}
}
2011-12-17 16:33:29 +01:00
protected def toAll(sender: ActorRef, routees: Iterable[ActorRef]): Iterable[Destination] = routees.map(Destination(sender, _))
def createRoutees(props: Props, context: ActorContext, nrOfInstances: Int, routees: Iterable[String]): IndexedSeq[ActorRef] = (nrOfInstances, routees) match {
case (0, Nil) throw new IllegalArgumentException("Insufficient information - missing configuration.")
case (x, Nil) (1 to x).map(_ context.actorOf(props))(scala.collection.breakOut)
case (_, xs) xs.map(context.actorFor(_))(scala.collection.breakOut)
}
2011-12-13 16:05:56 +01:00
2011-12-17 16:33:29 +01:00
protected def createAndRegisterRoutees(props: Props, context: ActorContext, nrOfInstances: Int, routees: Iterable[String]): Unit = {
if (resizer.isEmpty) {
registerRoutees(context, createRoutees(props, context, nrOfInstances, routees))
}
}
/**
* Adds new routees to the router.
*/
def registerRoutees(context: ActorContext, routees: IndexedSeq[ActorRef]): Unit = {
context.self.asInstanceOf[RoutedActorRef].addRoutees(routees)
2011-12-13 16:05:56 +01:00
}
/**
* Removes routees from the router. This method doesn't stop the routees.
*/
def unregisterRoutees(context: ActorContext, routees: IndexedSeq[ActorRef]): Unit = {
context.self.asInstanceOf[RoutedActorRef].removeRoutees(routees)
}
/**
* Routers with dynamically resizable number of routees return the [[akka.routing.Resizer]]
* to use.
*/
def resizer: Option[Resizer] = None
}
/**
* Java API for a custom router factory.
* @see akka.routing.RouterConfig
*/
abstract class CustomRouterConfig extends RouterConfig {
override def createRoute(props: Props, context: ActorContext): Route = {
// as a bonus, this prevents closing of props and context in the returned Route PartialFunction
val customRoute = createCustomRoute(props, context)
{
case (sender, message) customRoute.destinationsFor(sender, message)
}
}
def createCustomRoute(props: Props, context: ActorContext): CustomRoute
protected def registerRoutees(context: ActorContext, routees: java.util.List[ActorRef]): Unit = {
import scala.collection.JavaConverters._
registerRoutees(context, routees.asScala.toIndexedSeq)
}
}
trait CustomRoute {
def destinationsFor(sender: ActorRef, message: Any): java.lang.Iterable[Destination]
}
/**
* Base trait for `Router` actors. Override `receive` to handle custom
* messages which the corresponding [[akka.actor.RouterConfig]] lets
* through by returning an empty route.
*/
trait Router extends Actor {
2011-12-13 16:05:56 +01:00
val ref = self match {
case x: RoutedActorRef x
case _ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef")
}
final def receive = ({
case Terminated(child)
ref.removeRoutees(IndexedSeq(child))
if (ref.routees.isEmpty) context.stop(self)
2011-12-13 16:05:56 +01:00
}: Receive) orElse routerReceive
def routerReceive: Receive = {
case _
2011-07-28 15:48:03 +03:00
}
}
/**
* Used to broadcast a message to all connections in a router; only the
* contained message will be forwarded, i.e. the `Broadcast(...)`
* envelope will be stripped off.
*
* Router implementations may choose to handle this message differently.
*/
case class Broadcast(message: Any)
/**
* Sending this message to a router will make it send back its currently used routees.
* A RouterRoutees message is sent asynchronously to the "requester" containing information
* about what routees the router is routing over.
*/
case object CurrentRoutees
/**
* Message used to carry information about what routees the router is currently using.
*/
case class RouterRoutees(routees: Iterable[ActorRef])
2011-12-13 17:53:59 +01:00
/**
* For every message sent to a router, its route determines a set of destinations,
* where for each recipient a different sender may be specified; typically the
* sender should match the sender of the original request, but e.g. the scatter-
* gather router needs to receive the replies with an AskActorRef instead.
*/
case class Destination(sender: ActorRef, recipient: ActorRef)
/**
* Routing configuration that indicates no routing.
* Oxymoron style.
*/
case object NoRouter extends RouterConfig {
def createRoute(props: Props, actorContext: ActorContext): Route = null
}
/**
* Router configuration which has no default, i.e. external configuration is required.
*/
case object FromConfig extends RouterConfig {
def createRoute(props: Props, actorContext: ActorContext): Route =
throw new ConfigurationException("router " + actorContext.self + " needs external configuration from file (e.g. application.conf)")
}
2011-12-30 00:12:49 +01:00
/**
* Java API: Router configuration which has no default, i.e. external configuration is required.
*/
case class FromConfig() extends RouterConfig {
def createRoute(props: Props, actorContext: ActorContext): Route =
throw new ConfigurationException("router " + actorContext.self + " needs external configuration from file (e.g. application.conf)")
2011-12-30 00:12:49 +01:00
}
object RoundRobinRouter {
2011-12-17 16:33:29 +01:00
def apply(routees: Iterable[ActorRef]) = new RoundRobinRouter(routees = routees map (_.path.toString))
/**
* Java API to create router with the supplied 'routees' actors.
*/
def create(routees: java.lang.Iterable[ActorRef]): RoundRobinRouter = {
import scala.collection.JavaConverters._
apply(routees.asScala)
}
}
2011-07-28 15:48:03 +03:00
/**
* A Router that uses round-robin to select a connection. For concurrent calls, round robin is just a best effort.
* <br>
2011-12-17 16:33:29 +01:00
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical sense as this means
* that the round robin should both create new actors and use the 'routees' actor(s).
* In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
2011-12-17 16:33:29 +01:00
* if you provide either 'nrOfInstances' or 'routees' 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
*/
case class RoundRobinRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
extends RouterConfig with RoundRobinLike {
2011-07-28 15:48:03 +03:00
/**
* Constructor that sets nrOfInstances to be created.
* Java API
*/
def this(nr: Int) = {
this(nrOfInstances = nr)
}
/**
2011-12-17 16:33:29 +01:00
* Constructor that sets the routees to be used.
* Java API
*/
def this(t: java.lang.Iterable[String]) = {
this(routees = iterableAsScalaIterable(t))
}
/**
* Constructor that sets the resizer to be used.
* Java API
*/
def this(resizer: Resizer) = this(resizer = Some(resizer))
}
trait RoundRobinLike { this: RouterConfig
def nrOfInstances: Int
def routees: Iterable[String]
def createRoute(props: Props, context: ActorContext): Route = {
2011-12-17 16:33:29 +01:00
createAndRegisterRoutees(props, context, nrOfInstances, routees)
2011-07-28 15:48:03 +03:00
val ref = context.self.asInstanceOf[RoutedActorRef]
val next = new AtomicLong(0)
def getNext(): ActorRef = {
val _routees = ref.routees
_routees((next.getAndIncrement % _routees.size).asInstanceOf[Int])
}
{
case (sender, message)
message match {
2011-12-13 16:05:56 +01:00
case Broadcast(msg) toAll(sender, ref.routees)
case msg List(Destination(sender, getNext()))
}
2011-07-28 15:48:03 +03:00
}
}
}
object RandomRouter {
2011-12-17 16:33:29 +01:00
def apply(routees: Iterable[ActorRef]) = new RandomRouter(routees = routees map (_.path.toString))
/**
* Java API to create router with the supplied 'routees' actors.
*/
def create(routees: java.lang.Iterable[ActorRef]): RandomRouter = {
import scala.collection.JavaConverters._
apply(routees.asScala)
}
}
/**
* A Router that randomly selects one of the target connections to send a message to.
* <br>
2011-12-17 16:33:29 +01:00
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical sense as this means
* that the random router should both create new actors and use the 'routees' actor(s).
* In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
2011-12-17 16:33:29 +01:00
* if you provide either 'nrOfInstances' or 'routees' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class RandomRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
extends RouterConfig with RandomLike {
/**
* Constructor that sets nrOfInstances to be created.
* Java API
*/
def this(nr: Int) = {
this(nrOfInstances = nr)
}
/**
2011-12-17 16:33:29 +01:00
* Constructor that sets the routees to be used.
* Java API
*/
def this(t: java.lang.Iterable[String]) = {
this(routees = iterableAsScalaIterable(t))
}
/**
* Constructor that sets the resizer to be used.
* Java API
*/
def this(resizer: Resizer) = this(resizer = Some(resizer))
}
trait RandomLike { this: RouterConfig
import java.security.SecureRandom
def nrOfInstances: Int
def routees: Iterable[String]
private val random = new ThreadLocal[SecureRandom] {
override def initialValue = SecureRandom.getInstance("SHA1PRNG")
}
def createRoute(props: Props, context: ActorContext): Route = {
val ref = context.self.asInstanceOf[RoutedActorRef]
2011-12-17 16:33:29 +01:00
createAndRegisterRoutees(props, context, nrOfInstances, routees)
def getNext(): ActorRef = {
2011-12-13 16:05:56 +01:00
ref.routees(random.get.nextInt(ref.routees.size))
}
{
case (sender, message)
message match {
2011-12-13 16:05:56 +01:00
case Broadcast(msg) toAll(sender, ref.routees)
case msg List(Destination(sender, getNext()))
}
}
}
}
object BroadcastRouter {
2011-12-17 16:33:29 +01:00
def apply(routees: Iterable[ActorRef]) = new BroadcastRouter(routees = routees map (_.path.toString))
/**
* Java API to create router with the supplied 'routees' actors.
*/
def create(routees: java.lang.Iterable[ActorRef]): BroadcastRouter = {
import scala.collection.JavaConverters._
apply(routees.asScala)
}
}
/**
* A Router that uses broadcasts a message to all its connections.
* <br>
2011-12-17 16:33:29 +01:00
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical sense as this means
* that the random router should both create new actors and use the 'routees' actor(s).
* In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
2011-12-17 16:33:29 +01:00
* if you provide either 'nrOfInstances' or 'routees' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class BroadcastRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, override val resizer: Option[Resizer] = None)
extends RouterConfig with BroadcastLike {
/**
* Constructor that sets nrOfInstances to be created.
* Java API
*/
def this(nr: Int) = {
this(nrOfInstances = nr)
}
/**
2011-12-17 16:33:29 +01:00
* Constructor that sets the routees to be used.
* Java API
*/
def this(t: java.lang.Iterable[String]) = {
this(routees = iterableAsScalaIterable(t))
}
/**
* Constructor that sets the resizer to be used.
* Java API
*/
def this(resizer: Resizer) = this(resizer = Some(resizer))
}
trait BroadcastLike { this: RouterConfig
def nrOfInstances: Int
def routees: Iterable[String]
def createRoute(props: Props, context: ActorContext): Route = {
val ref = context.self.asInstanceOf[RoutedActorRef]
2011-12-17 16:33:29 +01:00
createAndRegisterRoutees(props, context, nrOfInstances, routees)
{
case (sender, message)
message match {
2011-12-13 16:05:56 +01:00
case _ toAll(sender, ref.routees)
}
}
}
}
object ScatterGatherFirstCompletedRouter {
def apply(routees: Iterable[ActorRef], within: Duration) = new ScatterGatherFirstCompletedRouter(routees = routees map (_.path.toString), within = within)
/**
* Java API to create router with the supplied 'routees' actors.
*/
def create(routees: java.lang.Iterable[ActorRef], within: Duration): ScatterGatherFirstCompletedRouter = {
import scala.collection.JavaConverters._
apply(routees.asScala, within)
}
}
/**
* Simple router that broadcasts the message to all routees, and replies with the first response.
* <br>
2011-12-17 16:33:29 +01:00
* Please note that providing both 'nrOfInstances' and 'routees' does not make logical sense as this means
* that the random router should both create new actors and use the 'routees' actor(s).
* In this case the 'nrOfInstances' will be ignored and the 'routees' will be used.
* <br>
* <b>The</b> configuration parameter trumps the constructor arguments. This means that
2011-12-17 16:33:29 +01:00
* if you provide either 'nrOfInstances' or 'routees' to during instantiation they will
* be ignored if the 'nrOfInstances' is defined in the configuration file for the actor being used.
*/
case class ScatterGatherFirstCompletedRouter(nrOfInstances: Int = 0, routees: Iterable[String] = Nil, within: Duration,
override val resizer: Option[Resizer] = None)
extends RouterConfig with ScatterGatherFirstCompletedLike {
/**
* Constructor that sets nrOfInstances to be created.
* Java API
*/
def this(nr: Int, w: Duration) = {
this(nrOfInstances = nr, within = w)
}
/**
2011-12-17 16:33:29 +01:00
* Constructor that sets the routees to be used.
* Java API
*/
def this(t: java.lang.Iterable[String], w: Duration) = {
this(routees = iterableAsScalaIterable(t), within = w)
}
/**
* Constructor that sets the resizer to be used.
* Java API
*/
def this(resizer: Resizer, w: Duration) = this(resizer = Some(resizer), within = w)
}
trait ScatterGatherFirstCompletedLike { this: RouterConfig
def nrOfInstances: Int
def routees: Iterable[String]
def within: Duration
def createRoute(props: Props, context: ActorContext): Route = {
val ref = context.self.asInstanceOf[RoutedActorRef]
2011-12-17 16:33:29 +01:00
createAndRegisterRoutees(props, context, nrOfInstances, routees)
{
case (sender, message)
val asker = context.asInstanceOf[ActorCell].systemImpl.provider.ask(Timeout(within)).get
asker.result.pipeTo(sender)
message match {
2011-12-13 16:05:56 +01:00
case _ toAll(asker, ref.routees)
}
}
}
}
/**
* Routers with dynamically resizable number of routees is implemented by providing a Resizer
* implementation in [[akka.routing.RouterConfig]].
*/
trait Resizer {
/**
* Is it time for resizing. Typically implemented with modulo of nth message, but
* could be based on elapsed time or something else. The messageCounter starts with 0
* for the initial resize and continues with 1 for the first message. Make sure to perform
* initial resize before first message (messageCounter == 0), because there is no guarantee
* that resize will be done when concurrent messages are in play.
*/
def isTimeForResize(messageCounter: Long): Boolean
/**
* Decide if the capacity of the router need to be changed. Will be invoked when `isTimeForResize`
* returns true and no other resize is in progress.
* Create and register more routees with `routerConfig.registerRoutees(actorContext, newRoutees)
* or remove routees with `routerConfig.unregisterRoutees(actorContext, abandonedRoutees)` and
* sending [[akka.actor.PoisonPill]] to them.
*/
def resize(props: Props, actorContext: ActorContext, currentRoutees: IndexedSeq[ActorRef], routerConfig: RouterConfig)
}
case class DefaultResizer(
/**
* The fewest number of routees the router should ever have.
*/
lowerBound: Int = 1,
/**
* The most number of routees the router should ever have.
* Must be greater than or equal to `lowerBound`.
*/
upperBound: Int = 10,
/**
* Threshold to evaluate if routee is considered to be busy (under pressure).
* Implementation depends on this value (default is 1).
* <ul>
* <li> 0: number of routees currently processing a message.</li>
* <li> 1: number of routees currently processing a message has
* some messages in mailbox.</li>
* <li> > 1: number of routees with at least the configured `pressureThreshold`
* messages in their mailbox. Note that estimating mailbox size of
* default UnboundedMailbox is O(N) operation.</li>
* </ul>
*/
pressureThreshold: Int = 1,
/**
* Percentage to increase capacity whenever all routees are busy.
* For example, 0.2 would increase 20% (rounded up), i.e. if current
* capacity is 6 it will request an increase of 2 more routees.
*/
rampupRate: Double = 0.2,
/**
* Minimum fraction of busy routees before backing off.
* For example, if this is 0.3, then we'll remove some routees only when
* less than 30% of routees are busy, i.e. if current capacity is 10 and
* 3 are busy then the capacity is unchanged, but if 2 or less are busy
* the capacity is decreased.
*
* Use 0.0 or negative to avoid removal of routees.
*/
backoffThreshold: Double = 0.3,
/**
* Fraction of routees to be removed when the resizer reaches the
* backoffThreshold.
* For example, 0.1 would decrease 10% (rounded up), i.e. if current
* capacity is 9 it will request an decrease of 1 routee.
*/
backoffRate: Double = 0.1,
/**
* When the resizer reduce the capacity the abandoned routee actors are stopped
* with PoisonPill after this delay. The reason for the delay is to give concurrent
* messages a chance to be placed in mailbox before sending PoisonPill.
* Use 0 seconds to skip delay.
*/
stopDelay: Duration = 1.second,
/**
* Number of messages between resize operation.
* Use 1 to resize before each message.
*/
messagesPerResize: Int = 10) extends Resizer {
/**
* Java API constructor for default values except bounds.
*/
def this(lower: Int, upper: Int) = this(lowerBound = lower, upperBound = upper)
if (lowerBound < 0) throw new IllegalArgumentException("lowerBound must be >= 0, was: [%s]".format(lowerBound))
if (upperBound < 0) throw new IllegalArgumentException("upperBound must be >= 0, was: [%s]".format(upperBound))
if (upperBound < lowerBound) throw new IllegalArgumentException("upperBound must be >= lowerBound, was: [%s] < [%s]".format(upperBound, lowerBound))
if (rampupRate < 0.0) throw new IllegalArgumentException("rampupRate must be >= 0.0, was [%s]".format(rampupRate))
if (backoffThreshold > 1.0) throw new IllegalArgumentException("backoffThreshold must be <= 1.0, was [%s]".format(backoffThreshold))
if (backoffRate < 0.0) throw new IllegalArgumentException("backoffRate must be >= 0.0, was [%s]".format(backoffRate))
if (messagesPerResize <= 0) throw new IllegalArgumentException("messagesPerResize must be > 0, was [%s]".format(messagesPerResize))
def isTimeForResize(messageCounter: Long): Boolean = (messageCounter % messagesPerResize == 0)
def resize(props: Props, actorContext: ActorContext, currentRoutees: IndexedSeq[ActorRef], routerConfig: RouterConfig) {
val requestedCapacity = capacity(currentRoutees)
if (requestedCapacity > 0) {
val newRoutees = routerConfig.createRoutees(props, actorContext, requestedCapacity, Nil)
routerConfig.registerRoutees(actorContext, newRoutees)
} else if (requestedCapacity < 0) {
val (keep, abandon) = currentRoutees.splitAt(currentRoutees.length + requestedCapacity)
routerConfig.unregisterRoutees(actorContext, abandon)
delayedStop(actorContext.system.scheduler, abandon)
}
}
/**
* Give concurrent messages a chance to be placed in mailbox before
* sending PoisonPill.
*/
protected def delayedStop(scheduler: Scheduler, abandon: IndexedSeq[ActorRef]) {
if (abandon.nonEmpty) {
if (stopDelay <= Duration.Zero) {
abandon foreach (_ ! PoisonPill)
} else {
scheduler.scheduleOnce(stopDelay) {
abandon foreach (_ ! PoisonPill)
}
}
}
}
/**
* Returns the overall desired change in resizer capacity. Positive value will
* add routees to the resizer. Negative value will remove routees from the
* resizer.
* @param routees The current actor in the resizer
* @return the number of routees by which the resizer should be adjusted (positive, negative or zero)
*/
def capacity(routees: IndexedSeq[ActorRef]): Int = {
val currentSize = routees.size
val delta = filter(pressure(routees), currentSize)
val proposed = currentSize + delta
if (proposed < lowerBound) delta + (lowerBound - proposed)
else if (proposed > upperBound) delta - (proposed - upperBound)
else delta
}
/**
* Number of routees considered busy, or above 'pressure level'.
*
* Implementation depends on the value of `pressureThreshold`
* (default is 1).
* <ul>
* <li> 0: number of routees currently processing a message.</li>
* <li> 1: number of routees currently processing a message has
* some messages in mailbox.</li>
* <li> > 1: number of routees with at least the configured `pressureThreshold`
* messages in their mailbox. Note that estimating mailbox size of
* default UnboundedMailbox is O(N) operation.</li>
* </ul>
*
* @param routees the current resizer of routees
* @return number of busy routees, between 0 and routees.size
*/
def pressure(routees: IndexedSeq[ActorRef]): Int = {
routees count {
case a: LocalActorRef
val cell = a.underlying
pressureThreshold match {
case 1 cell.mailbox.isScheduled && cell.currentMessage != null
case i if i < 1 cell.mailbox.isScheduled && cell.currentMessage != null
case threshold cell.mailbox.numberOfMessages >= threshold
}
case x
false
}
}
/**
* This method can be used to smooth the capacity delta by considering
* the current pressure and current capacity.
*
* @param pressure current number of busy routees
* @param capacity current number of routees
* @return proposed change in the capacity
*/
def filter(pressure: Int, capacity: Int): Int = {
rampup(pressure, capacity) + backoff(pressure, capacity)
}
/**
* Computes a proposed positive (or zero) capacity delta using
* the configured `rampupRate`.
* @param pressure the current number of busy routees
* @param capacity the current number of total routees
* @return proposed increase in capacity
*/
def rampup(pressure: Int, capacity: Int): Int =
if (pressure < capacity) 0 else math.ceil(rampupRate * capacity) toInt
/**
* Computes a proposed negative (or zero) capacity delta using
* the configured `backoffThreshold` and `backoffRate`
* @param pressure the current number of busy routees
* @param capacity the current number of total routees
* @return proposed decrease in capacity (as a negative number)
*/
def backoff(pressure: Int, capacity: Int): Int =
if (backoffThreshold > 0.0 && backoffRate > 0.0 && capacity > 0 && pressure.toDouble / capacity < backoffThreshold)
math.floor(-1.0 * backoffRate * capacity) toInt
else 0
}