From 3181905fedf274e7dfb3cd865222ec8865aac16c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 20 May 2011 22:41:41 +0200 Subject: [PATCH] Renaming EBEDD to Dispatcher, EBEDWSD to BalancingDispatcher, ThreadBasedDispatcher to PinnedDispatcher and PEBEDD to PriorityDispatcher, closing ticket #784 --- .../akka/actor/actor/TypedActorSpec.scala | 2 +- .../actor/supervisor/SupervisorMiscSpec.scala | 8 +- .../test/scala/akka/config/ConfigSpec.scala | 2 +- .../scala/akka/dispatch/ActorModelSpec.scala | 8 +- .../scala/akka/dispatch/DispatchersSpec.scala | 8 +- ...rBasedEventDrivenDispatcherActorSpec.scala | 16 ++-- ...BasedEventDrivenDispatcherActorsSpec.scala | 2 +- ...ventDrivenWorkStealingDispatcherSpec.scala | 8 +- .../dispatch/PriorityDispatcherSpec.scala | 4 +- .../akka/dispatch/ThreadBasedActorSpec.scala | 4 +- .../src/main/scala/akka/actor/ActorRef.scala | 2 +- ...atcher.scala => BalancingDispatcher.scala} | 6 +- ...rivenDispatcher.scala => Dispatcher.scala} | 28 +++---- .../scala/akka/dispatch/Dispatchers.scala | 76 +++++++++---------- ...ispatcher.scala => PinnedDispatcher.scala} | 8 +- .../main/scala/akka/event/EventHandler.scala | 2 +- .../src/main/scala/akka/cluster/Cluster.scala | 4 +- akka-docs/cluster/durable-mailbox.rst | 12 +-- akka-docs/general/configuration.rst | 2 +- akka-docs/java/dispatchers.rst | 32 ++++---- .../project/migration-guide-0.9.x-0.10.x.rst | 2 +- akka-docs/project/release-notes.rst | 8 +- akka-docs/scala/dispatchers.rst | 30 ++++---- .../actor/mailbox/DurableDispatcher.scala | 10 +-- .../akka/actor/mailbox/DurableMailbox.scala | 4 +- .../src/main/scala/akka/agent/Agent.scala | 2 +- config/akka-reference.conf | 20 ++--- project/build/AkkaProject.scala | 2 +- 28 files changed, 156 insertions(+), 156 deletions(-) rename akka-actor/src/main/scala/akka/dispatch/{ExecutorBasedEventDrivenWorkStealingDispatcher.scala => BalancingDispatcher.scala} (96%) rename akka-actor/src/main/scala/akka/dispatch/{ExecutorBasedEventDrivenDispatcher.scala => Dispatcher.scala} (90%) rename akka-actor/src/main/scala/akka/dispatch/{ThreadBasedDispatcher.scala => PinnedDispatcher.scala} (86%) diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/TypedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/TypedActorSpec.scala index 43111cc659..5b9f155d75 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/TypedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/TypedActorSpec.scala @@ -264,7 +264,7 @@ class TypedActorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach "be able to use work-stealing dispatcher" in { val config = Configuration( Duration(6600, "ms"), - Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher") + Dispatchers.newBalancingDispatcher("pooled-dispatcher") .withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity .setCorePoolSize(60) .setMaxPoolSize(60) diff --git a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala index c81455b9e8..6b1017e79d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala @@ -16,7 +16,7 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers { val countDownLatch = new CountDownLatch(4) val actor1 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) + self.dispatcher = Dispatchers.newPinnedDispatcher(self) override def postRestart(cause: Throwable) { countDownLatch.countDown() } protected def receive = { @@ -26,7 +26,7 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers { }).start() val actor2 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) + self.dispatcher = Dispatchers.newPinnedDispatcher(self) override def postRestart(cause: Throwable) { countDownLatch.countDown() } protected def receive = { @@ -36,7 +36,7 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers { }).start() val actor3 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("test").build + self.dispatcher = Dispatchers.newDispatcher("test").build override def postRestart(cause: Throwable) { countDownLatch.countDown() } protected def receive = { @@ -46,7 +46,7 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers { }).start() val actor4 = Actor.actorOf(new Actor { - self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) + self.dispatcher = Dispatchers.newPinnedDispatcher(self) override def postRestart(cause: Throwable) { countDownLatch.countDown() } protected def receive = { diff --git a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala index 916852a7b4..66d21435f4 100644 --- a/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/config/ConfigSpec.scala @@ -20,7 +20,7 @@ class ConfigSpec extends WordSpec with MustMatchers { getString("akka.time-unit") must equal(Some("seconds")) getString("akka.version") must equal(Some("2.0-SNAPSHOT")) - getString("akka.actor.default-dispatcher.type") must equal(Some("GlobalExecutorBasedEventDriven")) + getString("akka.actor.default-dispatcher.type") must equal(Some("GlobalDispatcher")) getInt("akka.actor.default-dispatcher.keep-alive-time") must equal(Some(60)) getDouble("akka.actor.default-dispatcher.core-pool-size-factor") must equal(Some(1.0)) getDouble("akka.actor.default-dispatcher.max-pool-size-factor") must equal(Some(4.0)) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala index c6af345d68..97dc67da22 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/ActorModelSpec.scala @@ -344,12 +344,12 @@ abstract class ActorModelSpec extends JUnitSuite { } } -class ExecutorBasedEventDrivenDispatcherModelTest extends ActorModelSpec { +class DispatcherModelTest extends ActorModelSpec { def newInterceptedDispatcher = - new ExecutorBasedEventDrivenDispatcher("foo") with MessageDispatcherInterceptor + new Dispatcher("foo") with MessageDispatcherInterceptor } -class ExecutorBasedEventDrivenWorkStealingDispatcherModelTest extends ActorModelSpec { +class BalancingDispatcherModelTest extends ActorModelSpec { def newInterceptedDispatcher = - new ExecutorBasedEventDrivenWorkStealingDispatcher("foo") with MessageDispatcherInterceptor + new BalancingDispatcher("foo") with MessageDispatcherInterceptor } diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/DispatchersSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/DispatchersSpec.scala index cc75c5d43a..f7df14e195 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/DispatchersSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/DispatchersSpec.scala @@ -21,15 +21,15 @@ object DispatchersSpec { val executorbounds = "executor-bounds" val allowcoretimeout = "allow-core-timeout" val rejectionpolicy = "rejection-policy" // abort, caller-runs, discard-oldest, discard - val throughput = "throughput" // Throughput for ExecutorBasedEventDrivenDispatcher + val throughput = "throughput" // Throughput for Dispatcher def instance(dispatcher: MessageDispatcher): (MessageDispatcher) ⇒ Boolean = _ == dispatcher def ofType[T <: MessageDispatcher: Manifest]: (MessageDispatcher) ⇒ Boolean = _.getClass == manifest[T].erasure def typesAndValidators: Map[String, (MessageDispatcher) ⇒ Boolean] = Map( - "ExecutorBasedEventDrivenWorkStealing" -> ofType[ExecutorBasedEventDrivenWorkStealingDispatcher], - "ExecutorBasedEventDriven" -> ofType[ExecutorBasedEventDrivenDispatcher], - "GlobalExecutorBasedEventDriven" -> instance(globalExecutorBasedEventDrivenDispatcher)) + "BalancingDispatcher" -> ofType[BalancingDispatcher], + "Dispatcher" -> ofType[Dispatcher], + "GlobalDispatcher" -> instance(globalDispatcher)) def validTypes = typesAndValidators.keys.toList diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala index c330d948d3..69fc9ea635 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala @@ -3,14 +3,14 @@ package akka.actor.dispatch import java.util.concurrent.{ CountDownLatch, TimeUnit } import org.scalatest.junit.JUnitSuite import org.junit.Test -import akka.dispatch.{ Dispatchers, ExecutorBasedEventDrivenDispatcher } +import akka.dispatch.{ Dispatchers, Dispatcher } import akka.actor.Actor import Actor._ import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger } -object ExecutorBasedEventDrivenDispatcherActorSpec { +object DispatcherActorSpec { class TestActor extends Actor { - self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString).build + self.dispatcher = Dispatchers.newDispatcher(self.uuid.toString).build def receive = { case "Hello" ⇒ self.reply("World") @@ -23,14 +23,14 @@ object ExecutorBasedEventDrivenDispatcherActorSpec { val oneWay = new CountDownLatch(1) } class OneWayTestActor extends Actor { - self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString).build + self.dispatcher = Dispatchers.newDispatcher(self.uuid.toString).build def receive = { case "OneWay" ⇒ OneWayTestActor.oneWay.countDown() } } } -class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite { - import ExecutorBasedEventDrivenDispatcherActorSpec._ +class DispatcherActorSpec extends JUnitSuite { + import DispatcherActorSpec._ private val unit = TimeUnit.MILLISECONDS @@ -74,7 +74,7 @@ class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite { @Test def shouldRespectThroughput { val throughputDispatcher = Dispatchers. - newExecutorBasedEventDrivenDispatcher("THROUGHPUT", 101, 0, Dispatchers.MAILBOX_TYPE). + newDispatcher("THROUGHPUT", 101, 0, Dispatchers.MAILBOX_TYPE). setCorePoolSize(1). build @@ -110,7 +110,7 @@ class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite { def shouldRespectThroughputDeadline { val deadlineMs = 100 val throughputDispatcher = Dispatchers. - newExecutorBasedEventDrivenDispatcher("THROUGHPUT", 2, deadlineMs, Dispatchers.MAILBOX_TYPE). + newDispatcher("THROUGHPUT", 2, deadlineMs, Dispatchers.MAILBOX_TYPE). setCorePoolSize(1). build val works = new AtomicBoolean(true) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala index d636ecc932..f7cc4a0956 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala @@ -12,7 +12,7 @@ import Actor._ * * @author Jan Van Besien */ -class ExecutorBasedEventDrivenDispatcherActorsSpec extends JUnitSuite with MustMatchers { +class DispatcherActorsSpec extends JUnitSuite with MustMatchers { class SlowActor(finishedCounter: CountDownLatch) extends Actor { def receive = { diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala index 064433c6ca..9ce2c9dccd 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala @@ -10,9 +10,9 @@ import akka.actor.{ IllegalActorStateException, Actor } import Actor._ import akka.dispatch.{ MessageQueue, Dispatchers } -object ExecutorBasedEventDrivenWorkStealingDispatcherSpec { +object BalancingDispatcherSpec { - def newWorkStealer() = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher", 1).build + def newWorkStealer() = Dispatchers.newBalancingDispatcher("pooled-dispatcher", 1).build val delayableActorDispatcher, sharedActorDispatcher, parentActorDispatcher = newWorkStealer() @@ -52,8 +52,8 @@ object ExecutorBasedEventDrivenWorkStealingDispatcherSpec { /** * @author Jan Van Besien */ -class ExecutorBasedEventDrivenWorkStealingDispatcherSpec extends JUnitSuite with MustMatchers { - import ExecutorBasedEventDrivenWorkStealingDispatcherSpec._ +class BalancingDispatcherSpec extends JUnitSuite with MustMatchers { + import BalancingDispatcherSpec._ @Test def fastActorShouldStealWorkFromSlowActor { diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala index ff2e85fb05..5740edb9b8 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -8,7 +8,7 @@ import java.util.concurrent.CountDownLatch class PriorityDispatcherSpec extends WordSpec with MustMatchers { - "A PriorityExecutorBasedEventDrivenDispatcher" must { + "A PriorityDispatcher" must { "Order it's messages according to the specified comparator using an unbounded mailbox" in { testOrdering(UnboundedMailbox()) } @@ -19,7 +19,7 @@ class PriorityDispatcherSpec extends WordSpec with MustMatchers { } def testOrdering(mboxType: MailboxType) { - val dispatcher = new PriorityExecutorBasedEventDrivenDispatcher("Test", + val dispatcher = new PriorityDispatcher("Test", PriorityGenerator({ case i: Int ⇒ i //Reverse order case 'Result ⇒ Int.MaxValue diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/ThreadBasedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/ThreadBasedActorSpec.scala index fd127c1a14..722c48fbdc 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/ThreadBasedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/ThreadBasedActorSpec.scala @@ -10,7 +10,7 @@ import Actor._ object ThreadBasedActorSpec { class TestActor extends Actor { - self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) + self.dispatcher = Dispatchers.newPinnedDispatcher(self) def receive = { case "Hello" ⇒ @@ -30,7 +30,7 @@ class ThreadBasedActorSpec extends JUnitSuite { def shouldSendOneWay { var oneWay = new CountDownLatch(1) val actor = actorOf(new Actor { - self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) + self.dispatcher = Dispatchers.newPinnedDispatcher(self) def receive = { case "OneWay" ⇒ oneWay.countDown() } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 79a768f85e..61fbf56be9 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -160,7 +160,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal /** * Akka Java API.

- * The default dispatcher is the Dispatchers.globalExecutorBasedEventDrivenDispatcher. + * The default dispatcher is the Dispatchers.globalDispatcher. * This means that all actors will share the same event-driven executor based dispatcher. *

* You can override it so it fits the specific use-case that the actor is used for. diff --git a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala similarity index 96% rename from akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala rename to akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index eaf4472901..0460720a73 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -19,18 +19,18 @@ import util.DynamicVariable * The preferred way of creating dispatchers is to use * the {@link akka.dispatch.Dispatchers} factory object. * - * @see akka.dispatch.ExecutorBasedEventDrivenWorkStealingDispatcher + * @see akka.dispatch.BalancingDispatcher * @see akka.dispatch.Dispatchers * * @author Viktor Klang */ -class ExecutorBasedEventDrivenWorkStealingDispatcher( +class BalancingDispatcher( _name: String, throughput: Int = Dispatchers.THROUGHPUT, throughputDeadlineTime: Int = Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType: MailboxType = Dispatchers.MAILBOX_TYPE, config: ThreadPoolConfig = ThreadPoolConfig()) - extends ExecutorBasedEventDrivenDispatcher(_name, throughput, throughputDeadlineTime, mailboxType, config) { + extends Dispatcher(_name, throughput, throughputDeadlineTime, mailboxType, config) { def this(_name: String, throughput: Int, throughputDeadlineTime: Int, mailboxType: MailboxType) = this(_name, throughput, throughputDeadlineTime, mailboxType, ThreadPoolConfig()) // Needed for Java API usage diff --git a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala similarity index 90% rename from akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala rename to akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 3c985e6409..ab2287a589 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -28,7 +28,7 @@ import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionExcept *

* Example usage: *

- *   val dispatcher = new ExecutorBasedEventDrivenDispatcher("name")
+ *   val dispatcher = new Dispatcher("name")
  *   dispatcher
  *     .withNewThreadPoolWithBoundedBlockingQueue(100)
  *     .setCorePoolSize(16)
@@ -43,7 +43,7 @@ import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionExcept
  * 

* Example usage: *

- *   ExecutorBasedEventDrivenDispatcher dispatcher = new ExecutorBasedEventDrivenDispatcher("name");
+ *   Dispatcher dispatcher = new Dispatcher("name");
  *   dispatcher
  *     .withNewThreadPoolWithBoundedBlockingQueue(100)
  *     .setCorePoolSize(16)
@@ -63,7 +63,7 @@ import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionExcept
  *                   always continues until the mailbox is empty.
  *                   Larger values (or zero or negative) increase throughput, smaller values increase fairness
  */
-class ExecutorBasedEventDrivenDispatcher(
+class Dispatcher(
   _name: String,
   val throughput: Int = Dispatchers.THROUGHPUT,
   val throughputDeadlineTime: Int = Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS,
@@ -117,7 +117,7 @@ class ExecutorBasedEventDrivenDispatcher(
     case b: UnboundedMailbox ⇒
       new ConcurrentLinkedQueue[MessageInvocation] with MessageQueue with ExecutableMailbox {
         @inline
-        final def dispatcher = ExecutorBasedEventDrivenDispatcher.this
+        final def dispatcher = Dispatcher.this
         @inline
         final def enqueue(m: MessageInvocation) = this.add(m)
         @inline
@@ -126,7 +126,7 @@ class ExecutorBasedEventDrivenDispatcher(
     case b: BoundedMailbox ⇒
       new DefaultBoundedMessageQueue(b.capacity, b.pushTimeOut) with ExecutableMailbox {
         @inline
-        final def dispatcher = ExecutorBasedEventDrivenDispatcher.this
+        final def dispatcher = Dispatcher.this
       }
   }
 
@@ -173,11 +173,11 @@ class ExecutorBasedEventDrivenDispatcher(
 }
 
 /**
- * This is the behavior of an ExecutorBasedEventDrivenDispatchers mailbox.
+ * This is the behavior of an Dispatchers mailbox.
  */
 trait ExecutableMailbox extends Runnable { self: MessageQueue ⇒
 
-  def dispatcher: ExecutorBasedEventDrivenDispatcher
+  def dispatcher: Dispatcher
 
   final def run = {
     try {
@@ -237,7 +237,7 @@ object PriorityGenerator {
 
 /**
  * A PriorityGenerator is a convenience API to create a Comparator that orders the messages of a
- * PriorityExecutorBasedEventDrivenDispatcher
+ * PriorityDispatcher
  */
 abstract class PriorityGenerator extends java.util.Comparator[MessageInvocation] {
   def gen(message: Any): Int
@@ -247,18 +247,18 @@ abstract class PriorityGenerator extends java.util.Comparator[MessageInvocation]
 }
 
 /**
- * A version of ExecutorBasedEventDrivenDispatcher that gives all actors registered to it a priority mailbox,
+ * A version of Dispatcher that gives all actors registered to it a priority mailbox,
  * prioritized according to the supplied comparator.
  *
  * The dispatcher will process the messages with the _lowest_ priority first.
  */
-class PriorityExecutorBasedEventDrivenDispatcher(
+class PriorityDispatcher(
   name: String,
   val comparator: java.util.Comparator[MessageInvocation],
   throughput: Int = Dispatchers.THROUGHPUT,
   throughputDeadlineTime: Int = Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS,
   mailboxType: MailboxType = Dispatchers.MAILBOX_TYPE,
-  config: ThreadPoolConfig = ThreadPoolConfig()) extends ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineTime, mailboxType, config) with PriorityMailbox {
+  config: ThreadPoolConfig = ThreadPoolConfig()) extends Dispatcher(name, throughput, throughputDeadlineTime, mailboxType, config) with PriorityMailbox {
 
   def this(name: String, comparator: java.util.Comparator[MessageInvocation], throughput: Int, throughputDeadlineTime: Int, mailboxType: MailboxType) =
     this(name, comparator, throughput, throughputDeadlineTime, mailboxType, ThreadPoolConfig()) // Needed for Java API usage
@@ -277,14 +277,14 @@ class PriorityExecutorBasedEventDrivenDispatcher(
 }
 
 /**
- * Can be used to give an ExecutorBasedEventDrivenDispatcher's actors priority-enabled mailboxes
+ * Can be used to give an Dispatcher's actors priority-enabled mailboxes
  *
  * Usage:
- * new ExecutorBasedEventDrivenDispatcher(...) with PriorityMailbox {
+ * new Dispatcher(...) with PriorityMailbox {
  *   val comparator = ...comparator that determines mailbox priority ordering...
  * }
  */
-trait PriorityMailbox { self: ExecutorBasedEventDrivenDispatcher ⇒
+trait PriorityMailbox { self: Dispatcher ⇒
   def comparator: java.util.Comparator[MessageInvocation]
 
   override def createMailbox(actorRef: ActorRef): AnyRef = self.mailboxType match {
diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
index 5de9e91d51..963927582f 100644
--- a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
@@ -18,7 +18,7 @@ import java.util.concurrent.TimeUnit
  * 

* Example usage: *

- *   val dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("name")
+ *   val dispatcher = Dispatchers.newDispatcher("name")
  *   dispatcher
  *     .withNewThreadPoolWithLinkedBlockingQueueWithCapacity(100)
  *     .setCorePoolSize(16)
@@ -32,7 +32,7 @@ import java.util.concurrent.TimeUnit
  * 

* Example usage: *

- *   MessageDispatcher dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("name");
+ *   MessageDispatcher dispatcher = Dispatchers.newDispatcher("name");
  *   dispatcher
  *     .withNewThreadPoolWithLinkedBlockingQueueWithCapacity(100)
  *     .setCorePoolSize(16)
@@ -57,10 +57,10 @@ object Dispatchers {
   val MAILBOX_TYPE: MailboxType = if (MAILBOX_CAPACITY < 1) UnboundedMailbox() else BoundedMailbox()
 
   lazy val defaultGlobalDispatcher = {
-    config.getSection("akka.actor.default-dispatcher").flatMap(from).getOrElse(globalExecutorBasedEventDrivenDispatcher)
+    config.getSection("akka.actor.default-dispatcher").flatMap(from).getOrElse(globalDispatcher)
   }
 
-  object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher("global", THROUGHPUT, THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE)
+  object globalDispatcher extends Dispatcher("global", THROUGHPUT, THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE)
 
   /**
    * Creates an thread based dispatcher serving a single actor through the same single thread.
@@ -68,9 +68,9 @@ object Dispatchers {
    * 

* E.g. each actor consumes its own thread. */ - def newThreadBasedDispatcher(actor: ActorRef) = actor match { - case null ⇒ new ThreadBasedDispatcher() - case some ⇒ new ThreadBasedDispatcher(some) + def newPinnedDispatcher(actor: ActorRef) = actor match { + case null ⇒ new PinnedDispatcher() + case some ⇒ new PinnedDispatcher(some) } /** @@ -79,9 +79,9 @@ object Dispatchers { *

* E.g. each actor consumes its own thread. */ - def newThreadBasedDispatcher(actor: ActorRef, mailboxType: MailboxType) = actor match { - case null ⇒ new ThreadBasedDispatcher(mailboxType) - case some ⇒ new ThreadBasedDispatcher(some, mailboxType) + def newPinnedDispatcher(actor: ActorRef, mailboxType: MailboxType) = actor match { + case null ⇒ new PinnedDispatcher(mailboxType) + case some ⇒ new PinnedDispatcher(some, mailboxType) } /** @@ -89,77 +89,77 @@ object Dispatchers { *

* E.g. each actor consumes its own thread. */ - def newThreadBasedDispatcher(name: String, mailboxType: MailboxType) = - new ThreadBasedDispatcher(name, mailboxType) + def newPinnedDispatcher(name: String, mailboxType: MailboxType) = + new PinnedDispatcher(name, mailboxType) /** * Creates an thread based dispatcher serving a single actor through the same single thread. *

* E.g. each actor consumes its own thread. */ - def newThreadBasedDispatcher(name: String) = - new ThreadBasedDispatcher(name) + def newPinnedDispatcher(name: String) = + new PinnedDispatcher(name) /** * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenDispatcher(name: String) = - ThreadPoolConfigDispatcherBuilder(config ⇒ new ExecutorBasedEventDrivenDispatcher(name, config), ThreadPoolConfig()) + def newDispatcher(name: String) = + ThreadPoolConfigDispatcherBuilder(config ⇒ new Dispatcher(name, config), ThreadPoolConfig()) /** * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, mailboxType: MailboxType) = + def newDispatcher(name: String, throughput: Int, mailboxType: MailboxType) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new ExecutorBasedEventDrivenDispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType, config), ThreadPoolConfig()) + new Dispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType, config), ThreadPoolConfig()) /** * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool. *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = + def newDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineMs, mailboxType, config), ThreadPoolConfig()) + new Dispatcher(name, throughput, throughputDeadlineMs, mailboxType, config), ThreadPoolConfig()) /** * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String) = - ThreadPoolConfigDispatcherBuilder(config ⇒ new ExecutorBasedEventDrivenWorkStealingDispatcher(name, config), ThreadPoolConfig()) + def newBalancingDispatcher(name: String) = + ThreadPoolConfigDispatcherBuilder(config ⇒ new BalancingDispatcher(name, config), ThreadPoolConfig()) /** * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String, throughput: Int) = + def newBalancingDispatcher(name: String, throughput: Int) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new ExecutorBasedEventDrivenWorkStealingDispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE, config), ThreadPoolConfig()) + new BalancingDispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE, config), ThreadPoolConfig()) /** * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) = + def newBalancingDispatcher(name: String, throughput: Int, mailboxType: MailboxType) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new ExecutorBasedEventDrivenWorkStealingDispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType, config), ThreadPoolConfig()) + new BalancingDispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType, config), ThreadPoolConfig()) /** * Creates a executor-based event-driven dispatcher, with work-stealing, serving multiple (millions) of actors through a thread pool. *

* Has a fluent builder interface for configuring its semantics. */ - def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = + def newBalancingDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) = ThreadPoolConfigDispatcherBuilder(config ⇒ - new ExecutorBasedEventDrivenWorkStealingDispatcher(name, throughput, throughputDeadlineMs, mailboxType, config), ThreadPoolConfig()) + new BalancingDispatcher(name, throughput, throughputDeadlineMs, mailboxType, config), ThreadPoolConfig()) /** * Utility function that tries to load the specified dispatcher config from the akka.conf * or else use the supplied default dispatcher @@ -181,7 +181,7 @@ object Dispatchers { * executor-bounds = -1 # Makes the Executor bounded, -1 is unbounded * allow-core-timeout = on # Allow core threads to time out * rejection-policy = "caller-runs" # abort, caller-runs, discard-oldest, discard - * throughput = 5 # Throughput for ExecutorBasedEventDrivenDispatcher + * throughput = 5 # Throughput for Dispatcher * } * ex: from(config.getConfigMap(identifier).get) * @@ -192,9 +192,9 @@ object Dispatchers { */ def from(cfg: Configuration): Option[MessageDispatcher] = { cfg.getString("type") map { - case "ExecutorBasedEventDriven" ⇒ new ExecutorBasedEventDrivenDispatcherConfigurator() - case "ExecutorBasedEventDrivenWorkStealing" ⇒ new ExecutorBasedEventDrivenWorkStealingDispatcherConfigurator() - case "GlobalExecutorBasedEventDriven" ⇒ GlobalExecutorBasedEventDrivenDispatcherConfigurator + case "Dispatcher" ⇒ new DispatcherConfigurator() + case "BalancingDispatcher" ⇒ new BalancingDispatcherConfigurator() + case "GlobalDispatcher" ⇒ GlobalDispatcherConfigurator case fqn ⇒ ReflectiveAccess.getClassFor[MessageDispatcherConfigurator](fqn) match { case r: Right[_, Class[MessageDispatcherConfigurator]] ⇒ @@ -212,13 +212,13 @@ object Dispatchers { } } -object GlobalExecutorBasedEventDrivenDispatcherConfigurator extends MessageDispatcherConfigurator { - def configure(config: Configuration): MessageDispatcher = Dispatchers.globalExecutorBasedEventDrivenDispatcher +object GlobalDispatcherConfigurator extends MessageDispatcherConfigurator { + def configure(config: Configuration): MessageDispatcher = Dispatchers.globalDispatcher } -class ExecutorBasedEventDrivenDispatcherConfigurator extends MessageDispatcherConfigurator { +class DispatcherConfigurator extends MessageDispatcherConfigurator { def configure(config: Configuration): MessageDispatcher = { - configureThreadPool(config, threadPoolConfig ⇒ new ExecutorBasedEventDrivenDispatcher( + configureThreadPool(config, threadPoolConfig ⇒ new Dispatcher( config.getString("name", newUuid.toString), config.getInt("throughput", Dispatchers.THROUGHPUT), config.getInt("throughput-deadline-time", Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS), @@ -227,9 +227,9 @@ class ExecutorBasedEventDrivenDispatcherConfigurator extends MessageDispatcherCo } } -class ExecutorBasedEventDrivenWorkStealingDispatcherConfigurator extends MessageDispatcherConfigurator { +class BalancingDispatcherConfigurator extends MessageDispatcherConfigurator { def configure(config: Configuration): MessageDispatcher = { - configureThreadPool(config, threadPoolConfig ⇒ new ExecutorBasedEventDrivenWorkStealingDispatcher( + configureThreadPool(config, threadPoolConfig ⇒ new BalancingDispatcher( config.getString("name", newUuid.toString), config.getInt("throughput", Dispatchers.THROUGHPUT), config.getInt("throughput-deadline-time", Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS), diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadBasedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala similarity index 86% rename from akka-actor/src/main/scala/akka/dispatch/ThreadBasedDispatcher.scala rename to akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala index 101a2ae2c5..e03e6af9e2 100644 --- a/akka-actor/src/main/scala/akka/dispatch/ThreadBasedDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala @@ -14,9 +14,9 @@ import akka.actor.{ Actor, ActorRef } * * @author Jonas Bonér */ -class ThreadBasedDispatcher(_actor: ActorRef, _name: String, _mailboxType: MailboxType) - extends ExecutorBasedEventDrivenDispatcher( - _name, Dispatchers.THROUGHPUT, -1, _mailboxType, ThreadBasedDispatcher.oneThread) { +class PinnedDispatcher(_actor: ActorRef, _name: String, _mailboxType: MailboxType) + extends Dispatcher( + _name, Dispatchers.THROUGHPUT, -1, _mailboxType, PinnedDispatcher.oneThread) { def this(_name: String, _mailboxType: MailboxType) = this(null, _name, _mailboxType) @@ -47,7 +47,7 @@ class ThreadBasedDispatcher(_actor: ActorRef, _name: String, _mailboxType: Mailb } } -object ThreadBasedDispatcher { +object PinnedDispatcher { val oneThread: ThreadPoolConfig = ThreadPoolConfig(allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1) } diff --git a/akka-actor/src/main/scala/akka/event/EventHandler.scala b/akka-actor/src/main/scala/akka/event/EventHandler.scala index 5c2d7b89cc..28adf4946d 100644 --- a/akka-actor/src/main/scala/akka/event/EventHandler.scala +++ b/akka-actor/src/main/scala/akka/event/EventHandler.scala @@ -89,7 +89,7 @@ object EventHandler extends ListenerManagement { class EventHandlerException extends AkkaException - lazy val EventHandlerDispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("event:handler").build + lazy val EventHandlerDispatcher = Dispatchers.newDispatcher("event:handler").build implicit object defaultListenerFormat extends StatelessActorFormat[DefaultListener] diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 1eb0f624b3..48771b5bed 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -1580,7 +1580,7 @@ object RemoteClusterDaemon { val ADDRESS = "akka-cluster-daemon".intern // FIXME configure functionServerDispatcher to what? - val functionServerDispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("akka:cloud:cluster:function:server").build + val functionServerDispatcher = Dispatchers.newDispatcher("akka:cloud:cluster:function:server").build } /** @@ -1591,7 +1591,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { import RemoteClusterDaemon._ import Cluster._ - self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) + self.dispatcher = Dispatchers.newPinnedDispatcher(self) def receive: Receive = { case message: RemoteDaemonMessageProtocol ⇒ diff --git a/akka-docs/cluster/durable-mailbox.rst b/akka-docs/cluster/durable-mailbox.rst index a1bc3add43..5ebc34df5a 100644 --- a/akka-docs/cluster/durable-mailbox.rst +++ b/akka-docs/cluster/durable-mailbox.rst @@ -59,13 +59,13 @@ actor is oblivious to which type of mailbox it is using. Here is an example:: or for a thread-based durable dispatcher:: - self.dispatcher = DurableThreadBasedDispatcher( + self.dispatcher = DurablePinnedDispatcher( self, FileDurableMailboxStorage) There are 2 different durable dispatchers, ``DurableEventBasedDispatcher`` and -``DurableThreadBasedDispatcher``, which are durable versions of -``ExecutorBasedEventDrivenDispatcher`` and ``ThreadBasedDispatcher``. +``DurablePinnedDispatcher``, which are durable versions of +``Dispatcher`` and ``PinnedDispatcher``. This gives you an excellent way of creating bulkheads in your application, where groups of actors sharing the same dispatcher also share the same backing @@ -120,7 +120,7 @@ Here is an example of how you can configure your dispatcher to use this mailbox: or for a thread-based durable dispatcher:: - self.dispatcher = DurableThreadBasedDispatcher( + self.dispatcher = DurablePinnedDispatcher( self, RedisDurableMailboxStorage) @@ -164,7 +164,7 @@ Here is an example of how you can configure your dispatcher to use this mailbox: or for a thread-based durable dispatcher:: - self.dispatcher = DurableThreadBasedDispatcher( + self.dispatcher = DurablePinnedDispatcher( self, ZooKeeperDurableMailboxStorage) @@ -202,7 +202,7 @@ Beanstalk documentation on how to do that. :: or for a thread-based durable dispatcher. :: - self.dispatcher = DurableThreadBasedDispatcher( + self.dispatcher = DurablePinnedDispatcher( self, BeanstalkDurableMailboxStorage) diff --git a/akka-docs/general/configuration.rst b/akka-docs/general/configuration.rst index bd6c9510e3..a6c045d42e 100644 --- a/akka-docs/general/configuration.rst +++ b/akka-docs/general/configuration.rst @@ -58,7 +58,7 @@ A custom ``akka.conf`` might look like this: "sample.myservice.Boot"] actor { - throughput = 10 # Throughput for ExecutorBasedEventDrivenDispatcher, set to 1 for complete fairness + throughput = 10 # Throughput for Dispatcher, set to 1 for complete fairness } remote { diff --git a/akka-docs/java/dispatchers.rst b/akka-docs/java/dispatchers.rst index 36bd22d8ba..b67cb1c02e 100644 --- a/akka-docs/java/dispatchers.rst +++ b/akka-docs/java/dispatchers.rst @@ -18,7 +18,7 @@ The event-based Actors currently consume ~600 bytes per Actor which means that y Default dispatcher ------------------ -For most scenarios the default settings are the best. Here we have one single event-based dispatcher for all Actors created. The dispatcher used is globalExecutorBasedEventDrivenDispatcher in akka.dispatch.Dispatchers. +For most scenarios the default settings are the best. Here we have one single event-based dispatcher for all Actors created. The dispatcher used is globalDispatcher in akka.dispatch.Dispatchers. But if you feel that you are starting to contend on the single dispatcher (the 'Executor' and its queue) or want to group a specific set of Actors for a dedicated dispatcher for better flexibility and configurability then you can override the defaults and define your own dispatcher. See below for details on which ones are available and how they can be configured. @@ -59,11 +59,11 @@ Let's now walk through the different dispatchers in more detail. Thread-based ^^^^^^^^^^^^ -The 'ThreadBasedDispatcher' binds a dedicated OS thread to each specific Actor. The messages are posted to a 'LinkedBlockingQueue' which feeds the messages to the dispatcher one by one. A 'ThreadBasedDispatcher' cannot be shared between actors. This dispatcher has worse performance and scalability than the event-based dispatcher but works great for creating "daemon" Actors that consumes a low frequency of messages and are allowed to go off and do their own thing for a longer period of time. Another advantage with this dispatcher is that Actors do not block threads for each other. +The 'PinnedDispatcher' binds a dedicated OS thread to each specific Actor. The messages are posted to a 'LinkedBlockingQueue' which feeds the messages to the dispatcher one by one. A 'PinnedDispatcher' cannot be shared between actors. This dispatcher has worse performance and scalability than the event-based dispatcher but works great for creating "daemon" Actors that consumes a low frequency of messages and are allowed to go off and do their own thing for a longer period of time. Another advantage with this dispatcher is that Actors do not block threads for each other. .. code-block:: java - Dispatcher dispatcher = Dispatchers.newThreadBasedDispatcher(actorRef); + Dispatcher dispatcher = Dispatchers.newPinnedDispatcher(actorRef); It would normally by used from within the actor like this: @@ -71,7 +71,7 @@ It would normally by used from within the actor like this: class MyActor extends UntypedActor { public MyActor() { - getContext().setDispatcher(Dispatchers.newThreadBasedDispatcher(getContext())); + getContext().setDispatcher(Dispatchers.newPinnedDispatcher(getContext())); } ... } @@ -79,7 +79,7 @@ It would normally by used from within the actor like this: Event-based ^^^^^^^^^^^ -The 'ExecutorBasedEventDrivenDispatcher' binds a set of Actors to a thread pool backed up by a 'BlockingQueue'. This dispatcher is highly configurable and supports a fluent configuration API to configure the 'BlockingQueue' (type of queue, max items etc.) as well as the thread pool. +The 'Dispatcher' binds a set of Actors to a thread pool backed up by a 'BlockingQueue'. This dispatcher is highly configurable and supports a fluent configuration API to configure the 'BlockingQueue' (type of queue, max items etc.) as well as the thread pool. The event-driven dispatchers **must be shared** between multiple Typed Actors and/or Actors. One best practice is to let each top-level Actor, e.g. the Actors you define in the declarative supervisor config, to get their own dispatcher but reuse the dispatcher for each new Actor that the top-level Actor creates. But you can also share dispatcher between multiple top-level Actors. This is very use-case specific and needs to be tried out on a case by case basis. The important thing is that Akka tries to provide you with the freedom you need to design and implement your system in the most efficient way in regards to performance, throughput and latency. @@ -109,7 +109,7 @@ Here is an example: class MyActor extends UntypedActor { public MyActor() { - getContext().setDispatcher(Dispatchers.newExecutorBasedEventDrivenDispatcher(name) + getContext().setDispatcher(Dispatchers.newDispatcher(name) .withNewThreadPoolWithLinkedBlockingQueueWithCapacity(100) .setCorePoolSize(16) .setMaxPoolSize(128) @@ -120,7 +120,7 @@ Here is an example: ... } -This 'ExecutorBasedEventDrivenDispatcher' allows you to define the 'throughput' it should have. This defines the number of messages for a specific Actor the dispatcher should process in one single sweep. +This 'Dispatcher' allows you to define the 'throughput' it should have. This defines the number of messages for a specific Actor the dispatcher should process in one single sweep. Setting this to a higher number will increase throughput but lower fairness, and vice versa. If you don't specify it explicitly then it uses the default value defined in the 'akka.conf' configuration file: .. code-block:: xml @@ -136,10 +136,10 @@ Browse the :ref:`scaladoc` or look at the code for all the options available. Priority event-based ^^^^^^^^^^^^^^^^^^^^ -Sometimes it's useful to be able to specify priority order of messages, that is done by using PriorityExecutorBasedEventDrivenDispatcher and supply +Sometimes it's useful to be able to specify priority order of messages, that is done by using PriorityDispatcher and supply a java.util.Comparator[MessageInvocation] or use a akka.dispatch.PriorityGenerator (recommended): -Creating a PriorityExecutorBasedEventDrivenDispatcher using PriorityGenerator: +Creating a PriorityDispatcher using PriorityGenerator: .. code-block:: java @@ -168,7 +168,7 @@ Creating a PriorityExecutorBasedEventDrivenDispatcher using PriorityGenerator: // We create an instance of the actor that will print out the messages it processes ActorRef ref = Actors.actorOf(MyActor.class); // We create a new Priority dispatcher and seed it with the priority generator - ref.setDispatcher(new PriorityExecutorBasedEventDrivenDispatcher("foo", gen)); + ref.setDispatcher(new PriorityDispatcher("foo", gen)); ref.start(); // Start the actor ref.getDispatcher().suspend(ref); // Suspending the actor so it doesn't start to treat the messages before we have enqueued all of them :-) @@ -196,14 +196,14 @@ lowpriority Work-stealing event-based ^^^^^^^^^^^^^^^^^^^^^^^^^ -The 'ExecutorBasedEventDrivenWorkStealingDispatcher' is a variation of the 'ExecutorBasedEventDrivenDispatcher' in which Actors of the same type can be set up to share this dispatcher and during execution time the different actors will steal messages from other actors if they have less messages to process. This can be a great way to improve throughput at the cost of a little higher latency. +The 'BalancingDispatcher' is a variation of the 'Dispatcher' in which Actors of the same type can be set up to share this dispatcher and during execution time the different actors will steal messages from other actors if they have less messages to process. This can be a great way to improve throughput at the cost of a little higher latency. Normally the way you use it is to define a static field to hold the dispatcher and then set in in the Actor explicitly. .. code-block:: java class MyActor extends UntypedActor { - public static MessageDispatcher dispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher(name).build(); + public static MessageDispatcher dispatcher = Dispatchers.newBalancingDispatcher(name).build(); public MyActor() { getContext().setDispatcher(dispatcher); @@ -236,7 +236,7 @@ Per-instance based configuration You can also do it on a specific dispatcher instance. -For the 'ExecutorBasedEventDrivenDispatcher' and the 'ExecutorBasedWorkStealingDispatcher' you can do it through their constructor +For the 'Dispatcher' and the 'ExecutorBasedWorkStealingDispatcher' you can do it through their constructor .. code-block:: java @@ -246,13 +246,13 @@ For the 'ExecutorBasedEventDrivenDispatcher' and the 'ExecutorBasedWorkStealingD Duration pushTimeout = new FiniteDuration(10, TimeUnit.SECONDS); MailboxType mailboxCapacity = new BoundedMailbox(false, capacity, pushTimeout); MessageDispatcher dispatcher = - Dispatchers.newExecutorBasedEventDrivenDispatcher(name, throughput, mailboxCapacity).build(); + Dispatchers.newDispatcher(name, throughput, mailboxCapacity).build(); getContext().setDispatcher(dispatcher); } ... } -For the 'ThreadBasedDispatcher', it is non-shareable between actors, and associates a dedicated Thread with the actor. +For the 'PinnedDispatcher', it is non-shareable between actors, and associates a dedicated Thread with the actor. Making it bounded (by specifying a capacity) is optional, but if you do, you need to provide a pushTimeout (default is 10 seconds). When trying to send a message to the Actor it will throw a MessageQueueAppendFailedException("BlockingMessageTransferQueue transfer timed out") if the message cannot be added to the mailbox within the time specified by the pushTimeout. .. code-block:: java @@ -261,7 +261,7 @@ Making it bounded (by specifying a capacity) is optional, but if you do, you nee public MyActor() { int mailboxCapacity = 100; Duration pushTimeout = new FiniteDuration(10, TimeUnit.SECONDS); - getContext().setDispatcher(Dispatchers.newThreadBasedDispatcher(getContext(), mailboxCapacity, pushTimeout)); + getContext().setDispatcher(Dispatchers.newPinnedDispatcher(getContext(), mailboxCapacity, pushTimeout)); } ... } diff --git a/akka-docs/project/migration-guide-0.9.x-0.10.x.rst b/akka-docs/project/migration-guide-0.9.x-0.10.x.rst index 78b2ddc303..ceaa42af9b 100644 --- a/akka-docs/project/migration-guide-0.9.x-0.10.x.rst +++ b/akka-docs/project/migration-guide-0.9.x-0.10.x.rst @@ -36,7 +36,7 @@ Configuration actor { timeout = 5 # default timeout for future based invocations - throughput = 5 # default throughput for ExecutorBasedEventDrivenDispatcher + throughput = 5 # default throughput for Dispatcher } ... } diff --git a/akka-docs/project/release-notes.rst b/akka-docs/project/release-notes.rst index 32b088c3a2..c6effb9b19 100644 --- a/akka-docs/project/release-notes.rst +++ b/akka-docs/project/release-notes.rst @@ -199,7 +199,7 @@ Release 1.0-MILESTONE1 - **FIX** - #420 REST endpoints should be able to be processed in parallel (Viktor Klang) - **FIX** - #422 Dispatcher config should work for ThreadPoolBuilder-based dispatchers (Viktor Klang) - **FIX** - #401 ActorRegistry should not leak memory (Viktor Klang) -- **FIX** - #250 Performance optimization for ExecutorBasedEventDrivenDispatcher (Viktor Klang) +- **FIX** - #250 Performance optimization for Dispatcher (Viktor Klang) - **FIX** - #419 Rename init and shutdown callbacks to preStart and postStop, and remove initTransactionalState (Viktor Klang) - **FIX** - #346 Make max no of restarts (and within) are now both optional (Viktor Klang) - **FIX** - #424 Actors self.supervisor not set by the time init() is called when started by startLink() (Viktor Klang) @@ -210,7 +210,7 @@ Release 1.0-MILESTONE1 - **FIX** - Logger.warn now properly works with varargs (Viktor Klang) - **FIX** - #450 Removed ActorRef lifeCycle boilerplate: Some(LifeCycle(Permanent)) => Permanent (Viktor Klang) - **FIX** - Moved ActorRef.trapExit into ActorRef.faultHandler and removed Option-boilerplate from faultHandler (Viktor Klang) -- **FIX** - ThreadBasedDispatcher cheaper for idling actors, also benefits from all that is ExecutorBasedEventDrivenDispatcher (Viktor Klang) +- **FIX** - PinnedDispatcher cheaper for idling actors, also benefits from all that is Dispatcher (Viktor Klang) - **FIX** - Fixing Futures.future, uses Actor.spawn under the hood, specify dispatcher to control where block is executed (Viktor Klang) - **FIX** - #469 Akka "dist" now uses a root folder to avoid loitering if unzipped in a folder (Viktor Klang) - **FIX** - Removed ScalaConfig, JavaConfig and rewrote Supervision configuration (Viktor Klang) @@ -224,7 +224,7 @@ Release 1.0-MILESTONE1 - **ADD** - #262 Add Java API for Agent (Viktor Klang) - **ADD** - #264 Add Java API for Dataflow (Viktor Klang) - **ADD** - Using JerseySimpleBroadcaster instead of JerseyBroadcaster in AkkaBroadcaster (Viktor Klang) -- **ADD** - #433 Throughput deadline added for ExecutorBasedEventDrivenDispatcher (Viktor Klang) +- **ADD** - #433 Throughput deadline added for Dispatcher (Viktor Klang) - **ADD** - Add possibility to set default cometSupport in akka.conf (Viktor Klang) - **ADD** - #451 Added possibility to use akka-http as a standalone REST server (Viktor Klang) - **ADD** - #446 Added support for Erlang-style receiveTimeout (Viktor Klang) @@ -308,7 +308,7 @@ Release 0.10 - Aug 21 2010 - **ADD** - Java API for the STM (Peter Vlugter) - **ADD** - #379 Create STM Atomic templates for Java API (Peter Vlugter) - **ADD** - #270 SBT plugin for Akka (Peter Vlugter) -- **ADD** - #198 support for ThreadBasedDispatcher in Spring config (Michael Kober) +- **ADD** - #198 support for PinnedDispatcher in Spring config (Michael Kober) - **ADD** - #377 support HawtDispatcher in Spring config (Michael Kober) - **ADD** - #376 support Spring config for untyped actors (Michael Kober) - **ADD** - #200 support WorkStealingDispatcher in Spring config (Michael Kober) diff --git a/akka-docs/scala/dispatchers.rst b/akka-docs/scala/dispatchers.rst index 91f3799d00..cc75b5c099 100644 --- a/akka-docs/scala/dispatchers.rst +++ b/akka-docs/scala/dispatchers.rst @@ -22,7 +22,7 @@ For most scenarios the default settings are the best. Here we have one single ev .. code-block:: scala - Dispatchers.globalExecutorBasedEventDrivenDispatcher + Dispatchers.globalDispatcher But if you feel that you are starting to contend on the single dispatcher (the 'Executor' and its queue) or want to group a specific set of Actors for a dedicated dispatcher for better flexibility and configurability then you can override the defaults and define your own dispatcher. See below for details on which ones are available and how they can be configured. @@ -61,7 +61,7 @@ Let's now walk through the different dispatchers in more detail. Thread-based ^^^^^^^^^^^^ -The 'ThreadBasedDispatcher' binds a dedicated OS thread to each specific Actor. The messages are posted to a 'LinkedBlockingQueue' which feeds the messages to the dispatcher one by one. A 'ThreadBasedDispatcher' cannot be shared between actors. This dispatcher has worse performance and scalability than the event-based dispatcher but works great for creating "daemon" Actors that consumes a low frequency of messages and are allowed to go off and do their own thing for a longer period of time. Another advantage with this dispatcher is that Actors do not block threads for each other. +The 'PinnedDispatcher' binds a dedicated OS thread to each specific Actor. The messages are posted to a 'LinkedBlockingQueue' which feeds the messages to the dispatcher one by one. A 'PinnedDispatcher' cannot be shared between actors. This dispatcher has worse performance and scalability than the event-based dispatcher but works great for creating "daemon" Actors that consumes a low frequency of messages and are allowed to go off and do their own thing for a longer period of time. Another advantage with this dispatcher is that Actors do not block threads for each other. It would normally by used from within the actor like this: @@ -69,7 +69,7 @@ It would normally by used from within the actor like this: class MyActor extends Actor { public MyActor() { - self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) + self.dispatcher = Dispatchers.newPinnedDispatcher(self) } ... } @@ -77,7 +77,7 @@ It would normally by used from within the actor like this: Event-based ^^^^^^^^^^^ -The 'ExecutorBasedEventDrivenDispatcher' binds a set of Actors to a thread pool backed up by a 'BlockingQueue'. This dispatcher is highly configurable and supports a fluent configuration API to configure the 'BlockingQueue' (type of queue, max items etc.) as well as the thread pool. +The 'Dispatcher' binds a set of Actors to a thread pool backed up by a 'BlockingQueue'. This dispatcher is highly configurable and supports a fluent configuration API to configure the 'BlockingQueue' (type of queue, max items etc.) as well as the thread pool. The event-driven dispatchers **must be shared** between multiple Actors. One best practice is to let each top-level Actor, e.g. the Actors you define in the declarative supervisor config, to get their own dispatcher but reuse the dispatcher for each new Actor that the top-level Actor creates. But you can also share dispatcher between multiple top-level Actors. This is very use-case specific and needs to be tried out on a case by case basis. The important thing is that Akka tries to provide you with the freedom you need to design and implement your system in the most efficient way in regards to performance, throughput and latency. @@ -106,7 +106,7 @@ Here is an example: import java.util.concurrent.ThreadPoolExecutor.CallerRunsPolicy class MyActor extends Actor { - self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(name) + self.dispatcher = Dispatchers.newDispatcher(name) .withNewThreadPoolWithLinkedBlockingQueueWithCapacity(100) .setCorePoolSize(16) .setMaxPoolSize(128) @@ -116,7 +116,7 @@ Here is an example: ... } -This 'ExecutorBasedEventDrivenDispatcher' allows you to define the 'throughput' it should have. This defines the number of messages for a specific Actor the dispatcher should process in one single sweep. +This 'Dispatcher' allows you to define the 'throughput' it should have. This defines the number of messages for a specific Actor the dispatcher should process in one single sweep. Setting this to a higher number will increase throughput but lower fairness, and vice versa. If you don't specify it explicitly then it uses the default value defined in the 'akka.conf' configuration file: .. code-block:: ruby @@ -132,10 +132,10 @@ Browse the `ScalaDoc `_ or look at the code for all the options availa Priority event-based ^^^^^^^^^^^^^^^^^^^^ -Sometimes it's useful to be able to specify priority order of messages, that is done by using PriorityExecutorBasedEventDrivenDispatcher and supply +Sometimes it's useful to be able to specify priority order of messages, that is done by using PriorityDispatcher and supply a java.util.Comparator[MessageInvocation] or use a akka.dispatch.PriorityGenerator (recommended): -Creating a PriorityExecutorBasedEventDrivenDispatcher using PriorityGenerator: +Creating a PriorityDispatcher using PriorityGenerator: .. code-block:: scala @@ -156,7 +156,7 @@ Creating a PriorityExecutorBasedEventDrivenDispatcher using PriorityGenerator: }) // We create a new Priority dispatcher and seed it with the priority generator - a.dispatcher = new PriorityExecutorBasedEventDrivenDispatcher("foo", gen) + a.dispatcher = new PriorityDispatcher("foo", gen) a.start // Start the Actor a.dispatcher.suspend(a) // Suspending the actor so it doesn't start to treat the messages before we have enqueued all of them :-) @@ -184,14 +184,14 @@ Prints: Work-stealing event-based ^^^^^^^^^^^^^^^^^^^^^^^^^ -The 'ExecutorBasedEventDrivenWorkStealingDispatcher' is a variation of the 'ExecutorBasedEventDrivenDispatcher' in which Actors of the same type can be set up to share this dispatcher and during execution time the different actors will steal messages from other actors if they have less messages to process. This can be a great way to improve throughput at the cost of a little higher latency. +The 'BalancingDispatcher' is a variation of the 'Dispatcher' in which Actors of the same type can be set up to share this dispatcher and during execution time the different actors will steal messages from other actors if they have less messages to process. This can be a great way to improve throughput at the cost of a little higher latency. Normally the way you use it is to create an Actor companion object to hold the dispatcher and then set in in the Actor explicitly. .. code-block:: scala object MyActor { - val dispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher(name).build + val dispatcher = Dispatchers.newBalancingDispatcher(name).build } class MyActor extends Actor { @@ -224,24 +224,24 @@ Per-instance based configuration You can also do it on a specific dispatcher instance. -For the 'ExecutorBasedEventDrivenDispatcher' and the 'ExecutorBasedWorkStealingDispatcher' you can do it through their constructor +For the 'Dispatcher' and the 'ExecutorBasedWorkStealingDispatcher' you can do it through their constructor .. code-block:: scala class MyActor extends Actor { val mailboxCapacity = BoundedMailbox(capacity = 100) - self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(name, throughput, mailboxCapacity).build + self.dispatcher = Dispatchers.newDispatcher(name, throughput, mailboxCapacity).build ... } -For the 'ThreadBasedDispatcher', it is non-shareable between actors, and associates a dedicated Thread with the actor. +For the 'PinnedDispatcher', it is non-shareable between actors, and associates a dedicated Thread with the actor. Making it bounded (by specifying a capacity) is optional, but if you do, you need to provide a pushTimeout (default is 10 seconds). When trying to send a message to the Actor it will throw a MessageQueueAppendFailedException("BlockingMessageTransferQueue transfer timed out") if the message cannot be added to the mailbox within the time specified by the pushTimeout. .. code-block:: scala class MyActor extends Actor { import akka.util.duration._ - self.dispatcher = Dispatchers.newThreadBasedDispatcher(self, mailboxCapacity = 100, + self.dispatcher = Dispatchers.newPinnedDispatcher(self, mailboxCapacity = 100, pushTimeOut = 10 seconds) ... } diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala index 247956ff7c..8a1b2c60f1 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableDispatcher.scala @@ -52,7 +52,7 @@ case object FileDurableMailboxStorage extends DurableMailboxStorage("akka.a case object ZooKeeperDurableMailboxStorage extends DurableMailboxStorage("akka.actor.mailbox.ZooKeeperBasedMailbox") /** - * The durable equivalent of ExecutorBasedEventDrivenDispatcher + * The durable equivalent of Dispatcher * * @author Jonas Bonér */ @@ -62,7 +62,7 @@ case class DurableEventBasedDispatcher( _throughput: Int = Dispatchers.THROUGHPUT, _throughputDeadlineTime: Int = Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, _mailboxType: MailboxType = Dispatchers.MAILBOX_TYPE, - _config: ThreadPoolConfig = ThreadPoolConfig()) extends ExecutorBasedEventDrivenDispatcher( + _config: ThreadPoolConfig = ThreadPoolConfig()) extends Dispatcher( _name, _throughput, _throughputDeadlineTime, @@ -101,14 +101,14 @@ case class DurableEventBasedDispatcher( } /** - * The durable equivalent of ThreadBasedDispatcher + * The durable equivalent of PinnedDispatcher * * @author Jonas Bonér */ -case class DurableThreadBasedDispatcher( +case class DurablePinnedDispatcher( _actor: ActorRef, _storage: DurableMailboxStorage, - _mailboxType: MailboxType) extends ThreadBasedDispatcher(_actor,_mailboxType) { + _mailboxType: MailboxType) extends PinnedDispatcher(_actor,_mailboxType) { def this(actor: ActorRef, _storage: DurableMailboxStorage) = this(actor, _storage, UnboundedMailbox()) // For Java API diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala index db5d98f04a..5b83f6a8c4 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/scala/akka/actor/mailbox/DurableMailbox.scala @@ -37,8 +37,8 @@ abstract class DurableExecutableMailbox(owner: ActorRef) extends MessageQueue wi EventHandler.debug(this, "Creating %s mailbox [%s]".format(getClass.getName, name)) - val dispatcher: ExecutorBasedEventDrivenDispatcher = owner.dispatcher match { - case e: ExecutorBasedEventDrivenDispatcher => e + val dispatcher: Dispatcher = owner.dispatcher match { + case e: Dispatcher => e case _ => null } diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala index 0f371ea168..b556b90e50 100644 --- a/akka-stm/src/main/scala/akka/agent/Agent.scala +++ b/akka-stm/src/main/scala/akka/agent/Agent.scala @@ -293,7 +293,7 @@ class AgentUpdater[T](agent: Agent[T]) extends Actor { * Thread-based agent updater actor. Used internally for `sendOff` actions. */ class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor { - self.dispatcher = Dispatchers.newThreadBasedDispatcher(self) + self.dispatcher = Dispatchers.newPinnedDispatcher(self) val txFactory = TransactionFactory(familyName = "ThreadBasedAgentUpdater", readonly = false) diff --git a/config/akka-reference.conf b/config/akka-reference.conf index efb7492e66..8e5a39b280 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -30,8 +30,8 @@ akka { # - UntypedActor: sendRequestReply && sendRequestReplyFuture # - TypedActor: methods with non-void return type serialize-messages = off # Does a deep clone of (non-primitive) messages to ensure immutability - throughput = 5 # Default throughput for all ExecutorBasedEventDrivenDispatcher, set to 1 for complete fairness - throughput-deadline-time = -1 # Default throughput deadline for all ExecutorBasedEventDrivenDispatcher, set to 0 or negative for no deadline + throughput = 5 # Default throughput for all Dispatcher, set to 1 for complete fairness + throughput-deadline-time = -1 # Default throughput deadline for all Dispatcher, set to 0 or negative for no deadline dispatcher-shutdown-timeout = 1 # Using the akka.time-unit, how long dispatchers by default will wait for new actors until they shut down deployment { @@ -75,22 +75,22 @@ akka { } default-dispatcher { - type = "GlobalExecutorBasedEventDriven" # Must be one of the following, all "Global*" are non-configurable - # - ExecutorBasedEventDriven - # - ExecutorBasedEventDrivenWorkStealing - # - GlobalExecutorBasedEventDriven + type = "GlobalDispatcher" # Must be one of the following, all "Global*" are non-configurable + # - Dispatcher + # - BalancingDispatcher + # - GlobalDispatcher keep-alive-time = 60 # Keep alive time for threads core-pool-size-factor = 1.0 # No of core threads ... ceil(available processors * factor) max-pool-size-factor = 4.0 # Max no of threads ... ceil(available processors * factor) executor-bounds = -1 # Makes the Executor bounded, -1 is unbounded allow-core-timeout = on # Allow core threads to time out rejection-policy = "caller-runs" # abort, caller-runs, discard-oldest, discard - throughput = 5 # Throughput for ExecutorBasedEventDrivenDispatcher, set to 1 for complete fairness - throughput-deadline-time = -1 # Throughput deadline for ExecutorBasedEventDrivenDispatcher, set to 0 or negative for no deadline + throughput = 5 # Throughput for Dispatcher, set to 1 for complete fairness + throughput-deadline-time = -1 # Throughput deadline for Dispatcher, set to 0 or negative for no deadline mailbox-capacity = -1 # If negative (or zero) then an unbounded mailbox is used (default) # If positive then a bounded mailbox is used and the capacity is set using the property # NOTE: setting a mailbox to 'blocking' can be a bit dangerous, could lead to deadlock, use with care - # The following are only used for ExecutorBasedEventDriven and only if mailbox-capacity > 0 + # The following are only used for Dispatcher and only if mailbox-capacity > 0 mailbox-push-timeout-time = 10 # Specifies the timeout to add a new message to a mailbox that is full - negative number means infinite timeout # (in unit defined by the time-unit property) } @@ -197,7 +197,7 @@ akka { # If you are using akka.http.AkkaMistServlet mist-dispatcher { - #type = "GlobalExecutorBasedEventDriven" # Uncomment if you want to use a different dispatcher than the default one for Comet + #type = "GlobalDispatcher" # Uncomment if you want to use a different dispatcher than the default one for Comet } connection-close = true # toggles the addition of the "Connection" response header with a "close" value root-actor-id = "_httproot" # the id of the actor to use as the root endpoint diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala index 35c487d70c..53e79bb5e9 100644 --- a/project/build/AkkaProject.scala +++ b/project/build/AkkaProject.scala @@ -19,7 +19,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec val scalaCompileSettings = Seq("-deprecation", //"-Xmigration", - //"-optimise", + "-optimise", "-encoding", "utf8") val javaCompileSettings = Seq("-Xlint:unchecked")