Skip to content

Commit

Permalink
Merge pull request akka#15822 from ktoso/clu-useRole-ignored-local-ktoso
Browse files Browse the repository at this point in the history
useRole ignored on local, because routers unaware of roles locally
  • Loading branch information
ktoso committed Oct 31, 2014
2 parents f2f88d9 + 3f12ef2 commit c2983c7
Show file tree
Hide file tree
Showing 16 changed files with 314 additions and 109 deletions.
7 changes: 4 additions & 3 deletions akka-actor/src/main/scala/akka/routing/Balancing.scala
Original file line number Diff line number Diff line change
Expand Up @@ -66,14 +66,13 @@ private[akka] final class BalancingRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
final case class BalancingPool(
override val nrOfInstances: Int,
nrOfInstances: Int,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
extends Pool {

def this(config: Config) =
this(
nrOfInstances = config.getInt("nr-of-instances"))
this(nrOfInstances = config.getInt("nr-of-instances"))

/**
* Java API
Expand All @@ -94,6 +93,8 @@ final case class BalancingPool(
*/
def withDispatcher(dispatcherId: String): BalancingPool = copy(routerDispatcher = dispatcherId)

def nrOfInstances(sys: ActorSystem) = this.nrOfInstances

/**
* INTERNAL API
*/
Expand Down
4 changes: 3 additions & 1 deletion akka-actor/src/main/scala/akka/routing/Broadcast.scala
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ final class BroadcastRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
final case class BroadcastPool(
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
nrOfInstances: Int, override val resizer: Option[Resizer] = None,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
Expand All @@ -78,6 +78,8 @@ final case class BroadcastPool(

override def createRouter(system: ActorSystem): Router = new Router(BroadcastRoutingLogic())

override def nrOfInstances(sys: ActorSystem) = this.nrOfInstances

/**
* Setting the supervisor strategy to be used for the “head” Router actor.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -260,7 +260,8 @@ final case class ConsistentHashingRoutingLogic(
*/
@SerialVersionUID(1L)
final case class ConsistentHashingPool(
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
val nrOfInstances: Int,
override val resizer: Option[Resizer] = None,
val virtualNodesFactor: Int = 0,
val hashMapping: ConsistentHashingRouter.ConsistentHashMapping = ConsistentHashingRouter.emptyConsistentHashMapping,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
Expand All @@ -283,6 +284,8 @@ final case class ConsistentHashingPool(
override def createRouter(system: ActorSystem): Router =
new Router(ConsistentHashingRoutingLogic(system, virtualNodesFactor, hashMapping))

override def nrOfInstances(sys: ActorSystem) = this.nrOfInstances

/**
* Setting the supervisor strategy to be used for the “head” Router actor.
*/
Expand Down
4 changes: 3 additions & 1 deletion akka-actor/src/main/scala/akka/routing/Random.scala
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ final class RandomRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
final case class RandomPool(
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
Expand All @@ -79,6 +79,8 @@ final case class RandomPool(

override def createRouter(system: ActorSystem): Router = new Router(RandomRoutingLogic())

override def nrOfInstances(sys: ActorSystem) = this.nrOfInstances

/**
* Setting the supervisor strategy to be used for the “head” Router actor.
*/
Expand Down
4 changes: 3 additions & 1 deletion akka-actor/src/main/scala/akka/routing/RoundRobin.scala
Original file line number Diff line number Diff line change
Expand Up @@ -64,7 +64,7 @@ final class RoundRobinRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
final case class RoundRobinPool(
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
Expand All @@ -83,6 +83,8 @@ final case class RoundRobinPool(

override def createRouter(system: ActorSystem): Router = new Router(RoundRobinRoutingLogic())

override def nrOfInstances(sys: ActorSystem) = this.nrOfInstances

/**
* Setting the supervisor strategy to be used for the “head” Router actor.
*/
Expand Down
21 changes: 9 additions & 12 deletions akka-actor/src/main/scala/akka/routing/RoutedActorCell.scala
Original file line number Diff line number Diff line change
Expand Up @@ -3,26 +3,22 @@
*/
package akka.routing

import scala.collection.immutable
import scala.concurrent.duration._
import akka.actor.Actor
import akka.actor.ActorCell
import akka.actor.ActorInitializationException
import akka.actor.ActorRef
import akka.actor.ActorSystemImpl
import akka.actor.AutoReceivedMessage
import akka.actor.IndirectActorProducer
import akka.actor.InternalActorRef
import akka.actor.PoisonPill
import akka.actor.Props
import akka.actor.SupervisorStrategy
import akka.actor.Terminated
import akka.dispatch.Envelope
import akka.dispatch.MessageDispatcher
import akka.actor.ActorContext
import akka.actor.PoisonPill
import akka.actor.SupervisorStrategy
import akka.actor.ActorRef
import akka.actor.ReceiveTimeout
import akka.actor.Identify
import akka.actor.ActorIdentity

import scala.collection.immutable
import scala.concurrent.duration._

/**
* INTERNAL API
Expand Down Expand Up @@ -106,8 +102,9 @@ private[akka] class RoutedActorCell(
_router = routerConfig.createRouter(system)
routerConfig match {
case pool: Pool
if (pool.nrOfInstances > 0)
addRoutees(Vector.fill(pool.nrOfInstances)(pool.newRoutee(routeeProps, this)))
val nrOfRoutees = pool.nrOfInstances(system)
if (nrOfRoutees > 0)
addRoutees(Vector.fill(nrOfRoutees)(pool.newRoutee(routeeProps, this)))
case group: Group
val paths = group.paths
if (paths.nonEmpty)
Expand Down
5 changes: 3 additions & 2 deletions akka-actor/src/main/scala/akka/routing/RouterConfig.scala
Original file line number Diff line number Diff line change
Expand Up @@ -176,10 +176,11 @@ abstract class PoolBase extends Pool
* them from the router if they terminate.
*/
trait Pool extends RouterConfig {

/**
* Initial number of routee instances
*/
def nrOfInstances: Int
def nrOfInstances(sys: ActorSystem): Int

/**
* Use a dedicated dispatcher for the routees of the pool.
Expand Down Expand Up @@ -315,7 +316,7 @@ class FromConfig(override val resizer: Option[Resizer],
def withDispatcher(dispatcherId: String): FromConfig =
new FromConfig(resizer, supervisorStrategy, dispatcherId)

override val nrOfInstances: Int = 0
override def nrOfInstances(sys: ActorSystem): Int = 0

/**
* [[akka.actor.Props]] for a group router based on the settings defined by
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ private[akka] final case class ScatterGatherFirstCompletedRoutees(
*/
@SerialVersionUID(1L)
final case class ScatterGatherFirstCompletedPool(
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
within: FiniteDuration,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
Expand All @@ -117,6 +117,8 @@ final case class ScatterGatherFirstCompletedPool(

override def createRouter(system: ActorSystem): Router = new Router(ScatterGatherFirstCompletedRoutingLogic(within))

override def nrOfInstances(sys: ActorSystem) = this.nrOfInstances

/**
* Setting the supervisor strategy to be used for the “head” Router actor.
*/
Expand Down
4 changes: 3 additions & 1 deletion akka-actor/src/main/scala/akka/routing/SmallestMailbox.scala
Original file line number Diff line number Diff line change
Expand Up @@ -174,7 +174,7 @@ class SmallestMailboxRoutingLogic extends RoutingLogic {
*/
@SerialVersionUID(1L)
final case class SmallestMailboxPool(
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
Expand All @@ -194,6 +194,8 @@ final case class SmallestMailboxPool(

override def createRouter(system: ActorSystem): Router = new Router(SmallestMailboxRoutingLogic())

override def nrOfInstances(sys: ActorSystem) = this.nrOfInstances

/**
* Setting the supervisor strategy to be used for the “head” Router actor.
*/
Expand Down
4 changes: 3 additions & 1 deletion akka-actor/src/main/scala/akka/routing/TailChopping.scala
Original file line number Diff line number Diff line change
Expand Up @@ -120,7 +120,7 @@ private[akka] final case class TailChoppingRoutees(
*/
@SerialVersionUID(1L)
final case class TailChoppingPool(
override val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
val nrOfInstances: Int, override val resizer: Option[Resizer] = None,
within: FiniteDuration,
interval: FiniteDuration,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
Expand Down Expand Up @@ -150,6 +150,8 @@ final case class TailChoppingPool(
new Router(TailChoppingRoutingLogic(system.scheduler, within,
interval, system.dispatchers.lookup(routerDispatcher)))

override def nrOfInstances(sys: ActorSystem) = this.nrOfInstances

/**
* Setting the supervisor strategy to be used for the “head” Router actor.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -3,40 +3,14 @@
*/
package akka.cluster

import com.typesafe.config.Config
import akka.ConfigurationException
import akka.actor.ActorSystem
import akka.actor.ActorSystemImpl
import akka.actor.Deploy
import akka.actor.DynamicAccess
import akka.actor.InternalActorRef
import akka.actor.NoScopeGiven
import akka.actor.Scheduler
import akka.actor.Scope
import akka.actor.Terminated
import akka.dispatch.sysmsg.DeathWatchNotification
import akka.actor.{ ActorRef, ActorSystem, ActorSystemImpl, Deploy, DynamicAccess, NoScopeGiven, Scope }
import akka.cluster.routing.{ ClusterRouterGroup, ClusterRouterGroupSettings, ClusterRouterPool, ClusterRouterPoolSettings }
import akka.event.EventStream
import akka.japi.Util.immutableSeq
import akka.remote.RemoteActorRefProvider
import akka.remote.RemoteDeployer
import akka.remote.{ RemoteActorRefProvider, RemoteDeployer }
import akka.remote.routing.RemoteRouterConfig
import akka.routing.RouterConfig
import akka.routing.DefaultResizer
import akka.cluster.routing.MixMetricsSelector
import akka.cluster.routing.HeapMetricsSelector
import akka.cluster.routing.SystemLoadAverageMetricsSelector
import akka.cluster.routing.CpuMetricsSelector
import akka.cluster.routing.MetricsSelector
import akka.dispatch.sysmsg.SystemMessage
import akka.actor.ActorRef
import akka.actor.Props
import akka.routing.Pool
import akka.routing.Group
import akka.cluster.routing.ClusterRouterPool
import akka.cluster.routing.ClusterRouterGroup
import com.typesafe.config.ConfigFactory
import akka.cluster.routing.ClusterRouterPoolSettings
import akka.cluster.routing.ClusterRouterGroupSettings
import akka.routing.{ Group, Pool }
import com.typesafe.config.Config

/**
* INTERNAL API
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -128,7 +128,7 @@ final case class AdaptiveLoadBalancingRoutingLogic(system: ActorSystem, metricsS
@SerialVersionUID(1L)
final case class AdaptiveLoadBalancingPool(
metricsSelector: MetricsSelector = MixMetricsSelector,
override val nrOfInstances: Int = 0,
val nrOfInstances: Int = 0,
override val supervisorStrategy: SupervisorStrategy = Pool.defaultSupervisorStrategy,
override val routerDispatcher: String = Dispatchers.DefaultDispatcherId,
override val usePoolDispatcher: Boolean = false)
Expand All @@ -149,6 +149,8 @@ final case class AdaptiveLoadBalancingPool(

override def resizer: Option[Resizer] = None

override def nrOfInstances(sys: ActorSystem) = this.nrOfInstances

override def createRouter(system: ActorSystem): Router =
new Router(AdaptiveLoadBalancingRoutingLogic(system, metricsSelector))

Expand Down
Loading

0 comments on commit c2983c7

Please sign in to comment.