Rename dispatcherFactory to dispatchers in ActorSystem. See #1458

This commit is contained in:
Patrik Nordwall 2011-12-21 19:07:54 +01:00
parent df260f8939
commit 6eb7e1d438
15 changed files with 44 additions and 44 deletions

View file

@ -444,18 +444,18 @@ class DispatcherModelSpec extends ActorModelSpec(DispatcherModelSpec.config) {
override def registerInterceptedDispatcher(): MessageDispatcherInterceptor = { override def registerInterceptedDispatcher(): MessageDispatcherInterceptor = {
// use new id for each invocation, since the MessageDispatcherInterceptor holds state // use new id for each invocation, since the MessageDispatcherInterceptor holds state
val id = "dispatcher-" + dispatcherCount.incrementAndGet() val id = "dispatcher-" + dispatcherCount.incrementAndGet()
val dispatcherConfigurator = new MessageDispatcherConfigurator(system.settings.config.getConfig("dispatcher"), system.dispatcherFactory.prerequisites) { val dispatcherConfigurator = new MessageDispatcherConfigurator(system.settings.config.getConfig("dispatcher"), system.dispatchers.prerequisites) {
val instance = { val instance = {
ThreadPoolConfigDispatcherBuilder(config ThreadPoolConfigDispatcherBuilder(config
new Dispatcher(system.dispatcherFactory.prerequisites, id, id, system.settings.DispatcherThroughput, new Dispatcher(system.dispatchers.prerequisites, id, id, system.settings.DispatcherThroughput,
system.settings.DispatcherThroughputDeadlineTime, system.dispatcherFactory.MailboxType, system.settings.DispatcherThroughputDeadlineTime, system.dispatchers.MailboxType,
config, system.settings.DispatcherDefaultShutdown) with MessageDispatcherInterceptor, config, system.settings.DispatcherDefaultShutdown) with MessageDispatcherInterceptor,
ThreadPoolConfig()).build ThreadPoolConfig()).build
} }
override def dispatcher(): MessageDispatcher = instance override def dispatcher(): MessageDispatcher = instance
} }
system.dispatcherFactory.register(id, dispatcherConfigurator) system.dispatchers.register(id, dispatcherConfigurator)
system.dispatcherFactory.lookup(id).asInstanceOf[MessageDispatcherInterceptor] system.dispatchers.lookup(id).asInstanceOf[MessageDispatcherInterceptor]
} }
override def dispatcherType = "Dispatcher" override def dispatcherType = "Dispatcher"
@ -505,19 +505,19 @@ class BalancingDispatcherModelSpec extends ActorModelSpec(BalancingDispatcherMod
override def registerInterceptedDispatcher(): MessageDispatcherInterceptor = { override def registerInterceptedDispatcher(): MessageDispatcherInterceptor = {
// use new id for each invocation, since the MessageDispatcherInterceptor holds state // use new id for each invocation, since the MessageDispatcherInterceptor holds state
val id = "dispatcher-" + dispatcherCount.incrementAndGet() val id = "dispatcher-" + dispatcherCount.incrementAndGet()
val dispatcherConfigurator = new MessageDispatcherConfigurator(system.settings.config.getConfig("dispatcher"), system.dispatcherFactory.prerequisites) { val dispatcherConfigurator = new MessageDispatcherConfigurator(system.settings.config.getConfig("dispatcher"), system.dispatchers.prerequisites) {
val instance = { val instance = {
ThreadPoolConfigDispatcherBuilder(config ThreadPoolConfigDispatcherBuilder(config
new BalancingDispatcher(system.dispatcherFactory.prerequisites, id, id, 1, // TODO check why 1 here? (came from old test) new BalancingDispatcher(system.dispatchers.prerequisites, id, id, 1, // TODO check why 1 here? (came from old test)
system.settings.DispatcherThroughputDeadlineTime, system.dispatcherFactory.MailboxType, system.settings.DispatcherThroughputDeadlineTime, system.dispatchers.MailboxType,
config, system.settings.DispatcherDefaultShutdown) with MessageDispatcherInterceptor, config, system.settings.DispatcherDefaultShutdown) with MessageDispatcherInterceptor,
ThreadPoolConfig()).build ThreadPoolConfig()).build
} }
override def dispatcher(): MessageDispatcher = instance override def dispatcher(): MessageDispatcher = instance
} }
system.dispatcherFactory.register(id, dispatcherConfigurator) system.dispatchers.register(id, dispatcherConfigurator)
system.dispatcherFactory.lookup(id).asInstanceOf[MessageDispatcherInterceptor] system.dispatchers.lookup(id).asInstanceOf[MessageDispatcherInterceptor]
} }
override def dispatcherType = "Balancing Dispatcher" override def dispatcherType = "Balancing Dispatcher"

View file

@ -23,7 +23,7 @@ object DispatchersSpec {
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) { class DispatchersSpec extends AkkaSpec(DispatchersSpec.config) {
val df = system.dispatcherFactory val df = system.dispatchers
import df._ import df._
val tipe = "type" val tipe = "type"

View file

@ -166,7 +166,7 @@ object CustomMailboxSpec {
class CustomMailboxSpec extends AkkaSpec(CustomMailboxSpec.config) { class CustomMailboxSpec extends AkkaSpec(CustomMailboxSpec.config) {
"Dispatcher configuration" must { "Dispatcher configuration" must {
"support custom mailboxType" in { "support custom mailboxType" in {
val dispatcher = system.dispatcherFactory.lookup("my-dispatcher") val dispatcher = system.dispatchers.lookup("my-dispatcher")
dispatcher.createMailbox(null).getClass must be(classOf[CustomMailboxSpec.MyMailbox]) dispatcher.createMailbox(null).getClass must be(classOf[CustomMailboxSpec.MyMailbox])
} }
} }

View file

@ -13,19 +13,19 @@ class PriorityDispatcherSpec extends AkkaSpec with DefaultTimeout {
// FIXME #1458: how should we make it easy to configure prio mailbox? // FIXME #1458: how should we make it easy to configure prio mailbox?
val dispatcherKey = "unbounded-prio-dispatcher" val dispatcherKey = "unbounded-prio-dispatcher"
val dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatcherFactory.defaultDispatcherConfig, system.dispatcherFactory.prerequisites) { val dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatchers.defaultDispatcherConfig, system.dispatchers.prerequisites) {
val instance = { val instance = {
val mailboxType = UnboundedPriorityMailbox(PriorityGenerator({ val mailboxType = UnboundedPriorityMailbox(PriorityGenerator({
case i: Int i //Reverse order case i: Int i //Reverse order
case 'Result Int.MaxValue case 'Result Int.MaxValue
}: Any Int)) }: Any Int))
system.dispatcherFactory.newDispatcher(dispatcherKey, 5, mailboxType).build system.dispatchers.newDispatcher(dispatcherKey, 5, mailboxType).build
} }
override def dispatcher(): MessageDispatcher = instance override def dispatcher(): MessageDispatcher = instance
} }
system.dispatcherFactory.register(dispatcherKey, dispatcherConfigurator) system.dispatchers.register(dispatcherKey, dispatcherConfigurator)
testOrdering(dispatcherKey) testOrdering(dispatcherKey)
} }
@ -34,19 +34,19 @@ class PriorityDispatcherSpec extends AkkaSpec with DefaultTimeout {
// FIXME #1458: how should we make it easy to configure prio mailbox? // FIXME #1458: how should we make it easy to configure prio mailbox?
val dispatcherKey = "bounded-prio-dispatcher" val dispatcherKey = "bounded-prio-dispatcher"
val dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatcherFactory.defaultDispatcherConfig, system.dispatcherFactory.prerequisites) { val dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatchers.defaultDispatcherConfig, system.dispatchers.prerequisites) {
val instance = { val instance = {
val mailboxType = BoundedPriorityMailbox(PriorityGenerator({ val mailboxType = BoundedPriorityMailbox(PriorityGenerator({
case i: Int i //Reverse order case i: Int i //Reverse order
case 'Result Int.MaxValue case 'Result Int.MaxValue
}: Any Int), 1000, system.settings.MailboxPushTimeout) }: Any Int), 1000, system.settings.MailboxPushTimeout)
system.dispatcherFactory.newDispatcher(dispatcherKey, 5, mailboxType).build system.dispatchers.newDispatcher(dispatcherKey, 5, mailboxType).build
} }
override def dispatcher(): MessageDispatcher = instance override def dispatcher(): MessageDispatcher = instance
} }
system.dispatcherFactory.register(dispatcherKey, dispatcherConfigurator) system.dispatchers.register(dispatcherKey, dispatcherConfigurator)
testOrdering(dispatcherKey) testOrdering(dispatcherKey)
} }

View file

@ -29,14 +29,14 @@ class CallingThreadDispatcherModelSpec extends ActorModelSpec(CallingThreadDispa
override def registerInterceptedDispatcher(): MessageDispatcherInterceptor = { override def registerInterceptedDispatcher(): MessageDispatcherInterceptor = {
// use new id for each invocation, since the MessageDispatcherInterceptor holds state // use new id for each invocation, since the MessageDispatcherInterceptor holds state
val dispatcherId = "test-calling-thread" + dispatcherCount.incrementAndGet() val dispatcherId = "test-calling-thread" + dispatcherCount.incrementAndGet()
val dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatcherFactory.defaultDispatcherConfig, system.dispatcherFactory.prerequisites) { val dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatchers.defaultDispatcherConfig, system.dispatchers.prerequisites) {
val instance = new CallingThreadDispatcher(prerequisites) with MessageDispatcherInterceptor { val instance = new CallingThreadDispatcher(prerequisites) with MessageDispatcherInterceptor {
override def id: String = dispatcherId override def id: String = dispatcherId
} }
override def dispatcher(): MessageDispatcher = instance override def dispatcher(): MessageDispatcher = instance
} }
system.dispatcherFactory.register(dispatcherId, dispatcherConfigurator) system.dispatchers.register(dispatcherId, dispatcherConfigurator)
system.dispatcherFactory.lookup(dispatcherId).asInstanceOf[MessageDispatcherInterceptor] system.dispatchers.lookup(dispatcherId).asInstanceOf[MessageDispatcherInterceptor]
} }
override def dispatcherType = "Calling Thread Dispatcher" override def dispatcherType = "Calling Thread Dispatcher"

View file

@ -258,7 +258,7 @@ private[akka] class ActorCell(
} }
@inline @inline
final def dispatcher: MessageDispatcher = system.dispatcherFactory.lookup(props.dispatcher) final def dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
/** /**
* UntypedActorContext impl * UntypedActorContext impl

View file

@ -271,7 +271,7 @@ abstract class ActorSystem extends ActorRefFactory {
/** /**
* Helper object for looking up configured dispatchers. * Helper object for looking up configured dispatchers.
*/ */
def dispatcherFactory: Dispatchers def dispatchers: Dispatchers
/** /**
* Default dispatcher as configured. This dispatcher is used for all actors * Default dispatcher as configured. This dispatcher is used for all actors
@ -412,8 +412,8 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor
} }
} }
val dispatcherFactory = new Dispatchers(settings, DefaultDispatcherPrerequisites(eventStream, deadLetterMailbox, scheduler)) val dispatchers = new Dispatchers(settings, DefaultDispatcherPrerequisites(eventStream, deadLetterMailbox, scheduler))
val dispatcher = dispatcherFactory.defaultGlobalDispatcher val dispatcher = dispatchers.defaultGlobalDispatcher
def terminationFuture: Future[Unit] = provider.terminationFuture def terminationFuture: Future[Unit] = provider.terminationFuture
def lookupRoot: InternalActorRef = provider.rootGuardian def lookupRoot: InternalActorRef = provider.rootGuardian

View file

@ -90,9 +90,9 @@ public class DispatcherDocTestBase {
// FIXME #1458: how should we make it easy to configure prio mailbox? // FIXME #1458: how should we make it easy to configure prio mailbox?
// We create a new Priority dispatcher and seed it with the priority generator // We create a new Priority dispatcher and seed it with the priority generator
final String dispatcherKey = "prio-dispatcher"; final String dispatcherKey = "prio-dispatcher";
MessageDispatcherConfigurator dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatcherFactory() MessageDispatcherConfigurator dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatchers()
.defaultDispatcherConfig(), system.dispatcherFactory().prerequisites()) { .defaultDispatcherConfig(), system.dispatchers().prerequisites()) {
private final MessageDispatcher instance = system.dispatcherFactory() private final MessageDispatcher instance = system.dispatchers()
.newDispatcher(dispatcherKey, 5, new UnboundedPriorityMailbox(generator)).build(); .newDispatcher(dispatcherKey, 5, new UnboundedPriorityMailbox(generator)).build();
@Override @Override
@ -100,7 +100,7 @@ public class DispatcherDocTestBase {
return instance; return instance;
} }
}; };
system.dispatcherFactory().register(dispatcherKey, dispatcherConfigurator); system.dispatchers().register(dispatcherKey, dispatcherConfigurator);
ActorRef myActor = system.actorOf( // We create a new Actor that just prints out what it processes ActorRef myActor = system.actorOf( // We create a new Actor that just prints out what it processes
new Props().withCreator(new UntypedActorFactory() { new Props().withCreator(new UntypedActorFactory() {

View file

@ -44,7 +44,7 @@ There are 4 different types of message dispatchers:
It is recommended to define the dispatcher in :ref:`configuration` to allow for tuning for different environments. It is recommended to define the dispatcher in :ref:`configuration` to allow for tuning for different environments.
Example of a custom event-based dispatcher, which can be fetched with ``system.dispatcherFactory().lookup("my-dispatcher")`` Example of a custom event-based dispatcher, which can be fetched with ``system.dispatchers().lookup("my-dispatcher")``
as in the example above: as in the example above:
.. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config .. includecode:: ../scala/code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config
@ -54,7 +54,7 @@ Default values are taken from ``default-dispatcher``, i.e. all options doesn't n
.. warning:: .. warning::
Factory methods for creating dispatchers programmatically are available in ``akka.dispatch.Dispatchers``, i.e. Factory methods for creating dispatchers programmatically are available in ``akka.dispatch.Dispatchers``, i.e.
``dispatcherFactory`` of the ``ActorSystem``. These methods will probably be changed or removed before ``dispatchers`` of the ``ActorSystem``. These methods will probably be changed or removed before
2.0 final release, because dispatchers need to be defined by configuration to work in a clustered setup. 2.0 final release, because dispatchers need to be defined by configuration to work in a clustered setup.
Let's now walk through the different dispatchers in more detail. Let's now walk through the different dispatchers in more detail.

View file

@ -78,7 +78,7 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) {
} }
"defining dispatcher with bounded queue" in { "defining dispatcher with bounded queue" in {
val dispatcher = system.dispatcherFactory.lookup("my-dispatcher-bounded-queue") val dispatcher = system.dispatchers.lookup("my-dispatcher-bounded-queue")
} }
"defining pinned dispatcher" in { "defining pinned dispatcher" in {
@ -100,11 +100,11 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) {
// FIXME #1458: how should we make it easy to configure prio mailbox? // FIXME #1458: how should we make it easy to configure prio mailbox?
// We create a new Priority dispatcher and seed it with the priority generator // We create a new Priority dispatcher and seed it with the priority generator
val dispatcherKey = "prio-dispatcher" val dispatcherKey = "prio-dispatcher"
val dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatcherFactory.defaultDispatcherConfig, system.dispatcherFactory.prerequisites) { val dispatcherConfigurator = new MessageDispatcherConfigurator(system.dispatchers.defaultDispatcherConfig, system.dispatchers.prerequisites) {
val instance = system.dispatcherFactory.newDispatcher(dispatcherKey, 5, UnboundedPriorityMailbox(gen)).build val instance = system.dispatchers.newDispatcher(dispatcherKey, 5, UnboundedPriorityMailbox(gen)).build
override def dispatcher(): MessageDispatcher = instance override def dispatcher(): MessageDispatcher = instance
} }
system.dispatcherFactory.register(dispatcherKey, dispatcherConfigurator) system.dispatchers.register(dispatcherKey, dispatcherConfigurator)
val a = system.actorOf( // We create a new Actor that just prints out what it processes val a = system.actorOf( // We create a new Actor that just prints out what it processes
Props(new Actor { Props(new Actor {
@ -140,7 +140,7 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) {
} }
"defining balancing dispatcher" in { "defining balancing dispatcher" in {
val dispatcher = system.dispatcherFactory.lookup("my-balancing-dispatcher") val dispatcher = system.dispatchers.lookup("my-balancing-dispatcher")
} }
} }

View file

@ -181,8 +181,8 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
val actor = system.actorOf(Props[MyDoubleEcho]) val actor = system.actorOf(Props[MyDoubleEcho])
actor ! (probe1.ref, probe2.ref) actor ! (probe1.ref, probe2.ref)
actor ! "hello" actor ! "hello"
probe1.expectMsg(50 millis, "hello") probe1.expectMsg(500 millis, "hello")
probe2.expectMsg(50 millis, "hello") probe2.expectMsg(500 millis, "hello")
//#test-probe //#test-probe
//#test-special-probe //#test-special-probe

View file

@ -44,7 +44,7 @@ There are 4 different types of message dispatchers:
It is recommended to define the dispatcher in :ref:`configuration` to allow for tuning for different environments. It is recommended to define the dispatcher in :ref:`configuration` to allow for tuning for different environments.
Example of a custom event-based dispatcher, which can be fetched with ``system.dispatcherFactory.lookup("my-dispatcher")`` Example of a custom event-based dispatcher, which can be fetched with ``system.dispatchers.lookup("my-dispatcher")``
as in the example above: as in the example above:
.. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config .. includecode:: code/akka/docs/dispatcher/DispatcherDocSpec.scala#my-dispatcher-config
@ -54,7 +54,7 @@ Default values are taken from ``default-dispatcher``, i.e. all options doesn't n
.. warning:: .. warning::
Factory methods for creating dispatchers programmatically are available in ``akka.dispatch.Dispatchers``, i.e. Factory methods for creating dispatchers programmatically are available in ``akka.dispatch.Dispatchers``, i.e.
``dispatcherFactory`` of the ``ActorSystem``. These methods will probably be changed or removed before ``dispatchers`` of the ``ActorSystem``. These methods will probably be changed or removed before
2.0 final release, because dispatchers need to be defined by configuration to work in a clustered setup. 2.0 final release, because dispatchers need to be defined by configuration to work in a clustered setup.
Let's now walk through the different dispatchers in more detail. Let's now walk through the different dispatchers in more detail.

View file

@ -75,7 +75,7 @@ class Remote(val settings: ActorSystem.Settings, val remoteSettings: RemoteSetti
_provider = provider _provider = provider
_serialization = SerializationExtension(system) _serialization = SerializationExtension(system)
_computeGridDispatcher = system.dispatcherFactory.lookup("akka.remote.compute-grid-dispatcher") _computeGridDispatcher = system.dispatchers.lookup("akka.remote.compute-grid-dispatcher")
_remoteDaemon = new RemoteSystemDaemon(system, this, system.provider.rootPath / "remote", system.provider.rootGuardian, log) _remoteDaemon = new RemoteSystemDaemon(system, this, system.provider.rootPath / "remote", system.provider.rootGuardian, log)
_eventStream = new NetworkEventStream(system) _eventStream = new NetworkEventStream(system)
_server = { _server = {

View file

@ -116,7 +116,7 @@ object TestActorRef {
apply[T](props, system.asInstanceOf[ActorSystemImpl].guardian, name) apply[T](props, system.asInstanceOf[ActorSystemImpl].guardian, name)
def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit system: ActorSystem): TestActorRef[T] = def apply[T <: Actor](props: Props, supervisor: ActorRef, name: String)(implicit system: ActorSystem): TestActorRef[T] =
new TestActorRef(system.asInstanceOf[ActorSystemImpl], system.dispatcherFactory.prerequisites, props, supervisor.asInstanceOf[InternalActorRef], name) new TestActorRef(system.asInstanceOf[ActorSystemImpl], system.dispatchers.prerequisites, props, supervisor.asInstanceOf[InternalActorRef], name)
def apply[T <: Actor](implicit m: Manifest[T], system: ActorSystem): TestActorRef[T] = apply[T](randomName) def apply[T <: Actor](implicit m: Manifest[T], system: ActorSystem): TestActorRef[T] = apply[T](randomName)

View file

@ -89,11 +89,11 @@ object TestFSMRef {
def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = { def apply[S, D, T <: Actor](factory: T)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = {
val impl = system.asInstanceOf[ActorSystemImpl] //FIXME should we rely on this cast to work here? val impl = system.asInstanceOf[ActorSystemImpl] //FIXME should we rely on this cast to work here?
new TestFSMRef(impl, system.dispatcherFactory.prerequisites, Props(creator = () factory), impl.guardian.asInstanceOf[InternalActorRef], TestActorRef.randomName) new TestFSMRef(impl, system.dispatchers.prerequisites, Props(creator = () factory), impl.guardian.asInstanceOf[InternalActorRef], TestActorRef.randomName)
} }
def apply[S, D, T <: Actor](factory: T, name: String)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = { def apply[S, D, T <: Actor](factory: T, name: String)(implicit ev: T <:< FSM[S, D], system: ActorSystem): TestFSMRef[S, D, T] = {
val impl = system.asInstanceOf[ActorSystemImpl] //FIXME should we rely on this cast to work here? val impl = system.asInstanceOf[ActorSystemImpl] //FIXME should we rely on this cast to work here?
new TestFSMRef(impl, system.dispatcherFactory.prerequisites, Props(creator = () factory), impl.guardian.asInstanceOf[InternalActorRef], name) new TestFSMRef(impl, system.dispatchers.prerequisites, Props(creator = () factory), impl.guardian.asInstanceOf[InternalActorRef], name)
} }
} }