From 3dc84a02290f6b95f859bb63343b78fc16563c84 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 17 Oct 2011 14:32:31 +0200 Subject: [PATCH 01/33] Renaming InVMMonitoring to LocalDeathWatch and moved it into AkkaApplication, also created a createDeathWatch method in ActorRefProvider so that it's seeded from there, and then removed @volatile from alot of vars in ActorCell since the fields are now protected by the Mailbox status field --- .../src/main/scala/akka/AkkaApplication.scala | 2 ++ .../src/main/scala/akka/actor/ActorCell.scala | 19 +++++++-------- .../src/main/scala/akka/actor/ActorRef.scala | 2 +- .../scala/akka/actor/ActorRefProvider.scala | 23 ++++++++++++++++++- .../main/scala/akka/dispatch/Dispatcher.scala | 7 ++++-- .../main/scala/akka/event/DeathWatch.scala | 19 +-------------- .../akka/remote/RemoteActorRefProvider.scala | 4 +++- 7 files changed, 42 insertions(+), 34 deletions(-) diff --git a/akka-actor/src/main/scala/akka/AkkaApplication.scala b/akka-actor/src/main/scala/akka/AkkaApplication.scala index b064630107..60e7d78df2 100644 --- a/akka-actor/src/main/scala/akka/AkkaApplication.scala +++ b/akka-actor/src/main/scala/akka/AkkaApplication.scala @@ -176,6 +176,8 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor // TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor val provider: ActorRefProvider = reflective.createProvider + val deathWatch = provider.createDeathWatch() + val typedActor = new TypedActor(this) val serialization = new Serialization(this) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 16a9444349..f95ed69e88 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -9,7 +9,6 @@ import akka.util._ import scala.annotation.tailrec import scala.collection.immutable.Stack import scala.collection.JavaConverters -import akka.event.InVMMonitoring import java.util.concurrent.{ ScheduledFuture, TimeUnit } import java.util.{ Collection ⇒ JCollection, Collections ⇒ JCollections } import akka.AkkaApplication @@ -207,27 +206,25 @@ private[akka] object ActorCell { } } +//vars don't need volatile since it's protected with the mailbox status +//Make sure that they are not read/written outside of a message processing (systemInvoke/invoke) private[akka] class ActorCell( val app: AkkaApplication, val self: ActorRef with ScalaActorRef, val props: Props, - @volatile var receiveTimeout: Option[Long], - @volatile var hotswap: Stack[PartialFunction[Any, Unit]]) extends ActorContext { + var receiveTimeout: Option[Long], + var hotswap: Stack[PartialFunction[Any, Unit]]) extends ActorContext { import ActorCell._ def provider = app.provider - @volatile var futureTimeout: Option[ScheduledFuture[AnyRef]] = None //FIXME TODO Doesn't need to be volatile either, since it will only ever be accessed when a message is processed - @volatile var _children: Vector[ChildRestartStats] = Vector.empty - @volatile //TODO FIXME Might be able to make this non-volatile since it should be guarded by a mailbox.isShutdown test (which will force volatile piggyback read) var currentMessage: Envelope = null - @volatile //TODO FIXME Might be able to make this non-volatile since it should be guarded by a mailbox.isShutdown test (which will force volatile piggyback read) var actor: Actor = _ //FIXME We can most probably make this just a regular reference to Actor def ref: ActorRef with ScalaActorRef = self @@ -238,7 +235,7 @@ private[akka] class ActorCell( def isShutdown: Boolean = mailbox.isClosed - @volatile //This must be volatile + @volatile //This must be volatile since it isn't protected by the mailbox status var mailbox: Mailbox = _ def start(): Unit = { @@ -404,7 +401,7 @@ private[akka] class ActorCell( if (supervisor.isDefined) supervisor.get ! ChildTerminated(self, cause) - InVMMonitoring.publish(Terminated(self, cause)) + app.deathWatch.publish(Terminated(self, cause)) currentMessage = null clearActorContext() @@ -426,10 +423,10 @@ private[akka] class ActorCell( case Create ⇒ create() case Recreate(cause) ⇒ recreate(cause) case Link(subject) ⇒ - akka.event.InVMMonitoring.subscribe(self, subject) + app.deathWatch.subscribe(self, subject) if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "now monitoring " + subject) case Unlink(subject) ⇒ - akka.event.InVMMonitoring.unsubscribe(self, subject) + app.deathWatch.unsubscribe(self, subject) if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopped monitoring " + subject) case Suspend ⇒ suspend() case Resume ⇒ resume() diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index d58ba2bf85..6e47f60300 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -13,7 +13,7 @@ import java.lang.{ UnsupportedOperationException, IllegalStateException } import akka.AkkaApplication import akka.remote.RemoteSupport import scala.util.DynamicVariable -import akka.event.{ EventHandler, InVMMonitoring } +import akka.event.{ EventHandler } /** * ActorRef is an immutable and serializable handle to an Actor. diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 4904c07d21..0b94e94809 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -4,7 +4,6 @@ package akka.actor -import akka.event.EventHandler import akka.config.ConfigurationException import akka.util.ReflectiveAccess import akka.routing._ @@ -14,6 +13,7 @@ import java.util.concurrent.ConcurrentHashMap import akka.dispatch.Promise import com.eaio.uuid.UUID import akka.AkkaException +import akka.event.{ ActorClassification, DeathWatch, EventHandler } /** * Interface for all ActorRef providers to implement. @@ -31,6 +31,8 @@ trait ActorRefProvider { private[akka] def evict(address: String): Boolean private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] + + private[akka] def createDeathWatch(): DeathWatch } /** @@ -168,4 +170,23 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { } private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = actorFor(actor.address) + + private[akka] def createDeathWatch(): DeathWatch = new LocalDeathWatch +} + +class LocalDeathWatch extends DeathWatch with ActorClassification { + + def mapSize = 1024 + + override def publish(event: Event): Unit = { + val monitors = dissociate(classify(event)) + if (monitors.nonEmpty) monitors.foreach(_ ! event) + } + + override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = { + if (!super.subscribe(subscriber, to)) { + subscriber ! Terminated(subscriber, new ActorKilledException("Already terminated when linking")) + false + } else true + } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index e44806b206..fd3dd697da 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -121,8 +121,11 @@ class Dispatcher( true } catch { case e: RejectedExecutionException ⇒ - app.eventHandler.warning(this, e.toString) - mbox.setAsIdle() + try { + app.eventHandler.warning(this, e.toString) + } finally { + mbox.setAsIdle() + } throw e } } else false diff --git a/akka-actor/src/main/scala/akka/event/DeathWatch.scala b/akka-actor/src/main/scala/akka/event/DeathWatch.scala index 31aabc076b..099d45a703 100644 --- a/akka-actor/src/main/scala/akka/event/DeathWatch.scala +++ b/akka-actor/src/main/scala/akka/event/DeathWatch.scala @@ -10,27 +10,10 @@ import akka.actor._ * The contract of DeathWatch is not properly expressed using the type system * Whenever there is a publish, all listeners to the Terminated Actor should be atomically removed * A failed subscribe should also only mean that the Classifier (ActorRef) that is listened to is already shut down - * See InVMMonitoring for semantics + * See LocalDeathWatch for semantics */ trait DeathWatch extends ActorEventBus with ActorClassifier { type Event = Terminated protected final def classify(event: Event): Classifier = event.actor -} - -object InVMMonitoring extends DeathWatch with ActorClassification { - - def mapSize = 1024 - - override def publish(event: Event): Unit = { - val monitors = dissociate(classify(event)) - if (monitors.nonEmpty) monitors.foreach(_ ! event) - } - - override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = { - if (!super.subscribe(subscriber, to)) { - subscriber ! Terminated(subscriber, new ActorKilledException("Already terminated when linking")) - false - } else true - } } \ No newline at end of file diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index ef7a82cfbe..5fcf8b8fee 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -9,7 +9,6 @@ import akka.routing._ import akka.actor.Actor._ import akka.actor.Status._ import akka.dispatch._ -import akka.event.EventHandler import akka.util.duration._ import akka.config.ConfigurationException import akka.AkkaException @@ -20,6 +19,7 @@ import Compression.LZF import java.net.InetSocketAddress import com.google.protobuf.ByteString import akka.AkkaApplication +import akka.event.{ DeathWatch, EventHandler } /** * Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it. @@ -225,6 +225,8 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider connection ! command } } + + private[akka] def createDeathWatch(): DeathWatch = local.createDeathWatch() //FIXME Implement Remote DeathWatch } /** From 2270395d3f92ff03d3d5a37387531ace62b7a585 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 17 Oct 2011 14:53:32 +0200 Subject: [PATCH 02/33] Adding try-finally in the system message processing to ensure that the cleanup is performed accurately --- .../src/main/scala/akka/actor/ActorCell.scala | 38 ++++++++++--------- 1 file changed, 21 insertions(+), 17 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index f95ed69e88..84a82351ec 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -385,26 +385,30 @@ private[akka] class ActorCell( dispatcher.detach(this) try { - val a = actor - if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopping") - if (a ne null) a.postStop() - - //Stop supervised actors - val links = _children - if (links.nonEmpty) { - _children = Vector.empty - links.foreach(_.child.stop()) + try { + val a = actor + if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopping") + if (a ne null) a.postStop() + } finally { + //Stop supervised actors + val links = _children + if (links.nonEmpty) { + _children = Vector.empty + links.foreach(_.child.stop()) + } } - } finally { val cause = new ActorKilledException("Stopped") //FIXME TODO make this an object, can be reused everywhere - - if (supervisor.isDefined) supervisor.get ! ChildTerminated(self, cause) - - app.deathWatch.publish(Terminated(self, cause)) - - currentMessage = null - clearActorContext() + try { + if (supervisor.isDefined) supervisor.get ! ChildTerminated(self, cause) + } finally { + try { + app.deathWatch.publish(Terminated(self, cause)) + } finally { + currentMessage = null + clearActorContext() + } + } } } From ab4f62cebe64c4e4a1d9042f0c0b271b3ba0387c Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 17 Oct 2011 16:17:16 +0200 Subject: [PATCH 03/33] add first draft of supervision spec --- akka-docs/general/index.rst | 1 + akka-docs/general/supervision.rst | 109 ++++++++++++++++++++++++++++++ 2 files changed, 110 insertions(+) create mode 100644 akka-docs/general/supervision.rst diff --git a/akka-docs/general/index.rst b/akka-docs/general/index.rst index ae8a8f30b3..a578ebacbd 100644 --- a/akka-docs/general/index.rst +++ b/akka-docs/general/index.rst @@ -8,4 +8,5 @@ General configuration event-handler slf4j + supervision diff --git a/akka-docs/general/supervision.rst b/akka-docs/general/supervision.rst new file mode 100644 index 0000000000..800b427a5c --- /dev/null +++ b/akka-docs/general/supervision.rst @@ -0,0 +1,109 @@ +Supervision +=========== + +This chapter outlines the concept behind supervision, the primitives offered +and their semantics. For details on how that translates into real code, please +refer to the corresponding chapters for Scala and Java APIs. + +What Supervision Means +---------------------- + +Supervision describes a dependency relationship between actors: the supervisor +delegates tasks to subordinates and therefore must respond to their failures. +When a subordinate detects a failure (i.e. throws an exception), it suspends +itself and all its children and sends a message to its supervisor, signaling +failure. Depending on the nature of the work to be supervised and the nature +of the failure, the supervisor has four basic choices: + +#. Resume the subordinate, keeping its accumulated internal state +#. Restart the subordinate, clearing out its accumulated internal state +#. Stop the subordinate permanently +#. Escalate the failure + +It is important to always view an actor as part of a supervision hierarchy, +which explains the existence of the fourth choice (as a supervisor also is +subordinate to another supervisor higher up) and has implications on the first +three: resuming an actor resumes all its subordinates, restarting an actor +entails restarting all its subordinates, similarly stopping an actor will also +stop all its subordinates. + +Each supervisor is configured with a function translating all possible failure +causes (i.e. exceptions) into one of the four choices given above; notably, +this function does not take the failed actor’s identity as an input. It is +quite easy to come up with examples of structures where this might not seem +flexible enough, e.g. wishing for different strategies to be applied to +different subordinates. At this point it is vital to understand that +supervision is about forming a recursive fault handling structure. If you try +to do too much at one level, it will become hard to reason about, hence the +recommended way in this case is to add a level of supervision. + +Akka implements a specific form called “parental supervision”. Actors can only +be created by other actors—where the top-level actor is provided by the +library—and each created actor is supervised by its parent. This restriction +makes the formation of actor supervision hierarchies explicit and encourages +sound design decisions. It should be noted that this also guarantees that +actors cannot be orphaned or attached to supervisors from the outside, which +might otherwise catch them unawares. In addition, this yields a natural and +clean shutdown procedure for (parts of) actor applications. + +What Restarting Means +--------------------- + +When presented with an actor which failed while processing a certain message, +causes for the failure fall into three categories: + +* Systematic (i.e. programming) error for the specific message received +* (Transient) failure of some external resource used during processing the message +* Corrupt internal state of the actor + +Unless the failure is specifically recognizable, the third cause cannot be +ruled out, which leads to the conclusion that the internal state needs to be +cleared out. If the supervisor decides that its other children or itself is not +affected by the corruption—e.g. because of conscious application of the error +kernel pattern—it is therefore best to restart the child. This is carried out +by creating a new instance of the underlying :class:`Actor` class and replacing +the failed instance with the fresh one inside the child’s :class:`ActorRef`; +the ability to do this is one of the reasons for encapsulating actors within +special references. The new actor then resumes processing its mailbox, meaning +that the restart is not visible outside of the actor itself with the notable +exception that the message during which the failure occurred is not +re-processed. + +Restarting an actor in this way recursively restarts all its children in the +same fashion, whereby all parent–child relationships are kept intact. If this +is not the right approach for certain sub-trees of the supervision hierarchy, +you should choose to stop the failed actor instead, which will terminate all +its children recursively, after which that part of the system may be recreated +from scratch. The second part of this action may be implemented using the +lifecycle monitoring described next or using lifecycle callbacks as described +in :class:`Actor`. + +What Lifecycle Monitoring Means +------------------------------- + +In contrast to the special relationship between parent and child described +above, each actor may monitor any other actor. Since actors emerge from +creation fully started and restarts are not visible outside of the affected +supervisors, the only state change available for monitoring is the transition +from alive to dead. Monitoring is thus used to tie one actor to another so that +it may react to the other actor’s termination, in contrast to supervision which +reacts to failure. + +Lifecycle monitoring is implemented using a :class:`Terminated` message to be +received by the behavior of the monitoring actor, where the default behavior is +to throw a special :class:`DeathPactException` if not otherwise handled. One +important property is that the message will be delivered irrespective of the +order in which the monitoring request and target’s termination occur, i.e. you +still get the message even if at the time of registration the target is already +dead. + +Monitoring is particularly useful if a supervisor cannot simply restart its +children and has to stop them, e.g. in case of errors during actor +initialization. In that case it should monitor those children and re-create +them or schedule itself to retry this at a later time. + +Another common use case is that an actor needs to fail in the absence of an +external resource, which may also be one of its own children. If a third party +terminates a child by way of the ``stop()`` method or sending a +:class:`PoisonPill`, the supervisor might well be affected. + From f75d16f7109e79f06b1293401bd05ef88569b0e9 Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 17 Oct 2011 17:59:04 +0200 Subject: [PATCH 04/33] fix small naming errors in supervision.rst --- akka-docs/general/supervision.rst | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/akka-docs/general/supervision.rst b/akka-docs/general/supervision.rst index 800b427a5c..0867d931f8 100644 --- a/akka-docs/general/supervision.rst +++ b/akka-docs/general/supervision.rst @@ -11,13 +11,13 @@ What Supervision Means Supervision describes a dependency relationship between actors: the supervisor delegates tasks to subordinates and therefore must respond to their failures. When a subordinate detects a failure (i.e. throws an exception), it suspends -itself and all its children and sends a message to its supervisor, signaling -failure. Depending on the nature of the work to be supervised and the nature -of the failure, the supervisor has four basic choices: +itself and all its subordinates and sends a message to its supervisor, +signaling failure. Depending on the nature of the work to be supervised and +the nature of the failure, the supervisor has four basic choices: #. Resume the subordinate, keeping its accumulated internal state #. Restart the subordinate, clearing out its accumulated internal state -#. Stop the subordinate permanently +#. Terminate the subordinate permanently #. Escalate the failure It is important to always view an actor as part of a supervision hierarchy, @@ -83,7 +83,7 @@ What Lifecycle Monitoring Means In contrast to the special relationship between parent and child described above, each actor may monitor any other actor. Since actors emerge from -creation fully started and restarts are not visible outside of the affected +creation fully alive and restarts are not visible outside of the affected supervisors, the only state change available for monitoring is the transition from alive to dead. Monitoring is thus used to tie one actor to another so that it may react to the other actor’s termination, in contrast to supervision which From 050411bf3b8d0b8c6c53086acc3fbafd18b0d4cf Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 17 Oct 2011 18:34:34 +0200 Subject: [PATCH 05/33] Making a Java API for Scheduler (JScheduler) and an abstract class Scheduler that extends it, to make the Scheduler pluggable, moving it into AkkaApplication and migrating the code. --- .../test/scala/akka/actor/SchedulerSpec.scala | 14 +++--- .../src/main/scala/akka/AkkaApplication.scala | 2 + .../src/main/scala/akka/actor/ActorCell.scala | 2 +- .../src/main/scala/akka/actor/FSM.scala | 9 ++-- .../src/main/scala/akka/actor/Scheduler.scala | 43 +++++++++++++++---- .../akka/dispatch/AbstractDispatcher.scala | 6 +-- .../src/main/scala/akka/dispatch/Future.scala | 10 ++--- akka-docs/common/scheduler.rst | 15 +------ .../akka/remote/RemoteActorRefProvider.scala | 2 +- .../main/scala/DiningHakkersOnBecome.scala | 8 ++-- 10 files changed, 65 insertions(+), 46 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index 9089929ce6..eb5865f5c2 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -33,14 +33,14 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { def receive = { case Tick ⇒ countDownLatch.countDown() } }) // run every 50 millisec - collectFuture(Scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS)) + collectFuture(app.scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS)) // after max 1 second it should be executed at least the 3 times already assert(countDownLatch.await(1, TimeUnit.SECONDS)) val countDownLatch2 = new CountDownLatch(3) - collectFuture(Scheduler.schedule(() ⇒ countDownLatch2.countDown(), 0, 50, TimeUnit.MILLISECONDS)) + collectFuture(app.scheduler.schedule(() ⇒ countDownLatch2.countDown(), 0, 50, TimeUnit.MILLISECONDS)) // after max 1 second it should be executed at least the 3 times already assert(countDownLatch2.await(2, TimeUnit.SECONDS)) @@ -53,8 +53,8 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { def receive = { case Tick ⇒ countDownLatch.countDown() } }) // run every 50 millisec - collectFuture(Scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS)) - collectFuture(Scheduler.scheduleOnce(() ⇒ countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS)) + collectFuture(app.scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS)) + collectFuture(app.scheduler.scheduleOnce(() ⇒ countDownLatch.countDown(), 50, TimeUnit.MILLISECONDS)) // after 1 second the wait should fail assert(countDownLatch.await(2, TimeUnit.SECONDS) == false) @@ -90,7 +90,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { }) (1 to 10).foreach { i ⇒ - val future = collectFuture(Scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.SECONDS)) + val future = collectFuture(app.scheduler.scheduleOnce(actor, Ping, 1, TimeUnit.SECONDS)) future.cancel(true) } assert(ticks.await(3, TimeUnit.SECONDS) == false) //No counting down should've been made @@ -116,9 +116,9 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { override def postRestart(reason: Throwable) = restartLatch.open }).withSupervisor(supervisor)) - collectFuture(Scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS)) + collectFuture(app.scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS)) // appx 2 pings before crash - collectFuture(Scheduler.scheduleOnce(actor, Crash, 1000, TimeUnit.MILLISECONDS)) + collectFuture(app.scheduler.scheduleOnce(actor, Crash, 1000, TimeUnit.MILLISECONDS)) assert(restartLatch.tryAwait(2, TimeUnit.SECONDS)) // should be enough time for the ping countdown to recover and reach 6 pings diff --git a/akka-actor/src/main/scala/akka/AkkaApplication.scala b/akka-actor/src/main/scala/akka/AkkaApplication.scala index 60e7d78df2..ce67f66745 100644 --- a/akka-actor/src/main/scala/akka/AkkaApplication.scala +++ b/akka-actor/src/main/scala/akka/AkkaApplication.scala @@ -181,4 +181,6 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor val typedActor = new TypedActor(this) val serialization = new Serialization(this) + + val scheduler = new DefaultScheduler } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 84a82351ec..1d46a48623 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -505,7 +505,7 @@ private[akka] class ActorCell( val recvtimeout = receiveTimeout if (recvtimeout.isDefined && dispatcher.mailboxIsEmpty(this)) { //Only reschedule if desired and there are currently no more messages to be processed - futureTimeout = Some(Scheduler.scheduleOnce(self, ReceiveTimeout, recvtimeout.get, TimeUnit.MILLISECONDS)) + futureTimeout = Some(app.scheduler.scheduleOnce(self, ReceiveTimeout, recvtimeout.get, TimeUnit.MILLISECONDS)) } } diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index 5cbeec8b09..4beff7229d 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -8,6 +8,7 @@ import akka.event.EventHandler import scala.collection.mutable import java.util.concurrent.ScheduledFuture +import akka.AkkaApplication object FSM { @@ -29,14 +30,14 @@ object FSM { case object StateTimeout case class TimeoutMarker(generation: Long) - case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int) { + case class Timer(name: String, msg: Any, repeat: Boolean, generation: Int)(implicit app: AkkaApplication) { private var ref: Option[ScheduledFuture[AnyRef]] = _ def schedule(actor: ActorRef, timeout: Duration) { if (repeat) { - ref = Some(Scheduler.schedule(actor, this, timeout.length, timeout.length, timeout.unit)) + ref = Some(app.scheduler.schedule(actor, this, timeout.length, timeout.length, timeout.unit)) } else { - ref = Some(Scheduler.scheduleOnce(actor, this, timeout.length, timeout.unit)) + ref = Some(app.scheduler.scheduleOnce(actor, this, timeout.length, timeout.unit)) } } @@ -525,7 +526,7 @@ trait FSM[S, D] extends ListenerManagement { if (timeout.isDefined) { val t = timeout.get if (t.finite_? && t.length >= 0) { - timeoutFuture = Some(Scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit)) + timeoutFuture = Some(app.scheduler.scheduleOnce(self, TimeoutMarker(generation), t.length, t.unit)) } } } diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index d68b262085..57a7ccbc8d 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -15,22 +15,49 @@ */ package akka.actor -import akka.event.EventHandler import akka.AkkaException import java.util.concurrent.atomic.AtomicLong import java.util.concurrent._ -import java.lang.RuntimeException +import akka.util.Duration -object Scheduler { +case class SchedulerException(msg: String, e: Throwable) extends AkkaException(msg, e) { + def this(msg: String) = this(msg, null) +} - case class SchedulerException(msg: String, e: Throwable) extends AkkaException(msg, e) +trait JScheduler { + def schedule(receiver: ActorRef, message: Any, initialDelay: Long, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] + def scheduleOnce(runnable: Runnable, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] + def scheduleOnce(receiver: ActorRef, message: Any, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] +} + +abstract class Scheduler extends JScheduler { + + def schedule(f: () ⇒ Unit, initialDelay: Long, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] + def scheduleOnce(f: () ⇒ Unit, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] + + def schedule(receiver: ActorRef, message: Any, initialDelay: Duration, delay: Duration): ScheduledFuture[AnyRef] = + schedule(receiver, message, initialDelay.toNanos, delay.toNanos, TimeUnit.NANOSECONDS) + + def schedule(f: () ⇒ Unit, initialDelay: Duration, delay: Duration): ScheduledFuture[AnyRef] = + schedule(f, initialDelay.toNanos, delay.toNanos, TimeUnit.NANOSECONDS) + + def scheduleOnce(receiver: ActorRef, message: Any, delay: Duration): ScheduledFuture[AnyRef] = + scheduleOnce(receiver, message, delay.length, delay.unit) + + def scheduleOnce(f: () ⇒ Unit, delay: Duration): ScheduledFuture[AnyRef] = + scheduleOnce(f, delay.length, delay.unit) +} + +class DefaultScheduler extends Scheduler { + private def createSendRunnable(receiver: ActorRef, message: Any, throwWhenReceiverExpired: Boolean): Runnable = new Runnable { + def run = { + receiver ! message + if (throwWhenReceiverExpired && receiver.isShutdown) throw new ActorKilledException("Receiver was terminated") + } + } private[akka] val service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory) - private def createSendRunnable(receiver: ActorRef, message: Any, throwWhenReceiverExpired: Boolean): Runnable = { - new Runnable { def run = receiver ! message } - } - /** * Schedules to send the specified message to the receiver after initialDelay and then repeated after delay. * The returned java.util.concurrent.ScheduledFuture can be used to cancel the diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index aef413579f..c7e7fab35a 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -127,7 +127,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable shutdownSchedule match { case UNSCHEDULED ⇒ shutdownSchedule = SCHEDULED - Scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) + app.scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) case SCHEDULED ⇒ shutdownSchedule = RESCHEDULED case RESCHEDULED ⇒ //Already marked for reschedule @@ -159,7 +159,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable shutdownSchedule match { case UNSCHEDULED ⇒ shutdownSchedule = SCHEDULED - Scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) + app.scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS) case SCHEDULED ⇒ shutdownSchedule = RESCHEDULED case RESCHEDULED ⇒ //Already marked for reschedule @@ -220,7 +220,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable shutdownSchedule match { case RESCHEDULED ⇒ shutdownSchedule = SCHEDULED - Scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) + app.scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) case SCHEDULED ⇒ if (uuids.isEmpty && _tasks.get == 0) { active switchOff { diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala index cfda93760f..dc302c5af9 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Future.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala @@ -7,7 +7,7 @@ package akka.dispatch import akka.AkkaException import akka.event.EventHandler -import akka.actor.{ Actor, UntypedChannel, Scheduler, Timeout, ExceptionChannel } +import akka.actor.{ Actor, UntypedChannel, Timeout, ExceptionChannel } import scala.Option import akka.japi.{ Procedure, Function ⇒ JFunc, Option ⇒ JOption } @@ -947,12 +947,12 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi val runnable = new Runnable { def run() { if (!isCompleted) { - if (!isExpired) Scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) + if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) else func(DefaultPromise.this) } } } - Scheduler.scheduleOnce(runnable, timeLeft(), NANOS) + dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS) false } else true } else false @@ -973,12 +973,12 @@ class DefaultPromise[T](val timeout: Timeout)(implicit val dispatcher: MessageDi val runnable = new Runnable { def run() { if (!isCompleted) { - if (!isExpired) Scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) + if (!isExpired) dispatcher.app.scheduler.scheduleOnce(this, timeLeftNoinline(), NANOS) else promise complete (try { Right(fallback) } catch { case e ⇒ Left(e) }) } } } - Scheduler.scheduleOnce(runnable, timeLeft(), NANOS) + dispatcher.app.scheduler.scheduleOnce(runnable, timeLeft(), NANOS) promise } } else this diff --git a/akka-docs/common/scheduler.rst b/akka-docs/common/scheduler.rst index bf2b813d2e..7fd28d37ee 100644 --- a/akka-docs/common/scheduler.rst +++ b/akka-docs/common/scheduler.rst @@ -1,23 +1,12 @@ Scheduler ========= -Module stability: **SOLID** - -``Akka`` has a little scheduler written using actors. -This can be convenient if you want to schedule some periodic task for maintenance or similar. - -It allows you to register a message that you want to be sent to a specific actor at a periodic interval. +//FIXME Here is an example: ------------------- .. code-block:: scala - import akka.actor.Scheduler - - //Sends messageToBeSent to receiverActor after initialDelayBeforeSending and then after each delayBetweenMessages - Scheduler.schedule(receiverActor, messageToBeSent, initialDelayBeforeSending, delayBetweenMessages, timeUnit) - - //Sends messageToBeSent to receiverActor after delayUntilSend - Scheduler.scheduleOnce(receiverActor, messageToBeSent, delayUntilSend, timeUnit) + //TODO FIXME diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 5fcf8b8fee..87a65d21bf 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -138,7 +138,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider actor } else { // we lost the race -- wait for future to complete - oldFuture.await.resultOrException.get + oldFuture.get } } diff --git a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala index 4bd6baafb2..d3326d85d6 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala @@ -3,7 +3,7 @@ package sample.fsm.dining.become //Akka adaptation of //http://www.dalnefre.com/wp/2010/08/dining-philosophers-in-humus/ -import akka.actor.{ Scheduler, ActorRef, Actor } +import akka.actor.{ ActorRef, Actor } import java.util.concurrent.TimeUnit import akka.AkkaApplication @@ -78,7 +78,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor { case Taken(`chopstickToWaitFor`) ⇒ println("%s has picked up %s and %s, and starts to eat", name, left.address, right.address) become(eating) - Scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS) + app.scheduler.scheduleOnce(self, Think, 5, TimeUnit.SECONDS) case Busy(chopstick) ⇒ become(thinking) @@ -107,7 +107,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor { left ! Put(self) right ! Put(self) println("%s puts down his chopsticks and starts to think", name) - Scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) + app.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) } //All hakkers start in a non-eating state @@ -115,7 +115,7 @@ class Hakker(name: String, left: ActorRef, right: ActorRef) extends Actor { case Think ⇒ println("%s starts to think", name) become(thinking) - Scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) + app.scheduler.scheduleOnce(self, Eat, 5, TimeUnit.SECONDS) } } From 379515771b0a89b904d32dadd3e4fc945dbd24a1 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 17 Oct 2011 19:31:04 +0200 Subject: [PATCH 06/33] Tidying up some superflous lines of code in Scheduler --- .../src/main/scala/akka/actor/Scheduler.scala | 16 ++++------------ 1 file changed, 4 insertions(+), 12 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala index 57a7ccbc8d..5d64eec5f1 100644 --- a/akka-actor/src/main/scala/akka/actor/Scheduler.scala +++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala @@ -67,9 +67,7 @@ class DefaultScheduler extends Scheduler { try { service.scheduleAtFixedRate(createSendRunnable(receiver, message, true), initialDelay, delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]] } catch { - case e: Exception ⇒ - val error = SchedulerException(message + " could not be scheduled on " + receiver, e) - throw error + case e: Exception ⇒ throw SchedulerException(message + " could not be scheduled on " + receiver, e) } } @@ -92,9 +90,7 @@ class DefaultScheduler extends Scheduler { try { service.scheduleAtFixedRate(runnable, initialDelay, delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]] } catch { - case e: Exception ⇒ - val error = SchedulerException("Failed to schedule a Runnable", e) - throw error + case e: Exception ⇒ throw SchedulerException("Failed to schedule a Runnable", e) } } @@ -107,9 +103,7 @@ class DefaultScheduler extends Scheduler { try { service.schedule(createSendRunnable(receiver, message, false), delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]] } catch { - case e: Exception ⇒ - val error = SchedulerException(message + " could not be scheduleOnce'd on " + receiver, e) - throw error + case e: Exception ⇒ throw SchedulerException(message + " could not be scheduleOnce'd on " + receiver, e) } } @@ -132,9 +126,7 @@ class DefaultScheduler extends Scheduler { try { service.schedule(runnable, delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]] } catch { - case e: Exception ⇒ - val error = SchedulerException("Failed to scheduleOnce a Runnable", e) - throw error + case e: Exception ⇒ throw SchedulerException("Failed to scheduleOnce a Runnable", e) } } From fa1a2610a7d0a943fe0a4fa5b451782172e89516 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 17 Oct 2011 19:31:59 +0200 Subject: [PATCH 07/33] Removing RemoteActorSystemMessage.Stop in favor of the sexier Terminate message --- .../src/main/scala/akka/cluster/ClusterActorRef.scala | 2 +- .../src/main/scala/akka/remote/RemoteActorRefProvider.scala | 2 +- .../main/scala/akka/remote/netty/NettyRemoteSupport.scala | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala index 43ea3586a9..a61ca3a6e1 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala @@ -117,7 +117,7 @@ private[akka] class ClusterActorRef(props: RoutedProps, val address: String) ext if (_status == ActorRefInternals.RUNNING) { Actor.registry.local.unregisterClusterActorRef(this) _status = ActorRefInternals.SHUTDOWN - postMessageToMailbox(RemoteActorSystemMessage.Stop, None) + postMessageToMailbox(Terminate, None) // FIXME here we need to fire off Actor.cluster.remove(address) (which needs to be properly implemented first, see ticket) connections.stopAll() diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 87a65d21bf..0cc5efa591 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -273,7 +273,7 @@ private[akka] case class RemoteActorRef private[akka] ( synchronized { if (running) { running = false - postMessageToMailbox(RemoteActorSystemMessage.Stop, None) + postMessageToMailbox(Terminate, None) } } } diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index 9c858d4701..519493704b 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -4,8 +4,7 @@ package akka.remote.netty -import akka.actor.{ ActorRef, Uuid, newUuid, uuidFrom, IllegalActorStateException, PoisonPill, RemoteActorSystemMessage, AutoReceivedMessage } -import akka.dispatch.{ ActorPromise, DefaultPromise, Promise } +import akka.actor.{ ActorRef, Uuid, newUuid, uuidFrom, IllegalActorStateException, PoisonPill, AutoReceivedMessage } import akka.remote._ import RemoteProtocol._ import akka.util._ @@ -28,6 +27,7 @@ import java.util.concurrent.atomic._ import akka.AkkaException import akka.AkkaApplication import akka.serialization.RemoteActorSerialization +import akka.dispatch.{ Terminate, ActorPromise, DefaultPromise, Promise } class RemoteClientMessageBufferException(message: String, cause: Throwable = null) extends AkkaException(message, cause) { def this(msg: String) = this(msg, null); @@ -958,7 +958,7 @@ class RemoteServerHandler( message match { // first match on system messages - case RemoteActorSystemMessage.Stop ⇒ + case Terminate ⇒ if (UNTRUSTED_MODE) throw new SecurityException("RemoteModule server is operating is untrusted mode, can not stop the actor") else actorRef.stop() From 4e960e536433a8ab09c07b41cf6dc5c657a869b3 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Mon, 17 Oct 2011 19:33:19 +0200 Subject: [PATCH 08/33] Changing so that the mailbox status is ack:ed after the _whole_ processing of the current batch, which means that Akka only does 1 volatile write per batch (of course the backing mailboxes might do their own volatile writes) --- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 4 ---- akka-actor/src/main/scala/akka/dispatch/Mailbox.scala | 5 ++++- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 1d46a48623..30f82e975f 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -443,8 +443,6 @@ private[akka] class ActorCell( app.eventHandler.error(e, self, "error while processing " + envelope.message) //TODO FIXME How should problems here be handled? throw e - } finally { - mailbox.acknowledgeStatus() //Volatile write } } @@ -487,8 +485,6 @@ private[akka] class ActorCell( messageHandle.channel sendException new ActorKilledException("Actor has been stopped") // throwing away message if actor is shut down, no use throwing an exception in receiving actor's thread, isShutdown is enforced on caller side } - } finally { - mailbox.acknowledgeStatus() } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 89f5c9bb62..5e21d5ea87 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -128,7 +128,10 @@ abstract class Mailbox extends AbstractMailbox with MessageQueue with SystemMess */ if (s >= Scheduled) { updateStatus(s, s & ~Scheduled) || setAsIdle() - } else false + } else { + acknowledgeStatus() // this write is needed to make memory consistent after processMailbox() + false + } } def shouldBeRegisteredForExecution(hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = status match { From cb8a0adbb8f30f4c77be8176659c4ac9cb974394 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 18 Oct 2011 11:26:35 +0200 Subject: [PATCH 09/33] Switching to a cached version of Stack.empty, saving 16 bytes per Actor. Switching to purging the Promises in the ActorRefProvider after successful creation to conserve memory. Stopping to clone the props everytime to set the application default dispatcher, and doing a conditional in ActorCell instead. --- .../src/main/scala/akka/actor/ActorCell.scala | 3 +- .../src/main/scala/akka/actor/ActorRef.scala | 13 +- .../scala/akka/actor/ActorRefProvider.scala | 94 +++++----- .../src/main/scala/akka/actor/Props.scala | 2 + .../main/scala/akka/actor/TypedActor.scala | 4 +- .../akka/routing/ConnectionManager.scala | 8 + .../akka/remote/RemoteActorRefProvider.scala | 163 +++++++++--------- .../akka/remote/RemoteConnectionManager.scala | 2 + 8 files changed, 139 insertions(+), 150 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 30f82e975f..23014cd884 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -231,7 +231,8 @@ private[akka] class ActorCell( def uuid: Uuid = self.uuid - def dispatcher: MessageDispatcher = props.dispatcher + @inline + final def dispatcher: MessageDispatcher = if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher def isShutdown: Boolean = mailbox.isClosed diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 6e47f60300..c511847261 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -152,12 +152,12 @@ abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyCha */ class LocalActorRef private[akka] ( app: AkkaApplication, - private[this] val props: Props, + props: Props, givenAddress: String, val systemService: Boolean = false, override private[akka] val uuid: Uuid = newUuid, receiveTimeout: Option[Long] = None, - hotswap: Stack[PartialFunction[Any, Unit]] = Stack.empty) + hotswap: Stack[PartialFunction[Any, Unit]] = Props.noHotSwap) extends ActorRef with ScalaActorRef { private[this] val actorCell = new ActorCell(app, this, props, receiveTimeout, hotswap) @@ -253,15 +253,6 @@ class LocalActorRef private[akka] ( } } -/** - * System messages for RemoteActorRef. - * - * @author Jonas Bonér - */ -object RemoteActorSystemMessage { - val Stop = "RemoteActorRef:stop".intern -} - /** * This trait represents the common (external) methods for all ActorRefs * Needed because implicit conversions aren't applied when instance imports are used diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 0b94e94809..a8a2465458 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -8,12 +8,11 @@ import akka.config.ConfigurationException import akka.util.ReflectiveAccess import akka.routing._ import akka.AkkaApplication -import akka.dispatch.MessageDispatcher import java.util.concurrent.ConcurrentHashMap -import akka.dispatch.Promise import com.eaio.uuid.UUID import akka.AkkaException import akka.event.{ ActorClassification, DeathWatch, EventHandler } +import akka.dispatch.{ Future, MessageDispatcher, Promise } /** * Interface for all ActorRef providers to implement. @@ -79,13 +78,14 @@ class ActorRefProviderException(message: String) extends AkkaException(message) */ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { - private val actors = new ConcurrentHashMap[String, Promise[ActorRef]] + private val actors = new ConcurrentHashMap[String, AnyRef] def actorOf(props: Props, address: String): ActorRef = actorOf(props, address, false) def actorFor(address: String): Option[ActorRef] = actors.get(address) match { - case null ⇒ None - case future ⇒ Some(future.get) + case null ⇒ None + case actor: ActorRef ⇒ Some(actor) + case future: Future[_] ⇒ Some(future.get.asInstanceOf[ActorRef]) } /** @@ -94,61 +94,53 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { private[akka] def evict(address: String): Boolean = actors.remove(address) ne null private[akka] def actorOf(props: Props, address: String, systemService: Boolean): ActorRef = { - Address.validate(address) - - val localProps = - if (props.dispatcher == Props.defaultDispatcher) - props.copy(dispatcher = app.dispatcher) - else - props - - val defaultTimeout = app.AkkaConfig.ActorTimeout val newFuture = Promise[ActorRef](5000)(app.dispatcher) // FIXME is this proper timeout? - val oldFuture = actors.putIfAbsent(address, newFuture) - if (oldFuture eq null) { // we won the race -- create the actor and resolve the future + actors.putIfAbsent(address, newFuture) match { + case null ⇒ + val actor: ActorRef = try { + app.deployer.lookupDeploymentFor(address) match { // see if the deployment already exists, if so use it, if not create actor - val actor: ActorRef = try { - app.deployer.lookupDeploymentFor(address) match { // see if the deployment already exists, if so use it, if not create actor + // create a local actor + case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, _, DeploymentConfig.LocalScope)) ⇒ + new LocalActorRef(app, props, address, systemService) // create a local actor - // create a local actor - case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, _, DeploymentConfig.LocalScope)) ⇒ - new LocalActorRef(app, localProps, address, systemService) // create a local actor + // create a routed actor ref + case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, _, DeploymentConfig.LocalScope)) ⇒ + val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match { + case RouterType.Direct ⇒ () ⇒ new DirectRouter + case RouterType.Random ⇒ () ⇒ new RandomRouter + case RouterType.RoundRobin ⇒ () ⇒ new RoundRobinRouter + case RouterType.ScatterGather ⇒ () ⇒ new ScatterGatherFirstCompletedRouter()( + if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher, app.AkkaConfig.ActorTimeout) + case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") + case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") + case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") + case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass) + } - // create a routed actor ref - case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, _, DeploymentConfig.LocalScope)) ⇒ - val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match { - case RouterType.Direct ⇒ () ⇒ new DirectRouter - case RouterType.Random ⇒ () ⇒ new RandomRouter - case RouterType.RoundRobin ⇒ () ⇒ new RoundRobinRouter - case RouterType.ScatterGather ⇒ () ⇒ new ScatterGatherFirstCompletedRouter()(props.dispatcher, defaultTimeout) - case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") - case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") - case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") - case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass) - } + val connections: Iterable[ActorRef] = + if (nrOfInstances.factor > 0) Vector.fill(nrOfInstances.factor)(new LocalActorRef(app, props, "", systemService)) else Nil - val connections: Iterable[ActorRef] = - if (nrOfInstances.factor > 0) - Vector.fill(nrOfInstances.factor)(new LocalActorRef(app, localProps, new UUID().toString, systemService)) - else Nil + actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), address) - actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), address) - - case _ ⇒ throw new Exception("Don't know how to create this actor ref! Why?") + case _ ⇒ throw new Exception("Don't know how to create this actor ref! Why?") + } + } catch { + case e: Exception ⇒ + newFuture completeWithException e // so the other threads gets notified of error + //TODO FIXME should we remove the mapping in "actors" here? + throw e } - } catch { - case e: Exception ⇒ - newFuture completeWithException e // so the other threads gets notified of error - throw e - } - newFuture completeWithResult actor - actor - - } else { // we lost the race -- wait for future to complete - oldFuture.await.resultOrException.get + newFuture completeWithResult actor + actors.replace(address, newFuture, actor) + actor + case actor: ActorRef ⇒ + actor + case future: Future[_] ⇒ + future.get.asInstanceOf[ActorRef] } } @@ -161,7 +153,7 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { //TODO If address matches an already created actor (Ahead-of-time deployed) return that actor //TODO If address exists in config, it will override the specified Props (should we attempt to merge?) //TODO If the actor deployed uses a different config, then ignore or throw exception? - if (props.connectionManager.size == 0) throw new ConfigurationException("RoutedProps used for creating actor [" + address + "] has zero connections configured; can't create a router") + if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + address + "] has zero connections configured; can't create a router") // val clusteringEnabled = ReflectiveAccess.ClusterModule.isEnabled // val localOnly = props.localOnly // if (clusteringEnabled && !props.localOnly) ReflectiveAccess.ClusterModule.newClusteredActorRef(props) diff --git a/akka-actor/src/main/scala/akka/actor/Props.scala b/akka-actor/src/main/scala/akka/actor/Props.scala index 2aa4eeb8e7..7aedfbdd0a 100644 --- a/akka-actor/src/main/scala/akka/actor/Props.scala +++ b/akka-actor/src/main/scala/akka/actor/Props.scala @@ -7,6 +7,7 @@ package akka.actor import akka.dispatch._ import akka.japi.Creator import akka.util._ +import collection.immutable.Stack /** * ActorRef configuration object, this is threadsafe and fully sharable @@ -20,6 +21,7 @@ object Props { final val defaultTimeout: Timeout = Timeout(Duration.MinusInf) final val defaultFaultHandler: FaultHandlingStrategy = OneForOneStrategy(classOf[Exception] :: Nil, None, None) final val defaultSupervisor: Option[ActorRef] = None + final val noHotSwap: Stack[Actor.Receive] = Stack.empty /** * The default Props instance, uses the settings from the Props object starting with default* diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 64e6fc64dd..d37b3f8c16 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -251,8 +251,8 @@ class TypedActor(val app: AkkaApplication) { //Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling val actorVar = new AtomVar[ActorRef](null) val timeout = props.timeout match { - case Timeout(Duration.MinusInf) ⇒ app.AkkaConfig.ActorTimeout - case x ⇒ x + case Props.`defaultTimeout` ⇒ app.AkkaConfig.ActorTimeout + case x ⇒ x } val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)(timeout)).asInstanceOf[T] proxyVar.set(proxy) // Chicken and egg situation we needed to solve, set the proxy so that we can set the self-reference inside each receive diff --git a/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala b/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala index 2d6d8c549e..80230e73ff 100644 --- a/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala +++ b/akka-actor/src/main/scala/akka/routing/ConnectionManager.scala @@ -40,6 +40,12 @@ trait ConnectionManager { */ def size: Int + /** + * Returns if the number of 'available' is 0 or not. Value could be stale as soon as received, and this method can't be combined (easily) + * with an atomic read of and isEmpty and version. + */ + def isEmpty: Boolean + /** * Shuts the connection manager down, which stops all managed actors */ @@ -90,6 +96,8 @@ class LocalConnectionManager(initialConnections: Iterable[ActorRef]) extends Con def size: Int = state.get.connections.size + def isEmpty: Boolean = state.get.connections.isEmpty + def connections = state.get def shutdown() { diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 0cc5efa591..0f4692a6af 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -34,7 +34,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider val local = new LocalActorRefProvider(app) val remote = new Remote(app) - private val actors = new ConcurrentHashMap[String, Promise[ActorRef]] + private val actors = new ConcurrentHashMap[String, AnyRef] private val remoteDaemonConnectionManager = new RemoteConnectionManager( app, @@ -47,98 +47,90 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider def actorOf(props: Props, address: String): ActorRef = actorOf(props, address, false) def actorOf(props: Props, address: String, systemService: Boolean): ActorRef = { - Address.validate(address) - val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout? - val oldFuture = actors.putIfAbsent(address, newFuture) - if (oldFuture eq null) { // we won the race -- create the actor and resolve the future - val actor: ActorRef = try { - app.deployer.lookupDeploymentFor(address) match { - case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, DeploymentConfig.RemoteScope(remoteAddresses))) ⇒ + actors.putIfAbsent(address, newFuture) match { // we won the race -- create the actor and resolve the future + case null ⇒ + val actor: ActorRef = try { + app.deployer.lookupDeploymentFor(address) match { + case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, DeploymentConfig.RemoteScope(remoteAddresses))) ⇒ - val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match { - case FailureDetectorType.NoOp ⇒ new NoOpFailureDetector - case FailureDetectorType.RemoveConnectionOnFirstFailure ⇒ new RemoveConnectionOnFirstFailureFailureDetector - case FailureDetectorType.BannagePeriod(timeToBan) ⇒ new BannagePeriodFailureDetector(timeToBan) - case FailureDetectorType.Custom(implClass) ⇒ FailureDetector.createCustomFailureDetector(implClass) - } - - val thisHostname = remote.address.getHostName - val thisPort = remote.address.getPort - - def isReplicaNode: Boolean = remoteAddresses exists { remoteAddress ⇒ - remoteAddress.hostname == thisHostname && remoteAddress.port == thisPort - } - - if (isReplicaNode) { - // we are on one of the replica node for this remote actor - val localProps = - if (props.dispatcher == Props.defaultDispatcher) props.copy(dispatcher = app.dispatcher) - else props - new LocalActorRef(app, localProps, address, false) - } else { - - // we are on the single "reference" node uses the remote actors on the replica nodes - val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match { - case RouterType.Direct ⇒ - if (remoteAddresses.size != 1) throw new ConfigurationException( - "Actor [%s] configured with Direct router must have exactly 1 remote node configured. Found [%s]" - .format(address, remoteAddresses.mkString(", "))) - () ⇒ new DirectRouter - - case RouterType.Random ⇒ - if (remoteAddresses.size < 1) throw new ConfigurationException( - "Actor [%s] configured with Random router must have at least 1 remote node configured. Found [%s]" - .format(address, remoteAddresses.mkString(", "))) - () ⇒ new RandomRouter - - case RouterType.RoundRobin ⇒ - if (remoteAddresses.size < 1) throw new ConfigurationException( - "Actor [%s] configured with RoundRobin router must have at least 1 remote node configured. Found [%s]" - .format(address, remoteAddresses.mkString(", "))) - () ⇒ new RoundRobinRouter - - case RouterType.ScatterGather ⇒ - if (remoteAddresses.size < 1) throw new ConfigurationException( - "Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]" - .format(address, remoteAddresses.mkString(", "))) - () ⇒ new ScatterGatherFirstCompletedRouter()(defaultDispatcher, defaultTimeout) - - case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") - case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") - case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") - case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass) + val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match { + case FailureDetectorType.NoOp ⇒ new NoOpFailureDetector + case FailureDetectorType.RemoveConnectionOnFirstFailure ⇒ new RemoveConnectionOnFirstFailureFailureDetector + case FailureDetectorType.BannagePeriod(timeToBan) ⇒ new BannagePeriodFailureDetector(timeToBan) + case FailureDetectorType.Custom(implClass) ⇒ FailureDetector.createCustomFailureDetector(implClass) } - var connections = Map.empty[InetSocketAddress, ActorRef] - remoteAddresses foreach { remoteAddress: DeploymentConfig.RemoteAddress ⇒ - val inetSocketAddress = new InetSocketAddress(remoteAddress.hostname, remoteAddress.port) - connections += (inetSocketAddress -> RemoteActorRef(remote.server, inetSocketAddress, address, None)) + val thisHostname = remote.address.getHostName + val thisPort = remote.address.getPort + + def isReplicaNode: Boolean = remoteAddresses exists { some ⇒ some.hostname == thisHostname && some.port == thisPort } + + if (isReplicaNode) { + // we are on one of the replica node for this remote actor + new LocalActorRef(app, props, address, false) + } else { + + // we are on the single "reference" node uses the remote actors on the replica nodes + val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match { + case RouterType.Direct ⇒ + if (remoteAddresses.size != 1) throw new ConfigurationException( + "Actor [%s] configured with Direct router must have exactly 1 remote node configured. Found [%s]" + .format(address, remoteAddresses.mkString(", "))) + () ⇒ new DirectRouter + + case RouterType.Random ⇒ + if (remoteAddresses.size < 1) throw new ConfigurationException( + "Actor [%s] configured with Random router must have at least 1 remote node configured. Found [%s]" + .format(address, remoteAddresses.mkString(", "))) + () ⇒ new RandomRouter + + case RouterType.RoundRobin ⇒ + if (remoteAddresses.size < 1) throw new ConfigurationException( + "Actor [%s] configured with RoundRobin router must have at least 1 remote node configured. Found [%s]" + .format(address, remoteAddresses.mkString(", "))) + () ⇒ new RoundRobinRouter + + case RouterType.ScatterGather ⇒ + if (remoteAddresses.size < 1) throw new ConfigurationException( + "Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]" + .format(address, remoteAddresses.mkString(", "))) + () ⇒ new ScatterGatherFirstCompletedRouter()(defaultDispatcher, defaultTimeout) + + case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") + case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") + case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") + case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass) + } + + val connections = (Map.empty[InetSocketAddress, ActorRef] /: remoteAddresses) { (conns, a) ⇒ + val inetAddr = new InetSocketAddress(a.hostname, a.port) + conns + (inetAddr -> RemoteActorRef(remote.server, inetAddr, address, None)) + } + + val connectionManager = new RemoteConnectionManager(app, remote, connections, failureDetector) + + connections.keys foreach { useActorOnNode(_, address, props.creator) } + + actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), address) } - val connectionManager = new RemoteConnectionManager(app, remote, connections, failureDetector) - - connections.keys foreach { useActorOnNode(_, address, props.creator) } - - actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), address) - } - - case deploy ⇒ local.actorOf(props, address, systemService) + case deploy ⇒ local.actorOf(props, address, systemService) + } + } catch { + case e: Exception ⇒ + newFuture completeWithException e // so the other threads gets notified of error + throw e } - } catch { - case e: Exception ⇒ - newFuture completeWithException e // so the other threads gets notified of error - throw e - } - // actor foreach app.registry.register // only for ActorRegistry backward compat, will be removed later + // actor foreach app.registry.register // only for ActorRegistry backward compat, will be removed later - newFuture completeWithResult actor - actor - - } else { // we lost the race -- wait for future to complete - oldFuture.get + newFuture completeWithResult actor + actors.replace(address, newFuture, actor) + actor + case actor: ActorRef ⇒ actor + case future: Future[_] ⇒ future.get.asInstanceOf[ActorRef] } } @@ -151,8 +143,9 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider } def actorFor(address: String): Option[ActorRef] = actors.get(address) match { - case null ⇒ None - case future ⇒ Some(future.get) + case null ⇒ None + case actor: ActorRef ⇒ Some(actor) + case future: Future[_] ⇒ Some(future.get.asInstanceOf[ActorRef]) } /** diff --git a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala index b89b9310a5..d5a99fe893 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteConnectionManager.scala @@ -52,6 +52,8 @@ class RemoteConnectionManager( def size: Int = connections.connections.size + def isEmpty: Boolean = connections.connections.isEmpty + def shutdown() { state.get.iterable foreach (_.stop()) // shut down all remote connections } From 7a6508973fa02d9e7758a0570d015d71a1a3bdc6 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 18 Oct 2011 12:40:44 +0200 Subject: [PATCH 10/33] Adding extra output to give more hope in reproducing weird test failure that only happens in Jenkins --- akka-testkit/src/main/scala/akka/testkit/TestKit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index c3411fc3de..f4541f97f6 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -267,7 +267,7 @@ class TestKit(_app: AkkaApplication) { val _max = if (max eq Duration.MinusInf) remaining else max.dilated val o = receiveOne(_max) assert(o ne null, "timeout during expectMsg: " + hint) - assert(f.isDefinedAt(o), "does not match: " + o) + assert(f.isDefinedAt(o), "expected: " + hint + " but got unexpected message " + o) f(o) } From 1c3b9a389b444b8897150acf1d6b568d95d98227 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 18 Oct 2011 14:00:46 +0200 Subject: [PATCH 11/33] Adding clarification to DeathWatchSpec as well as making sure that systemServices aren't passed into the deployer --- .../scala/akka/actor/DeathWatchSpec.scala | 2 +- .../scala/akka/actor/ActorRefProvider.scala | 80 ++++++++++--------- .../akka/remote/RemoteActorRefProvider.scala | 2 +- 3 files changed, 44 insertions(+), 40 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 3db1131fec..2ecfffb12e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -12,7 +12,7 @@ import java.util.concurrent.atomic._ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender { "The Death Watch" must { - def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(2 seconds, "stopped") { + def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(2 seconds, actorRef + ": Stopped") { case Terminated(`actorRef`, ex: ActorKilledException) if ex.getMessage == "Stopped" ⇒ true } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index a8a2465458..b9116fbec6 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -95,53 +95,57 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { private[akka] def actorOf(props: Props, address: String, systemService: Boolean): ActorRef = { - val newFuture = Promise[ActorRef](5000)(app.dispatcher) // FIXME is this proper timeout? + if (systemService) new LocalActorRef(app, props, address, systemService = true) + else { + val newFuture = Promise[ActorRef](5000)(app.dispatcher) // FIXME is this proper timeout? - actors.putIfAbsent(address, newFuture) match { - case null ⇒ - val actor: ActorRef = try { - app.deployer.lookupDeploymentFor(address) match { // see if the deployment already exists, if so use it, if not create actor + actors.putIfAbsent(address, newFuture) match { + case null ⇒ + val actor: ActorRef = try { + app.deployer.lookupDeploymentFor(address) match { // see if the deployment already exists, if so use it, if not create actor - // create a local actor - case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, _, DeploymentConfig.LocalScope)) ⇒ - new LocalActorRef(app, props, address, systemService) // create a local actor + // create a local actor + case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, _, DeploymentConfig.LocalScope)) ⇒ + new LocalActorRef(app, props, address, systemService) // create a local actor - // create a routed actor ref - case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, _, DeploymentConfig.LocalScope)) ⇒ - val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match { - case RouterType.Direct ⇒ () ⇒ new DirectRouter - case RouterType.Random ⇒ () ⇒ new RandomRouter - case RouterType.RoundRobin ⇒ () ⇒ new RoundRobinRouter - case RouterType.ScatterGather ⇒ () ⇒ new ScatterGatherFirstCompletedRouter()( - if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher, app.AkkaConfig.ActorTimeout) - case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") - case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") - case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") - case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass) - } + // create a routed actor ref + case deploy @ Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, _, DeploymentConfig.LocalScope)) ⇒ + val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match { + case RouterType.Direct ⇒ () ⇒ new DirectRouter + case RouterType.Random ⇒ () ⇒ new RandomRouter + case RouterType.RoundRobin ⇒ () ⇒ new RoundRobinRouter + case RouterType.ScatterGather ⇒ () ⇒ new ScatterGatherFirstCompletedRouter()( + if (props.dispatcher == Props.defaultDispatcher) app.dispatcher else props.dispatcher, app.AkkaConfig.ActorTimeout) + case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") + case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") + case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") + case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass) + } - val connections: Iterable[ActorRef] = - if (nrOfInstances.factor > 0) Vector.fill(nrOfInstances.factor)(new LocalActorRef(app, props, "", systemService)) else Nil + val connections: Iterable[ActorRef] = + if (nrOfInstances.factor > 0) Vector.fill(nrOfInstances.factor)(new LocalActorRef(app, props, "", systemService)) else Nil - actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), address) + actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = new LocalConnectionManager(connections)), address) - case _ ⇒ throw new Exception("Don't know how to create this actor ref! Why?") + case _ ⇒ throw new Exception("Don't know how to create this actor ref! Why?") + } + } catch { + case e: Exception ⇒ + newFuture completeWithException e // so the other threads gets notified of error + //TODO FIXME should we remove the mapping in "actors" here? + throw e } - } catch { - case e: Exception ⇒ - newFuture completeWithException e // so the other threads gets notified of error - //TODO FIXME should we remove the mapping in "actors" here? - throw e - } - newFuture completeWithResult actor - actors.replace(address, newFuture, actor) - actor - case actor: ActorRef ⇒ - actor - case future: Future[_] ⇒ - future.get.asInstanceOf[ActorRef] + newFuture completeWithResult actor + actors.replace(address, newFuture, actor) + actor + case actor: ActorRef ⇒ + actor + case future: Future[_] ⇒ + future.get.asInstanceOf[ActorRef] + } } + } /** diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 0f4692a6af..af89a719e5 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -138,7 +138,7 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider * Copied from LocalActorRefProvider... */ def actorOf(props: RoutedProps, address: String): ActorRef = { - if (props.connectionManager.size == 0) throw new ConfigurationException("RoutedProps used for creating actor [" + address + "] has zero connections configured; can't create a router") + if (props.connectionManager.isEmpty) throw new ConfigurationException("RoutedProps used for creating actor [" + address + "] has zero connections configured; can't create a router") new RoutedActorRef(props, address) } From 65868d7c96f3bfabc64b44c7073e93fe13fb39a8 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 18 Oct 2011 14:21:48 +0200 Subject: [PATCH 12/33] Making sure that the RemoteActorRefProvider delegates systemServices down to the LocalActorRefProvider --- .../akka/remote/RemoteActorRefProvider.scala | 156 +++++++++--------- 1 file changed, 79 insertions(+), 77 deletions(-) diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index af89a719e5..2663abbf35 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -46,93 +46,95 @@ class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider def actorOf(props: Props, address: String): ActorRef = actorOf(props, address, false) - def actorOf(props: Props, address: String, systemService: Boolean): ActorRef = { - val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout? + def actorOf(props: Props, address: String, systemService: Boolean): ActorRef = + if (systemService) local.actorOf(props, address, systemService) + else { + val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout? - actors.putIfAbsent(address, newFuture) match { // we won the race -- create the actor and resolve the future - case null ⇒ - val actor: ActorRef = try { - app.deployer.lookupDeploymentFor(address) match { - case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, DeploymentConfig.RemoteScope(remoteAddresses))) ⇒ + actors.putIfAbsent(address, newFuture) match { // we won the race -- create the actor and resolve the future + case null ⇒ + val actor: ActorRef = try { + app.deployer.lookupDeploymentFor(address) match { + case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, DeploymentConfig.RemoteScope(remoteAddresses))) ⇒ - val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match { - case FailureDetectorType.NoOp ⇒ new NoOpFailureDetector - case FailureDetectorType.RemoveConnectionOnFirstFailure ⇒ new RemoveConnectionOnFirstFailureFailureDetector - case FailureDetectorType.BannagePeriod(timeToBan) ⇒ new BannagePeriodFailureDetector(timeToBan) - case FailureDetectorType.Custom(implClass) ⇒ FailureDetector.createCustomFailureDetector(implClass) - } - - val thisHostname = remote.address.getHostName - val thisPort = remote.address.getPort - - def isReplicaNode: Boolean = remoteAddresses exists { some ⇒ some.hostname == thisHostname && some.port == thisPort } - - if (isReplicaNode) { - // we are on one of the replica node for this remote actor - new LocalActorRef(app, props, address, false) - } else { - - // we are on the single "reference" node uses the remote actors on the replica nodes - val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match { - case RouterType.Direct ⇒ - if (remoteAddresses.size != 1) throw new ConfigurationException( - "Actor [%s] configured with Direct router must have exactly 1 remote node configured. Found [%s]" - .format(address, remoteAddresses.mkString(", "))) - () ⇒ new DirectRouter - - case RouterType.Random ⇒ - if (remoteAddresses.size < 1) throw new ConfigurationException( - "Actor [%s] configured with Random router must have at least 1 remote node configured. Found [%s]" - .format(address, remoteAddresses.mkString(", "))) - () ⇒ new RandomRouter - - case RouterType.RoundRobin ⇒ - if (remoteAddresses.size < 1) throw new ConfigurationException( - "Actor [%s] configured with RoundRobin router must have at least 1 remote node configured. Found [%s]" - .format(address, remoteAddresses.mkString(", "))) - () ⇒ new RoundRobinRouter - - case RouterType.ScatterGather ⇒ - if (remoteAddresses.size < 1) throw new ConfigurationException( - "Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]" - .format(address, remoteAddresses.mkString(", "))) - () ⇒ new ScatterGatherFirstCompletedRouter()(defaultDispatcher, defaultTimeout) - - case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") - case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") - case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") - case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass) + val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match { + case FailureDetectorType.NoOp ⇒ new NoOpFailureDetector + case FailureDetectorType.RemoveConnectionOnFirstFailure ⇒ new RemoveConnectionOnFirstFailureFailureDetector + case FailureDetectorType.BannagePeriod(timeToBan) ⇒ new BannagePeriodFailureDetector(timeToBan) + case FailureDetectorType.Custom(implClass) ⇒ FailureDetector.createCustomFailureDetector(implClass) } - val connections = (Map.empty[InetSocketAddress, ActorRef] /: remoteAddresses) { (conns, a) ⇒ - val inetAddr = new InetSocketAddress(a.hostname, a.port) - conns + (inetAddr -> RemoteActorRef(remote.server, inetAddr, address, None)) + val thisHostname = remote.address.getHostName + val thisPort = remote.address.getPort + + def isReplicaNode: Boolean = remoteAddresses exists { some ⇒ some.hostname == thisHostname && some.port == thisPort } + + if (isReplicaNode) { + // we are on one of the replica node for this remote actor + new LocalActorRef(app, props, address, false) + } else { + + // we are on the single "reference" node uses the remote actors on the replica nodes + val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match { + case RouterType.Direct ⇒ + if (remoteAddresses.size != 1) throw new ConfigurationException( + "Actor [%s] configured with Direct router must have exactly 1 remote node configured. Found [%s]" + .format(address, remoteAddresses.mkString(", "))) + () ⇒ new DirectRouter + + case RouterType.Random ⇒ + if (remoteAddresses.size < 1) throw new ConfigurationException( + "Actor [%s] configured with Random router must have at least 1 remote node configured. Found [%s]" + .format(address, remoteAddresses.mkString(", "))) + () ⇒ new RandomRouter + + case RouterType.RoundRobin ⇒ + if (remoteAddresses.size < 1) throw new ConfigurationException( + "Actor [%s] configured with RoundRobin router must have at least 1 remote node configured. Found [%s]" + .format(address, remoteAddresses.mkString(", "))) + () ⇒ new RoundRobinRouter + + case RouterType.ScatterGather ⇒ + if (remoteAddresses.size < 1) throw new ConfigurationException( + "Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]" + .format(address, remoteAddresses.mkString(", "))) + () ⇒ new ScatterGatherFirstCompletedRouter()(defaultDispatcher, defaultTimeout) + + case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") + case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") + case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") + case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass) + } + + val connections = (Map.empty[InetSocketAddress, ActorRef] /: remoteAddresses) { (conns, a) ⇒ + val inetAddr = new InetSocketAddress(a.hostname, a.port) + conns + (inetAddr -> RemoteActorRef(remote.server, inetAddr, address, None)) + } + + val connectionManager = new RemoteConnectionManager(app, remote, connections, failureDetector) + + connections.keys foreach { useActorOnNode(_, address, props.creator) } + + actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), address) } - val connectionManager = new RemoteConnectionManager(app, remote, connections, failureDetector) - - connections.keys foreach { useActorOnNode(_, address, props.creator) } - - actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), address) - } - - case deploy ⇒ local.actorOf(props, address, systemService) + case deploy ⇒ local.actorOf(props, address, systemService) + } + } catch { + case e: Exception ⇒ + newFuture completeWithException e // so the other threads gets notified of error + throw e } - } catch { - case e: Exception ⇒ - newFuture completeWithException e // so the other threads gets notified of error - throw e - } - // actor foreach app.registry.register // only for ActorRegistry backward compat, will be removed later + // actor foreach app.registry.register // only for ActorRegistry backward compat, will be removed later - newFuture completeWithResult actor - actors.replace(address, newFuture, actor) - actor - case actor: ActorRef ⇒ actor - case future: Future[_] ⇒ future.get.asInstanceOf[ActorRef] + newFuture completeWithResult actor + actors.replace(address, newFuture, actor) + actor + case actor: ActorRef ⇒ actor + case future: Future[_] ⇒ future.get.asInstanceOf[ActorRef] + } } - } /** * Copied from LocalActorRefProvider... From 304d39d839d15b3969456ea9df060b12f62efd29 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 18 Oct 2011 15:09:35 +0200 Subject: [PATCH 13/33] Removing uuid tracking in MessageDispatcher, isn't needed and will be reducing the overall memory footprint per actor --- .../akka/actor/dispatch/ActorModelSpec.scala | 6 --- .../actor/dispatch/DispatcherActorSpec.scala | 2 +- .../akka/dispatch/AbstractDispatcher.scala | 39 ++++++++++--------- 3 files changed, 22 insertions(+), 25 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index ca2bbed25f..5a0fd6ee0d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -120,17 +120,11 @@ object ActorModelSpec { protected[akka] abstract override def register(actor: ActorCell) { getStats(actor.ref).registers.incrementAndGet() super.register(actor) - //printMembers("after registering " + actor) } protected[akka] abstract override def unregister(actor: ActorCell) { getStats(actor.ref).unregisters.incrementAndGet() super.unregister(actor) - //printMembers("after unregistering " + actor) - } - - def printMembers(when: String) { - System.err.println(when + " then " + uuids.toArray.toList.map(_.toString.split("-")(0)).mkString("==> ", ", ", "<==")) } protected[akka] abstract override def dispatch(invocation: Envelope) { diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala index 9877d4eac6..1aae1091f6 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala @@ -81,7 +81,7 @@ class DispatcherActorSpec extends AkkaSpec { (1 to 100) foreach { _ ⇒ slowOne ! "ping" } fastOne ! "sabotage" start.countDown() - val result = latch.await(5, TimeUnit.SECONDS) + val result = latch.await(10, TimeUnit.SECONDS) fastOne.stop() slowOne.stop() assert(result === true) diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index c7e7fab35a..4f6cdb73ec 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -70,8 +70,8 @@ object MessageDispatcher { abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable { import MessageDispatcher._ - protected val uuids = new ConcurrentSkipListSet[Uuid] protected val _tasks = new AtomicLong(0L) + protected val _actors = new AtomicLong(0L) protected val guard = new ReentrantGuard protected val active = new Switch(false) @@ -123,7 +123,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable final def detach(actor: ActorCell) { guard withGuard { unregister(actor) - if (uuids.isEmpty && _tasks.get == 0) { + if (_tasks.get == 0 && _actors.get == 0) { shutdownSchedule match { case UNSCHEDULED ⇒ shutdownSchedule = SCHEDULED @@ -155,7 +155,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable private val taskCleanup: () ⇒ Unit = () ⇒ if (_tasks.decrementAndGet() == 0) { guard withGuard { - if (_tasks.get == 0 && uuids.isEmpty) { + if (_tasks.get == 0 && _actors.get == 0) { shutdownSchedule match { case UNSCHEDULED ⇒ shutdownSchedule = SCHEDULED @@ -173,9 +173,8 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable * and only call it under the dispatcher-guard, see "attach" for the only invocation */ protected[akka] def register(actor: ActorCell) { - if (uuids add actor.uuid) { - systemDispatch(SystemEnvelope(actor, Create, NullChannel)) //FIXME should this be here or moved into ActorCell.start perhaps? - } else System.err.println("Couldn't register: " + actor) + _actors.incrementAndGet() + systemDispatch(SystemEnvelope(actor, Create, NullChannel)) //FIXME should this be here or moved into ActorCell.start perhaps? } /** @@ -183,12 +182,11 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable * and only call it under the dispatcher-guard, see "detach" for the only invocation */ protected[akka] def unregister(actor: ActorCell) { - if (uuids remove actor.uuid) { - val mailBox = actor.mailbox - mailBox.becomeClosed() - actor.mailbox = deadLetterMailbox //FIXME getAndSet would be preferrable here - cleanUpMailboxFor(actor, mailBox) - } else System.err.println("Couldn't unregister: " + actor) + _actors.decrementAndGet() + val mailBox = actor.mailbox + mailBox.becomeClosed() + actor.mailbox = deadLetterMailbox //FIXME getAndSet would be preferrable here + cleanUpMailboxFor(actor, mailBox) } /** @@ -222,7 +220,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable shutdownSchedule = SCHEDULED app.scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS) case SCHEDULED ⇒ - if (uuids.isEmpty && _tasks.get == 0) { + if (_tasks.get == 0) { active switchOff { shutdown() // shut down in the dispatcher's references is zero } @@ -243,16 +241,21 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable /** * After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference */ - def suspend(actor: ActorCell): Unit = - if (uuids.contains(actor.uuid)) actor.mailbox.becomeSuspended() + def suspend(actor: ActorCell): Unit = { + val mbox = actor.mailbox + if (mbox.dispatcher eq this) + mbox.becomeSuspended() + } /* * After the call to this method, the dispatcher must begin any new message processing for the specified reference */ - def resume(actor: ActorCell): Unit = if (uuids.contains(actor.uuid)) { + def resume(actor: ActorCell): Unit = { val mbox = actor.mailbox - mbox.becomeOpen() - registerForExecution(mbox, false, false) + if (mbox.dispatcher eq this) { + mbox.becomeOpen() + registerForExecution(mbox, false, false) + } } /** From 183dfb4d7fbe1ff1c35daf540b28e0fd1f8ffc0a Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 18 Oct 2011 16:44:35 +0200 Subject: [PATCH 14/33] remove SystemEnvelope - channel field was always set to NullChannel and not used - receiver field is better put into the Mailbox, because there it takes space only once per actor - leaves only the bare SystemMessage to be queued --- .../akka/dispatch/MailboxConfigSpec.scala | 8 +-- .../src/main/scala/akka/actor/ActorCell.scala | 38 ++++++------ .../akka/dispatch/AbstractDispatcher.scala | 38 +++++------- .../akka/dispatch/BalancingDispatcher.scala | 2 +- .../main/scala/akka/dispatch/Dispatcher.scala | 6 +- .../main/scala/akka/dispatch/Mailbox.scala | 59 ++++++++++--------- .../testkit/CallingThreadDispatcher.scala | 10 ++-- 7 files changed, 78 insertions(+), 83 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala index 7e8d6ae929..5181776db1 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala @@ -146,8 +146,8 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn class DefaultMailboxSpec extends MailboxSpec { lazy val name = "The default mailbox implementation" def factory = { - case u: UnboundedMailbox ⇒ u.create(null) - case b: BoundedMailbox ⇒ b.create(null) + case u: UnboundedMailbox ⇒ u.create(null, null) + case b: BoundedMailbox ⇒ b.create(null, null) } } @@ -155,7 +155,7 @@ class PriorityMailboxSpec extends MailboxSpec { val comparator = PriorityGenerator(_.##) lazy val name = "The priority mailbox implementation" def factory = { - case UnboundedMailbox() ⇒ UnboundedPriorityMailbox(comparator).create(null) - case BoundedMailbox(capacity, pushTimeOut) ⇒ BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(null) + case UnboundedMailbox() ⇒ UnboundedPriorityMailbox(comparator).create(null, null) + case BoundedMailbox(capacity, pushTimeOut) ⇒ BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(null, null) } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 23014cd884..f010e06045 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -245,7 +245,7 @@ private[akka] class ActorCell( if (props.supervisor.isDefined) { props.supervisor.get match { case l: LocalActorRef ⇒ - l.underlying.dispatcher.systemDispatch(SystemEnvelope(l.underlying, akka.dispatch.Supervise(self), NullChannel)) + l.underlying.dispatcher.systemDispatch(l.underlying, akka.dispatch.Supervise(self)) case other ⇒ throw new UnsupportedOperationException("Supervision failure: " + other + " cannot be a supervisor, only LocalActorRefs can") } } @@ -253,20 +253,20 @@ private[akka] class ActorCell( dispatcher.attach(this) } - def suspend(): Unit = dispatcher.systemDispatch(SystemEnvelope(this, Suspend, NullChannel)) + def suspend(): Unit = dispatcher.systemDispatch(this, Suspend()) - def resume(): Unit = dispatcher.systemDispatch(SystemEnvelope(this, Resume, NullChannel)) + def resume(): Unit = dispatcher.systemDispatch(this, Resume()) private[akka] def stop(): Unit = - dispatcher.systemDispatch(SystemEnvelope(this, Terminate, NullChannel)) + dispatcher.systemDispatch(this, Terminate()) def startsMonitoring(subject: ActorRef): ActorRef = { - dispatcher.systemDispatch(SystemEnvelope(this, Link(subject), NullChannel)) + dispatcher.systemDispatch(this, Link(subject)) subject } def stopsMonitoring(subject: ActorRef): ActorRef = { - dispatcher.systemDispatch(SystemEnvelope(this, Unlink(subject), NullChannel)) + dispatcher.systemDispatch(this, Unlink(subject)) subject } @@ -324,7 +324,7 @@ private[akka] class ActorCell( } } - def systemInvoke(envelope: SystemEnvelope) { + def systemInvoke(message: SystemMessage) { def create(): Unit = try { val created = newActor() @@ -337,7 +337,6 @@ private[akka] class ActorCell( app.eventHandler.error(e, self, "error while creating actor") // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) - envelope.channel.sendException(e) } finally { if (supervisor.isDefined) supervisor.get ! Failed(self, e) else self.stop() } @@ -369,7 +368,6 @@ private[akka] class ActorCell( app.eventHandler.error(e, self, "error while creating actor") // prevent any further messages to be processed until the actor has been restarted dispatcher.suspend(this) - envelope.channel.sendException(e) } finally { if (supervisor.isDefined) supervisor.get ! Failed(self, e) else self.stop() } @@ -424,24 +422,24 @@ private[akka] class ActorCell( try { val isClosed = mailbox.isClosed //Fence plus volatile read if (!isClosed) { - envelope.message match { - case Create ⇒ create() - case Recreate(cause) ⇒ recreate(cause) - case Link(subject) ⇒ + message match { + case Create(_) ⇒ create() + case Recreate(cause, _) ⇒ recreate(cause) + case Link(subject, _) ⇒ app.deathWatch.subscribe(self, subject) if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "now monitoring " + subject) - case Unlink(subject) ⇒ + case Unlink(subject, _) ⇒ app.deathWatch.unsubscribe(self, subject) if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopped monitoring " + subject) - case Suspend ⇒ suspend() - case Resume ⇒ resume() - case Terminate ⇒ terminate() - case Supervise(child) ⇒ supervise(child) + case Suspend(_) ⇒ suspend() + case Resume(_) ⇒ resume() + case Terminate(_) ⇒ terminate() + case Supervise(child, _) ⇒ supervise(child) } } } catch { case e ⇒ //Should we really catch everything here? - app.eventHandler.error(e, self, "error while processing " + envelope.message) + app.eventHandler.error(e, self, "error while processing " + message) //TODO FIXME How should problems here be handled? throw e } @@ -495,7 +493,7 @@ private[akka] class ActorCell( def handleChildTerminated(child: ActorRef): Unit = _children = props.faultHandler.handleChildTerminated(child, _children) - def restart(cause: Throwable): Unit = dispatcher.systemDispatch(SystemEnvelope(this, Recreate(cause), NullChannel)) + def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause)) def checkReceiveTimeout() { cancelReceiveTimeout() diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index c7e7fab35a..279543bb23 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -24,25 +24,17 @@ final case class Envelope(val receiver: ActorCell, val message: Any, val channel } } -sealed trait SystemMessage extends PossiblyHarmful -case object Create extends SystemMessage -case class Recreate(cause: Throwable) extends SystemMessage -case object Suspend extends SystemMessage -case object Resume extends SystemMessage -case object Terminate extends SystemMessage -case class Supervise(child: ActorRef) extends SystemMessage -case class Link(subject: ActorRef) extends SystemMessage -case class Unlink(subject: ActorRef) extends SystemMessage - -final case class SystemEnvelope(val receiver: ActorCell, val message: SystemMessage, val channel: UntypedChannel) { - if (receiver eq null) throw new IllegalArgumentException("Receiver can't be null") - /** - * @return whether to proceed with processing other messages - */ - final def invoke() { - receiver systemInvoke this - } +sealed trait SystemMessage extends PossiblyHarmful { + def next: SystemMessage } +case class Create(next: SystemMessage = null) extends SystemMessage +case class Recreate(cause: Throwable, next: SystemMessage = null) extends SystemMessage +case class Suspend(next: SystemMessage = null) extends SystemMessage +case class Resume(next: SystemMessage = null) extends SystemMessage +case class Terminate(next: SystemMessage = null) extends SystemMessage +case class Supervise(child: ActorRef, next: SystemMessage = null) extends SystemMessage +case class Link(subject: ActorRef, next: SystemMessage = null) extends SystemMessage +case class Unlink(subject: ActorRef, next: SystemMessage = null) extends SystemMessage final case class TaskInvocation(app: AkkaApplication, function: () ⇒ Unit, cleanup: () ⇒ Unit) extends Runnable { def run() { @@ -87,13 +79,13 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable */ protected[akka] val deadLetterMailbox: Mailbox = DeadLetterMailbox - object DeadLetterMailbox extends Mailbox { + object DeadLetterMailbox extends Mailbox(null) { becomeClosed() override def dispatcher = null //MessageDispatcher.this override def enqueue(envelope: Envelope) { envelope.channel sendException new ActorKilledException("Actor has been stopped") } override def dequeue() = null - override def systemEnqueue(handle: SystemEnvelope): Unit = () - override def systemDequeue(): SystemEnvelope = null + override def systemEnqueue(handle: SystemMessage): Unit = () + override def systemDequeue(): SystemMessage = null override def hasMessages = false override def hasSystemMessages = false override def numberOfMessages = 0 @@ -174,7 +166,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable */ protected[akka] def register(actor: ActorCell) { if (uuids add actor.uuid) { - systemDispatch(SystemEnvelope(actor, Create, NullChannel)) //FIXME should this be here or moved into ActorCell.start perhaps? + systemDispatch(actor, Create()) //FIXME should this be here or moved into ActorCell.start perhaps? } else System.err.println("Couldn't register: " + actor) } @@ -258,7 +250,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable /** * Will be called when the dispatcher is to queue an invocation for execution */ - protected[akka] def systemDispatch(invocation: SystemEnvelope) + protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage) /** * Will be called when the dispatcher is to queue an invocation for execution diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index e5c5cf198b..551eea21ce 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -54,7 +54,7 @@ class BalancingDispatcher( protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor) - class SharingMailbox(val actor: ActorCell) extends Mailbox with DefaultSystemMessageQueue { + class SharingMailbox(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue { final def enqueue(handle: Envelope) = messageQueue.enqueue(handle) final def dequeue(): Envelope = { diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index fd3dd697da..53d28a79a3 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -84,8 +84,8 @@ class Dispatcher( registerForExecution(mbox, true, false) } - protected[akka] def systemDispatch(invocation: SystemEnvelope) = { - val mbox = invocation.receiver.mailbox + protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage) = { + val mbox = receiver.mailbox mbox systemEnqueue invocation registerForExecution(mbox, false, true) } @@ -100,7 +100,7 @@ class Dispatcher( } } - protected[akka] def createMailbox(actor: ActorCell): Mailbox = mailboxType.create(this) + protected[akka] def createMailbox(actor: ActorCell): Mailbox = mailboxType.create(this, actor) protected[akka] def start {} diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 5e21d5ea87..095a0f9cd5 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -8,7 +8,7 @@ import akka.AkkaException import java.util.{ Comparator, PriorityQueue } import akka.util._ import java.util.Queue -import akka.actor.ActorContext +import akka.actor.{ ActorContext, ActorCell } import java.util.concurrent._ import atomic.{ AtomicInteger, AtomicReferenceFieldUpdater } import annotation.tailrec @@ -34,7 +34,7 @@ private[dispatch] object Mailbox { /** * @author Jonas Bonér */ -abstract class Mailbox extends AbstractMailbox with MessageQueue with SystemMessageQueue with Runnable { +abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with MessageQueue with SystemMessageQueue with Runnable { import Mailbox._ @inline @@ -186,7 +186,7 @@ abstract class Mailbox extends AbstractMailbox with MessageQueue with SystemMess def processAllSystemMessages() { var nextMessage = systemDequeue() while (nextMessage ne null) { - nextMessage.invoke() + actor systemInvoke nextMessage nextMessage = systemDequeue() } } @@ -208,19 +208,20 @@ trait MessageQueue { } trait SystemMessageQueue { - def systemEnqueue(handle: SystemEnvelope): Unit + def systemEnqueue(message: SystemMessage): Unit - def systemDequeue(): SystemEnvelope + def systemDequeue(): SystemMessage def hasSystemMessages: Boolean } trait DefaultSystemMessageQueue { self: SystemMessageQueue ⇒ - final val systemMessages = new ConcurrentLinkedQueue[SystemEnvelope]() - def systemEnqueue(handle: SystemEnvelope): Unit = systemMessages offer handle + final val systemMessages = new ConcurrentLinkedQueue[SystemMessage]() - def systemDequeue(): SystemEnvelope = systemMessages.poll() + def systemEnqueue(message: SystemMessage): Unit = systemMessages offer message + + def systemDequeue(): SystemMessage = systemMessages.poll() def hasSystemMessages: Boolean = !systemMessages.isEmpty } @@ -255,17 +256,18 @@ trait QueueBasedMessageQueue extends MessageQueue { * Mailbox configuration. */ trait MailboxType { - def create(dispatcher: MessageDispatcher): Mailbox + def create(dispatcher: MessageDispatcher, receiver: ActorCell): Mailbox } /** * It's a case class for Java (new UnboundedMailbox) */ case class UnboundedMailbox() extends MailboxType { - override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { - final val queue = new ConcurrentLinkedQueue[Envelope]() - final val dispatcher = _dispatcher - } + override def create(_dispatcher: MessageDispatcher, receiver: ActorCell) = + new Mailbox(receiver) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { + final val queue = new ConcurrentLinkedQueue[Envelope]() + final val dispatcher = _dispatcher + } } case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType { @@ -273,18 +275,20 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative") if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null") - override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue { - final val queue = new LinkedBlockingQueue[Envelope](capacity) - final val pushTimeOut = BoundedMailbox.this.pushTimeOut - final val dispatcher = _dispatcher - } + override def create(_dispatcher: MessageDispatcher, receiver: ActorCell) = + new Mailbox(receiver) with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue { + final val queue = new LinkedBlockingQueue[Envelope](capacity) + final val pushTimeOut = BoundedMailbox.this.pushTimeOut + final val dispatcher = _dispatcher + } } case class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType { - override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { - final val queue = new PriorityBlockingQueue[Envelope](11, cmp) - final val dispatcher = _dispatcher - } + override def create(_dispatcher: MessageDispatcher, receiver: ActorCell) = + new Mailbox(receiver) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { + final val queue = new PriorityBlockingQueue[Envelope](11, cmp) + final val dispatcher = _dispatcher + } } case class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType { @@ -292,10 +296,11 @@ case class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final va if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative") if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null") - override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue { - final val queue = new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) - final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut - final val dispatcher = _dispatcher - } + override def create(_dispatcher: MessageDispatcher, receiver: ActorCell) = + new Mailbox(receiver) with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue { + final val queue = new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) + final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut + final val dispatcher = _dispatcher + } } diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 9cf9992809..829cce72c3 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -106,7 +106,7 @@ object CallingThreadDispatcher { class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling-thread", val warnings: Boolean = true) extends MessageDispatcher(_app) { import CallingThreadDispatcher._ - protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(this) + protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(this, actor) private def getMailbox(actor: ActorCell) = actor.mailbox.asInstanceOf[CallingThreadMailbox] @@ -140,11 +140,11 @@ class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling override def mailboxIsEmpty(actor: ActorCell): Boolean = getMailbox(actor).queue.isEmpty - protected[akka] override def systemDispatch(handle: SystemEnvelope) { - val mbox = getMailbox(handle.receiver) + protected[akka] override def systemDispatch(receiver: ActorCell, message: SystemMessage) { + val mbox = getMailbox(receiver) mbox.lock.lock try { - handle.invoke() + receiver systemInvoke message } finally { mbox.lock.unlock } @@ -241,7 +241,7 @@ class NestingQueue { def isActive = active } -class CallingThreadMailbox(val dispatcher: MessageDispatcher) extends Mailbox with DefaultSystemMessageQueue { +class CallingThreadMailbox(val dispatcher: MessageDispatcher, _receiver: ActorCell) extends Mailbox(_receiver) with DefaultSystemMessageQueue { private val q = new ThreadLocal[NestingQueue]() { override def initialValue = new NestingQueue From 6150beb3334b99654d4c5d5faaf6ec105bc681a9 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 18 Oct 2011 16:59:57 +0200 Subject: [PATCH 15/33] Pushing the memory per actor down to 464 bytes. Returning None for the Deploy if there is no config --- .../src/main/scala/akka/actor/Actor.scala | 6 ++-- .../src/main/scala/akka/actor/ActorRef.scala | 11 +++---- .../src/main/scala/akka/actor/Deployer.scala | 30 ++++++------------- .../scala/akka/actor/DeploymentConfig.scala | 13 +++++--- .../src/main/scala/akka/routing/Routing.scala | 1 + .../akka/remote/RemoteActorRefProvider.scala | 2 ++ 6 files changed, 28 insertions(+), 35 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index a3290b67b7..c5834ac633 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -217,7 +217,7 @@ trait Actor { /** * The default timeout, based on the config setting 'akka.actor.timeout' */ - implicit val defaultTimeout = config.ActorTimeout + implicit def defaultTimeout = config.ActorTimeout /** * Wrap a Receive partial function in a logging enclosure, which sends a @@ -419,7 +419,7 @@ trait Actor { private[akka] final def apply(msg: Any) = { if (msg.isInstanceOf[AnyRef] && (msg.asInstanceOf[AnyRef] eq null)) - throw new InvalidMessageException("Message from [" + channel + "] to [" + self.toString + "] is null") + throw new InvalidMessageException("Message from [" + channel + "] to [" + self + "] is null") def autoReceiveMessage(msg: AutoReceivedMessage) { if (config.DebugAutoReceive) app.eventHandler.debug(this, "received AutoReceiveMessage " + msg) @@ -449,7 +449,7 @@ trait Actor { } } - private lazy val processingBehavior = receive //ProcessingBehavior is the original behavior + private val processingBehavior = receive //ProcessingBehavior is the original behavior } /** diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index c511847261..05091d414b 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -6,14 +6,9 @@ package akka.actor import akka.dispatch._ import akka.util._ -import akka.serialization.{ Serializer, Serialization } -import java.net.InetSocketAddress import scala.collection.immutable.Stack import java.lang.{ UnsupportedOperationException, IllegalStateException } import akka.AkkaApplication -import akka.remote.RemoteSupport -import scala.util.DynamicVariable -import akka.event.{ EventHandler } /** * ActorRef is an immutable and serializable handle to an Actor. @@ -49,7 +44,7 @@ abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyCha scalaRef: ScalaActorRef ⇒ // Only mutable for RemoteServer in order to maintain identity across nodes - private[akka] val uuid = newUuid + private[akka] def uuid: Uuid def address: String @@ -155,7 +150,7 @@ class LocalActorRef private[akka] ( props: Props, givenAddress: String, val systemService: Boolean = false, - override private[akka] val uuid: Uuid = newUuid, + private[akka] val uuid: Uuid = newUuid, receiveTimeout: Option[Long] = None, hotswap: Stack[PartialFunction[Any, Unit]] = Props.noHotSwap) extends ActorRef with ScalaActorRef { @@ -351,6 +346,8 @@ class DeadLetterActorRef(app: AkkaApplication) extends UnsupportedActorRef { val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher) val address: String = "akka:internal:DeadLetterActorRef" + private[akka] val uuid: akka.actor.Uuid = new com.eaio.uuid.UUID(0L, 0L) //Nil UUID + override def startsMonitoring(actorRef: ActorRef): ActorRef = actorRef override def stopsMonitoring(actorRef: ActorRef): ActorRef = actorRef diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 1a7b814308..7533c55579 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -34,11 +34,7 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer { // val defaultAddress = Node(Config.nodename) lazy val instance: ActorDeployer = { - val deployer = if (app.reflective.ClusterModule.isEnabled) { - app.reflective.ClusterModule.clusterDeployer - } else { - LocalDeployer - } + val deployer = if (app.reflective.ClusterModule.isEnabled) app.reflective.ClusterModule.clusterDeployer else LocalDeployer deployer.init(deploymentsInConfig) deployer } @@ -47,7 +43,7 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer { private[akka] def init(deployments: Seq[Deploy]) = instance.init(deployments) - def shutdown(): Unit = instance.shutdown() //TODO Why should we have "shutdown", should be crash only? + def shutdown(): Unit = instance.shutdown() //TODO FIXME Why should we have "shutdown", should be crash only? def deploy(deployment: Deploy): Unit = instance.deploy(deployment) @@ -81,20 +77,14 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer { lookupInConfig(address) } catch { case e: ConfigurationException ⇒ - app.eventHandler.error(e, this, e.getMessage) + app.eventHandler.error(e, this, e.getMessage) //TODO FIXME I do not condone log AND rethrow throw e } - newDeployment foreach { d ⇒ - if (d eq null) { - val e = new IllegalStateException("Deployment for address [" + address + "] is null") - app.eventHandler.error(e, this, e.getMessage) - throw e - } - deploy(d) // deploy and cache it + newDeployment match { + case None | Some(null) ⇒ None + case Some(d) ⇒ deploy(d); newDeployment // deploy and cache it } - - newDeployment } } @@ -127,9 +117,7 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer { // -------------------------------- val addressPath = "akka.actor.deployment." + address configuration.getSection(addressPath) match { - case None ⇒ - Some(Deploy(address, None, Direct, NrOfInstances(1), NoOpFailureDetector, LocalScope)) - + case None ⇒ None case Some(addressConfig) ⇒ // -------------------------------- @@ -150,11 +138,11 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer { // akka.actor.deployment.
.nr-of-instances // -------------------------------- val nrOfInstances = { - if (router == Direct) NrOfInstances(1) + if (router == Direct) OneNrOfInstances else { addressConfig.getAny("nr-of-instances", "1") match { case "auto" ⇒ AutoNrOfInstances - case "1" ⇒ NrOfInstances(1) + case "1" ⇒ OneNrOfInstances case "0" ⇒ ZeroNrOfInstances case nrOfReplicas: String ⇒ try { diff --git a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala index 2e10680e0a..5e7bd9a2f7 100644 --- a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala +++ b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala @@ -103,7 +103,12 @@ object DeploymentConfig { } object NrOfInstances { - def apply(factor: Int): NrOfInstances = new NrOfInstances(factor) + def apply(factor: Int): NrOfInstances = factor match { + case -1 ⇒ AutoNrOfInstances + case 0 ⇒ ZeroNrOfInstances + case 1 ⇒ OneNrOfInstances + case x ⇒ new NrOfInstances(x) + } def unapply(other: Any) = other match { case x: NrOfInstances ⇒ import x._; Some(factor) case _ ⇒ None @@ -113,10 +118,12 @@ object DeploymentConfig { // For Java API class AutoNrOfInstances extends NrOfInstances(-1) class ZeroNrOfInstances extends NrOfInstances(0) + class OneNrOfInstances extends NrOfInstances(0) // For Scala API case object AutoNrOfInstances extends AutoNrOfInstances case object ZeroNrOfInstances extends ZeroNrOfInstances + case object OneNrOfInstances extends OneNrOfInstances // -------------------------------- // --- Replication @@ -252,9 +259,7 @@ class DeploymentConfig(val app: AkkaApplication) { import DeploymentConfig._ - case class ClusterScope( - preferredNodes: Iterable[Home] = Vector(Node(app.nodename)), - replication: ReplicationScheme = Transient) extends Scope + case class ClusterScope(preferredNodes: Iterable[Home] = Vector(Node(app.nodename)), replication: ReplicationScheme = Transient) extends Scope def isHomeNode(homes: Iterable[Home]): Boolean = homes exists (home ⇒ nodeNameFor(home) == app.nodename) diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index d9c149cfff..d499b80efb 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -94,6 +94,7 @@ object Routing { * An Abstract convenience implementation for building an ActorReference that uses a Router. */ abstract private[akka] class AbstractRoutedActorRef(val props: RoutedProps) extends UnsupportedActorRef { + private[akka] val uuid: Uuid = newUuid val router = props.routerFactory() diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 2663abbf35..225612812f 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -237,6 +237,8 @@ private[akka] case class RemoteActorRef private[akka] ( loader: Option[ClassLoader]) extends ActorRef with ScalaActorRef { + private[akka] val uuid: Uuid = newUuid + @volatile private var running: Boolean = true From 474787a81d27490ca8fd38f797b32b3778598730 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 18 Oct 2011 17:56:23 +0200 Subject: [PATCH 16/33] Renaming createActor to actorOf --- .../src/test/java/akka/actor/JavaAPI.java | 6 +- .../ActorFireForgetRequestReplySpec.scala | 12 +-- .../scala/akka/actor/ActorLifeCycleSpec.scala | 12 +-- .../test/scala/akka/actor/ActorRefSpec.scala | 56 +++++++------- .../scala/akka/actor/ActorTimeoutSpec.scala | 2 +- .../src/test/scala/akka/actor/Bench.scala | 4 +- .../scala/akka/actor/DeathWatchSpec.scala | 14 ++-- .../test/scala/akka/actor/FSMActorSpec.scala | 12 +-- .../test/scala/akka/actor/FSMTimingSpec.scala | 2 +- .../scala/akka/actor/FSMTransitionSpec.scala | 8 +- .../scala/akka/actor/ForwardActorSpec.scala | 6 +- .../test/scala/akka/actor/HotSwapSpec.scala | 8 +- .../src/test/scala/akka/actor/IOActor.scala | 32 ++++---- .../scala/akka/actor/ReceiveTimeoutSpec.scala | 10 +-- .../akka/actor/RestartStrategySpec.scala | 20 ++--- .../test/scala/akka/actor/SchedulerSpec.scala | 12 +-- .../akka/actor/SupervisorHierarchySpec.scala | 10 +-- .../scala/akka/actor/SupervisorMiscSpec.scala | 10 +-- .../scala/akka/actor/SupervisorSpec.scala | 46 ++++++------ .../scala/akka/actor/SupervisorTreeSpec.scala | 6 +- .../test/scala/akka/actor/Ticket669Spec.scala | 8 +- .../akka/actor/dispatch/ActorModelSpec.scala | 2 +- .../dispatch/BalancingDispatcherSpec.scala | 4 +- .../actor/dispatch/DispatcherActorSpec.scala | 14 ++-- .../actor/dispatch/DispatcherActorsSpec.scala | 4 +- .../akka/actor/dispatch/PinnedActorSpec.scala | 6 +- .../akka/actor/routing/ListenerSpec.scala | 4 +- .../test/scala/akka/dispatch/FutureSpec.scala | 42 +++++------ .../akka/dispatch/MailboxConfigSpec.scala | 2 +- .../dispatch/PriorityDispatcherSpec.scala | 2 +- .../test/scala/akka/event/EventBusSpec.scala | 2 +- .../trading/common/AkkaPerformanceTest.scala | 2 +- .../trading/common/TradingSystem.scala | 8 +- .../oneway/OneWayPerformanceTest.scala | 4 +- .../trading/oneway/OneWayTradingSystem.scala | 8 +- .../scala/akka/routing/ActorPoolSpec.scala | 46 ++++++------ .../routing/ConfiguredLocalRoutingSpec.scala | 12 +-- .../test/scala/akka/routing/RoutingSpec.scala | 74 +++++++++---------- .../scala/akka/ticket/Ticket703Spec.scala | 4 +- .../scala/akka/actor/ActorRefProvider.scala | 20 ++--- .../main/scala/akka/actor/TypedActor.scala | 2 +- .../src/main/scala/akka/cluster/Cluster.scala | 4 +- akka-docs/intro/code/tutorials/first/Pi.scala | 6 +- akka-docs/scala/code/ActorDocSpec.scala | 6 +- .../main/scala/akka/remote/RemoteDaemon.scala | 6 +- .../remote/netty/NettyRemoteSupport.scala | 4 +- .../DirectRoutedRemoteActorMultiJvmSpec.scala | 2 +- .../NewRemoteActorMultiJvmSpec.scala | 2 +- .../RandomRoutedRemoteActorMultiJvmSpec.scala | 2 +- ...ndRobinRoutedRemoteActorMultiJvmSpec.scala | 2 +- .../main/scala/DiningHakkersOnBecome.scala | 4 +- .../src/main/scala/DiningHakkersOnFsm.scala | 4 +- .../src/main/scala/akka/agent/Agent.scala | 6 +- .../akka/stm/example/EitherOrElseExample.java | 2 +- .../java/akka/stm/example/RetryExample.java | 2 +- .../example/UntypedCoordinatedExample.java | 4 +- .../example/UntypedTransactorExample.java | 4 +- .../test/UntypedCoordinatedIncrementTest.java | 4 +- .../test/UntypedTransactorTest.java | 4 +- .../transactor/CoordinatedIncrementSpec.scala | 10 +-- .../scala/transactor/FickleFriendsSpec.scala | 8 +- .../scala/transactor/TransactorSpec.scala | 6 +- .../test/scala/akka/testkit/AkkaSpec.scala | 10 +-- .../scala/akka/testkit/TestActorRefSpec.scala | 2 +- .../java/akka/tutorial/first/java/Pi.java | 6 +- .../src/main/scala/Pi.scala | 6 +- .../java/akka/tutorial/java/second/Pi.java | 6 +- .../src/main/scala/Pi.scala | 6 +- 68 files changed, 343 insertions(+), 343 deletions(-) diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java b/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java index 72105b455b..d0388cb9c0 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaAPI.java @@ -12,12 +12,12 @@ public class JavaAPI { private AkkaApplication app = new AkkaApplication(); @Test void mustBeAbleToCreateActorRefFromClass() { - ActorRef ref = app.createActor(JavaAPITestActor.class); + ActorRef ref = app.actorOf(JavaAPITestActor.class); assertNotNull(ref); } @Test void mustBeAbleToCreateActorRefFromFactory() { - ActorRef ref = app.createActor(new Props().withCreator(new Creator() { + ActorRef ref = app.actorOf(new Props().withCreator(new Creator() { public Actor create() { return new JavaAPITestActor(); } @@ -26,7 +26,7 @@ public class JavaAPI { } @Test void mustAcceptSingleArgTryTell() { - ActorRef ref = app.createActor(JavaAPITestActor.class); + ActorRef ref = app.actorOf(JavaAPITestActor.class); ref.tryTell("hallo"); ref.tryTell("hallo", ref); } diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala index 1bce22b52a..461868df21 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala @@ -65,16 +65,16 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach { "An Actor" must { "reply to bang message using reply" in { - val replyActor = createActor[ReplyActor] - val senderActor = createActor(new SenderActor(replyActor)) + val replyActor = actorOf[ReplyActor] + val senderActor = actorOf(new SenderActor(replyActor)) senderActor ! "Init" state.finished.await state.s must be("Reply") } "reply to bang message using implicit sender" in { - val replyActor = createActor[ReplyActor] - val senderActor = createActor(new SenderActor(replyActor)) + val replyActor = actorOf[ReplyActor] + val senderActor = actorOf(new SenderActor(replyActor)) senderActor ! "InitImplicit" state.finished.await state.s must be("ReplyImplicit") @@ -82,8 +82,8 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach { "should shutdown crashed temporary actor" in { filterEvents(EventFilter[Exception]("Expected")) { - val supervisor = createActor(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) - val actor = createActor(Props[CrashingActor].withSupervisor(supervisor)) + val supervisor = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) + val actor = actorOf(Props[CrashingActor].withSupervisor(supervisor)) actor.isShutdown must be(false) actor ! "Die" state.finished.await diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala index 319f9a14d2..52e1bd6d58 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala @@ -33,9 +33,9 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "invoke preRestart, preStart, postRestart when using OneForOneStrategy" in { filterException[ActorKilledException] { val id = newUuid().toString - val supervisor = createActor(Props(self ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) + val supervisor = actorOf(Props(self ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) val gen = new AtomicInteger(0) - val restarter = createActor(Props(new LifeCycleTestActor(id, gen) { + val restarter = actorOf(Props(new LifeCycleTestActor(id, gen) { override def preRestart(reason: Throwable, message: Option[Any]) { report("preRestart") } override def postRestart(reason: Throwable) { report("postRestart") } }).withSupervisor(supervisor)) @@ -66,9 +66,9 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "default for preRestart and postRestart is to call postStop and preStart respectively" in { filterException[ActorKilledException] { val id = newUuid().toString - val supervisor = createActor(Props(self ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) + val supervisor = actorOf(Props(self ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) val gen = new AtomicInteger(0) - val restarter = createActor(Props(new LifeCycleTestActor(id, gen)).withSupervisor(supervisor)) + val restarter = actorOf(Props(new LifeCycleTestActor(id, gen)).withSupervisor(supervisor)) expectMsg(("preStart", id, 0)) restarter ! Kill @@ -95,9 +95,9 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS "not invoke preRestart and postRestart when never restarted using OneForOneStrategy" in { val id = newUuid().toString - val supervisor = createActor(Props(self ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) + val supervisor = actorOf(Props(self ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) val gen = new AtomicInteger(0) - val a = createActor(Props(new LifeCycleTestActor(id, gen)).withSupervisor(supervisor)) + val a = actorOf(Props(new LifeCycleTestActor(id, gen)).withSupervisor(supervisor)) expectMsg(("preStart", id, 0)) a ! "status" expectMsg(("OK", id, 0)) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index 703e74c90b..9ab8615594 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -28,11 +28,11 @@ object ActorRefSpec { def receive = { case "complexRequest" ⇒ { replyTo = channel - val worker = context.createActor(Props[WorkerActor]) + val worker = context.actorOf(Props[WorkerActor]) worker ! "work" } case "complexRequest2" ⇒ - val worker = context.createActor(Props[WorkerActor]) + val worker = context.actorOf(Props[WorkerActor]) worker ! ReplyTo(channel) case "workDone" ⇒ replyTo ! "complexReply" case "simpleRequest" ⇒ reply("simpleReply") @@ -135,7 +135,7 @@ class ActorRefSpec extends AkkaSpec { "An ActorRef" must { - "not allow Actors to be created outside of an createActor" in { + "not allow Actors to be created outside of an actorOf" in { intercept[akka.actor.ActorInitializationException] { new Actor { def receive = { case _ ⇒ } } } @@ -145,7 +145,7 @@ class ActorRefSpec extends AkkaSpec { filterException[akka.actor.ActorInitializationException] { intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(new Actor { + actorOf(new Actor { val nested = promiseIntercept(new Actor { def receive = { case _ ⇒ } })(result) def receive = { case _ ⇒ } })) @@ -155,49 +155,49 @@ class ActorRefSpec extends AkkaSpec { intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(promiseIntercept(new FailingOuterActor(createActor(new InnerActor)))(result))) + actorOf(promiseIntercept(new FailingOuterActor(actorOf(new InnerActor)))(result))) } contextStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(new OuterActor(createActor(promiseIntercept(new FailingInnerActor)(result))))) + actorOf(new OuterActor(actorOf(promiseIntercept(new FailingInnerActor)(result))))) } contextStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(promiseIntercept(new FailingInheritingOuterActor(createActor(new InnerActor)))(result))) + actorOf(promiseIntercept(new FailingInheritingOuterActor(actorOf(new InnerActor)))(result))) } contextStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(new FailingOuterActor(createActor(promiseIntercept(new FailingInheritingInnerActor)(result))))) + actorOf(new FailingOuterActor(actorOf(promiseIntercept(new FailingInheritingInnerActor)(result))))) } contextStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(new FailingInheritingOuterActor(createActor(promiseIntercept(new FailingInheritingInnerActor)(result))))) + actorOf(new FailingInheritingOuterActor(actorOf(promiseIntercept(new FailingInheritingInnerActor)(result))))) } contextStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(new FailingInheritingOuterActor(createActor(promiseIntercept(new FailingInnerActor)(result))))) + actorOf(new FailingInheritingOuterActor(actorOf(promiseIntercept(new FailingInnerActor)(result))))) } contextStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(new OuterActor(createActor(new InnerActor { + actorOf(new OuterActor(actorOf(new InnerActor { val a = promiseIntercept(new InnerActor)(result) })))) } @@ -206,21 +206,21 @@ class ActorRefSpec extends AkkaSpec { intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(new FailingOuterActor(createActor(promiseIntercept(new FailingInheritingInnerActor)(result))))) + actorOf(new FailingOuterActor(actorOf(promiseIntercept(new FailingInheritingInnerActor)(result))))) } contextStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(new OuterActor(createActor(promiseIntercept(new FailingInheritingInnerActor)(result))))) + actorOf(new OuterActor(actorOf(promiseIntercept(new FailingInheritingInnerActor)(result))))) } contextStackMustBeEmpty intercept[akka.actor.ActorInitializationException] { wrap(result ⇒ - createActor(new OuterActor(createActor(promiseIntercept({ new InnerActor; new InnerActor })(result))))) + actorOf(new OuterActor(actorOf(promiseIntercept({ new InnerActor; new InnerActor })(result))))) } contextStackMustBeEmpty @@ -229,7 +229,7 @@ class ActorRefSpec extends AkkaSpec { filterException[java.lang.IllegalStateException] { (intercept[java.lang.IllegalStateException] { wrap(result ⇒ - createActor(new OuterActor(createActor(promiseIntercept({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor })(result))))) + actorOf(new OuterActor(actorOf(promiseIntercept({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor })(result))))) }).getMessage must be === "Ur state be b0rked" contextStackMustBeEmpty @@ -237,7 +237,7 @@ class ActorRefSpec extends AkkaSpec { } "be serializable using Java Serialization on local node" in { - val a = createActor[InnerActor] + val a = actorOf[InnerActor] import java.io._ @@ -260,7 +260,7 @@ class ActorRefSpec extends AkkaSpec { } "throw an exception on deserialize if no app in scope" in { - val a = createActor[InnerActor] + val a = actorOf[InnerActor] import java.io._ @@ -282,7 +282,7 @@ class ActorRefSpec extends AkkaSpec { "must throw exception on deserialize if not present in local registry and remoting is not enabled" in { val latch = new CountDownLatch(1) - val a = createActor(new InnerActor { + val a = actorOf(new InnerActor { override def postStop { // app.registry.unregister(self) latch.countDown @@ -318,9 +318,9 @@ class ActorRefSpec extends AkkaSpec { } } - "support nested createActors" in { - val a = createActor(new Actor { - val nested = createActor(new Actor { def receive = { case _ ⇒ } }) + "support nested actorOfs" in { + val a = actorOf(new Actor { + val nested = actorOf(new Actor { def receive = { case _ ⇒ } }) def receive = { case _ ⇒ reply(nested) } }) @@ -330,8 +330,8 @@ class ActorRefSpec extends AkkaSpec { (a ne nested) must be === true } - "support advanced nested createActors" in { - val a = createActor(Props(new OuterActor(createActor(Props(new InnerActor))))) + "support advanced nested actorOfs" in { + val a = actorOf(Props(new OuterActor(actorOf(Props(new InnerActor))))) val inner = (a ? "innerself").as[Any].get (a ? a).as[ActorRef].get must be(a) @@ -342,8 +342,8 @@ class ActorRefSpec extends AkkaSpec { } "support reply via channel" in { - val serverRef = createActor(Props[ReplyActor]) - val clientRef = createActor(Props(new SenderActor(serverRef))) + val serverRef = actorOf(Props[ReplyActor]) + val clientRef = actorOf(Props(new SenderActor(serverRef))) clientRef ! "complex" clientRef ! "simple" @@ -367,7 +367,7 @@ class ActorRefSpec extends AkkaSpec { "stop when sent a poison pill" in { val timeout = Timeout(20000) - val ref = createActor(Props(new Actor { + val ref = actorOf(Props(new Actor { def receive = { case 5 ⇒ tryReply("five") case null ⇒ tryReply("null") @@ -392,9 +392,9 @@ class ActorRefSpec extends AkkaSpec { filterException[ActorKilledException] { val latch = new CountDownLatch(2) - val boss = createActor(Props(new Actor { + val boss = actorOf(Props(new Actor { - val ref = createActor( + val ref = actorOf( Props(new Actor { def receive = { case _ ⇒ } override def preRestart(reason: Throwable, msg: Option[Any]) = latch.countDown() diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala index acfd440628..35ddfecd22 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorTimeoutSpec.scala @@ -10,7 +10,7 @@ import akka.testkit.AkkaSpec class ActorTimeoutSpec extends AkkaSpec with BeforeAndAfterAll { - def actorWithTimeout(t: Timeout): ActorRef = createActor(Props(creator = () ⇒ new Actor { + def actorWithTimeout(t: Timeout): ActorRef = actorOf(Props(creator = () ⇒ new Actor { def receive = { case x ⇒ } diff --git a/akka-actor-tests/src/test/scala/akka/actor/Bench.scala b/akka-actor-tests/src/test/scala/akka/actor/Bench.scala index bc5bb0dc1d..9aa5ffc05d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Bench.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Bench.scala @@ -78,7 +78,7 @@ object Chameneos { var numFaded = 0 override def preStart() = { - for (i ← 0 until numChameneos) context.createActor(new Chameneo(self, colours(i % 3), i)) + for (i ← 0 until numChameneos) context.actorOf(new Chameneo(self, colours(i % 3), i)) } def receive = { @@ -109,7 +109,7 @@ object Chameneos { def run { // System.setProperty("akka.config", "akka.conf") Chameneos.start = System.currentTimeMillis - AkkaApplication().createActor(new Mall(1000000, 4)) + AkkaApplication().actorOf(new Mall(1000000, 4)) Thread.sleep(10000) println("Elapsed: " + (end - start)) } diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 2ecfffb12e..ca8fdad334 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -17,7 +17,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } "notify with one Terminated message when an Actor is stopped" in { - val terminal = createActor(Props(context ⇒ { case _ ⇒ context.self.stop() })) + val terminal = actorOf(Props(context ⇒ { case _ ⇒ context.self.stop() })) testActor startsMonitoring terminal @@ -29,8 +29,8 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } "notify with all monitors with one Terminated message when an Actor is stopped" in { - val monitor1, monitor2 = createActor(Props(context ⇒ { case t: Terminated ⇒ testActor ! t })) - val terminal = createActor(Props(context ⇒ { case _ ⇒ context.self.stop() })) + val monitor1, monitor2 = actorOf(Props(context ⇒ { case t: Terminated ⇒ testActor ! t })) + val terminal = actorOf(Props(context ⇒ { case _ ⇒ context.self.stop() })) monitor1 startsMonitoring terminal monitor2 startsMonitoring terminal @@ -48,8 +48,8 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } "notify with _current_ monitors with one Terminated message when an Actor is stopped" in { - val monitor1, monitor2 = createActor(Props(context ⇒ { case t: Terminated ⇒ testActor ! t })) - val terminal = createActor(Props(context ⇒ { case _ ⇒ context.self.stop() })) + val monitor1, monitor2 = actorOf(Props(context ⇒ { case t: Terminated ⇒ testActor ! t })) + val terminal = actorOf(Props(context ⇒ { case _ ⇒ context.self.stop() })) monitor1 startsMonitoring terminal monitor2 startsMonitoring terminal @@ -69,8 +69,8 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende "notify with a Terminated message once when an Actor is stopped but not when restarted" in { filterException[ActorKilledException] { - val supervisor = createActor(Props(context ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(2)))) - val terminal = createActor(Props(context ⇒ { case x ⇒ context.channel ! x }).withSupervisor(supervisor)) + val supervisor = actorOf(Props(context ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(2)))) + val terminal = actorOf(Props(context ⇒ { case x ⇒ context.channel ! x }).withSupervisor(supervisor)) testActor startsMonitoring terminal diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala index b671f77761..d7b8ec79d3 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMActorSpec.scala @@ -107,9 +107,9 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true "unlock the lock" in { // lock that locked after being open for 1 sec - val lock = createActor(new Lock("33221", 1 second)) + val lock = actorOf(new Lock("33221", 1 second)) - val transitionTester = createActor(new Actor { + val transitionTester = actorOf(new Actor { def receive = { case Transition(_, _, _) ⇒ transitionCallBackLatch.open case CurrentState(_, Locked) ⇒ initialStateLatch.open @@ -141,7 +141,7 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true val answerLatch = TestLatch() object Hello object Bye - val tester = createActor(new Actor { + val tester = actorOf(new Actor { protected def receive = { case Hello ⇒ lock ! "hello" case "world" ⇒ answerLatch.open @@ -162,7 +162,7 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true case Ev("go") ⇒ goto(2) } }) - val logger = createActor(new Actor { + val logger = actorOf(new Actor { def receive = { case x ⇒ testActor forward x } @@ -187,7 +187,7 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true case x ⇒ testActor ! x } } - val ref = createActor(fsm) + val ref = actorOf(fsm) started.await ref.stop() expectMsg(1 second, fsm.StopEvent(Shutdown, 1, null)) @@ -217,7 +217,7 @@ class FSMActorSpec extends AkkaSpec(Configuration("akka.actor.debug.fsm" -> true case StopEvent(r, _, _) ⇒ testActor ! r } }) - val logger = createActor(new Actor { + val logger = actorOf(new Actor { def receive = { case x ⇒ testActor forward x } diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index c5c2b21299..ab787b12bf 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -12,7 +12,7 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender { import FSMTimingSpec._ import FSM._ - val fsm = createActor(new StateMachine(testActor)) + val fsm = actorOf(new StateMachine(testActor)) fsm ! SubscribeTransitionCallBack(testActor) expectMsg(200 millis, CurrentState(fsm, Initial)) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala index ab7d632862..6a98ac5448 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala @@ -42,7 +42,7 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender { "A FSM transition notifier" must { "notify listeners" in { - val fsm = createActor(new MyFSM(testActor)) + val fsm = actorOf(new MyFSM(testActor)) within(1 second) { fsm ! SubscribeTransitionCallBack(testActor) expectMsg(CurrentState(fsm, 0)) @@ -54,9 +54,9 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender { } "not fail when listener goes away" in { - val forward = createActor(new Forwarder(testActor)) - val sup = createActor(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) - val fsm = sup startsMonitoring createActor(new MyFSM(testActor)) + val forward = actorOf(new Forwarder(testActor)) + val sup = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) + val fsm = sup startsMonitoring actorOf(new MyFSM(testActor)) within(300 millis) { fsm ! SubscribeTransitionCallBack(forward) expectMsg(CurrentState(fsm, 0)) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala index ebbf663d60..ce8bacf12b 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala @@ -14,11 +14,11 @@ object ForwardActorSpec { val ExpectedMessage = "FOO" def createForwardingChain(app: AkkaApplication): ActorRef = { - val replier = app.createActor(new Actor { + val replier = app.actorOf(new Actor { def receive = { case x ⇒ reply(x) } }) - def mkforwarder(forwardTo: ActorRef) = app.createActor( + def mkforwarder(forwardTo: ActorRef) = app.actorOf( new Actor { def receive = { case x ⇒ forwardTo forward x } }) @@ -35,7 +35,7 @@ class ForwardActorSpec extends AkkaSpec { "forward actor reference when invoking forward on bang" in { val latch = new TestLatch(1) - val replyTo = createActor(new Actor { def receive = { case ExpectedMessage ⇒ latch.countDown() } }) + val replyTo = actorOf(new Actor { def receive = { case ExpectedMessage ⇒ latch.countDown() } }) val chain = createForwardingChain(app) diff --git a/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala index 9e7d0804a2..9c5b1d99af 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/HotSwapSpec.scala @@ -14,7 +14,7 @@ class HotSwapSpec extends AkkaSpec { val barrier = TestBarrier(2) @volatile var _log = "" - val a = createActor(new Actor { + val a = actorOf(new Actor { def receive = { case _ ⇒ _log += "default" } }) a ! HotSwap(self ⇒ { @@ -31,7 +31,7 @@ class HotSwapSpec extends AkkaSpec { val barrier = TestBarrier(2) @volatile var _log = "" - val a = createActor(new Actor { + val a = actorOf(new Actor { def receive = { case "init" ⇒ _log += "init" @@ -60,7 +60,7 @@ class HotSwapSpec extends AkkaSpec { val barrier = TestBarrier(2) @volatile var _log = "" - val a = createActor(new Actor { + val a = actorOf(new Actor { def receive = { case "init" ⇒ _log += "init" @@ -106,7 +106,7 @@ class HotSwapSpec extends AkkaSpec { val barrier = TestBarrier(2) @volatile var _log = "" - val a = createActor(new Actor { + val a = actorOf(new Actor { def receive = { case "init" ⇒ _log += "init" diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 710bd48a7d..85c29e1033 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -22,7 +22,7 @@ object IOActorSpec { started.open() } - def createWorker = context.createActor(Props(new Actor with IO { + def createWorker = context.actorOf(Props(new Actor with IO { def receiveIO = { case NewClient(server) ⇒ val socket = server.accept() @@ -43,7 +43,7 @@ object IOActorSpec { class SimpleEchoClient(host: String, port: Int, ioManager: ActorRef) extends Actor with IO { lazy val socket: SocketHandle = connect(ioManager, host, port, reader) - lazy val reader: ActorRef = context.createActor { + lazy val reader: ActorRef = context.actorOf { new Actor with IO { def receiveIO = { case length: Int ⇒ @@ -70,7 +70,7 @@ object IOActorSpec { started.open() } - def createWorker = context.createActor(Props(new Actor with IO { + def createWorker = context.actorOf(Props(new Actor with IO { def receiveIO = { case NewClient(server) ⇒ val socket = server.accept() @@ -174,10 +174,10 @@ class IOActorSpec extends AkkaSpec with BeforeAndAfterEach { "an IO Actor" must { "run echo server" in { val started = TestLatch(1) - val ioManager = createActor(new IOManager(2)) // teeny tiny buffer - val server = createActor(new SimpleEchoServer("localhost", 8064, ioManager, started)) + val ioManager = actorOf(new IOManager(2)) // teeny tiny buffer + val server = actorOf(new SimpleEchoServer("localhost", 8064, ioManager, started)) started.await - val client = createActor(new SimpleEchoClient("localhost", 8064, ioManager)) + val client = actorOf(new SimpleEchoClient("localhost", 8064, ioManager)) val f1 = client ? ByteString("Hello World!1") val f2 = client ? ByteString("Hello World!2") val f3 = client ? ByteString("Hello World!3") @@ -191,10 +191,10 @@ class IOActorSpec extends AkkaSpec with BeforeAndAfterEach { "run echo server under high load" in { val started = TestLatch(1) - val ioManager = createActor(new IOManager()) - val server = createActor(new SimpleEchoServer("localhost", 8065, ioManager, started)) + val ioManager = actorOf(new IOManager()) + val server = actorOf(new SimpleEchoServer("localhost", 8065, ioManager, started)) started.await - val client = createActor(new SimpleEchoClient("localhost", 8065, ioManager)) + val client = actorOf(new SimpleEchoClient("localhost", 8065, ioManager)) val list = List.range(0, 1000) val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString)) assert(f.get.size === 1000) @@ -205,10 +205,10 @@ class IOActorSpec extends AkkaSpec with BeforeAndAfterEach { "run echo server under high load with small buffer" in { val started = TestLatch(1) - val ioManager = createActor(new IOManager(2)) - val server = createActor(new SimpleEchoServer("localhost", 8066, ioManager, started)) + val ioManager = actorOf(new IOManager(2)) + val server = actorOf(new SimpleEchoServer("localhost", 8066, ioManager, started)) started.await - val client = createActor(new SimpleEchoClient("localhost", 8066, ioManager)) + val client = actorOf(new SimpleEchoClient("localhost", 8066, ioManager)) val list = List.range(0, 1000) val f = Future.traverse(list)(i ⇒ client ? ByteString(i.toString)) assert(f.get.size === 1000) @@ -219,11 +219,11 @@ class IOActorSpec extends AkkaSpec with BeforeAndAfterEach { "run key-value store" in { val started = TestLatch(1) - val ioManager = createActor(new IOManager(2)) // teeny tiny buffer - val server = createActor(new KVStore("localhost", 8067, ioManager, started)) + val ioManager = actorOf(new IOManager(2)) // teeny tiny buffer + val server = actorOf(new KVStore("localhost", 8067, ioManager, started)) started.await - val client1 = createActor(new KVClient("localhost", 8067, ioManager)) - val client2 = createActor(new KVClient("localhost", 8067, ioManager)) + val client1 = actorOf(new KVClient("localhost", 8067, ioManager)) + val client2 = actorOf(new KVClient("localhost", 8067, ioManager)) val f1 = client1 ? (('set, "hello", ByteString("World"))) val f2 = client1 ? (('set, "test", ByteString("No one will read me"))) val f3 = client1 ? (('get, "hello")) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala index ea6b378b18..11fa223377 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ReceiveTimeoutSpec.scala @@ -16,7 +16,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { "get timeout" in { val timeoutLatch = TestLatch() - val timeoutActor = createActor(new Actor { + val timeoutActor = actorOf(new Actor { receiveTimeout = Some(500L) protected def receive = { @@ -31,7 +31,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { "get timeout when swapped" in { val timeoutLatch = TestLatch() - val timeoutActor = createActor(new Actor { + val timeoutActor = actorOf(new Actor { receiveTimeout = Some(500L) protected def receive = { @@ -55,7 +55,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { val timeoutLatch = TestLatch() case object Tick - val timeoutActor = createActor(new Actor { + val timeoutActor = actorOf(new Actor { receiveTimeout = Some(500L) protected def receive = { @@ -75,7 +75,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { val timeoutLatch = TestLatch() case object Tick - val timeoutActor = createActor(new Actor { + val timeoutActor = actorOf(new Actor { receiveTimeout = Some(500L) protected def receive = { @@ -97,7 +97,7 @@ class ReceiveTimeoutSpec extends AkkaSpec { "not receive timeout message when not specified" in { val timeoutLatch = TestLatch() - val timeoutActor = createActor(new Actor { + val timeoutActor = actorOf(new Actor { protected def receive = { case ReceiveTimeout ⇒ timeoutLatch.open } diff --git a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala index 412cf62b27..a43665e91a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/RestartStrategySpec.scala @@ -28,7 +28,7 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll { "A RestartStrategy" must { "ensure that slave stays dead after max restarts within time range" in { - val boss = createActor(Props(new Actor { + val boss = actorOf(Props(new Actor { protected def receive = { case _ ⇒ () } }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 1000))) @@ -37,7 +37,7 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll { val countDownLatch = new CountDownLatch(3) val stopLatch = new StandardLatch - val slave = createActor(Props(new Actor { + val slave = actorOf(Props(new Actor { protected def receive = { case Ping ⇒ countDownLatch.countDown() @@ -75,13 +75,13 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll { } "ensure that slave is immortal without max restarts and time range" in { - val boss = createActor(Props(new Actor { + val boss = actorOf(Props(new Actor { def receive = { case _ ⇒ () } }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) val countDownLatch = new CountDownLatch(100) - val slave = createActor(Props(new Actor { + val slave = actorOf(Props(new Actor { protected def receive = { case Crash ⇒ throw new Exception("Crashing...") @@ -98,7 +98,7 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll { } "ensure that slave restarts after number of crashes not within time range" in { - val boss = createActor(Props(new Actor { + val boss = actorOf(Props(new Actor { def receive = { case _ ⇒ () } }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 500))) @@ -108,7 +108,7 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll { val pingLatch = new StandardLatch val secondPingLatch = new StandardLatch - val slave = createActor(Props(new Actor { + val slave = actorOf(Props(new Actor { protected def receive = { case Ping ⇒ @@ -156,7 +156,7 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll { } "ensure that slave is not restarted after max retries" in { - val boss = createActor(Props(new Actor { + val boss = actorOf(Props(new Actor { def receive = { case _ ⇒ () } }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), Some(2), None))) @@ -165,7 +165,7 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll { val countDownLatch = new CountDownLatch(3) val stopLatch = new StandardLatch - val slave = createActor(Props(new Actor { + val slave = actorOf(Props(new Actor { protected def receive = { case Ping ⇒ countDownLatch.countDown() @@ -211,11 +211,11 @@ class RestartStrategySpec extends AkkaSpec with BeforeAndAfterAll { val restartLatch, stopLatch, maxNoOfRestartsLatch = new StandardLatch val countDownLatch = new CountDownLatch(2) - val boss = createActor(Props(new Actor { + val boss = actorOf(Props(new Actor { def receive = { case t: Terminated ⇒ maxNoOfRestartsLatch.open } }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, Some(1000)))) - val slave = createActor(Props(new Actor { + val slave = actorOf(Props(new Actor { protected def receive = { case Ping ⇒ countDownLatch.countDown() diff --git a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala index eb5865f5c2..4e3ea20fe1 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SchedulerSpec.scala @@ -29,7 +29,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { "schedule more than once" in { case object Tick val countDownLatch = new CountDownLatch(3) - val tickActor = createActor(new Actor { + val tickActor = actorOf(new Actor { def receive = { case Tick ⇒ countDownLatch.countDown() } }) // run every 50 millisec @@ -49,7 +49,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { "schedule once" in { case object Tick val countDownLatch = new CountDownLatch(3) - val tickActor = createActor(new Actor { + val tickActor = actorOf(new Actor { def receive = { case Tick ⇒ countDownLatch.countDown() } }) // run every 50 millisec @@ -69,7 +69,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { // "not create actors" in { // object Ping // val ticks = new CountDownLatch(1000) - // val actor = createActor(new Actor { + // val actor = actorOf(new Actor { // def receive = { case Ping ⇒ ticks.countDown } // }) // val numActors = app.registry.local.actors.length @@ -85,7 +85,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { object Ping val ticks = new CountDownLatch(1) - val actor = createActor(new Actor { + val actor = actorOf(new Actor { def receive = { case Ping ⇒ ticks.countDown() } }) @@ -106,8 +106,8 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach { val restartLatch = new StandardLatch val pingLatch = new CountDownLatch(6) - val supervisor = createActor(Props(context ⇒ { case _ ⇒ }).withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, 1000))) - val actor = createActor(Props(new Actor { + val supervisor = actorOf(Props(context ⇒ { case _ ⇒ }).withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, 1000))) + val actor = actorOf(Props(new Actor { def receive = { case Ping ⇒ pingLatch.countDown() case Crash ⇒ throw new Exception("CRASH") diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index da33ba33f9..4404e53c20 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -27,12 +27,12 @@ class SupervisorHierarchySpec extends AkkaSpec { "restart manager and workers in AllForOne" in { val countDown = new CountDownLatch(4) - val boss = createActor(Props(self ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), None, None))) + val boss = actorOf(Props(self ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), None, None))) - val manager = createActor(Props(new CountDownActor(countDown)).withFaultHandler(AllForOneStrategy(List(), None, None)).withSupervisor(boss)) + val manager = actorOf(Props(new CountDownActor(countDown)).withFaultHandler(AllForOneStrategy(List(), None, None)).withSupervisor(boss)) val workerProps = Props(new CountDownActor(countDown)).withSupervisor(manager) - val workerOne, workerTwo, workerThree = createActor(workerProps) + val workerOne, workerTwo, workerThree = actorOf(workerProps) filterException[ActorKilledException] { workerOne ! Kill @@ -47,8 +47,8 @@ class SupervisorHierarchySpec extends AkkaSpec { "send notification to supervisor when permanent failure" in { val countDownMessages = new CountDownLatch(1) val countDownMax = new CountDownLatch(1) - val boss = createActor(Props(new Actor { - val crasher = self startsMonitoring createActor(Props(new CountDownActor(countDownMessages)).withSupervisor(self)) + val boss = actorOf(Props(new Actor { + val crasher = self startsMonitoring actorOf(Props(new CountDownActor(countDownMessages)).withSupervisor(self)) protected def receive = { case "killCrasher" ⇒ crasher ! Kill diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index 01d19696bb..ea4776981d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -16,7 +16,7 @@ class SupervisorMiscSpec extends AkkaSpec { filterEvents(EventFilter[Exception]("Kill")) { val countDownLatch = new CountDownLatch(4) - val supervisor = createActor(Props(new Actor { + val supervisor = actorOf(Props(new Actor { def receive = { case _ ⇒ } }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 5000))) @@ -29,13 +29,13 @@ class SupervisorMiscSpec extends AkkaSpec { } }).withSupervisor(supervisor) - val actor1 = createActor(workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))) + val actor1 = actorOf(workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))) - val actor2 = createActor(workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))) + val actor2 = actorOf(workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))) - val actor3 = createActor(workerProps.withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) + val actor3 = actorOf(workerProps.withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) - val actor4 = createActor(workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))) + val actor4 = actorOf(workerProps.withDispatcher(app.dispatcherFactory.newPinnedDispatcher("pinned"))) actor1 ! Kill actor2 ! Kill diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index a227f5c4cb..c4604253c8 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -54,7 +54,7 @@ object SupervisorSpec { class Master extends Actor { - val temp = context.createActor(Props[PingPongActor].withSupervisor(self)) + val temp = context.actorOf(Props[PingPongActor].withSupervisor(self)) def receive = { case Die ⇒ (temp.?(Die, TimeoutMillis)).get @@ -72,51 +72,51 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte // ===================================================== def temporaryActorAllForOne = { - val supervisor = createActor(Props(AllForOneStrategy(List(classOf[Exception]), Some(0)))) - val temporaryActor = createActor(Props[PingPongActor].withSupervisor(supervisor)) + val supervisor = actorOf(Props(AllForOneStrategy(List(classOf[Exception]), Some(0)))) + val temporaryActor = actorOf(Props[PingPongActor].withSupervisor(supervisor)) (temporaryActor, supervisor) } def singleActorAllForOne = { - val supervisor = createActor(Props(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) - val pingpong = createActor(Props[PingPongActor].withSupervisor(supervisor)) + val supervisor = actorOf(Props(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val pingpong = actorOf(Props[PingPongActor].withSupervisor(supervisor)) (pingpong, supervisor) } def singleActorOneForOne = { - val supervisor = createActor(Props(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) - val pingpong = createActor(Props[PingPongActor].withSupervisor(supervisor)) + val supervisor = actorOf(Props(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val pingpong = actorOf(Props[PingPongActor].withSupervisor(supervisor)) (pingpong, supervisor) } def multipleActorsAllForOne = { - val supervisor = createActor(Props(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) - val pingpong1 = createActor(Props[PingPongActor].withSupervisor(supervisor)) - val pingpong2 = createActor(Props[PingPongActor].withSupervisor(supervisor)) - val pingpong3 = createActor(Props[PingPongActor].withSupervisor(supervisor)) + val supervisor = actorOf(Props(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val pingpong1 = actorOf(Props[PingPongActor].withSupervisor(supervisor)) + val pingpong2 = actorOf(Props[PingPongActor].withSupervisor(supervisor)) + val pingpong3 = actorOf(Props[PingPongActor].withSupervisor(supervisor)) (pingpong1, pingpong2, pingpong3, supervisor) } def multipleActorsOneForOne = { - val supervisor = createActor(Props(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) - val pingpong1 = createActor(Props[PingPongActor].withSupervisor(supervisor)) - val pingpong2 = createActor(Props[PingPongActor].withSupervisor(supervisor)) - val pingpong3 = createActor(Props[PingPongActor].withSupervisor(supervisor)) + val supervisor = actorOf(Props(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val pingpong1 = actorOf(Props[PingPongActor].withSupervisor(supervisor)) + val pingpong2 = actorOf(Props[PingPongActor].withSupervisor(supervisor)) + val pingpong3 = actorOf(Props[PingPongActor].withSupervisor(supervisor)) (pingpong1, pingpong2, pingpong3, supervisor) } def nestedSupervisorsAllForOne = { - val topSupervisor = createActor(Props(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) - val pingpong1 = createActor(Props[PingPongActor].withSupervisor(topSupervisor)) + val topSupervisor = actorOf(Props(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) + val pingpong1 = actorOf(Props[PingPongActor].withSupervisor(topSupervisor)) - val middleSupervisor = createActor(Props(AllForOneStrategy(Nil, 3, TimeoutMillis)).withSupervisor(topSupervisor)) - val pingpong2 = createActor(Props[PingPongActor].withSupervisor(middleSupervisor)) - val pingpong3 = createActor(Props[PingPongActor].withSupervisor(middleSupervisor)) + val middleSupervisor = actorOf(Props(AllForOneStrategy(Nil, 3, TimeoutMillis)).withSupervisor(topSupervisor)) + val pingpong2 = actorOf(Props[PingPongActor].withSupervisor(middleSupervisor)) + val pingpong3 = actorOf(Props[PingPongActor].withSupervisor(middleSupervisor)) (pingpong1, pingpong2, pingpong3, topSupervisor) } @@ -148,7 +148,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte "A supervisor" must { "not restart programmatically linked temporary actor" in { - val master = createActor(Props[Master].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) + val master = actorOf(Props[Master].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) intercept[RuntimeException] { (master.?(Die, TimeoutMillis)).get @@ -290,9 +290,9 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte "must attempt restart when exception during restart" in { val inits = new AtomicInteger(0) - val supervisor = createActor(Props(OneForOneStrategy(classOf[Exception] :: Nil, 3, 10000))) + val supervisor = actorOf(Props(OneForOneStrategy(classOf[Exception] :: Nil, 3, 10000))) - val dyingActor = createActor(Props(new Actor { + val dyingActor = actorOf(Props(new Actor { inits.incrementAndGet if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!") diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala index 48767e8747..fb4ae089c9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorTreeSpec.scala @@ -24,9 +24,9 @@ class SupervisorTreeSpec extends AkkaSpec with ImplicitSender { def receive = { case false ⇒ } override def preRestart(reason: Throwable, msg: Option[Any]) { testActor ! self.address } }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 1000)) - val headActor = createActor(p) - val middleActor = createActor(p.withSupervisor(headActor)) - val lastActor = createActor(p.withSupervisor(middleActor)) + val headActor = actorOf(p) + val middleActor = actorOf(p.withSupervisor(headActor)) + val lastActor = actorOf(p.withSupervisor(middleActor)) middleActor ! Kill expectMsg(middleActor.address) diff --git a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala index 27b7ef343d..c32832fbc1 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala @@ -18,8 +18,8 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender "A supervised actor with lifecycle PERMANENT" should { "be able to reply on failure during preRestart" in { filterEvents(EventFilter[Exception]("test")) { - val supervisor = createActor(Props(AllForOneStrategy(List(classOf[Exception]), 5, 10000))) - val supervised = createActor(Props[Supervised].withSupervisor(supervisor)) + val supervisor = actorOf(Props(AllForOneStrategy(List(classOf[Exception]), 5, 10000))) + val supervised = actorOf(Props[Supervised].withSupervisor(supervisor)) supervised.!("test")(Some(testActor)) expectMsg("failure1") @@ -29,8 +29,8 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender "be able to reply on failure during postStop" in { filterEvents(EventFilter[Exception]("test")) { - val supervisor = createActor(Props(AllForOneStrategy(List(classOf[Exception]), Some(0), None))) - val supervised = createActor(Props[Supervised].withSupervisor(supervisor)) + val supervisor = actorOf(Props(AllForOneStrategy(List(classOf[Exception]), Some(0), None))) + val supervised = actorOf(Props[Supervised].withSupervisor(supervisor)) supervised.!("test")(Some(testActor)) expectMsg("failure2") diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index 5a0fd6ee0d..64f594b1e2 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -237,7 +237,7 @@ abstract class ActorModelSpec extends AkkaSpec { import ActorModelSpec._ - def newTestActor(dispatcher: MessageDispatcher) = app.createActor(Props[DispatcherActor].withDispatcher(dispatcher)) + def newTestActor(dispatcher: MessageDispatcher) = app.actorOf(Props[DispatcherActor].withDispatcher(dispatcher)) protected def newInterceptedDispatcher: MessageDispatcherInterceptor protected def dispatcherType: String diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala index 76cf43ddc2..d0353b86eb 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/BalancingDispatcherSpec.scala @@ -43,8 +43,8 @@ class BalancingDispatcherSpec extends AkkaSpec { "have fast actor stealing work from slow actor" in { val finishedCounter = new CountDownLatch(110) - val slow = createActor(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef] - val fast = createActor(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef] + val slow = actorOf(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef] + val fast = actorOf(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef] var sentToFast = 0 diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala index 1aae1091f6..59255bd473 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala @@ -32,14 +32,14 @@ class DispatcherActorSpec extends AkkaSpec { "A Dispatcher and an Actor" must { "support tell" in { - val actor = createActor(Props[OneWayTestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) + val actor = actorOf(Props[OneWayTestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) val result = actor ! "OneWay" assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS)) actor.stop() } "support ask/reply" in { - val actor = createActor(Props[TestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) + val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) val result = (actor ? "Hello").as[String] assert("World" === result.get) actor.stop() @@ -47,7 +47,7 @@ class DispatcherActorSpec extends AkkaSpec { "support ask/exception" in { filterEvents(EventFilter[RuntimeException]("Expected")) { - val actor = createActor(Props[TestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) + val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newDispatcher("test").build)) try { (actor ? "Failure").get fail("Should have thrown an exception") @@ -68,10 +68,10 @@ class DispatcherActorSpec extends AkkaSpec { val works = new AtomicBoolean(true) val latch = new CountDownLatch(100) val start = new CountDownLatch(1) - val fastOne = createActor( + val fastOne = actorOf( Props(context ⇒ { case "sabotage" ⇒ works.set(false) }).withDispatcher(throughputDispatcher)) - val slowOne = createActor( + val slowOne = actorOf( Props(context ⇒ { case "hogexecutor" ⇒ start.await case "ping" ⇒ if (works.get) latch.countDown() @@ -98,12 +98,12 @@ class DispatcherActorSpec extends AkkaSpec { val start = new CountDownLatch(1) val ready = new CountDownLatch(1) - val fastOne = createActor( + val fastOne = actorOf( Props(context ⇒ { case "ping" ⇒ if (works.get) latch.countDown(); context.self.stop() }).withDispatcher(throughputDispatcher)) - val slowOne = createActor( + val slowOne = actorOf( Props(context ⇒ { case "hogexecutor" ⇒ ready.countDown(); start.await case "ping" ⇒ works.set(false); context.self.stop() diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorsSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorsSpec.scala index 101b0e14b7..f18658f3b8 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorsSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorsSpec.scala @@ -32,8 +32,8 @@ class DispatcherActorsSpec extends AkkaSpec { "not block fast actors by slow actors" in { val sFinished = new CountDownLatch(50) val fFinished = new CountDownLatch(10) - val s = createActor(new SlowActor(sFinished)) - val f = createActor(new FastActor(fFinished)) + val s = actorOf(new SlowActor(sFinished)) + val f = actorOf(new FastActor(fFinished)) // send a lot of stuff to s for (i ← 1 to 50) { diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala index 3f3a7c156f..46d05c18ea 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala @@ -35,21 +35,21 @@ class PinnedActorSpec extends AkkaSpec with BeforeAndAfterEach { "support tell" in { var oneWay = new CountDownLatch(1) - val actor = createActor(Props(self ⇒ { case "OneWay" ⇒ oneWay.countDown() }).withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) + val actor = actorOf(Props(self ⇒ { case "OneWay" ⇒ oneWay.countDown() }).withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) val result = actor ! "OneWay" assert(oneWay.await(1, TimeUnit.SECONDS)) actor.stop() } "support ask/reply" in { - val actor = createActor(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) + val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) val result = (actor ? "Hello").as[String] assert("World" === result.get) actor.stop() } "support ask/exception" in { - val actor = createActor(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) + val actor = actorOf(Props[TestActor].withDispatcher(app.dispatcherFactory.newPinnedDispatcher("test"))) app.eventHandler.notify(Mute(EventFilter[RuntimeException]("Expected exception; to test fault-tolerance"))) try { (actor ? "Failure").get diff --git a/akka-actor-tests/src/test/scala/akka/actor/routing/ListenerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/routing/ListenerSpec.scala index 9bd9432bd5..2422538467 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/routing/ListenerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/routing/ListenerSpec.scala @@ -15,13 +15,13 @@ class ListenerSpec extends AkkaSpec { val barLatch = TestLatch(2) val barCount = new AtomicInteger(0) - val broadcast = createActor(new Actor with Listeners { + val broadcast = actorOf(new Actor with Listeners { def receive = listenerManagement orElse { case "foo" ⇒ gossip("bar") } }) - def newListener = createActor(new Actor { + def newListener = actorOf(new Actor { def receive = { case "bar" ⇒ barCount.incrementAndGet diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index b0d9616ace..aef33a4874 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -116,7 +116,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "from an Actor" that { "returns a result" must { behave like futureWithResult { test ⇒ - val actor = createActor[TestActor] + val actor = actorOf[TestActor] val future = actor ? "Hello" future.await test(future, "World") @@ -126,7 +126,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "throws an exception" must { behave like futureWithException[RuntimeException] { test ⇒ filterException[RuntimeException] { - val actor = createActor[TestActor] + val actor = actorOf[TestActor] val future = actor ? "Failure" future.await test(future, "Expected exception; to test fault-tolerance") @@ -139,8 +139,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "using flatMap with an Actor" that { "will return a result" must { behave like futureWithResult { test ⇒ - val actor1 = createActor[TestActor] - val actor2 = createActor(new Actor { def receive = { case s: String ⇒ reply(s.toUpperCase) } }) + val actor1 = actorOf[TestActor] + val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ reply(s.toUpperCase) } }) val future = actor1 ? "Hello" flatMap { case s: String ⇒ actor2 ? s } future.await test(future, "WORLD") @@ -151,8 +151,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "will throw an exception" must { behave like futureWithException[ArithmeticException] { test ⇒ filterException[ArithmeticException] { - val actor1 = createActor[TestActor] - val actor2 = createActor(new Actor { def receive = { case s: String ⇒ reply(s.length / 0) } }) + val actor1 = actorOf[TestActor] + val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ reply(s.length / 0) } }) val future = actor1 ? "Hello" flatMap { case s: String ⇒ actor2 ? s } future.await test(future, "/ by zero") @@ -164,8 +164,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "will throw a MatchError when matching wrong type" must { behave like futureWithException[MatchError] { test ⇒ filterException[MatchError] { - val actor1 = createActor[TestActor] - val actor2 = createActor(new Actor { def receive = { case s: String ⇒ reply(s.toUpperCase) } }) + val actor1 = actorOf[TestActor] + val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ reply(s.toUpperCase) } }) val future = actor1 ? "Hello" flatMap { case i: Int ⇒ actor2 ? i } future.await test(future, "World (of class java.lang.String)") @@ -180,7 +180,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "compose with for-comprehensions" in { filterException[ClassCastException] { - val actor = createActor(new Actor { + val actor = actorOf(new Actor { def receive = { case s: String ⇒ reply(s.length) case i: Int ⇒ reply((i * 2).toString) @@ -212,7 +212,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { filterException[MatchError] { case class Req[T](req: T) case class Res[T](res: T) - val actor = createActor(new Actor { + val actor = actorOf(new Actor { def receive = { case Req(s: String) ⇒ reply(Res(s.length)) case Req(i: Int) ⇒ reply(Res((i * 2).toString)) @@ -257,7 +257,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { val future7 = future3 recover { case e: ArithmeticException ⇒ "You got ERROR" } - val actor = createActor[TestActor] + val actor = actorOf[TestActor] val future8 = actor ? "Failure" val future9 = actor ? "Failure" recover { @@ -300,7 +300,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "fold" in { val actors = (1 to 10).toList map { _ ⇒ - createActor(new Actor { + actorOf(new Actor { def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); tryReply(add) } }) } @@ -311,7 +311,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "fold by composing" in { val actors = (1 to 10).toList map { _ ⇒ - createActor(new Actor { + actorOf(new Actor { def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); tryReply(add) } }) } @@ -322,7 +322,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "fold with an exception" in { filterException[IllegalArgumentException] { val actors = (1 to 10).toList map { _ ⇒ - createActor(new Actor { + actorOf(new Actor { def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait) @@ -358,7 +358,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "shouldReduceResults" in { val actors = (1 to 10).toList map { _ ⇒ - createActor(new Actor { + actorOf(new Actor { def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); tryReply(add) } }) } @@ -370,7 +370,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "shouldReduceResultsWithException" in { filterException[IllegalArgumentException] { val actors = (1 to 10).toList map { _ ⇒ - createActor(new Actor { + actorOf(new Actor { def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait) @@ -393,14 +393,14 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "receiveShouldExecuteOnComplete" in { val latch = new StandardLatch - val actor = createActor[TestActor] + val actor = actorOf[TestActor] actor ? "Hello" onResult { case "World" ⇒ latch.open } assert(latch.tryAwait(5, TimeUnit.SECONDS)) actor.stop() } "shouldTraverseFutures" in { - val oddActor = createActor(new Actor { + val oddActor = actorOf(new Actor { var counter = 1 def receive = { case 'GetNext ⇒ @@ -467,7 +467,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "futureComposingWithContinuations" in { import Future.flow - val actor = createActor[TestActor] + val actor = actorOf[TestActor] val x = Future("Hello") val y = x flatMap (actor ? _) mapTo manifest[String] @@ -496,7 +496,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { filterException[ClassCastException] { import Future.flow - val actor = createActor[TestActor] + val actor = actorOf[TestActor] val x = Future(3) val y = (actor ? "Hello").mapTo[Int] @@ -511,7 +511,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { filterException[ClassCastException] { import Future.flow - val actor = createActor[TestActor] + val actor = actorOf[TestActor] val x = Future("Hello") val y = actor ? "Hello" mapTo manifest[Nothing] diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala index 5181776db1..b51896f659 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala @@ -82,7 +82,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn def createMessageInvocation(msg: Any): Envelope = { new Envelope( - createActor(new Actor { //Dummy actor + actorOf(new Actor { //Dummy actor def receive = { case _ ⇒ } }).asInstanceOf[LocalActorRef].underlying, msg, NullChannel) } 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 e4c47b11b9..a54f739e23 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -24,7 +24,7 @@ class PriorityDispatcherSpec extends AkkaSpec { def testOrdering(mboxType: MailboxType) { val dispatcher = app.dispatcherFactory.newDispatcher("Test", 1, -1, mboxType).build - val actor = createActor(Props(new Actor { + val actor = actorOf(Props(new Actor { var acc: List[Int] = Nil def receive = { diff --git a/akka-actor-tests/src/test/scala/akka/event/EventBusSpec.scala b/akka-actor-tests/src/test/scala/akka/event/EventBusSpec.scala index e810b3840d..c96a6b85fc 100644 --- a/akka-actor-tests/src/test/scala/akka/event/EventBusSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/event/EventBusSpec.scala @@ -161,7 +161,7 @@ class ActorEventBusSpec extends EventBusSpec("ActorEventBus") { def createEvents(numberOfEvents: Int) = (0 until numberOfEvents) - def createSubscriber(pipeTo: ActorRef) = createActor(Props(new TestActorWrapperActor(pipeTo))) + def createSubscriber(pipeTo: ActorRef) = actorOf(Props(new TestActorWrapperActor(pipeTo))) def classifierFor(event: BusType#Event) = event.toString diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala index bd4aeb43fe..3bd7ea96d9 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/AkkaPerformanceTest.scala @@ -34,7 +34,7 @@ abstract class AkkaPerformanceTest(val app: AkkaApplication) extends BenchmarkSc val clients = (for (i ← 0 until numberOfClients) yield { val receiver = receivers(i % receivers.size) Props(new Client(receiver, orders, latch, repeatsPerClient + (if (i < oddRepeats) 1 else 0), sampling, delayMs)).withDispatcher(clientDispatcher) - }).toList.map(app.createActor(_)) + }).toList.map(app.actorOf(_)) clients.foreach(_ ! "run") val ok = latch.await((5000 + (2 + delayMs) * totalNumberOfRequests) * timeDilation, TimeUnit.MILLISECONDS) diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/common/TradingSystem.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/common/TradingSystem.scala index a454c48a5c..076639e4ae 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/common/TradingSystem.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/common/TradingSystem.scala @@ -70,8 +70,8 @@ class AkkaTradingSystem(val app: AkkaApplication) extends TradingSystem { def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match { - case Some(d) ⇒ app.createActor(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d)) - case _ ⇒ app.createActor(Props(new AkkaMatchingEngine(meId, orderbooks))) + case Some(d) ⇒ app.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks)).withDispatcher(d)) + case _ ⇒ app.actorOf(Props(new AkkaMatchingEngine(meId, orderbooks))) } override def createOrderReceivers: List[ActorRef] = { @@ -91,8 +91,8 @@ class AkkaTradingSystem(val app: AkkaApplication) extends TradingSystem { } def createOrderReceiver() = orDispatcher match { - case Some(d) ⇒ app.createActor(Props(new AkkaOrderReceiver()).withDispatcher(d)) - case _ ⇒ app.createActor(Props(new AkkaOrderReceiver())) + case Some(d) ⇒ app.actorOf(Props(new AkkaOrderReceiver()).withDispatcher(d)) + case _ ⇒ app.actorOf(Props(new AkkaOrderReceiver())) } override def start() { diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayPerformanceTest.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayPerformanceTest.scala index 1fe9efbdef..320254fdc9 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayPerformanceTest.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayPerformanceTest.scala @@ -12,8 +12,8 @@ class OneWayPerformanceTest extends AkkaPerformanceTest(AkkaApplication()) { override def createTradingSystem: TS = new OneWayTradingSystem(app) { override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match { - case Some(d) ⇒ app.createActor(Props(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown).withDispatcher(d)) - case _ ⇒ app.createActor(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown) + case Some(d) ⇒ app.actorOf(Props(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown).withDispatcher(d)) + case _ ⇒ app.actorOf(new OneWayMatchingEngine(meId, orderbooks) with LatchMessageCountDown) } } diff --git a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayTradingSystem.scala b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayTradingSystem.scala index b0b17ea7c5..f841c8288b 100644 --- a/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayTradingSystem.scala +++ b/akka-actor-tests/src/test/scala/akka/performance/trading/oneway/OneWayTradingSystem.scala @@ -8,13 +8,13 @@ import akka.AkkaApplication class OneWayTradingSystem(_app: AkkaApplication) extends AkkaTradingSystem(_app) { override def createMatchingEngine(meId: String, orderbooks: List[Orderbook]) = meDispatcher match { - case Some(d) ⇒ app.createActor(Props(new OneWayMatchingEngine(meId, orderbooks)).withDispatcher(d)) - case _ ⇒ app.createActor(Props(new OneWayMatchingEngine(meId, orderbooks))) + case Some(d) ⇒ app.actorOf(Props(new OneWayMatchingEngine(meId, orderbooks)).withDispatcher(d)) + case _ ⇒ app.actorOf(Props(new OneWayMatchingEngine(meId, orderbooks))) } override def createOrderReceiver() = orDispatcher match { - case Some(d) ⇒ app.createActor(Props[OneWayOrderReceiver].withDispatcher(d)) - case _ ⇒ app.createActor(Props[OneWayOrderReceiver]) + case Some(d) ⇒ app.actorOf(Props[OneWayOrderReceiver].withDispatcher(d)) + case _ ⇒ app.actorOf(Props[OneWayOrderReceiver]) } } diff --git a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala index 699b9e18e0..0c1f381b1b 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala @@ -34,9 +34,9 @@ class ActorPoolSpec extends AkkaSpec { val latch = TestLatch(2) val count = new AtomicInteger(0) - val pool = createActor( + val pool = actorOf( Props(new Actor with DefaultActorPool with FixedCapacityStrategy with SmallestMailboxSelector { - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case _ ⇒ count.incrementAndGet @@ -52,7 +52,7 @@ class ActorPoolSpec extends AkkaSpec { }).withFaultHandler(faultHandler)) val successes = TestLatch(2) - val successCounter = createActor(new Actor { + val successCounter = actorOf(new Actor { def receive = { case "success" ⇒ successes.countDown() } @@ -73,7 +73,7 @@ class ActorPoolSpec extends AkkaSpec { } "pass ticket #705" in { - val pool = createActor( + val pool = actorOf( Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicFilter { def lowerBound = 2 def upperBound = 20 @@ -84,7 +84,7 @@ class ActorPoolSpec extends AkkaSpec { def selectionCount = 1 def receive = _route def pressureThreshold = 1 - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case req: String ⇒ { sleepFor(10 millis) @@ -110,9 +110,9 @@ class ActorPoolSpec extends AkkaSpec { var latch = TestLatch(3) val count = new AtomicInteger(0) - val pool = createActor( + val pool = actorOf( Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case n: Int ⇒ sleepFor(n millis) @@ -174,9 +174,9 @@ class ActorPoolSpec extends AkkaSpec { var latch = TestLatch(3) val count = new AtomicInteger(0) - val pool = createActor( + val pool = actorOf( Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case n: Int ⇒ sleepFor(n millis) @@ -227,10 +227,10 @@ class ActorPoolSpec extends AkkaSpec { val latch1 = TestLatch(2) val delegates = new java.util.concurrent.ConcurrentHashMap[String, String] - val pool1 = createActor( + val pool1 = actorOf( Props(new Actor with DefaultActorPool with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter { - def instance(p: Props): ActorRef = createActor(p.withCreator(new Actor { + def instance(p: Props): ActorRef = actorOf(p.withCreator(new Actor { def receive = { case _ ⇒ delegates put (self.uuid.toString, "") @@ -256,9 +256,9 @@ class ActorPoolSpec extends AkkaSpec { val latch2 = TestLatch(2) delegates.clear() - val pool2 = createActor( + val pool2 = actorOf( Props(new Actor with DefaultActorPool with FixedCapacityStrategy with RoundRobinSelector with BasicNoBackoffFilter { - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case _ ⇒ delegates put (self.uuid.toString, "") @@ -285,9 +285,9 @@ class ActorPoolSpec extends AkkaSpec { "backoff" in { val latch = TestLatch(10) - val pool = createActor( + val pool = actorOf( Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with Filter with RunningMeanBackoff with BasicRampup { - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case n: Int ⇒ sleepFor(n millis) @@ -357,7 +357,7 @@ class ActorPoolSpec extends AkkaSpec { val deathCount = new AtomicInteger(0) val keepDying = new AtomicBoolean(false) - val pool1 = createActor( + val pool1 = actorOf( Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { def lowerBound = 2 def upperBound = 5 @@ -368,7 +368,7 @@ class ActorPoolSpec extends AkkaSpec { def selectionCount = 1 def receive = _route def pressureThreshold = 1 - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { if (deathCount.get > 5) deathCount.set(0) if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } def receive = { @@ -380,7 +380,7 @@ class ActorPoolSpec extends AkkaSpec { })) }).withFaultHandler(faultHandler)) - val pool2 = createActor( + val pool2 = actorOf( Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { def lowerBound = 2 def upperBound = 5 @@ -391,7 +391,7 @@ class ActorPoolSpec extends AkkaSpec { def selectionCount = 1 def receive = _route def pressureThreshold = 1 - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { if (deathCount.get > 5) deathCount.set(0) if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } def receive = { @@ -403,7 +403,7 @@ class ActorPoolSpec extends AkkaSpec { })) }).withFaultHandler(faultHandler)) - val pool3 = createActor( + val pool3 = actorOf( Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with RoundRobinSelector with BasicFilter { def lowerBound = 2 def upperBound = 5 @@ -414,7 +414,7 @@ class ActorPoolSpec extends AkkaSpec { def selectionCount = 1 def receive = _route def pressureThreshold = 1 - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { if (deathCount.get > 5) deathCount.set(0) if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } @@ -501,7 +501,7 @@ class ActorPoolSpec extends AkkaSpec { object BadState - val pool1 = createActor( + val pool1 = actorOf( Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with ActiveFuturesPressureCapacitor with SmallestMailboxSelector with BasicFilter { def lowerBound = 2 def upperBound = 5 @@ -512,7 +512,7 @@ class ActorPoolSpec extends AkkaSpec { def selectionCount = 1 def receive = _route def pressureThreshold = 1 - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { if (deathCount.get > 5) deathCount.set(0) if (deathCount.get > 0) { deathCount.incrementAndGet; throw new IllegalStateException("keep dying") } def receive = { diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala index ad2f2a3a17..5e5da7ee23 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala @@ -27,7 +27,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val helloLatch = new CountDownLatch(5) val stopLatch = new CountDownLatch(5) - val actor = app.createActor(Props(new Actor { + val actor = app.actorOf(Props(new Actor { def receive = { case "hello" ⇒ helloLatch.countDown() } @@ -70,7 +70,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { replies = replies + (i -> 0) } - val actor = app.createActor(Props(new Actor { + val actor = app.actorOf(Props(new Actor { lazy val id = counter.getAndIncrement() def receive = { case "hit" ⇒ reply(id) @@ -108,7 +108,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val helloLatch = new CountDownLatch(5) val stopLatch = new CountDownLatch(5) - val actor = app.createActor(Props(new Actor { + val actor = app.actorOf(Props(new Actor { def receive = { case "hello" ⇒ helloLatch.countDown() } @@ -142,7 +142,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val stopLatch = new CountDownLatch(7) - val actor = app.createActor(Props(new Actor { + val actor = app.actorOf(Props(new Actor { def receive = { case "hello" ⇒ {} } @@ -184,7 +184,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { replies = replies + (i -> 0) } - val actor = app.createActor(Props(new Actor { + val actor = app.actorOf(Props(new Actor { lazy val id = counter.getAndIncrement() def receive = { case "hit" ⇒ reply(id) @@ -222,7 +222,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val helloLatch = new CountDownLatch(6) val stopLatch = new CountDownLatch(6) - val actor = app.createActor(Props(new Actor { + val actor = app.actorOf(Props(new Actor { def receive = { case "hello" ⇒ helloLatch.countDown() } diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index a1001a0da7..5937597149 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -25,17 +25,17 @@ class RoutingSpec extends AkkaSpec { "direct router" must { "be started when constructed" in { - val actor1 = createActor[TestActor] + val actor1 = actorOf[TestActor] val props = RoutedProps().withDirectRouter.withLocalConnections(List(actor1)) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor.isShutdown must be(false) } "throw ConfigurationException at construction when no connections" in { try { val props = RoutedProps().withDirectRouter - app.createActor(props, "foo") + app.actorOf(props, "foo") fail() } catch { case e: ConfigurationException ⇒ @@ -46,7 +46,7 @@ class RoutingSpec extends AkkaSpec { val doneLatch = new CountDownLatch(1) val counter = new AtomicInteger(0) - val connection1 = createActor(new Actor { + val connection1 = actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case _ ⇒ counter.incrementAndGet @@ -54,7 +54,7 @@ class RoutingSpec extends AkkaSpec { }) val props = RoutedProps().withDirectRouter.withLocalConnections(List(connection1)) - val routedActor = app.createActor(props, "foo") + val routedActor = app.actorOf(props, "foo") routedActor ! "hello" routedActor ! "end" @@ -67,7 +67,7 @@ class RoutingSpec extends AkkaSpec { val doneLatch = new CountDownLatch(1) val counter1 = new AtomicInteger - val connection1 = createActor(new Actor { + val connection1 = actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counter1.addAndGet(msg) @@ -75,7 +75,7 @@ class RoutingSpec extends AkkaSpec { }) val props = RoutedProps().withDirectRouter.withLocalConnections(List(connection1)) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor ! Broadcast(1) actor ! "end" @@ -89,17 +89,17 @@ class RoutingSpec extends AkkaSpec { "round robin router" must { "be started when constructed" in { - val actor1 = createActor[TestActor] + val actor1 = actorOf[TestActor] val props = RoutedProps().withRoundRobinRouter.withLocalConnections(List(actor1)) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor.isShutdown must be(false) } "throw ConfigurationException at construction when no connections" in { try { val props = RoutedProps().withRoundRobinRouter - app.createActor(props, "foo") + app.actorOf(props, "foo") fail() } catch { case e: ConfigurationException ⇒ @@ -121,7 +121,7 @@ class RoutingSpec extends AkkaSpec { for (i ← 0 until connectionCount) { counters = counters :+ new AtomicInteger() - val connection = createActor(new Actor { + val connection = actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counters.get(i).get.addAndGet(msg) @@ -132,7 +132,7 @@ class RoutingSpec extends AkkaSpec { //create the routed actor. val props = RoutedProps().withRoundRobinRouter.withLocalConnections(connections) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") //send messages to the actor. for (i ← 0 until iterationCount) { @@ -155,7 +155,7 @@ class RoutingSpec extends AkkaSpec { val doneLatch = new CountDownLatch(2) val counter1 = new AtomicInteger - val connection1 = createActor(new Actor { + val connection1 = actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counter1.addAndGet(msg) @@ -163,7 +163,7 @@ class RoutingSpec extends AkkaSpec { }) val counter2 = new AtomicInteger - val connection2 = createActor(new Actor { + val connection2 = actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counter2.addAndGet(msg) @@ -171,7 +171,7 @@ class RoutingSpec extends AkkaSpec { }) val props = RoutedProps().withRoundRobinRouter.withLocalConnections(List(connection1, connection2)) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor ! Broadcast(1) actor ! Broadcast("end") @@ -186,7 +186,7 @@ class RoutingSpec extends AkkaSpec { val doneLatch = new CountDownLatch(1) val counter1 = new AtomicInteger - val connection1 = createActor(new Actor { + val connection1 = actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case _ ⇒ counter1.incrementAndGet() @@ -194,7 +194,7 @@ class RoutingSpec extends AkkaSpec { }) val props = RoutedProps().withRoundRobinRouter.withLocalConnections(List(connection1)) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") try { actor ? Broadcast(1) @@ -213,17 +213,17 @@ class RoutingSpec extends AkkaSpec { "be started when constructed" in { - val actor1 = createActor[TestActor] + val actor1 = actorOf[TestActor] val props = RoutedProps().withRandomRouter.withLocalConnections(List(actor1)) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor.isShutdown must be(false) } "throw ConfigurationException at construction when no connections" in { try { val props = RoutedProps().withRandomRouter - app.createActor(props, "foo") + app.actorOf(props, "foo") fail() } catch { case e: ConfigurationException ⇒ @@ -238,7 +238,7 @@ class RoutingSpec extends AkkaSpec { val doneLatch = new CountDownLatch(2) val counter1 = new AtomicInteger - val connection1 = createActor(new Actor { + val connection1 = actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counter1.addAndGet(msg) @@ -246,7 +246,7 @@ class RoutingSpec extends AkkaSpec { }) val counter2 = new AtomicInteger - val connection2 = createActor(new Actor { + val connection2 = actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counter2.addAndGet(msg) @@ -254,7 +254,7 @@ class RoutingSpec extends AkkaSpec { }) val props = RoutedProps().withRandomRouter.withLocalConnections(List(connection1, connection2)) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor ! Broadcast(1) actor ! Broadcast("end") @@ -269,7 +269,7 @@ class RoutingSpec extends AkkaSpec { val doneLatch = new CountDownLatch(1) val counter1 = new AtomicInteger - val connection1 = createActor(new Actor { + val connection1 = actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case _ ⇒ counter1.incrementAndGet() @@ -277,7 +277,7 @@ class RoutingSpec extends AkkaSpec { }) val props = RoutedProps().withRandomRouter.withLocalConnections(List(connection1)) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") try { actor ? Broadcast(1) @@ -302,7 +302,7 @@ class RoutingSpec extends AkkaSpec { .withLocalConnections(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch)))) .withRouter(() ⇒ new ScatterGatherFirstCompletedRouter()) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor ! Broadcast(Stop(Some(0))) @@ -319,7 +319,7 @@ class RoutingSpec extends AkkaSpec { .withLocalConnections(List(newActor(0, Some(shutdownLatch)), newActor(1, Some(shutdownLatch)))) .withRouter(() ⇒ new ScatterGatherFirstCompletedRouter()) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor ! Broadcast(Stop()) @@ -337,7 +337,7 @@ class RoutingSpec extends AkkaSpec { .withLocalConnections(List(newActor(0), newActor(1))) .withRouter(() ⇒ new ScatterGatherFirstCompletedRouter()) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") (actor ? Broadcast("Hi!")).get.asInstanceOf[Int] must be(0) @@ -348,7 +348,7 @@ class RoutingSpec extends AkkaSpec { .withLocalConnections(List(newActor(0), newActor(1))) .withRouter(() ⇒ new ScatterGatherFirstCompletedRouter()) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") (actor ? Broadcast(0)).get.asInstanceOf[Int] must be(1) } @@ -357,7 +357,7 @@ class RoutingSpec extends AkkaSpec { val props = RoutedProps() .withLocalConnections(List(newActor(0))) .withRouter(() ⇒ new ScatterGatherFirstCompletedRouter()) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor.isShutdown must be(false) @@ -369,7 +369,7 @@ class RoutingSpec extends AkkaSpec { .withRouter(() ⇒ new ScatterGatherFirstCompletedRouter()) try { - app.createActor(props, "foo") + app.actorOf(props, "foo") fail() } catch { case e: ConfigurationException ⇒ @@ -386,7 +386,7 @@ class RoutingSpec extends AkkaSpec { for (i ← 0 until connectionCount) { counters = counters :+ new AtomicInteger() - val connection = app.createActor(new Actor { + val connection = app.actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counters.get(i).get.addAndGet(msg) @@ -399,7 +399,7 @@ class RoutingSpec extends AkkaSpec { .withLocalConnections(connections) .withRouter(() ⇒ new ScatterGatherFirstCompletedRouter()) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") for (i ← 0 until iterationCount) { for (k ← 0 until connectionCount) { @@ -421,7 +421,7 @@ class RoutingSpec extends AkkaSpec { val doneLatch = new TestLatch(2) val counter1 = new AtomicInteger - val connection1 = app.createActor(new Actor { + val connection1 = app.actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counter1.addAndGet(msg) @@ -429,7 +429,7 @@ class RoutingSpec extends AkkaSpec { }) val counter2 = new AtomicInteger - val connection2 = app.createActor(new Actor { + val connection2 = app.actorOf(new Actor { def receive = { case "end" ⇒ doneLatch.countDown() case msg: Int ⇒ counter2.addAndGet(msg) @@ -440,7 +440,7 @@ class RoutingSpec extends AkkaSpec { .withLocalConnections(List(connection1, connection2)) .withRouter(() ⇒ new ScatterGatherFirstCompletedRouter()) - val actor = app.createActor(props, "foo") + val actor = app.actorOf(props, "foo") actor ! Broadcast(1) actor ! Broadcast("end") @@ -453,7 +453,7 @@ class RoutingSpec extends AkkaSpec { case class Stop(id: Option[Int] = None) - def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = app.createActor(new Actor { + def newActor(id: Int, shudownLatch: Option[TestLatch] = None) = app.actorOf(new Actor { def receive = { case Stop(None) ⇒ self.stop() case Stop(Some(_id)) if (_id == id) ⇒ self.stop() diff --git a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala index 5ed1aa7db7..0367956c50 100644 --- a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala @@ -8,7 +8,7 @@ class Ticket703Spec extends AkkaSpec { "A ? call to an actor pool" should { "reuse the proper timeout" in { - val actorPool = createActor( + val actorPool = actorOf( Props(new Actor with DefaultActorPool with BoundedCapacityStrategy with MailboxPressureCapacitor with SmallestMailboxSelector with BasicNoBackoffFilter { def lowerBound = 2 def upperBound = 20 @@ -17,7 +17,7 @@ class Ticket703Spec extends AkkaSpec { def selectionCount = 1 def receive = _route def pressureThreshold = 1 - def instance(p: Props) = createActor(p.withCreator(new Actor { + def instance(p: Props) = actorOf(p.withCreator(new Actor { def receive = { case req: String ⇒ Thread.sleep(6000L) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index b9116fbec6..5cbc7cc8c0 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -43,29 +43,29 @@ trait ActorRefFactory { def dispatcher: MessageDispatcher - def createActor(props: Props): ActorRef = createActor(props, new UUID().toString) + def actorOf(props: Props): ActorRef = actorOf(props, new UUID().toString) /* * TODO this will have to go at some point, because creating two actors with * the same address can race on the cluster, and then you never know which * implementation wins */ - def createActor(props: Props, address: String): ActorRef = provider.actorOf(props, address) + def actorOf(props: Props, address: String): ActorRef = provider.actorOf(props, address) - def createActor[T <: Actor](implicit m: Manifest[T]): ActorRef = createActor(Props(m.erasure.asInstanceOf[Class[_ <: Actor]])) + def actorOf[T <: Actor](implicit m: Manifest[T]): ActorRef = actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]])) - def createActor[T <: Actor](address: String)(implicit m: Manifest[T]): ActorRef = - createActor(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]), address) + def actorOf[T <: Actor](address: String)(implicit m: Manifest[T]): ActorRef = + actorOf(Props(m.erasure.asInstanceOf[Class[_ <: Actor]]), address) - def createActor[T <: Actor](clazz: Class[T]): ActorRef = createActor(Props(clazz)) + def actorOf[T <: Actor](clazz: Class[T]): ActorRef = actorOf(Props(clazz)) - def createActor(factory: ⇒ Actor): ActorRef = createActor(Props(() ⇒ factory)) + def actorOf(factory: ⇒ Actor): ActorRef = actorOf(Props(() ⇒ factory)) - def createActor(creator: UntypedActorFactory): ActorRef = createActor(Props(() ⇒ creator.create())) + def actorOf(creator: UntypedActorFactory): ActorRef = actorOf(Props(() ⇒ creator.create())) - def createActor(props: RoutedProps): ActorRef = createActor(props, new UUID().toString) + def actorOf(props: RoutedProps): ActorRef = actorOf(props, new UUID().toString) - def createActor(props: RoutedProps, address: String): ActorRef = provider.actorOf(props, address) + def actorOf(props: RoutedProps, address: String): ActorRef = provider.actorOf(props, address) def findActor(address: String): Option[ActorRef] = provider.actorFor(address) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index d37b3f8c16..ea23e77bd7 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -256,7 +256,7 @@ class TypedActor(val app: AkkaApplication) { } val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)(timeout)).asInstanceOf[T] proxyVar.set(proxy) // Chicken and egg situation we needed to solve, set the proxy so that we can set the self-reference inside each receive - val ref = app.createActor(props) + val ref = app.actorOf(props) actorVar.set(ref) //Make sure the InvocationHandler gets ahold of the actor reference, this is not a problem since the proxy hasn't escaped this method yet proxyVar.get } diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 8464759f60..f8441500de 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -1761,7 +1761,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { } } - def createActorRefToUseForReplay(snapshotAsBytes: Option[Array[Byte]], actorAddress: String, newActorRef: LocalActorRef): ActorRef = { + def actorOfRefToUseForReplay(snapshotAsBytes: Option[Array[Byte]], actorAddress: String, newActorRef: LocalActorRef): ActorRef = { snapshotAsBytes match { // we have a new actor ref - the snapshot @@ -1816,7 +1816,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { val (snapshotAsBytes, entriesAsBytes) = readonlyTxLog.latestSnapshotAndSubsequentEntries // deserialize and restore actor snapshot. This call will automatically recreate a transaction log. - val actorRef = createActorRefToUseForReplay(snapshotAsBytes, actorAddress, newActorRef) + val actorRef = actorOfRefToUseForReplay(snapshotAsBytes, actorAddress, newActorRef) // deserialize the messages val messages: Vector[AnyRef] = deserializeMessages(entriesAsBytes) diff --git a/akka-docs/intro/code/tutorials/first/Pi.scala b/akka-docs/intro/code/tutorials/first/Pi.scala index 1e3033055d..57e5b3558e 100644 --- a/akka-docs/intro/code/tutorials/first/Pi.scala +++ b/akka-docs/intro/code/tutorials/first/Pi.scala @@ -68,10 +68,10 @@ object Pi extends App { //#create-workers // create the workers - val workers = Vector.fill(nrOfWorkers)(app.createActor[Worker]) + val workers = Vector.fill(nrOfWorkers)(app.actorOf[Worker]) // wrap them with a load-balancing router - val router = app.createActor(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi") + val router = app.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi") //#create-workers //#master-receive @@ -121,7 +121,7 @@ object Pi extends App { val latch = new CountDownLatch(1) // create the master - val master = app.createActor(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)) + val master = app.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)) // start the calculation master ! Calculate diff --git a/akka-docs/scala/code/ActorDocSpec.scala b/akka-docs/scala/code/ActorDocSpec.scala index c263b02ea1..a8998d7691 100644 --- a/akka-docs/scala/code/ActorDocSpec.scala +++ b/akka-docs/scala/code/ActorDocSpec.scala @@ -22,9 +22,9 @@ class MyActor extends Actor { class ActorDocSpec extends AkkaSpec { - "creating actor with AkkaSpec.createActor" in { + "creating actor with AkkaSpec.actorOf" in { //#creating-actorOf - val myActor = createActor[MyActor] + val myActor = actorOf[MyActor] //#creating-actorOf // testing the actor @@ -58,7 +58,7 @@ class ActorDocSpec extends AkkaSpec { //#creating-constructor // allows passing in arguments to the MyActor constructor - val myActor = createActor(new MyActor("...")) + val myActor = actorOf(new MyActor("...")) //#creating-constructor myActor.stop() diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index 574ab6a18e..965060bd10 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -41,7 +41,7 @@ class Remote(val app: AkkaApplication) extends RemoteService { // FIXME configure computeGridDispatcher to what? val computeGridDispatcher = app.dispatcherFactory.newDispatcher("akka:compute-grid").build - private[remote] lazy val remoteDaemonSupervisor = app.createActor(Props( + private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props( OneForOneStrategy(List(classOf[Exception]), None, None))) // is infinite restart what we want? private[remote] lazy val remoteDaemon = @@ -51,7 +51,7 @@ class Remote(val app: AkkaApplication) extends RemoteService { givenAddress = remoteDaemonServiceName, systemService = true) - private[remote] lazy val remoteClientLifeCycleHandler = app.createActor(Props(new Actor { + private[remote] lazy val remoteClientLifeCycleHandler = app.actorOf(Props(new Actor { def receive = { case RemoteClientError(cause, client, address) ⇒ client.shutdownClientModule() case RemoteClientDisconnected(client, address) ⇒ client.shutdownClientModule() @@ -141,7 +141,7 @@ class RemoteDaemon(val remote: Remote) extends Actor { } val actorAddress = message.getActorAddress - val newActorRef = app.createActor(Props(creator = actorFactory), actorAddress) + val newActorRef = app.actorOf(Props(creator = actorFactory), actorAddress) remote.server.register(actorAddress, newActorRef) diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala index 519493704b..b1274ef1c2 100644 --- a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala +++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala @@ -942,7 +942,7 @@ class RemoteServerHandler( val actorRef = try { - createActor(actorInfo, channel) + actorOf(actorInfo, channel) } catch { case e: SecurityException ⇒ app.eventHandler.error(e, this, e.getMessage) @@ -998,7 +998,7 @@ class RemoteServerHandler( * * Does not start the actor. */ - private def createActor(actorInfo: ActorInfoProtocol, channel: Channel): ActorRef = { + private def actorOf(actorInfo: ActorInfoProtocol, channel: Channel): ActorRef = { val uuid = actorInfo.getUuid val address = actorInfo.getAddress diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala index 9e14609a6b..70c9bf99ef 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala @@ -49,7 +49,7 @@ class DirectRoutedRemoteActorMultiJvmNode2 extends AkkaRemoteSpec { barrier("start") - val actor = app.createActor[SomeActor]("service-hello") + val actor = app.actorOf[SomeActor]("service-hello") actor.isInstanceOf[RoutedActorRef] must be(true) val result = (actor ? "identify").get diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala index 0de114f4bb..c38009cbeb 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala @@ -48,7 +48,7 @@ class NewRemoteActorMultiJvmNode2 extends AkkaRemoteSpec { barrier("start") - val actor = app.createActor[SomeActor]("service-hello") + val actor = app.actorOf[SomeActor]("service-hello") val result = (actor ? "identify").get result must equal("node1") diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala index c3688ae178..e7b6139b7a 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala @@ -70,7 +70,7 @@ class RandomRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec { remote.start() barrier("start") - val actor = app.createActor[SomeActor]("service-hello") + val actor = app.actorOf[SomeActor]("service-hello") actor.isInstanceOf[RoutedActorRef] must be(true) val connectionCount = NrOfNodes - 1 diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala index 1a6a17041d..7bc65fe7db 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala @@ -70,7 +70,7 @@ class RoundRobinRoutedRemoteActorMultiJvmNode4 extends AkkaRemoteSpec { remote.start() barrier("start") - val actor = app.createActor[SomeActor]("service-hello") + val actor = app.actorOf[SomeActor]("service-hello") actor.isInstanceOf[RoutedActorRef] must be(true) val connectionCount = NrOfNodes - 1 diff --git a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala index d3326d85d6..64a19296da 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnBecome.scala @@ -126,11 +126,11 @@ object DiningHakkers { val app = AkkaApplication() def run { //Create 5 chopsticks - val chopsticks = for (i ← 1 to 5) yield app.createActor(new Chopstick("Chopstick " + i)) + val chopsticks = for (i ← 1 to 5) yield app.actorOf(new Chopstick("Chopstick " + i)) //Create 5 awesome hakkers and assign them their left and right chopstick val hakkers = for { (name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex - } yield app.createActor(new Hakker(name, chopsticks(i), chopsticks((i + 1) % 5))) + } yield app.actorOf(new Hakker(name, chopsticks(i), chopsticks((i + 1) % 5))) //Signal all hakkers that they should start thinking, and watch the show hakkers.foreach(_ ! Think) diff --git a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala index 943d15412b..937bb2488c 100644 --- a/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala +++ b/akka-samples/akka-sample-fsm/src/main/scala/DiningHakkersOnFsm.scala @@ -168,11 +168,11 @@ object DiningHakkersOnFsm { def run = { // Create 5 chopsticks - val chopsticks = for (i ← 1 to 5) yield app.createActor(new Chopstick("Chopstick " + i)) + val chopsticks = for (i ← 1 to 5) yield app.actorOf(new Chopstick("Chopstick " + i)) // Create 5 awesome fsm hakkers and assign them their left and right chopstick val hakkers = for { (name, i) ← List("Ghosh", "Bonér", "Klang", "Krasser", "Manie").zipWithIndex - } yield app.createActor(new FSMHakker(name, chopsticks(i), chopsticks((i + 1) % 5))) + } yield app.actorOf(new FSMHakker(name, chopsticks(i), chopsticks((i + 1) % 5))) hakkers.foreach(_ ! Think) } diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala index 33152d5aff..a9ae2a63c4 100644 --- a/akka-stm/src/main/scala/akka/agent/Agent.scala +++ b/akka-stm/src/main/scala/akka/agent/Agent.scala @@ -95,7 +95,7 @@ object Agent { */ class Agent[T](initialValue: T, app: AkkaApplication) { private[akka] val ref = Ref(initialValue) - private[akka] val updater = app.createActor(Props(new AgentUpdater(this))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow? + private[akka] val updater = app.actorOf(Props(new AgentUpdater(this))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow? /** * Read the internal state of the agent. @@ -154,7 +154,7 @@ class Agent[T](initialValue: T, app: AkkaApplication) { send((value: T) ⇒ { suspend() val pinnedDispatcher = new PinnedDispatcher(app, null, "agent-send-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) - val threadBased = app.createActor(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) + val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) threadBased ! Update(f) value }) @@ -172,7 +172,7 @@ class Agent[T](initialValue: T, app: AkkaApplication) { send((value: T) ⇒ { suspend() val pinnedDispatcher = new PinnedDispatcher(app, null, "agent-alter-off", UnboundedMailbox(), app.AkkaConfig.ActorTimeoutMillis) - val threadBased = app.createActor(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) + val threadBased = app.actorOf(Props(new ThreadBasedAgentUpdater(this)).withDispatcher(pinnedDispatcher)) result completeWith threadBased.?(Update(f), timeout).asInstanceOf[Future[T]] value }) diff --git a/akka-stm/src/test/java/akka/stm/example/EitherOrElseExample.java b/akka-stm/src/test/java/akka/stm/example/EitherOrElseExample.java index efc585539b..bb893af77d 100644 --- a/akka-stm/src/test/java/akka/stm/example/EitherOrElseExample.java +++ b/akka-stm/src/test/java/akka/stm/example/EitherOrElseExample.java @@ -15,7 +15,7 @@ public class EitherOrElseExample { final Ref left = new Ref(100); final Ref right = new Ref(100); - ActorRef brancher = application.createActor(new Props().withCreator(Brancher.class)); + ActorRef brancher = application.actorOf(new Props().withCreator(Brancher.class)); brancher.tell(new Branch(left, right, 500)); diff --git a/akka-stm/src/test/java/akka/stm/example/RetryExample.java b/akka-stm/src/test/java/akka/stm/example/RetryExample.java index f6e3aa89b6..70471e40b1 100644 --- a/akka-stm/src/test/java/akka/stm/example/RetryExample.java +++ b/akka-stm/src/test/java/akka/stm/example/RetryExample.java @@ -15,7 +15,7 @@ public class RetryExample { final Ref account1 = new Ref(100.0); final Ref account2 = new Ref(100.0); - ActorRef transferer = application.createActor(new Props().withCreator(Transferer.class)); + ActorRef transferer = application.actorOf(new Props().withCreator(Transferer.class)); transferer.tell(new Transfer(account1, account2, 500.0)); // Transferer: not enough money - retrying diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java index dc39f6961d..229722f1e5 100644 --- a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java +++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java @@ -14,8 +14,8 @@ public class UntypedCoordinatedExample { AkkaApplication application = new AkkaApplication("UntypedCoordinatedExample"); - ActorRef counter1 = application.createActor(new Props().withCreator(UntypedCoordinatedCounter.class)); - ActorRef counter2 = application.createActor(new Props().withCreator(UntypedCoordinatedCounter.class)); + ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class)); + ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCoordinatedCounter.class)); counter1.tell(new Coordinated(new Increment(counter2))); diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java b/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java index be23b0e545..5884077b28 100644 --- a/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java +++ b/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java @@ -13,8 +13,8 @@ public class UntypedTransactorExample { AkkaApplication application = new AkkaApplication("UntypedTransactorExample"); - ActorRef counter1 = application.createActor(new Props().withCreator(UntypedCounter.class)); - ActorRef counter2 = application.createActor(new Props().withCreator(UntypedCounter.class)); + ActorRef counter1 = application.actorOf(new Props().withCreator(UntypedCounter.class)); + ActorRef counter2 = application.actorOf(new Props().withCreator(UntypedCounter.class)); counter1.tell(new Increment(counter2)); diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java index 0e0daa9bb1..6eeb1546b5 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java @@ -42,14 +42,14 @@ public class UntypedCoordinatedIncrementTest { counters = new ArrayList(); for (int i = 1; i <= numCounters; i++) { final String name = "counter" + i; - ActorRef counter = application.createActor(new Props().withCreator(new UntypedActorFactory() { + ActorRef counter = application.actorOf(new Props().withCreator(new UntypedActorFactory() { public UntypedActor create() { return new UntypedCoordinatedCounter(name); } })); counters.add(counter); } - failer = application.createActor(new Props().withCreator(UntypedFailer.class)); + failer = application.actorOf(new Props().withCreator(UntypedFailer.class)); } @Test public void incrementAllCountersWithSuccessfulTransaction() { diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java index 938ac39ae8..8dc2c2beae 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java @@ -41,14 +41,14 @@ public class UntypedTransactorTest { counters = new ArrayList(); for (int i = 1; i <= numCounters; i++) { final String name = "counter" + i; - ActorRef counter = application.createActor(new Props().withCreator(new UntypedActorFactory() { + ActorRef counter = application.actorOf(new Props().withCreator(new UntypedActorFactory() { public UntypedActor create() { return new UntypedCounter(name); } })); counters.add(counter); } - failer = application.createActor(new Props().withCreator(UntypedFailer.class)); + failer = application.actorOf(new Props().withCreator(UntypedFailer.class)); } @Test public void incrementAllCountersWithSuccessfulTransaction() { diff --git a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala index b4507b4534..38d0496f43 100644 --- a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala +++ b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala @@ -60,16 +60,16 @@ class CoordinatedIncrementSpec extends AkkaSpec with BeforeAndAfterAll { val numCounters = 5 - def createActors = { - def createCounter(i: Int) = app.createActor(Props(new Counter("counter" + i))) + def actorOfs = { + def createCounter(i: Int) = app.actorOf(Props(new Counter("counter" + i))) val counters = (1 to numCounters) map createCounter - val failer = app.createActor(Props(new Failer)) + val failer = app.actorOf(Props(new Failer)) (counters, failer) } "Coordinated increment" should { "increment all counters by one with successful transactions" in { - val (counters, failer) = createActors + val (counters, failer) = actorOfs val coordinated = Coordinated() counters(0) ! coordinated(Increment(counters.tail)) coordinated.await @@ -86,7 +86,7 @@ class CoordinatedIncrementSpec extends AkkaSpec with BeforeAndAfterAll { EventFilter[CoordinatedTransactionException], EventFilter[ActorTimeoutException]) app.eventHandler.notify(TestEvent.Mute(ignoreExceptions)) - val (counters, failer) = createActors + val (counters, failer) = actorOfs val coordinated = Coordinated() counters(0) ! Coordinated(Increment(counters.tail :+ failer)) coordinated.await diff --git a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala index 201b34904a..29a1bd82e3 100644 --- a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala +++ b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala @@ -105,10 +105,10 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll { val numCounters = 2 - def createActors = { - def createCounter(i: Int) = app.createActor(Props(new FickleCounter("counter" + i))) + def actorOfs = { + def createCounter(i: Int) = app.actorOf(Props(new FickleCounter("counter" + i))) val counters = (1 to numCounters) map createCounter - val coordinator = app.createActor(Props(new Coordinator("coordinator"))) + val coordinator = app.actorOf(Props(new Coordinator("coordinator"))) (counters, coordinator) } @@ -119,7 +119,7 @@ class FickleFriendsSpec extends AkkaSpec with BeforeAndAfterAll { EventFilter[CoordinatedTransactionException], EventFilter[ActorTimeoutException]) app.eventHandler.notify(TestEvent.Mute(ignoreExceptions)) - val (counters, coordinator) = createActors + val (counters, coordinator) = actorOfs val latch = new CountDownLatch(1) coordinator ! FriendlyIncrement(counters, latch) latch.await // this could take a while diff --git a/akka-stm/src/test/scala/transactor/TransactorSpec.scala b/akka-stm/src/test/scala/transactor/TransactorSpec.scala index 1ebb79d81c..32b5034847 100644 --- a/akka-stm/src/test/scala/transactor/TransactorSpec.scala +++ b/akka-stm/src/test/scala/transactor/TransactorSpec.scala @@ -84,9 +84,9 @@ class TransactorSpec extends AkkaSpec { val numCounters = 5 def createTransactors = { - def createCounter(i: Int) = app.createActor(Props(new Counter("counter" + i))) + def createCounter(i: Int) = app.actorOf(Props(new Counter("counter" + i))) val counters = (1 to numCounters) map createCounter - val failer = app.createActor(Props(new Failer)) + val failer = app.actorOf(Props(new Failer)) (counters, failer) } @@ -124,7 +124,7 @@ class TransactorSpec extends AkkaSpec { "Transactor" should { "be usable without overriding normally" in { - val transactor = app.createActor(Props(new Setter)) + val transactor = app.actorOf(Props(new Setter)) val ref = Ref(0) val latch = TestLatch(1) transactor ! Set(ref, 5, latch) diff --git a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala index 3eadab4af6..e08f3b7d9a 100644 --- a/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/AkkaSpec.scala @@ -15,15 +15,15 @@ abstract class AkkaSpec(_application: AkkaApplication = AkkaApplication()) def this(config: Configuration) = this(new AkkaApplication(getClass.getSimpleName, AkkaApplication.defaultConfig ++ config)) - def createActor(props: Props): ActorRef = app.createActor(props) + def actorOf(props: Props): ActorRef = app.actorOf(props) - def createActor[T <: Actor](clazz: Class[T]): ActorRef = createActor(Props(clazz)) + def actorOf[T <: Actor](clazz: Class[T]): ActorRef = actorOf(Props(clazz)) - def createActor[T <: Actor: Manifest]: ActorRef = createActor(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]]) + def actorOf[T <: Actor: Manifest]: ActorRef = actorOf(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]]) - def createActor[T <: Actor](factory: ⇒ T): ActorRef = createActor(Props(factory)) + def actorOf[T <: Actor](factory: ⇒ T): ActorRef = actorOf(Props(factory)) def spawn(body: ⇒ Unit)(implicit dispatcher: MessageDispatcher) { - createActor(Props(ctx ⇒ { case "go" ⇒ try body finally ctx.self.stop() }).withDispatcher(dispatcher)) ! "go" + actorOf(Props(ctx ⇒ { case "go" ⇒ try body finally ctx.self.stop() }).withDispatcher(dispatcher)) ! "go" } } \ No newline at end of file diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index c147a496e8..4546c60a85 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -119,7 +119,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach { "used with ActorRef" in { val a = TestActorRef(Props(new Actor { - val nested = context.createActor(Props(self ⇒ { case _ ⇒ })) + val nested = context.actorOf(Props(self ⇒ { case _ ⇒ })) def receive = { case _ ⇒ reply(nested) } })) a must not be (null) diff --git a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java index 180149ece2..8db364396f 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java +++ b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java @@ -110,11 +110,11 @@ public class Pi { LinkedList workers = new LinkedList(); for (int i = 0; i < nrOfWorkers; i++) { - ActorRef worker = app.createActor(Worker.class); + ActorRef worker = app.actorOf(Worker.class); workers.add(worker); } - router = app.createActor(new RoutedProps().withRoundRobinRouter().withLocalConnections(workers), "pi"); + router = app.actorOf(new RoutedProps().withRoundRobinRouter().withLocalConnections(workers), "pi"); } // message handler @@ -168,7 +168,7 @@ public class Pi { final CountDownLatch latch = new CountDownLatch(1); // create the master - ActorRef master = app.createActor(new UntypedActorFactory() { + ActorRef master = app.actorOf(new UntypedActorFactory() { public UntypedActor create() { return new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch); } diff --git a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala index 6aab7ade00..1150538109 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala @@ -58,10 +58,10 @@ object Pi extends App { var start: Long = _ // create the workers - val workers = Vector.fill(nrOfWorkers)(app.createActor[Worker]) + val workers = Vector.fill(nrOfWorkers)(app.actorOf[Worker]) // wrap them with a load-balancing router - val router = app.createActor(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi") + val router = app.actorOf(RoutedProps().withRoundRobinRouter.withLocalConnections(workers), "pi") // message handler def receive = { @@ -104,7 +104,7 @@ object Pi extends App { val latch = new CountDownLatch(1) // create the master - val master = app.createActor(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)) + val master = app.actorOf(new Master(nrOfWorkers, nrOfMessages, nrOfElements, latch)) // start the calculation master ! Calculate diff --git a/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java b/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java index 525373b5ad..8b1e924101 100644 --- a/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java +++ b/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java @@ -103,11 +103,11 @@ public class Pi { LinkedList workers = new LinkedList(); for (int i = 0; i < nrOfWorkers; i++) { - ActorRef worker = app.createActor(Worker.class); + ActorRef worker = app.actorOf(Worker.class); workers.add(worker); } - router = app.createActor(new RoutedProps().withRoundRobinRouter().withLocalConnections(workers), "pi"); + router = app.actorOf(new RoutedProps().withRoundRobinRouter().withLocalConnections(workers), "pi"); } @Override @@ -163,7 +163,7 @@ public class Pi { public void calculate(final int nrOfWorkers, final int nrOfElements, final int nrOfMessages) throws Exception { // create the master - ActorRef master = app.createActor(new UntypedActorFactory() { + ActorRef master = app.actorOf(new UntypedActorFactory() { public UntypedActor create() { return new Master(nrOfWorkers, nrOfMessages, nrOfElements); } diff --git a/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala index dec9d7268e..84541abc8a 100644 --- a/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala @@ -53,10 +53,10 @@ object Pi extends App { var nrOfResults: Int = _ // create the workers - val workers = Vector.fill(nrOfWorkers)(app.createActor[Worker]) + val workers = Vector.fill(nrOfWorkers)(app.actorOf[Worker]) // wrap them with a load-balancing router - val router = app.createActor(RoutedProps( + val router = app.actorOf(RoutedProps( routerFactory = () ⇒ new RoundRobinRouter, connectionManager = new LocalConnectionManager(workers)), "pi") @@ -101,7 +101,7 @@ object Pi extends App { // ================== def calculate(nrOfWorkers: Int, nrOfElements: Int, nrOfMessages: Int) { // create the master - val master = app.createActor(new Master(nrOfWorkers, nrOfElements, nrOfMessages)) + val master = app.actorOf(new Master(nrOfWorkers, nrOfElements, nrOfMessages)) //start the calculation val start = now From 01efcd7b500c03c30213ab013086793b4454d9cd Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Tue, 18 Oct 2011 19:14:42 +0200 Subject: [PATCH 17/33] Removing ActorCell.ref (use ActorCell.self instead), introducing Props.randomAddress which will use the toString of the uuid of the actor ref as address, bypassing deployer for actors with 'randomAddress' since it isn't possible to know what the address will be anyway, removing Address.validate since it serves no useful purpose, removing guard.withGuard in MessageDispatcher in favor of the less costly lock try-finally unlock strategy --- .../akka/actor/dispatch/ActorModelSpec.scala | 10 ++++----- .../src/main/scala/akka/actor/ActorCell.scala | 4 +--- .../src/main/scala/akka/actor/ActorRef.scala | 12 +++++------ .../scala/akka/actor/ActorRefProvider.scala | 16 ++++++++------ .../src/main/scala/akka/actor/Deployer.scala | 8 ++++--- .../scala/akka/actor/DeploymentConfig.scala | 4 +--- .../src/main/scala/akka/actor/Props.scala | 1 + .../akka/dispatch/AbstractDispatcher.scala | 21 ++++++++++++------- .../main/scala/akka/event/EventHandler.scala | 2 +- .../main/scala/akka/camel/TypedCamel.scala | 4 ++-- .../main/scala/akka/camel/CamelService.scala | 6 +++--- .../src/main/scala/akka/cluster/Cluster.scala | 8 +++---- .../actor/mailbox/DurableMailboxSpec.scala | 4 ++-- .../akka/remote/NetworkEventStream.scala | 2 +- .../main/scala/akka/remote/RemoteDaemon.scala | 8 +++---- .../serialization/SerializationProtocol.scala | 2 +- .../serialization/ActorSerializeSpec.scala | 8 +++---- .../scala/akka/testkit/TestActorRef.scala | 6 +++--- .../main/scala/akka/testkit/TestFSMRef.scala | 2 +- 19 files changed, 69 insertions(+), 59 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index 64f594b1e2..b8a672aaf1 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -108,27 +108,27 @@ object ActorModelSpec { } abstract override def suspend(actor: ActorCell) { - getStats(actor.ref).suspensions.incrementAndGet() + getStats(actor.self).suspensions.incrementAndGet() super.suspend(actor) } abstract override def resume(actor: ActorCell) { super.resume(actor) - getStats(actor.ref).resumes.incrementAndGet() + getStats(actor.self).resumes.incrementAndGet() } protected[akka] abstract override def register(actor: ActorCell) { - getStats(actor.ref).registers.incrementAndGet() + getStats(actor.self).registers.incrementAndGet() super.register(actor) } protected[akka] abstract override def unregister(actor: ActorCell) { - getStats(actor.ref).unregisters.incrementAndGet() + getStats(actor.self).unregisters.incrementAndGet() super.unregister(actor) } protected[akka] abstract override def dispatch(invocation: Envelope) { - val stats = getStats(invocation.receiver.ref) + val stats = getStats(invocation.receiver.self) stats.msgsReceived.incrementAndGet() super.dispatch(invocation) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index f010e06045..268beb890a 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -227,8 +227,6 @@ private[akka] class ActorCell( var actor: Actor = _ //FIXME We can most probably make this just a regular reference to Actor - def ref: ActorRef with ScalaActorRef = self - def uuid: Uuid = self.uuid @inline @@ -245,7 +243,7 @@ private[akka] class ActorCell( if (props.supervisor.isDefined) { props.supervisor.get match { case l: LocalActorRef ⇒ - l.underlying.dispatcher.systemDispatch(l.underlying, akka.dispatch.Supervise(self)) + l.underlying.dispatcher.systemDispatch(l.underlying, akka.dispatch.Supervise(self)) //FIXME TODO Support all ActorRefs? case other ⇒ throw new UnsupportedOperationException("Supervision failure: " + other + " cannot be a supervisor, only LocalActorRefs can") } } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 05091d414b..41e8a6aeac 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -148,21 +148,21 @@ abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyCha class LocalActorRef private[akka] ( app: AkkaApplication, props: Props, - givenAddress: String, + givenAddress: String, //Never refer to this internally instead use "address" val systemService: Boolean = false, private[akka] val uuid: Uuid = newUuid, receiveTimeout: Option[Long] = None, hotswap: Stack[PartialFunction[Any, Unit]] = Props.noHotSwap) extends ActorRef with ScalaActorRef { + final val address: String = givenAddress match { + case null | Props.randomAddress ⇒ uuid.toString + case other ⇒ other + } + private[this] val actorCell = new ActorCell(app, this, props, receiveTimeout, hotswap) actorCell.start() - final def address: String = givenAddress match { - case null | "" ⇒ uuid.toString - case other ⇒ other - } - /** * Is the actor shut down? * If this method returns true, it will never return false again, but if it returns false, you cannot be sure if it's alive still (race condition) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 5cbc7cc8c0..7133c82fcd 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -43,7 +43,7 @@ trait ActorRefFactory { def dispatcher: MessageDispatcher - def actorOf(props: Props): ActorRef = actorOf(props, new UUID().toString) + def actorOf(props: Props): ActorRef = actorOf(props, Props.randomAddress) /* * TODO this will have to go at some point, because creating two actors with @@ -63,7 +63,7 @@ trait ActorRefFactory { def actorOf(creator: UntypedActorFactory): ActorRef = actorOf(Props(() ⇒ creator.create())) - def actorOf(props: RoutedProps): ActorRef = actorOf(props, new UUID().toString) + def actorOf(props: RoutedProps): ActorRef = actorOf(props, Props.randomAddress) def actorOf(props: RoutedProps, address: String): ActorRef = provider.actorOf(props, address) @@ -94,15 +94,19 @@ class LocalActorRefProvider(val app: AkkaApplication) extends ActorRefProvider { private[akka] def evict(address: String): Boolean = actors.remove(address) ne null private[akka] def actorOf(props: Props, address: String, systemService: Boolean): ActorRef = { - - if (systemService) new LocalActorRef(app, props, address, systemService = true) - else { + if ((address eq null) || address == Props.randomAddress) { + val actor = new LocalActorRef(app, props, address, systemService = true) + actors.putIfAbsent(actor.address, actor) match { + case null ⇒ actor + case other ⇒ throw new IllegalStateException("Same uuid generated twice for: " + actor + " and " + other) + } + } else { val newFuture = Promise[ActorRef](5000)(app.dispatcher) // FIXME is this proper timeout? actors.putIfAbsent(address, newFuture) match { case null ⇒ val actor: ActorRef = try { - app.deployer.lookupDeploymentFor(address) match { // see if the deployment already exists, if so use it, if not create actor + (if (systemService) None else app.deployer.lookupDeploymentFor(address)) match { // see if the deployment already exists, if so use it, if not create actor // create a local actor case None | Some(DeploymentConfig.Deploy(_, _, DeploymentConfig.Direct, _, _, DeploymentConfig.LocalScope)) ⇒ diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 7533c55579..82af1d9c56 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -19,6 +19,10 @@ trait ActorDeployer { private[akka] def shutdown(): Unit //TODO Why should we have "shutdown", should be crash only? private[akka] def deploy(deployment: Deploy): Unit private[akka] def lookupDeploymentFor(address: String): Option[Deploy] + def lookupDeployment(address: String): Option[Deploy] = address match { + case null | Props.`randomAddress` ⇒ None + case some ⇒ lookupDeploymentFor(some) + } private[akka] def deploy(deployment: Seq[Deploy]): Unit = deployment foreach (deploy(_)) } @@ -256,9 +260,7 @@ class Deployer(val app: AkkaApplication) extends ActorDeployer { // akka.actor.deployment.
.cluster // -------------------------------- addressConfig.getSection("cluster") match { - case None ⇒ - Some(Deploy(address, recipe, router, nrOfInstances, NoOpFailureDetector, LocalScope)) // deploy locally - + case None ⇒ None case Some(clusterConfig) ⇒ // -------------------------------- diff --git a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala index 5e7bd9a2f7..fdafe85f91 100644 --- a/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala +++ b/akka-actor/src/main/scala/akka/actor/DeploymentConfig.scala @@ -20,9 +20,7 @@ object DeploymentConfig { routing: Routing = Direct, nrOfInstances: NrOfInstances = ZeroNrOfInstances, failureDetector: FailureDetector = NoOpFailureDetector, - scope: Scope = LocalScope) { - Address.validate(address) - } + scope: Scope = LocalScope) // -------------------------------- // --- Actor Recipe diff --git a/akka-actor/src/main/scala/akka/actor/Props.scala b/akka-actor/src/main/scala/akka/actor/Props.scala index 7aedfbdd0a..45528091f7 100644 --- a/akka-actor/src/main/scala/akka/actor/Props.scala +++ b/akka-actor/src/main/scala/akka/actor/Props.scala @@ -22,6 +22,7 @@ object Props { final val defaultFaultHandler: FaultHandlingStrategy = OneForOneStrategy(classOf[Exception] :: Nil, None, None) final val defaultSupervisor: Option[ActorRef] = None final val noHotSwap: Stack[Actor.Receive] = Stack.empty + final val randomAddress: String = "" /** * The default Props instance, uses the settings from the Props object starting with default* diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 0d4166ab8b..48b72d576b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -113,7 +113,8 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable * Detaches the specified actor instance from this dispatcher */ final def detach(actor: ActorCell) { - guard withGuard { + guard.lock.lock() + try { unregister(actor) if (_tasks.get == 0 && _actors.get == 0) { shutdownSchedule match { @@ -125,11 +126,15 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable case RESCHEDULED ⇒ //Already marked for reschedule } } - } + } finally { guard.lock.unlock() } } protected final def startIfUnstarted() { - if (active.isOff) guard withGuard { active.switchOn { start() } } + if (active.isOff) { + guard.lock.lock() + try { active.switchOn { start() } } + finally { guard.lock.unlock() } + } } protected[akka] final def dispatchTask(block: () ⇒ Unit) { @@ -146,7 +151,8 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable private val taskCleanup: () ⇒ Unit = () ⇒ if (_tasks.decrementAndGet() == 0) { - guard withGuard { + guard.lock.lock() + try { if (_tasks.get == 0 && _actors.get == 0) { shutdownSchedule match { case UNSCHEDULED ⇒ @@ -157,7 +163,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable case RESCHEDULED ⇒ //Already marked for reschedule } } - } + } finally { guard.lock.unlock() } } /** @@ -206,7 +212,8 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable private val shutdownAction = new Runnable { def run() { - guard withGuard { + guard.lock.lock() + try { shutdownSchedule match { case RESCHEDULED ⇒ shutdownSchedule = SCHEDULED @@ -220,7 +227,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable shutdownSchedule = UNSCHEDULED case UNSCHEDULED ⇒ //Do nothing } - } + } finally { guard.lock.unlock() } } } diff --git a/akka-actor/src/main/scala/akka/event/EventHandler.scala b/akka-actor/src/main/scala/akka/event/EventHandler.scala index 5382e95076..a668c16249 100644 --- a/akka-actor/src/main/scala/akka/event/EventHandler.scala +++ b/akka-actor/src/main/scala/akka/event/EventHandler.scala @@ -204,7 +204,7 @@ class EventHandler(app: AkkaApplication) extends ListenerManagement { defaultListeners foreach { listenerName ⇒ try { ReflectiveAccess.getClassFor[Actor](listenerName) match { - case Right(actorClass) ⇒ addListener(new LocalActorRef(app, Props(actorClass).withDispatcher(EventHandlerDispatcher), newUuid.toString, systemService = true)) + case Right(actorClass) ⇒ addListener(new LocalActorRef(app, Props(actorClass).withDispatcher(EventHandlerDispatcher), Props.randomAddress, systemService = true)) case Left(exception) ⇒ throw exception } } catch { diff --git a/akka-camel-typed/src/main/scala/akka/camel/TypedCamel.scala b/akka-camel-typed/src/main/scala/akka/camel/TypedCamel.scala index b82dfefa7d..2e1d19817b 100644 --- a/akka-camel-typed/src/main/scala/akka/camel/TypedCamel.scala +++ b/akka-camel-typed/src/main/scala/akka/camel/TypedCamel.scala @@ -32,8 +32,8 @@ private[camel] object TypedCamel { * and re-uses the activationTracker of service. */ def onCamelServiceStart(service: CamelService) { - consumerPublisher = new LocalActorRef(Props(new TypedConsumerPublisher(service.activationTracker)), newUuid.toString, true) - publishRequestor = new LocalActorRef(Props(new TypedConsumerPublishRequestor), newUuid.toString, true) + consumerPublisher = new LocalActorRef(Props(new TypedConsumerPublisher(service.activationTracker)), Props.randomAddress, true) + publishRequestor = new LocalActorRef(Props(new TypedConsumerPublishRequestor), Props.randomAddress, true) registerPublishRequestor diff --git a/akka-camel/src/main/scala/akka/camel/CamelService.scala b/akka-camel/src/main/scala/akka/camel/CamelService.scala index 66c89a5381..eeb039cb8d 100644 --- a/akka-camel/src/main/scala/akka/camel/CamelService.scala +++ b/akka-camel/src/main/scala/akka/camel/CamelService.scala @@ -26,9 +26,9 @@ import TypedCamelAccess._ * @author Martin Krasser */ trait CamelService extends Bootable { - private[camel] val activationTracker = new LocalActorRef(Props[ActivationTracker], newUuid.toString, true) - private[camel] val consumerPublisher = new LocalActorRef(Props(new ConsumerPublisher(activationTracker)), newUuid.toString, true) - private[camel] val publishRequestor = new LocalActorRef(Props(new ConsumerPublishRequestor), newUuid.toString, true) + private[camel] val activationTracker = new LocalActorRef(Props[ActivationTracker], Props.randomAddress, true) + private[camel] val consumerPublisher = new LocalActorRef(Props(new ConsumerPublisher(activationTracker)), Props.randomAddress, true) + private[camel] val publishRequestor = new LocalActorRef(Props(new ConsumerPublishRequestor), Props.randomAddress, true) private val serviceEnabled = config.getList("akka.enabled-modules").exists(_ == "camel") diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index f8441500de..0dcd0447aa 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -1860,7 +1860,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { Props( self ⇒ { case f: Function0[_] ⇒ try { f() } finally { self.stop() } - }).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) ! payloadFor(message, classOf[Function0[Unit]]) + }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Function0[Unit]]) } def handle_fun0_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) { @@ -1868,7 +1868,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { Props( self ⇒ { case f: Function0[_] ⇒ try { self.reply(f()) } finally { self.stop() } - }).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) forward payloadFor(message, classOf[Function0[Any]]) + }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Function0[Any]]) } def handle_fun1_arg_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) { @@ -1876,7 +1876,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { Props( self ⇒ { case (fun: Function[_, _], param: Any) ⇒ try { fun.asInstanceOf[Any ⇒ Unit].apply(param) } finally { self.stop() } - }).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) + }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) } def handle_fun1_arg_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) { @@ -1884,7 +1884,7 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor { Props( self ⇒ { case (fun: Function[_, _], param: Any) ⇒ try { self.reply(fun.asInstanceOf[Any ⇒ Any](param)) } finally { self.stop() } - }).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) + }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) } def handleFailover(message: RemoteProtocol.RemoteDaemonMessageProtocol) { diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala b/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala index 24d8e471b4..e7f4f2c4b5 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/test/scala/akka/actor/mailbox/DurableMailboxSpec.scala @@ -31,7 +31,7 @@ abstract class DurableMailboxSpec(val backendName: String, val storage: DurableM "should handle reply to ! for 1 message" in { val latch = new CountDownLatch(1) val queueActor = createMailboxTestActor(backendName + " should handle reply to !") - val sender = new LocalActorRef(Props(self ⇒ { case "sum" ⇒ latch.countDown }), newUuid.toString, true) + val sender = new LocalActorRef(Props(self ⇒ { case "sum" ⇒ latch.countDown }), Props.randomAddress, true) queueActor.!("sum")(Some(sender)) latch.await(10, TimeUnit.SECONDS) must be(true) @@ -40,7 +40,7 @@ abstract class DurableMailboxSpec(val backendName: String, val storage: DurableM "should handle reply to ! for multiple messages" in { val latch = new CountDownLatch(5) val queueActor = createMailboxTestActor(backendName + " should handle reply to !") - val sender = new LocalActorRef(Props(self ⇒ { case "sum" ⇒ latch.countDown }), newUuid.toString, true) + val sender = new LocalActorRef(Props(self ⇒ { case "sum" ⇒ latch.countDown }), Props.randomAddress, true) for (i ← 1 to 5) queueActor.!("sum")(Some(sender)) diff --git a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala index 0a0058c525..b67f782c36 100644 --- a/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala +++ b/akka-remote/src/main/scala/akka/remote/NetworkEventStream.scala @@ -65,7 +65,7 @@ class NetworkEventStream(val app: AkkaApplication) { import NetworkEventStream._ private[akka] val channel = app.provider.actorOf( - Props[Channel].copy(dispatcher = app.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")), newUuid.toString, systemService = true) + Props[Channel].copy(dispatcher = app.dispatcherFactory.newPinnedDispatcher("NetworkEventStream")), Props.randomAddress, systemService = true) /** * Registers a network event stream listener (asyncronously). diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index 965060bd10..449badf073 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -177,7 +177,7 @@ class RemoteDaemon(val remote: Remote) extends Actor { Props( context ⇒ { case f: Function0[_] ⇒ try { f() } finally { context.self.stop() } - }).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) ! payloadFor(message, classOf[Function0[Unit]]) + }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Function0[Unit]]) } def handle_fun0_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) { @@ -185,7 +185,7 @@ class RemoteDaemon(val remote: Remote) extends Actor { Props( context ⇒ { case f: Function0[_] ⇒ try { reply(f()) } finally { context.self.stop() } - }).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) forward payloadFor(message, classOf[Function0[Any]]) + }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Function0[Any]]) } def handle_fun1_arg_unit(message: RemoteProtocol.RemoteDaemonMessageProtocol) { @@ -193,7 +193,7 @@ class RemoteDaemon(val remote: Remote) extends Actor { Props( context ⇒ { case (fun: Function[_, _], param: Any) ⇒ try { fun.asInstanceOf[Any ⇒ Unit].apply(param) } finally { context.self.stop() } - }).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) + }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]]) } def handle_fun1_arg_any(message: RemoteProtocol.RemoteDaemonMessageProtocol) { @@ -201,7 +201,7 @@ class RemoteDaemon(val remote: Remote) extends Actor { Props( context ⇒ { case (fun: Function[_, _], param: Any) ⇒ try { reply(fun.asInstanceOf[Any ⇒ Any](param)) } finally { context.self.stop() } - }).copy(dispatcher = computeGridDispatcher), newUuid.toString, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) + }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) } def handleFailover(message: RemoteProtocol.RemoteDaemonMessageProtocol) { diff --git a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala index ae8a204eb6..2e4d177aba 100644 --- a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala @@ -101,7 +101,7 @@ class ActorSerialization(val app: AkkaApplication, remote: RemoteSupport) { l map { m ⇒ remoteActorSerialization.createRemoteMessageProtocolBuilder( - Option(m.receiver.ref), + Option(m.receiver.self), Left(actorRef.uuid), actorRef.address, app.AkkaConfig.ActorTimeoutMillis, diff --git a/akka-remote/src/test/scala/akka/serialization/ActorSerializeSpec.scala b/akka-remote/src/test/scala/akka/serialization/ActorSerializeSpec.scala index 67df328a64..6a31c6eea3 100644 --- a/akka-remote/src/test/scala/akka/serialization/ActorSerializeSpec.scala +++ b/akka-remote/src/test/scala/akka/serialization/ActorSerializeSpec.scala @@ -23,7 +23,7 @@ class ActorSerializeSpec extends AkkaSpec with BeforeAndAfterAll { "Serializable actor" must { "must be able to serialize and de-serialize a stateful actor with a given serializer" ignore { - val actor1 = new LocalActorRef(app, Props[MyJavaSerializableActor], newUuid.toString, systemService = true) + val actor1 = new LocalActorRef(app, Props[MyJavaSerializableActor], Props.randomAddress, systemService = true) (actor1 ? "hello").get must equal("world 1") (actor1 ? "hello").get must equal("world 2") @@ -39,7 +39,7 @@ class ActorSerializeSpec extends AkkaSpec with BeforeAndAfterAll { "must be able to serialize and deserialize a MyStatelessActorWithMessagesInMailbox" ignore { - val actor1 = new LocalActorRef(app, Props[MyStatelessActorWithMessagesInMailbox], newUuid.toString, systemService = true) + val actor1 = new LocalActorRef(app, Props[MyStatelessActorWithMessagesInMailbox], Props.randomAddress, systemService = true) for (i ← 1 to 10) actor1 ! "hello" actor1.underlying.dispatcher.mailboxSize(actor1.underlying) must be > (0) @@ -57,7 +57,7 @@ class ActorSerializeSpec extends AkkaSpec with BeforeAndAfterAll { "must be able to serialize and deserialize a PersonActorWithMessagesInMailbox" ignore { val p1 = Person("debasish ghosh", 25, SerializeSpec.Address("120", "Monroe Street", "Santa Clara", "95050")) - val actor1 = new LocalActorRef(app, Props[PersonActorWithMessagesInMailbox], newUuid.toString, systemService = true) + val actor1 = new LocalActorRef(app, Props[PersonActorWithMessagesInMailbox], Props.randomAddress, systemService = true) (actor1 ! p1) (actor1 ! p1) (actor1 ! p1) @@ -103,7 +103,7 @@ class ActorSerializeSpec extends AkkaSpec with BeforeAndAfterAll { "serialize actor that accepts protobuf message" ignore { "must serialize" ignore { - val actor1 = new LocalActorRef(app, Props[MyActorWithProtobufMessagesInMailbox], newUuid.toString, systemService = true) + val actor1 = new LocalActorRef(app, Props[MyActorWithProtobufMessagesInMailbox], Props.randomAddress, systemService = true) val msg = MyMessage(123, "debasish ghosh", true) val b = ProtobufProtocol.MyMessage.newBuilder.setId(msg.id).setName(msg.name).setStatus(msg.status).build for (i ← 1 to 10) actor1 ! b diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 5bfc4131d6..8794d00c50 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -42,15 +42,15 @@ class TestActorRef[T <: Actor](_app: AkkaApplication, props: Props, address: Str object TestActorRef { - def apply[T <: Actor](factory: ⇒ T)(implicit app: AkkaApplication): TestActorRef[T] = apply[T](Props(factory), new UUID().toString) + def apply[T <: Actor](factory: ⇒ T)(implicit app: AkkaApplication): TestActorRef[T] = apply[T](Props(factory), Props.randomAddress) def apply[T <: Actor](factory: ⇒ T, address: String)(implicit app: AkkaApplication): TestActorRef[T] = apply[T](Props(factory), address) - def apply[T <: Actor](props: Props)(implicit app: AkkaApplication): TestActorRef[T] = apply[T](props, new UUID().toString) + def apply[T <: Actor](props: Props)(implicit app: AkkaApplication): TestActorRef[T] = apply[T](props, Props.randomAddress) def apply[T <: Actor](props: Props, address: String)(implicit app: AkkaApplication): TestActorRef[T] = new TestActorRef(app, props, address) - def apply[T <: Actor](implicit m: Manifest[T], app: AkkaApplication): TestActorRef[T] = apply[T](new UUID().toString) + def apply[T <: Actor](implicit m: Manifest[T], app: AkkaApplication): TestActorRef[T] = apply[T](Props.randomAddress) def apply[T <: Actor](address: String)(implicit m: Manifest[T], app: AkkaApplication): TestActorRef[T] = apply[T](Props({ import ReflectiveAccess.{ createInstance, noParams, noArgs } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala index 5dee240f36..9b3b59c6bd 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestFSMRef.scala @@ -81,7 +81,7 @@ class TestFSMRef[S, D, T <: Actor](app: AkkaApplication, props: Props, address: object TestFSMRef { def apply[S, D, T <: Actor](factory: ⇒ T)(implicit ev: T <:< FSM[S, D], app: AkkaApplication): TestFSMRef[S, D, T] = - new TestFSMRef(app, Props(creator = () ⇒ factory), new UUID().toString) + new TestFSMRef(app, Props(creator = () ⇒ factory), Props.randomAddress) def apply[S, D, T <: Actor](factory: ⇒ T, address: String)(implicit ev: T <:< FSM[S, D], app: AkkaApplication): TestFSMRef[S, D, T] = new TestFSMRef(app, Props(creator = () ⇒ factory), address) From 5c823ad50d85d484a68b407279faeb173bb7c164 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 18 Oct 2011 18:06:17 +0200 Subject: [PATCH 18/33] replace ConcurrentLinkedQueue with single-linked list for Mailbox.systemQueue cost zero when empty, non-blocking, shave off 84 bytes per actor --- .../java/akka/dispatch/AbstractMailbox.java | 8 ++- .../src/main/scala/akka/actor/ActorCell.scala | 26 ++++---- .../akka/dispatch/AbstractDispatcher.scala | 60 ++++++++++++++----- .../akka/dispatch/BalancingDispatcher.scala | 8 +-- .../main/scala/akka/dispatch/Mailbox.scala | 55 +++++++++++++---- 5 files changed, 116 insertions(+), 41 deletions(-) diff --git a/akka-actor/src/main/java/akka/dispatch/AbstractMailbox.java b/akka-actor/src/main/java/akka/dispatch/AbstractMailbox.java index 21d41ac921..0f37dec003 100644 --- a/akka-actor/src/main/java/akka/dispatch/AbstractMailbox.java +++ b/akka-actor/src/main/java/akka/dispatch/AbstractMailbox.java @@ -5,8 +5,14 @@ package akka.dispatch; import java.util.concurrent.atomic.AtomicIntegerFieldUpdater; +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; abstract class AbstractMailbox { private volatile int _status; // not initialized because this is faster: 0 == Open - protected final static AtomicIntegerFieldUpdater updater = AtomicIntegerFieldUpdater.newUpdater(AbstractMailbox.class, "_status"); + protected final static AtomicIntegerFieldUpdater updater = + AtomicIntegerFieldUpdater.newUpdater(AbstractMailbox.class, "_status"); + + private volatile SystemMessage _systemQueue; // not initialized because this is faster + protected final static AtomicReferenceFieldUpdater systemQueueUpdater = + AtomicReferenceFieldUpdater.newUpdater(AbstractMailbox.class, SystemMessage.class, "_systemQueue"); } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 268beb890a..910777685c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -243,6 +243,7 @@ private[akka] class ActorCell( if (props.supervisor.isDefined) { props.supervisor.get match { case l: LocalActorRef ⇒ + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ l.underlying.dispatcher.systemDispatch(l.underlying, akka.dispatch.Supervise(self)) //FIXME TODO Support all ActorRefs? case other ⇒ throw new UnsupportedOperationException("Supervision failure: " + other + " cannot be a supervisor, only LocalActorRefs can") } @@ -251,19 +252,23 @@ private[akka] class ActorCell( dispatcher.attach(this) } + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ def suspend(): Unit = dispatcher.systemDispatch(this, Suspend()) + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ def resume(): Unit = dispatcher.systemDispatch(this, Resume()) - private[akka] def stop(): Unit = - dispatcher.systemDispatch(this, Terminate()) + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ + private[akka] def stop(): Unit = dispatcher.systemDispatch(this, Terminate()) def startsMonitoring(subject: ActorRef): ActorRef = { + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ dispatcher.systemDispatch(this, Link(subject)) subject } def stopsMonitoring(subject: ActorRef): ActorRef = { + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ dispatcher.systemDispatch(this, Unlink(subject)) subject } @@ -421,18 +426,18 @@ private[akka] class ActorCell( val isClosed = mailbox.isClosed //Fence plus volatile read if (!isClosed) { message match { - case Create(_) ⇒ create() - case Recreate(cause, _) ⇒ recreate(cause) - case Link(subject, _) ⇒ + case Create() ⇒ create() + case Recreate(cause) ⇒ recreate(cause) + case Link(subject) ⇒ app.deathWatch.subscribe(self, subject) if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "now monitoring " + subject) - case Unlink(subject, _) ⇒ + case Unlink(subject) ⇒ app.deathWatch.unsubscribe(self, subject) if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopped monitoring " + subject) - case Suspend(_) ⇒ suspend() - case Resume(_) ⇒ resume() - case Terminate(_) ⇒ terminate() - case Supervise(child, _) ⇒ supervise(child) + case Suspend() ⇒ suspend() + case Resume() ⇒ resume() + case Terminate() ⇒ terminate() + case Supervise(child) ⇒ supervise(child) } } } catch { @@ -491,6 +496,7 @@ private[akka] class ActorCell( def handleChildTerminated(child: ActorRef): Unit = _children = props.faultHandler.handleChildTerminated(child, _children) + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause)) def checkReceiveTimeout() { diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 48b72d576b..58502fa334 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -12,6 +12,7 @@ import akka.util.{ Duration, Switch, ReentrantGuard } import java.util.concurrent.ThreadPoolExecutor.{ AbortPolicy, CallerRunsPolicy, DiscardOldestPolicy, DiscardPolicy } import akka.actor._ import akka.AkkaApplication +import scala.annotation.tailrec /** * @author Jonas Bonér @@ -24,17 +25,45 @@ final case class Envelope(val receiver: ActorCell, val message: Any, val channel } } -sealed trait SystemMessage extends PossiblyHarmful { - def next: SystemMessage +object SystemMessage { + @tailrec + final def size(list: SystemMessage, acc: Int = 0): Int = { + if (list eq null) acc else size(list.next, acc + 1) + } + + @tailrec + final def reverse(list: SystemMessage, acc: SystemMessage = null): SystemMessage = { + if (list eq null) acc else { + val next = list.next + list.next = acc + reverse(next, list) + } + } } -case class Create(next: SystemMessage = null) extends SystemMessage -case class Recreate(cause: Throwable, next: SystemMessage = null) extends SystemMessage -case class Suspend(next: SystemMessage = null) extends SystemMessage -case class Resume(next: SystemMessage = null) extends SystemMessage -case class Terminate(next: SystemMessage = null) extends SystemMessage -case class Supervise(child: ActorRef, next: SystemMessage = null) extends SystemMessage -case class Link(subject: ActorRef, next: SystemMessage = null) extends SystemMessage -case class Unlink(subject: ActorRef, next: SystemMessage = null) extends SystemMessage + +/** + * System messages are handled specially: they form their own queue within + * each actor’s mailbox. This queue is encoded in the messages themselves to + * avoid extra allocations and overhead. The next pointer is a normal var, and + * it does not need to be volatile because in the enqueuing method its update + * is immediately succeeded by a volatile write and all reads happen after the + * volatile read in the dequeuing thread. Afterwards, the obtained list of + * system messages is handled in a single thread only and not ever passed around, + * hence no further synchronization is needed. + * + * ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ + */ +sealed trait SystemMessage extends PossiblyHarmful { + var next: SystemMessage = _ +} +case class Create() extends SystemMessage +case class Recreate(cause: Throwable) extends SystemMessage +case class Suspend() extends SystemMessage +case class Resume() extends SystemMessage +case class Terminate() extends SystemMessage +case class Supervise(child: ActorRef) extends SystemMessage +case class Link(subject: ActorRef) extends SystemMessage +case class Unlink(subject: ActorRef) extends SystemMessage final case class TaskInvocation(app: AkkaApplication, function: () ⇒ Unit, cleanup: () ⇒ Unit) extends Runnable { def run() { @@ -85,7 +114,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable override def enqueue(envelope: Envelope) { envelope.channel sendException new ActorKilledException("Actor has been stopped") } override def dequeue() = null override def systemEnqueue(handle: SystemMessage): Unit = () - override def systemDequeue(): SystemMessage = null + override def systemDrain(): SystemMessage = null override def hasMessages = false override def hasSystemMessages = false override def numberOfMessages = 0 @@ -172,6 +201,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable */ protected[akka] def register(actor: ActorCell) { _actors.incrementAndGet() + // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ systemDispatch(actor, Create()) //FIXME should this be here or moved into ActorCell.start perhaps? } @@ -194,10 +224,10 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable protected def cleanUpMailboxFor(actor: ActorCell, mailBox: Mailbox) { if (mailBox.hasSystemMessages) { - var envelope = mailBox.systemDequeue() - while (envelope ne null) { - deadLetterMailbox.systemEnqueue(envelope) - envelope = mailBox.systemDequeue() + var message = mailBox.systemDrain() + while (message ne null) { + deadLetterMailbox.systemEnqueue(message) + message = message.next } } diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 551eea21ce..f38e8d8f54 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -91,10 +91,10 @@ class BalancingDispatcher( protected override def cleanUpMailboxFor(actor: ActorCell, mailBox: Mailbox) { if (mailBox.hasSystemMessages) { - var envelope = mailBox.systemDequeue() - while (envelope ne null) { - deadLetterMailbox.systemEnqueue(envelope) //Send to dead letter queue - envelope = mailBox.systemDequeue() + var messages = mailBox.systemDrain() + while (messages ne null) { + deadLetterMailbox.systemEnqueue(messages) //Send to dead letter queue + messages = messages.next } } } diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index 095a0f9cd5..df730efd8c 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -134,6 +134,12 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag } } + /* + * AtomicReferenceFieldUpdater for system queue + */ + protected final def systemQueueGet: SystemMessage = AbstractMailbox.systemQueueUpdater.get(this) + protected final def systemQueuePut(_old: SystemMessage, _new: SystemMessage): Boolean = AbstractMailbox.systemQueueUpdater.compareAndSet(this, _old, _new) + def shouldBeRegisteredForExecution(hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = status match { case Open | Scheduled ⇒ hasMessageHint || hasSystemMessageHint || hasSystemMessages || hasMessages case Closed ⇒ false @@ -184,10 +190,18 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag } def processAllSystemMessages() { - var nextMessage = systemDequeue() - while (nextMessage ne null) { - actor systemInvoke nextMessage - nextMessage = systemDequeue() + var nextMessage = systemDrain() + try { + while (nextMessage ne null) { + actor systemInvoke nextMessage + nextMessage = nextMessage.next + // don’t ever execute normal message when system message present! + if (nextMessage eq null) nextMessage = systemDrain() + } + } catch { + case e ⇒ + actor.app.eventHandler.error(e, this, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!") + throw e } } @@ -208,22 +222,41 @@ trait MessageQueue { } trait SystemMessageQueue { + /** + * Enqueue a new system message, e.g. by prepending atomically as new head of a single-linked list. + */ def systemEnqueue(message: SystemMessage): Unit - def systemDequeue(): SystemMessage + /** + * Dequeue all messages from system queue and return them as single-linked list. + */ + def systemDrain(): SystemMessage def hasSystemMessages: Boolean } -trait DefaultSystemMessageQueue { self: SystemMessageQueue ⇒ +trait DefaultSystemMessageQueue { self: Mailbox ⇒ - final val systemMessages = new ConcurrentLinkedQueue[SystemMessage]() + @tailrec + final def systemEnqueue(message: SystemMessage): Unit = { + val head = systemQueueGet + /* + * this write is safely published by the compareAndSet contained within + * systemQueuePut; “Intra-Thread Semantics” on page 12 of the JSR133 spec + * guarantees that “head” uses the value obtained from systemQueueGet above. + * Hence, SystemMessage.next does not need to be volatile. + */ + message.next = head + if (!systemQueuePut(head, message)) systemEnqueue(message) + } - def systemEnqueue(message: SystemMessage): Unit = systemMessages offer message + @tailrec + final def systemDrain(): SystemMessage = { + val head = systemQueueGet + if (systemQueuePut(head, null)) SystemMessage.reverse(head) else systemDrain() + } - def systemDequeue(): SystemMessage = systemMessages.poll() - - def hasSystemMessages: Boolean = !systemMessages.isEmpty + def hasSystemMessages: Boolean = systemQueueGet ne null } trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue { From 7d879944ce592afb8a157773b635e1ad9afefa04 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 10:10:03 +0200 Subject: [PATCH 19/33] #1210 - fixing typo --- akka-docs/scala/futures.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-docs/scala/futures.rst b/akka-docs/scala/futures.rst index 77dff1856c..622c26f778 100644 --- a/akka-docs/scala/futures.rst +++ b/akka-docs/scala/futures.rst @@ -179,7 +179,7 @@ This is fine when dealing with a known amount of Actors, but can grow unwieldy i To better explain what happened in the example, ``Future.sequence`` is taking the ``List[Future[Int]]`` and turning it into a ``Future[List[Int]]``. We can then use ``map`` to work with the ``List[Int]`` directly, and we find the sum of the ``List``. -The ``traverse`` method is similar to ``sequence``, but it takes a ``T[A]`` and a function ``T => Future[B]`` to return a ``Future[T[B]]``, where ``T`` is again a subclass of Traversable. For example, to use ``traverse`` to sum the first 100 odd numbers: +The ``traverse`` method is similar to ``sequence``, but it takes a ``T[A]`` and a function ``A => Future[B]`` to return a ``Future[T[B]]``, where ``T`` is again a subclass of Traversable. For example, to use ``traverse`` to sum the first 100 odd numbers: .. code-block:: scala From bde3969f65bad15a9a345edf8512fe445b762c5f Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 11:01:00 +0200 Subject: [PATCH 20/33] #1297 - Fixing two tests that have been failing on Jenkins but working everywhere else --- .../src/test/scala/akka/actor/DeathWatchSpec.scala | 2 +- .../test/scala/akka/actor/dispatch/DispatcherActorSpec.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index ca8fdad334..ece9ec3d61 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -13,7 +13,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende "The Death Watch" must { def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(2 seconds, actorRef + ": Stopped") { - case Terminated(`actorRef`, ex: ActorKilledException) if ex.getMessage == "Stopped" ⇒ true + case Terminated(`actorRef`, ex: ActorKilledException) if ex.getMessage == "Stopped" || ex.getMessage == "Already terminated when linking" ⇒ true } "notify with one Terminated message when an Actor is stopped" in { diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala index 59255bd473..c8f23375c6 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala @@ -73,11 +73,11 @@ class DispatcherActorSpec extends AkkaSpec { val slowOne = actorOf( Props(context ⇒ { - case "hogexecutor" ⇒ start.await + case "hogexecutor" ⇒ context.channel ! "OK"; start.await case "ping" ⇒ if (works.get) latch.countDown() }).withDispatcher(throughputDispatcher)) - slowOne ! "hogexecutor" + assert((slowOne ? "hogexecutor").get === "OK") (1 to 100) foreach { _ ⇒ slowOne ! "ping" } fastOne ! "sabotage" start.countDown() From 0dc3c5ad3d865d00f87d89c2b256f2792c404ed9 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 13:19:44 +0200 Subject: [PATCH 21/33] Removing receiver from Envelope and switch to use the Mailbox.actor instead, this should speed up the BalancingDispatcher by some since it doesn't entail any allocations in adopting a message --- .../scala/akka/actor/dispatch/ActorModelSpec.scala | 6 +++--- .../test/scala/akka/dispatch/MailboxConfigSpec.scala | 7 +------ akka-actor/src/main/scala/akka/actor/ActorCell.scala | 4 ++-- .../scala/akka/dispatch/AbstractDispatcher.scala | 10 ++-------- .../scala/akka/dispatch/BalancingDispatcher.scala | 10 ++-------- .../src/main/scala/akka/dispatch/Dispatcher.scala | 4 ++-- .../src/main/scala/akka/dispatch/Mailbox.scala | 4 ++-- .../akka/serialization/SerializationProtocol.scala | 2 +- .../scala/akka/testkit/CallingThreadDispatcher.scala | 12 ++++++------ 9 files changed, 21 insertions(+), 38 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index b8a672aaf1..a9085d2a81 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -127,10 +127,10 @@ object ActorModelSpec { super.unregister(actor) } - protected[akka] abstract override def dispatch(invocation: Envelope) { - val stats = getStats(invocation.receiver.self) + protected[akka] abstract override def dispatch(receiver: ActorCell, invocation: Envelope) { + val stats = getStats(receiver.self) stats.msgsReceived.incrementAndGet() - super.dispatch(invocation) + super.dispatch(receiver, invocation) } protected[akka] abstract override def start() { diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala index b51896f659..b4cc86cca2 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/MailboxConfigSpec.scala @@ -80,12 +80,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn result } - def createMessageInvocation(msg: Any): Envelope = { - new Envelope( - actorOf(new Actor { //Dummy actor - def receive = { case _ ⇒ } - }).asInstanceOf[LocalActorRef].underlying, msg, NullChannel) - } + def createMessageInvocation(msg: Any): Envelope = Envelope(msg, NullChannel) def ensureInitialMailboxState(config: MailboxType, q: Mailbox) { q must not be null diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 910777685c..392e935b4a 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -278,7 +278,7 @@ private[akka] class ActorCell( //TODO FIXME remove this method def supervisor: Option[ActorRef] = props.supervisor - def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = dispatcher dispatch Envelope(this, message, channel) + def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = dispatcher.dispatch(this, Envelope(message, channel)) def postMessageToMailboxAndCreateFutureResultWithTimeout( message: Any, @@ -288,7 +288,7 @@ private[akka] class ActorCell( case f: ActorPromise ⇒ f case _ ⇒ new ActorPromise(timeout)(dispatcher) } - dispatcher dispatch Envelope(this, message, future) + dispatcher.dispatch(this, Envelope(message, future)) future } diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 58502fa334..d7bdb68229 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -17,13 +17,7 @@ import scala.annotation.tailrec /** * @author Jonas Bonér */ -final case class Envelope(val receiver: ActorCell, val message: Any, val channel: UntypedChannel) { - if (receiver eq null) throw new IllegalArgumentException("Receiver can't be null") - - final def invoke() { - receiver invoke this - } -} +final case class Envelope(val message: Any, val channel: UntypedChannel) object SystemMessage { @tailrec @@ -295,7 +289,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable /** * Will be called when the dispatcher is to queue an invocation for execution */ - protected[akka] def dispatch(invocation: Envelope) + protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) /** * Suggest to register the provided mailbox for execution diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index f38e8d8f54..f38e3a657e 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -57,12 +57,7 @@ class BalancingDispatcher( class SharingMailbox(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue { final def enqueue(handle: Envelope) = messageQueue.enqueue(handle) - final def dequeue(): Envelope = { - val envelope = messageQueue.dequeue() - if (envelope eq null) null - else if (envelope.receiver eq actor) envelope - else envelope.copy(receiver = actor) - } + final def dequeue(): Envelope = messageQueue.dequeue() final def numberOfMessages: Int = messageQueue.numberOfMessages @@ -106,8 +101,7 @@ class BalancingDispatcher( } else true } - override protected[akka] def dispatch(invocation: Envelope) = { - val receiver = invocation.receiver + override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { messageQueue enqueue invocation val buddy = buddies.pollFirst() diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 53d28a79a3..f9819c3cd3 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -78,8 +78,8 @@ class Dispatcher( protected[akka] val executorServiceFactory = executorServiceFactoryProvider.createExecutorServiceFactory(name) protected[akka] val executorService = new AtomicReference[ExecutorService](new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService)) - protected[akka] def dispatch(invocation: Envelope) = { - val mbox = invocation.receiver.mailbox + protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = { + val mbox = receiver.mailbox mbox enqueue invocation registerForExecution(mbox, true, false) } diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index df730efd8c..6ea3afb3d7 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -170,7 +170,7 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag var processedMessages = 0 val deadlineNs = if (dispatcher.isThroughputDeadlineTimeDefined) System.nanoTime + TimeUnit.MILLISECONDS.toNanos(dispatcher.throughputDeadlineTime) else 0 do { - nextMessage.invoke + actor invoke nextMessage processAllSystemMessages() //After we're done, process all system messages @@ -182,7 +182,7 @@ abstract class Mailbox(val actor: ActorCell) extends AbstractMailbox with Messag } else null //Abort } while (nextMessage ne null) } else { //If we only run one message per process - nextMessage.invoke //Just run it + actor invoke nextMessage //Just run it processAllSystemMessages() //After we're done, process all system messages } } diff --git a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala index 2e4d177aba..6a8b431e39 100644 --- a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala @@ -101,7 +101,7 @@ class ActorSerialization(val app: AkkaApplication, remote: RemoteSupport) { l map { m ⇒ remoteActorSerialization.createRemoteMessageProtocolBuilder( - Option(m.receiver.self), + localRef, Left(actorRef.uuid), actorRef.address, app.AkkaConfig.ActorTimeoutMillis, diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 829cce72c3..b6cb07cb02 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -150,20 +150,20 @@ class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling } } - protected[akka] override def dispatch(handle: Envelope) { - val mbox = getMailbox(handle.receiver) + protected[akka] override def dispatch(receiver: ActorCell, handle: Envelope) { + val mbox = getMailbox(receiver) val queue = mbox.queue val execute = mbox.suspendSwitch.fold { queue.push(handle) if (warnings && handle.channel.isInstanceOf[Promise[_]]) { - app.eventHandler.warning(this, "suspendSwitch, creating Future could deadlock; target: %s" format handle.receiver) + app.eventHandler.warning(this, "suspendSwitch, creating Future could deadlock; target: %s" format receiver) } false } { queue.push(handle) if (queue.isActive) { if (warnings && handle.channel.isInstanceOf[Promise[_]]) { - app.eventHandler.warning(this, "blocked on this thread, creating Future could deadlock; target: %s" format handle.receiver) + app.eventHandler.warning(this, "blocked on this thread, creating Future could deadlock; target: %s" format receiver) } false } else { @@ -200,10 +200,10 @@ class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling } if (handle ne null) { try { - handle.invoke + mbox.actor.invoke(handle) if (warnings) handle.channel match { case f: ActorPromise if !f.isCompleted ⇒ - app.eventHandler.warning(this, "calling %s with message %s did not reply as expected, might deadlock" format (handle.receiver, handle.message)) + app.eventHandler.warning(this, "calling %s with message %s did not reply as expected, might deadlock" format (mbox.actor, handle.message)) case _ ⇒ } true From 2d4251fcee90a54c1b62faf41a5c6e67c41b1b12 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 14:26:53 +0200 Subject: [PATCH 22/33] Fixing a race in DeathWatchSpec --- .../scala/akka/actor/DeathWatchSpec.scala | 24 ++++++++++++------- .../akka/actor/dispatch/ActorModelSpec.scala | 22 +---------------- .../akka/dispatch/BalancingDispatcher.scala | 2 ++ .../src/main/scala/akka/event/EventBus.scala | 24 ++++++------------- 4 files changed, 25 insertions(+), 47 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index ece9ec3d61..c4622b081f 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -17,11 +17,11 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } "notify with one Terminated message when an Actor is stopped" in { - val terminal = actorOf(Props(context ⇒ { case _ ⇒ context.self.stop() })) + val terminal = actorOf(Props(context ⇒ { case _ ⇒ })) testActor startsMonitoring terminal - terminal ! "anything" + terminal ! PoisonPill expectTerminationOf(terminal) @@ -30,13 +30,13 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende "notify with all monitors with one Terminated message when an Actor is stopped" in { val monitor1, monitor2 = actorOf(Props(context ⇒ { case t: Terminated ⇒ testActor ! t })) - val terminal = actorOf(Props(context ⇒ { case _ ⇒ context.self.stop() })) + val terminal = actorOf(Props(context ⇒ { case _ ⇒ })) monitor1 startsMonitoring terminal monitor2 startsMonitoring terminal testActor startsMonitoring terminal - terminal ! "anything" + terminal ! PoisonPill expectTerminationOf(terminal) expectTerminationOf(terminal) @@ -48,8 +48,11 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } "notify with _current_ monitors with one Terminated message when an Actor is stopped" in { - val monitor1, monitor2 = actorOf(Props(context ⇒ { case t: Terminated ⇒ testActor ! t })) - val terminal = actorOf(Props(context ⇒ { case _ ⇒ context.self.stop() })) + val monitor1, monitor2 = actorOf(Props(context ⇒ { + case t: Terminated ⇒ testActor ! t + case "ping" ⇒ context.channel ! "pong" + })) + val terminal = actorOf(Props(context ⇒ { case _ ⇒ })) monitor1 startsMonitoring terminal monitor2 startsMonitoring terminal @@ -57,12 +60,15 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende monitor2 stopsMonitoring terminal - terminal ! "anything" + monitor2 ! "ping" + + expectMsg("pong") //Needs to be here since startsMonitoring and stopsMonitoring are asynchronous + + terminal ! PoisonPill expectTerminationOf(terminal) expectTerminationOf(terminal) - terminal.stop() monitor1.stop() monitor2.stop() } @@ -80,8 +86,8 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende terminal ! Kill expectTerminationOf(terminal) + terminal.isShutdown must be === true - terminal.stop() supervisor.stop() } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index a9085d2a81..d4aa3e90e6 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -392,31 +392,11 @@ abstract class ActorModelSpec extends AkkaSpec { } catch { case e ⇒ System.err.println("Error: " + e.getMessage + " missing count downs == " + cachedMessage.latch.getCount() + " out of " + num) - //app.eventHandler.error(new Exception with NoStackTrace, null, cachedMessage.latch.getCount()) } } for (run ← 1 to 3) { flood(40000) - try { - assertDispatcher(dispatcher)(starts = run, stops = run) - } catch { - case e ⇒ - - // FIXME: registry has been removed - // app.registry.local.foreach { - // case actor: LocalActorRef ⇒ - // val cell = actor.underlying - // val mbox = cell.mailbox - // System.err.println("Left in the registry: " + actor.address + " => " + cell + " => " + mbox.hasMessages + " " + mbox.hasSystemMessages + " " + mbox.numberOfMessages + " " + mbox.isScheduled) - // var message = mbox.dequeue() - // while (message ne null) { - // System.err.println("Lingering message for " + cell + " " + message) - // message = mbox.dequeue() - // } - // } - - throw e - } + assertDispatcher(dispatcher)(starts = run, stops = run) } } diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index f38e3a657e..93e7c99e7b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -90,6 +90,8 @@ class BalancingDispatcher( while (messages ne null) { deadLetterMailbox.systemEnqueue(messages) //Send to dead letter queue messages = messages.next + if (messages eq null) //Make sure that any system messages received after the current drain are also sent to the dead letter mbox + messages = mailBox.systemDrain() } } } diff --git a/akka-actor/src/main/scala/akka/event/EventBus.scala b/akka-actor/src/main/scala/akka/event/EventBus.scala index 33319fbb13..9dd76f5344 100644 --- a/akka-actor/src/main/scala/akka/event/EventBus.scala +++ b/akka-actor/src/main/scala/akka/event/EventBus.scala @@ -182,10 +182,7 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒ if (monitored.isShutdown) false else { if (mappings.putIfAbsent(monitored, Vector(monitor)) ne null) associate(monitored, monitor) - else { - if (monitored.isShutdown) !dissociate(monitored, monitor) - else true - } + else if (monitored.isShutdown) !dissociate(monitored, monitor) else true } case raw: Vector[_] ⇒ val v = raw.asInstanceOf[Vector[ActorRef]] @@ -194,10 +191,7 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒ else { val added = v :+ monitor if (!mappings.replace(monitored, v, added)) associate(monitored, monitor) - else { - if (monitored.isShutdown) !dissociate(monitored, monitor) - else true - } + else if (monitored.isShutdown) !dissociate(monitored, monitor) else true } } } @@ -241,13 +235,11 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒ case raw: Vector[_] ⇒ val v = raw.asInstanceOf[Vector[ActorRef]] val removed = v.filterNot(monitor ==) - if (removed eq v) false + if (removed eq raw) false else if (removed.isEmpty) { - if (!mappings.remove(monitored, v)) dissociate(monitored, monitor) - else true + if (!mappings.remove(monitored, v)) dissociate(monitored, monitor) else true } else { - if (!mappings.replace(monitored, v, removed)) dissociate(monitored, monitor) - else true + if (!mappings.replace(monitored, v, removed)) dissociate(monitored, monitor) else true } } } @@ -263,10 +255,8 @@ trait ActorClassification { self: ActorEventBus with ActorClassifier ⇒ protected def mapSize: Int def publish(event: Event): Unit = mappings.get(classify(event)) match { - case null ⇒ - case raw: Vector[_] ⇒ - val v = raw.asInstanceOf[Vector[ActorRef]] - v foreach { _ ! event } + case null ⇒ + case raw: Vector[_] ⇒ raw.asInstanceOf[Vector[ActorRef]] foreach { _ ! event } } def subscribe(subscriber: Subscriber, to: Classifier): Boolean = associate(to, subscriber) From 77dc9e9c351df29f4287bf26354de62ad3fe6c53 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 16:59:47 +0200 Subject: [PATCH 23/33] #1299 - Removing reply and tryReply, preparing the way for 'sender ! response' --- .../java/akka/actor/JavaAPITestActor.java | 2 +- .../ActorFireForgetRequestReplySpec.scala | 2 +- .../scala/akka/actor/ActorLifeCycleSpec.scala | 2 +- .../test/scala/akka/actor/ActorRefSpec.scala | 22 +++++----- .../scala/akka/actor/ForwardActorSpec.scala | 2 +- .../src/test/scala/akka/actor/IOActor.scala | 16 ++++---- .../scala/akka/actor/LoggingReceiveSpec.scala | 4 +- .../scala/akka/actor/SupervisorMiscSpec.scala | 2 +- .../scala/akka/actor/SupervisorSpec.scala | 4 +- .../test/scala/akka/actor/Ticket669Spec.scala | 4 +- .../akka/actor/dispatch/ActorModelSpec.scala | 4 +- .../actor/dispatch/DispatcherActorSpec.scala | 2 +- .../akka/actor/dispatch/PinnedActorSpec.scala | 2 +- .../test/scala/akka/dispatch/FutureSpec.scala | 40 +++++++++---------- .../dispatch/PriorityDispatcherSpec.scala | 2 +- .../scala/akka/routing/ActorPoolSpec.scala | 4 +- .../routing/ConfiguredLocalRoutingSpec.scala | 4 +- .../test/scala/akka/routing/RoutingSpec.scala | 2 +- .../scala/akka/ticket/Ticket703Spec.scala | 2 +- .../src/main/scala/akka/actor/Actor.scala | 27 +------------ .../src/main/scala/akka/actor/ActorRef.scala | 2 +- .../src/main/scala/akka/actor/FSM.scala | 4 +- .../main/scala/akka/actor/TypedActor.scala | 2 +- .../main/scala/akka/actor/UntypedActor.scala | 2 +- .../src/main/scala/akka/routing/Pool.scala | 2 +- .../akka/camel/TypedCamelTestSupport.scala | 8 ++-- .../scala/akka/camel/ConsumerPublisher.scala | 4 +- .../src/main/scala/akka/camel/Producer.scala | 2 +- .../akka/camel/SampleUntypedConsumer.java | 2 +- .../camel/SampleUntypedConsumerBlocking.java | 2 +- .../scala/akka/camel/CamelTestSupport.scala | 8 ++-- .../scala/akka/camel/ConsumerScalaTest.scala | 12 +++--- .../akka/camel/ProducerFeatureTest.scala | 10 ++--- .../component/ActorComponentFeatureTest.scala | 4 +- akka-docs/intro/code/tutorials/first/Pi.scala | 3 +- .../main/scala/akka/remote/RemoteDaemon.scala | 8 ++-- .../DirectRoutedRemoteActorMultiJvmSpec.scala | 4 +- .../NewRemoteActorMultiJvmSpec.scala | 4 +- .../RandomRoutedRemoteActorMultiJvmSpec.scala | 2 +- ...ndRobinRoutedRemoteActorMultiJvmSpec.scala | 2 +- .../serialization/ActorSerializeSpec.scala | 8 ++-- .../java/sample/camel/UntypedConsumer1.java | 2 +- .../src/main/scala/sample/camel/Actors.scala | 22 +++++----- .../camel/SampleRemoteUntypedConsumer.java | 2 +- .../camel/HttpConcurrencyTestStress.scala | 4 +- .../sample/camel/RemoteConsumerTest.scala | 4 +- .../src/main/scala/akka/agent/Agent.scala | 6 +-- .../example/UntypedCoordinatedCounter.java | 2 +- .../transactor/example/UntypedCounter.java | 2 +- .../test/UntypedCoordinatedCounter.java | 2 +- .../akka/transactor/test/UntypedCounter.java | 2 +- .../transactor/CoordinatedIncrementSpec.scala | 2 +- .../scala/transactor/FickleFriendsSpec.scala | 4 +- .../scala/transactor/TransactorSpec.scala | 2 +- .../src/main/scala/akka/testkit/TestKit.scala | 8 ---- .../scala/akka/testkit/TestActorRefSpec.scala | 8 ++-- .../scala/akka/testkit/TestProbeSpec.scala | 6 +-- .../java/akka/tutorial/first/java/Pi.java | 2 +- .../src/main/scala/Pi.scala | 2 +- .../java/akka/tutorial/java/second/Pi.java | 2 +- .../src/main/scala/Pi.scala | 2 +- 61 files changed, 144 insertions(+), 186 deletions(-) diff --git a/akka-actor-tests/src/test/java/akka/actor/JavaAPITestActor.java b/akka-actor-tests/src/test/java/akka/actor/JavaAPITestActor.java index 588307ee6c..4952d1b2c9 100644 --- a/akka-actor-tests/src/test/java/akka/actor/JavaAPITestActor.java +++ b/akka-actor-tests/src/test/java/akka/actor/JavaAPITestActor.java @@ -2,6 +2,6 @@ package akka.actor; public class JavaAPITestActor extends UntypedActor { public void onReceive(Object msg) { - tryReply("got it!"); + getChannel().tryTell("got it!"); } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala index 461868df21..a74fe3ecbb 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorFireForgetRequestReplySpec.scala @@ -15,7 +15,7 @@ object ActorFireForgetRequestReplySpec { class ReplyActor extends Actor { def receive = { case "Send" ⇒ - reply("Reply") + channel ! "Reply" case "SendImplicit" ⇒ channel ! "ReplyImplicit" } diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala index 52e1bd6d58..5b9c1b96f9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorLifeCycleSpec.scala @@ -25,7 +25,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS val currentGen = generationProvider.getAndIncrement() override def preStart() { report("preStart") } override def postStop() { report("postStop") } - def receive = { case "status" ⇒ this reply message("OK") } + def receive = { case "status" ⇒ channel ! message("OK") } } "An Actor" must { diff --git a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala index 9ab8615594..1fde830821 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -35,7 +35,7 @@ object ActorRefSpec { val worker = context.actorOf(Props[WorkerActor]) worker ! ReplyTo(channel) case "workDone" ⇒ replyTo ! "complexReply" - case "simpleRequest" ⇒ reply("simpleReply") + case "simpleRequest" ⇒ channel ! "simpleReply" } } @@ -43,7 +43,7 @@ object ActorRefSpec { def receive = { case "work" ⇒ { work - reply("workDone") + channel ! "workDone" self.stop() } case ReplyTo(replyTo) ⇒ { @@ -74,7 +74,7 @@ object ActorRefSpec { class OuterActor(val inner: ActorRef) extends Actor { def receive = { - case "self" ⇒ reply(self) + case "self" ⇒ channel ! self case x ⇒ inner forward x } } @@ -83,7 +83,7 @@ object ActorRefSpec { val fail = new InnerActor def receive = { - case "self" ⇒ reply(self) + case "self" ⇒ channel ! self case x ⇒ inner forward x } } @@ -94,8 +94,8 @@ object ActorRefSpec { class InnerActor extends Actor { def receive = { - case "innerself" ⇒ reply(self) - case other ⇒ reply(other) + case "innerself" ⇒ channel ! self + case other ⇒ channel ! other } } @@ -103,8 +103,8 @@ object ActorRefSpec { val fail = new InnerActor def receive = { - case "innerself" ⇒ reply(self) - case other ⇒ reply(other) + case "innerself" ⇒ channel ! self + case other ⇒ channel ! other } } @@ -321,7 +321,7 @@ class ActorRefSpec extends AkkaSpec { "support nested actorOfs" in { val a = actorOf(new Actor { val nested = actorOf(new Actor { def receive = { case _ ⇒ } }) - def receive = { case _ ⇒ reply(nested) } + def receive = { case _ ⇒ channel ! nested } }) val nested = (a ? "any").as[ActorRef].get @@ -369,8 +369,8 @@ class ActorRefSpec extends AkkaSpec { val timeout = Timeout(20000) val ref = actorOf(Props(new Actor { def receive = { - case 5 ⇒ tryReply("five") - case null ⇒ tryReply("null") + case 5 ⇒ channel.tryTell("five") + case null ⇒ channel.tryTell("null") } })) diff --git a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala index ce8bacf12b..55cfbd5fd7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ForwardActorSpec.scala @@ -15,7 +15,7 @@ object ForwardActorSpec { def createForwardingChain(app: AkkaApplication): ActorRef = { val replier = app.actorOf(new Actor { - def receive = { case x ⇒ reply(x) } + def receive = { case x ⇒ channel ! x } }) def mkforwarder(forwardTo: ActorRef) = app.actorOf( diff --git a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala index 85c29e1033..eee4553b59 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/IOActor.scala @@ -48,7 +48,7 @@ object IOActorSpec { def receiveIO = { case length: Int ⇒ val bytes = socket.read(length) - reply(bytes) + channel ! bytes } } } @@ -108,9 +108,9 @@ object IOActorSpec { case msg: NewClient ⇒ createWorker forward msg case ('set, key: String, value: ByteString) ⇒ kvs += (key -> value) - tryReply(()) - case ('get, key: String) ⇒ tryReply(kvs.get(key)) - case 'getall ⇒ tryReply(kvs) + channel.tryTell(())(self) + case ('get, key: String) ⇒ channel.tryTell(kvs.get(key))(self) + case 'getall ⇒ channel.tryTell(kvs)(self) } } @@ -123,18 +123,20 @@ object IOActorSpec { socket = connect(ioManager, host, port) } + def reply(msg: Any) = channel.tryTell(msg)(self) + def receiveIO = { case ('set, key: String, value: ByteString) ⇒ socket write (ByteString("SET " + key + " " + value.length + "\r\n") ++ value) - tryReply(readResult) + reply(readResult) case ('get, key: String) ⇒ socket write ByteString("GET " + key + "\r\n") - tryReply(readResult) + reply(readResult) case 'getall ⇒ socket write ByteString("GETALL\r\n") - tryReply(readResult) + reply(readResult) } def readResult = { diff --git a/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala index 0af11ba652..d00356afd3 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/LoggingReceiveSpec.scala @@ -62,7 +62,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd app.eventHandler.addListener(testActor) val actor = TestActorRef(new Actor { def receive = loggable(this) { - case _ ⇒ reply("x") + case _ ⇒ channel ! "x" } }) actor ! "buh" @@ -91,7 +91,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd app.eventHandler.addListener(testActor) val actor = TestActorRef(new Actor { def receive = loggable(this)(loggable(this) { - case _ ⇒ reply("x") + case _ ⇒ channel ! "x" }) }) actor ! "buh" diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala index ea4776981d..aaebabb764 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorMiscSpec.scala @@ -24,7 +24,7 @@ class SupervisorMiscSpec extends AkkaSpec { override def postRestart(cause: Throwable) { countDownLatch.countDown() } protected def receive = { - case "status" ⇒ this.reply("OK") + case "status" ⇒ this.channel ! "OK" case _ ⇒ this.self.stop() } }).withSupervisor(supervisor) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala index c4604253c8..80e015a9b7 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorSpec.scala @@ -42,7 +42,7 @@ object SupervisorSpec { def receive = { case Ping ⇒ messageLog.put(PingMessage) - tryReply(PongMessage) + channel.tryTell(PongMessage) case Die ⇒ throw new RuntimeException(ExceptionMessage) } @@ -298,7 +298,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with BeforeAndAfte if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!") def receive = { - case Ping ⇒ tryReply(PongMessage) + case Ping ⇒ channel.tryTell(PongMessage) case Die ⇒ throw new RuntimeException("Expected") } }).withSupervisor(supervisor)) diff --git a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala index c32832fbc1..e0330bca9a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Ticket669Spec.scala @@ -47,11 +47,11 @@ object Ticket669Spec { } override def preRestart(reason: scala.Throwable, msg: Option[Any]) { - tryReply("failure1") + channel.tryTell("failure1") } override def postStop() { - tryReply("failure2") + channel.tryTell("failure2") } } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index d4aa3e90e6..97aa58eddc 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -70,8 +70,8 @@ object ActorModelSpec { case Meet(sign, wait) ⇒ ack; sign.countDown(); wait.await(); busy.switchOff() case Wait(time) ⇒ ack; Thread.sleep(time); busy.switchOff() case WaitAck(time, l) ⇒ ack; Thread.sleep(time); l.countDown(); busy.switchOff() - case Reply(msg) ⇒ ack; reply(msg); busy.switchOff() - case TryReply(msg) ⇒ ack; tryReply(msg); busy.switchOff() + case Reply(msg) ⇒ ack; channel ! msg; busy.switchOff() + case TryReply(msg) ⇒ ack; channel.tryTell(msg); busy.switchOff() case Forward(to, msg) ⇒ ack; to.forward(msg); busy.switchOff() case CountDown(latch) ⇒ ack; latch.countDown(); busy.switchOff() case Increment(count) ⇒ ack; count.incrementAndGet(); busy.switchOff() diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala index c8f23375c6..7ffaf365ea 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/DispatcherActorSpec.scala @@ -9,7 +9,7 @@ import akka.actor.{ Props, Actor } object DispatcherActorSpec { class TestActor extends Actor { def receive = { - case "Hello" ⇒ reply("World") + case "Hello" ⇒ channel ! "World" case "Failure" ⇒ throw new RuntimeException("Expected exception; to test fault-tolerance") } } diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala index 46d05c18ea..3564ab3dcb 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/PinnedActorSpec.scala @@ -12,7 +12,7 @@ import org.scalatest.BeforeAndAfterEach object PinnedActorSpec { class TestActor extends Actor { def receive = { - case "Hello" ⇒ reply("World") + case "Hello" ⇒ channel ! "World" case "Failure" ⇒ throw new RuntimeException("Expected exception; to test fault-tolerance") } } diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala index aef33a4874..6d5f8b9ea1 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/FutureSpec.scala @@ -17,20 +17,16 @@ import org.scalatest.junit.JUnitSuite object FutureSpec { class TestActor extends Actor { def receive = { - case "Hello" ⇒ - reply("World") - case "NoReply" ⇒ {} - case "Failure" ⇒ - throw new RuntimeException("Expected exception; to test fault-tolerance") + case "Hello" ⇒ channel ! "World" + case "Failure" ⇒ throw new RuntimeException("Expected exception; to test fault-tolerance") + case "NoReply" ⇒ } } class TestDelayActor(await: StandardLatch) extends Actor { def receive = { - case "Hello" ⇒ - await.await - reply("World") - case "NoReply" ⇒ { await.await } + case "Hello" ⇒ await.await; channel ! "World" + case "NoReply" ⇒ await.await case "Failure" ⇒ await.await throw new RuntimeException("Expected exception; to test fault-tolerance") @@ -140,7 +136,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "will return a result" must { behave like futureWithResult { test ⇒ val actor1 = actorOf[TestActor] - val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ reply(s.toUpperCase) } }) + val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ channel ! s.toUpperCase } }) val future = actor1 ? "Hello" flatMap { case s: String ⇒ actor2 ? s } future.await test(future, "WORLD") @@ -152,7 +148,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { behave like futureWithException[ArithmeticException] { test ⇒ filterException[ArithmeticException] { val actor1 = actorOf[TestActor] - val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ reply(s.length / 0) } }) + val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ channel ! s.length / 0 } }) val future = actor1 ? "Hello" flatMap { case s: String ⇒ actor2 ? s } future.await test(future, "/ by zero") @@ -165,7 +161,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { behave like futureWithException[MatchError] { test ⇒ filterException[MatchError] { val actor1 = actorOf[TestActor] - val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ reply(s.toUpperCase) } }) + val actor2 = actorOf(new Actor { def receive = { case s: String ⇒ channel ! s.toUpperCase } }) val future = actor1 ? "Hello" flatMap { case i: Int ⇒ actor2 ? i } future.await test(future, "World (of class java.lang.String)") @@ -182,8 +178,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { filterException[ClassCastException] { val actor = actorOf(new Actor { def receive = { - case s: String ⇒ reply(s.length) - case i: Int ⇒ reply((i * 2).toString) + case s: String ⇒ channel ! s.length + case i: Int ⇒ channel ! (i * 2).toString } }) @@ -214,8 +210,8 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { case class Res[T](res: T) val actor = actorOf(new Actor { def receive = { - case Req(s: String) ⇒ reply(Res(s.length)) - case Req(i: Int) ⇒ reply(Res((i * 2).toString)) + case Req(s: String) ⇒ channel ! Res(s.length) + case Req(i: Int) ⇒ channel ! Res((i * 2).toString) } }) @@ -301,7 +297,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "fold" in { val actors = (1 to 10).toList map { _ ⇒ actorOf(new Actor { - def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); tryReply(add) } + def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); channel.tryTell(add) } }) } val timeout = 10000 @@ -312,7 +308,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "fold by composing" in { val actors = (1 to 10).toList map { _ ⇒ actorOf(new Actor { - def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); tryReply(add) } + def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); channel.tryTell(add) } }) } def futures = actors.zipWithIndex map { case (actor: ActorRef, idx: Int) ⇒ actor.?((idx, idx * 200), 10000).mapTo[Int] } @@ -327,7 +323,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { case (add: Int, wait: Int) ⇒ Thread.sleep(wait) if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected") - tryReply(add) + channel.tryTell(add) } }) } @@ -359,7 +355,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { "shouldReduceResults" in { val actors = (1 to 10).toList map { _ ⇒ actorOf(new Actor { - def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); tryReply(add) } + def receive = { case (add: Int, wait: Int) ⇒ Thread.sleep(wait); channel.tryTell(add) } }) } val timeout = 10000 @@ -375,7 +371,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { case (add: Int, wait: Int) ⇒ Thread.sleep(wait) if (add == 6) throw new IllegalArgumentException("shouldFoldResultsWithException: expected") - tryReply(add) + channel.tryTell(add) } }) } @@ -404,7 +400,7 @@ class FutureSpec extends AkkaSpec with Checkers with BeforeAndAfterAll { var counter = 1 def receive = { case 'GetNext ⇒ - reply(counter) + channel ! counter counter += 2 } }) 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 a54f739e23..e3ab6d2ed7 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -29,7 +29,7 @@ class PriorityDispatcherSpec extends AkkaSpec { def receive = { case i: Int ⇒ acc = i :: acc - case 'Result ⇒ tryReply(acc) + case 'Result ⇒ channel.tryTell(acc) } }).withDispatcher(dispatcher)).asInstanceOf[LocalActorRef] diff --git a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala index 0c1f381b1b..1ad0820f23 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ActorPoolSpec.scala @@ -41,7 +41,7 @@ class ActorPoolSpec extends AkkaSpec { case _ ⇒ count.incrementAndGet latch.countDown() - tryReply("success") + channel.tryTell("success") } })) @@ -88,7 +88,7 @@ class ActorPoolSpec extends AkkaSpec { def receive = { case req: String ⇒ { sleepFor(10 millis) - tryReply("Response") + channel.tryTell("Response") } } })) diff --git a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala index 5e5da7ee23..ca566f726a 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/ConfiguredLocalRoutingSpec.scala @@ -73,7 +73,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val actor = app.actorOf(Props(new Actor { lazy val id = counter.getAndIncrement() def receive = { - case "hit" ⇒ reply(id) + case "hit" ⇒ channel ! id case "end" ⇒ doneLatch.countDown() } }), address) @@ -187,7 +187,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec { val actor = app.actorOf(Props(new Actor { lazy val id = counter.getAndIncrement() def receive = { - case "hit" ⇒ reply(id) + case "hit" ⇒ channel ! id case "end" ⇒ doneLatch.countDown() } }), address) diff --git a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala index 5937597149..dab51d076f 100644 --- a/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/routing/RoutingSpec.scala @@ -458,7 +458,7 @@ class RoutingSpec extends AkkaSpec { case Stop(None) ⇒ self.stop() case Stop(Some(_id)) if (_id == id) ⇒ self.stop() case _id: Int if (_id == id) ⇒ - case _ ⇒ Thread sleep 100 * id; tryReply(id) + case _ ⇒ Thread sleep 100 * id; channel.tryTell(id) } override def postStop = { diff --git a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala index 0367956c50..5e46f3ec05 100644 --- a/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala +++ b/akka-actor-tests/src/test/scala/akka/ticket/Ticket703Spec.scala @@ -21,7 +21,7 @@ class Ticket703Spec extends AkkaSpec { def receive = { case req: String ⇒ Thread.sleep(6000L) - tryReply("Response") + channel.tryTell("Response") } })) }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 5, 1000))) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index c5834ac633..34b95174f6 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -295,31 +295,6 @@ trait Actor { */ def receiveTimeout_=(timeout: Option[Long]) = context.receiveTimeout = timeout - /** - * Akka Scala & Java API - * Use reply(..) to reply with a message to the original sender of the message currently - * being processed. This method fails if the original sender of the message could not be determined with an - * IllegalStateException. - * - * If you don't want deal with this IllegalStateException, but just a boolean, just use the tryReply(...) - * version. - * - *

- * Throws an IllegalStateException if unable to determine what to reply to. - */ - def reply(message: Any) = channel.!(message)(self) - - /** - * Akka Scala & Java API - * Use tryReply(..) to try reply with a message to the original sender of the message currently - * being processed. This method - *

- * Returns true if reply was sent, and false if unable to determine what to reply to. - * - * If you would rather have an exception, check the reply(..) version. - */ - def tryReply(message: Any): Boolean = channel.tryTell(message)(self) - /** * Same as ActorContext.children */ @@ -341,7 +316,7 @@ trait Actor { * def receive = { * case Ping => * println("got a 'Ping' message") - * reply("pong") + * channel ! "pong" * * case OneWay => * println("got a 'OneWay' message") diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 41e8a6aeac..188cb6b00f 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -68,7 +68,7 @@ abstract class ActorRef extends ActorRefShared with UntypedChannel with ReplyCha * Use this method with care. In most cases it is better to use 'tell' together with the 'getContext().getSender()' to * implement request/response message exchanges. *

- * If you are sending messages using ask then you have to use getContext().reply(..) + * If you are sending messages using ask then you have to use getContext().channel().tell(...) * to send a reply message to the original sender. If not then the sender will block until the timeout expires. */ def ask(message: AnyRef, timeout: Long, sender: ActorRef): Future[AnyRef] = diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala index 4beff7229d..efdb748708 100644 --- a/akka-actor/src/main/scala/akka/actor/FSM.scala +++ b/akka-actor/src/main/scala/akka/actor/FSM.scala @@ -506,7 +506,7 @@ trait FSM[S, D] extends ListenerManagement { nextState.stopReason match { case None ⇒ makeTransition(nextState) case _ ⇒ - nextState.replies.reverse foreach reply + nextState.replies.reverse foreach { r ⇒ channel ! r } terminate(nextState) self.stop() } @@ -516,7 +516,7 @@ trait FSM[S, D] extends ListenerManagement { if (!stateFunctions.contains(nextState.stateName)) { terminate(stay withStopReason Failure("Next state %s does not exist".format(nextState.stateName))) } else { - nextState.replies.reverse foreach reply + nextState.replies.reverse foreach { r ⇒ channel ! r } if (currentState.stateName != nextState.stateName) { handleTransition(currentState.stateName, nextState.stateName) notifyListeners(Transition(self, currentState.stateName, nextState.stateName)) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index ea23e77bd7..43d98a5941 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -281,7 +281,7 @@ class TypedActor(val app: AkkaApplication) { case p: ActorPromise ⇒ p completeWith m(me).asInstanceOf[Future[Any]] case _ ⇒ throw new IllegalStateException("Future-returning TypedActor didn't use ?/ask so cannot reply") } - } else reply(m(me)) + } else channel ! m(me) } finally { TypedActor.selfReference set null diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index 5016b0c590..4a4237b59d 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -21,7 +21,7 @@ import akka.dispatch.{ MessageDispatcher, Promise } * * if (msg.equals("UseReply")) { * // Reply to original sender of message using the 'reply' method - * reply(msg + ":" + getSelf().getAddress()); + * getContext().getChannel().tell(msg + ":" + getSelf().getAddress()); * * } else if (msg.equals("UseSender") && getSender().isDefined()) { * // Reply to original sender of message using the sender reference diff --git a/akka-actor/src/main/scala/akka/routing/Pool.scala b/akka-actor/src/main/scala/akka/routing/Pool.scala index 803bdf1a51..5414159388 100644 --- a/akka-actor/src/main/scala/akka/routing/Pool.scala +++ b/akka-actor/src/main/scala/akka/routing/Pool.scala @@ -103,7 +103,7 @@ trait DefaultActorPool extends ActorPool { this: Actor ⇒ protected def _route(): Actor.Receive = { // for testing... case Stat ⇒ - tryReply(Stats(_delegates length)) + channel.tryTell(Stats(_delegates length)) case Terminated(victim, _) ⇒ _delegates = _delegates filterNot { victim == } case msg ⇒ diff --git a/akka-camel-typed/src/test/scala/akka/camel/TypedCamelTestSupport.scala b/akka-camel-typed/src/test/scala/akka/camel/TypedCamelTestSupport.scala index 4e5d92aa22..eb4204ce0e 100644 --- a/akka-camel-typed/src/test/scala/akka/camel/TypedCamelTestSupport.scala +++ b/akka-camel-typed/src/test/scala/akka/camel/TypedCamelTestSupport.scala @@ -24,7 +24,7 @@ object TypedCamelTestSupport { def countdown: Handler = { case SetExpectedMessageCount(num) ⇒ { latch = new CountDownLatch(num) - reply(latch) + channel ! latch } case msg ⇒ latch.countDown } @@ -32,7 +32,7 @@ object TypedCamelTestSupport { trait Respond { this: Actor ⇒ def respond: Handler = { - case msg: Message ⇒ reply(response(msg)) + case msg: Message ⇒ channel ! response(msg) } def response(msg: Message): Any = "Hello %s" format msg.body @@ -42,8 +42,8 @@ object TypedCamelTestSupport { val messages = Buffer[Any]() def retain: Handler = { - case GetRetainedMessage ⇒ reply(messages.last) - case GetRetainedMessages(p) ⇒ reply(messages.toList.filter(p)) + case GetRetainedMessage ⇒ channel ! messages.last + case GetRetainedMessages(p) ⇒ channel ! messages.filter(p).toList case msg ⇒ { messages += msg msg diff --git a/akka-camel/src/main/scala/akka/camel/ConsumerPublisher.scala b/akka-camel/src/main/scala/akka/camel/ConsumerPublisher.scala index b02587f236..062c6246b5 100644 --- a/akka-camel/src/main/scala/akka/camel/ConsumerPublisher.scala +++ b/akka-camel/src/main/scala/akka/camel/ConsumerPublisher.scala @@ -127,11 +127,11 @@ private[camel] class ActivationTracker extends Actor { def receive = { case SetExpectedActivationCount(num) ⇒ { activationLatch = new CountDownLatch(num) - reply(activationLatch) + channel ! activationLatch } case SetExpectedDeactivationCount(num) ⇒ { deactivationLatch = new CountDownLatch(num) - reply(deactivationLatch) + channel ! deactivationLatch } case EndpointActivated ⇒ activationLatch.countDown case EndpointDeactivated ⇒ deactivationLatch.countDown diff --git a/akka-camel/src/main/scala/akka/camel/Producer.scala b/akka-camel/src/main/scala/akka/camel/Producer.scala index b29d00e7e5..bdc0079251 100644 --- a/akka-camel/src/main/scala/akka/camel/Producer.scala +++ b/akka-camel/src/main/scala/akka/camel/Producer.scala @@ -159,7 +159,7 @@ trait ProducerSupport { this: Actor ⇒ * actor). */ protected def receiveAfterProduce: Receive = { - case msg ⇒ if (!oneway) reply(msg) + case msg ⇒ if (!oneway) channel ! msg } /** diff --git a/akka-camel/src/test/java/akka/camel/SampleUntypedConsumer.java b/akka-camel/src/test/java/akka/camel/SampleUntypedConsumer.java index d86d4d5cc7..93d0427902 100644 --- a/akka-camel/src/test/java/akka/camel/SampleUntypedConsumer.java +++ b/akka-camel/src/test/java/akka/camel/SampleUntypedConsumer.java @@ -15,7 +15,7 @@ public class SampleUntypedConsumer extends UntypedConsumerActor { Message msg = (Message)message; String body = msg.getBodyAs(String.class); String header = msg.getHeaderAs("test", String.class); - tryReply(String.format("%s %s", body, header)); + channel.tryTell(String.format("%s %s", body, header)); } } diff --git a/akka-camel/src/test/java/akka/camel/SampleUntypedConsumerBlocking.java b/akka-camel/src/test/java/akka/camel/SampleUntypedConsumerBlocking.java index 0e58677f54..db3ea7665d 100644 --- a/akka-camel/src/test/java/akka/camel/SampleUntypedConsumerBlocking.java +++ b/akka-camel/src/test/java/akka/camel/SampleUntypedConsumerBlocking.java @@ -17,7 +17,7 @@ public class SampleUntypedConsumerBlocking extends UntypedConsumerActor { Message msg = (Message)message; String body = msg.getBodyAs(String.class); String header = msg.getHeaderAs("test", String.class); - tryReply(String.format("%s %s", body, header)); + channel.tryTell(String.format("%s %s", body, header)); } } diff --git a/akka-camel/src/test/scala/akka/camel/CamelTestSupport.scala b/akka-camel/src/test/scala/akka/camel/CamelTestSupport.scala index be5099d050..d099dba708 100644 --- a/akka-camel/src/test/scala/akka/camel/CamelTestSupport.scala +++ b/akka-camel/src/test/scala/akka/camel/CamelTestSupport.scala @@ -36,7 +36,7 @@ object CamelTestSupport { def countdown: Handler = { case SetExpectedMessageCount(num) ⇒ { latch = new CountDownLatch(num) - reply(latch) + channel ! latch } case msg ⇒ latch.countDown } @@ -44,7 +44,7 @@ object CamelTestSupport { trait Respond { this: Actor ⇒ def respond: Handler = { - case msg: Message ⇒ reply(response(msg)) + case msg: Message ⇒ channel ! response(msg) } def response(msg: Message): Any = "Hello %s" format msg.body @@ -54,8 +54,8 @@ object CamelTestSupport { val messages = Buffer[Any]() def retain: Handler = { - case GetRetainedMessage ⇒ reply(messages.last) - case GetRetainedMessages(p) ⇒ reply(messages.toList.filter(p)) + case GetRetainedMessage ⇒ channel ! messages.last + case GetRetainedMessages(p) ⇒ channel ! messages.filter(p).toList case msg ⇒ { messages += msg msg diff --git a/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala b/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala index 7ee64f5d65..7a83e09de7 100644 --- a/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala +++ b/akka-camel/src/test/scala/akka/camel/ConsumerScalaTest.scala @@ -211,7 +211,7 @@ object ConsumerScalaTest { class TestConsumer(uri: String) extends Actor with Consumer { def endpointUri = uri protected def receive = { - case msg: Message ⇒ reply("received %s" format msg.body) + case msg: Message ⇒ channel ! "received %s" format msg.body } } @@ -226,7 +226,7 @@ object ConsumerScalaTest { def endpointUri = uri override def autoack = false protected def receive = { - case msg: Message ⇒ reply(Ack) + case msg: Message ⇒ channel ! Ack } } @@ -247,15 +247,15 @@ object ConsumerScalaTest { protected def receive = { case "fail" ⇒ { throw new Exception("test") } - case "succeed" ⇒ reply("ok") + case "succeed" ⇒ channel ! "ok" } override def preRestart(reason: scala.Throwable, msg: Option[Any]) { - tryReply("pr") + channel.tryTell("pr") } override def postStop { - tryReply("ps") + channel.tryTell("ps") } } @@ -288,7 +288,7 @@ object ConsumerScalaTest { } private def respondTo(msg: Message) = - if (valid) reply("accepted: %s" format msg.body) + if (valid) channel ! ("accepted: %s" format msg.body) else throw new Exception("rejected: %s" format msg.body) } diff --git a/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala b/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala index d7fbc783f0..ad1ec6b5cf 100644 --- a/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala +++ b/akka-camel/src/test/scala/akka/camel/ProducerFeatureTest.scala @@ -253,18 +253,16 @@ object ProducerFeatureTest { class TestResponder extends Actor { protected def receive = { case msg: Message ⇒ msg.body match { - case "fail" ⇒ reply(Failure(new Exception("failure"), msg.headers)) - case _ ⇒ reply(msg.transformBody { body: String ⇒ "received %s" format body }) + case "fail" ⇒ channel ! Failure(new Exception("failure"), msg.headers) + case _ ⇒ channel ! (msg.transformBody { body: String ⇒ "received %s" format body }) } } } class ReplyingForwardTarget extends Actor { protected def receive = { - case msg: Message ⇒ - reply(msg.addHeader("test" -> "result")) - case msg: Failure ⇒ - reply(Failure(msg.cause, msg.headers + ("test" -> "failure"))) + case msg: Message ⇒ channel ! msg.addHeader("test" -> "result") + case msg: Failure ⇒ channel ! Failure(msg.cause, msg.headers + ("test" -> "failure")) } } diff --git a/akka-camel/src/test/scala/akka/camel/component/ActorComponentFeatureTest.scala b/akka-camel/src/test/scala/akka/camel/component/ActorComponentFeatureTest.scala index d2b4067e2b..c05ceffaaa 100644 --- a/akka-camel/src/test/scala/akka/camel/component/ActorComponentFeatureTest.scala +++ b/akka-camel/src/test/scala/akka/camel/component/ActorComponentFeatureTest.scala @@ -96,13 +96,13 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with object ActorComponentFeatureTest { class CustomIdActor extends Actor { protected def receive = { - case msg: Message ⇒ reply("Received %s" format msg.body) + case msg: Message ⇒ channel ! ("Received %s" format msg.body) } } class FailWithMessage extends Actor { protected def receive = { - case msg: Message ⇒ reply(Failure(new Exception("test"))) + case msg: Message ⇒ channel ! Failure(new Exception("test")) } } diff --git a/akka-docs/intro/code/tutorials/first/Pi.scala b/akka-docs/intro/code/tutorials/first/Pi.scala index 57e5b3558e..c346db3820 100644 --- a/akka-docs/intro/code/tutorials/first/Pi.scala +++ b/akka-docs/intro/code/tutorials/first/Pi.scala @@ -50,8 +50,7 @@ object Pi extends App { //#calculatePiFor def receive = { - case Work(start, nrOfElements) ⇒ - reply(Result(calculatePiFor(start, nrOfElements))) // perform the work + case Work(start, nrOfElements) ⇒ channel ! Result(calculatePiFor(start, nrOfElements)) // perform the work } } //#worker diff --git a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala index 449badf073..0266d14b60 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteDaemon.scala @@ -149,10 +149,10 @@ class RemoteDaemon(val remote: Remote) extends Actor { app.eventHandler.error(this, "Actor 'address' is not defined, ignoring remote daemon command [%s]".format(message)) } - reply(Success(address.toString)) + channel ! Success(address.toString) } catch { case error: Throwable ⇒ - reply(Failure(error)) + channel ! Failure(error) throw error } } @@ -184,7 +184,7 @@ class RemoteDaemon(val remote: Remote) extends Actor { new LocalActorRef(app, Props( context ⇒ { - case f: Function0[_] ⇒ try { reply(f()) } finally { context.self.stop() } + case f: Function0[_] ⇒ try { channel ! f() } finally { context.self.stop() } }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Function0[Any]]) } @@ -200,7 +200,7 @@ class RemoteDaemon(val remote: Remote) extends Actor { new LocalActorRef(app, Props( context ⇒ { - case (fun: Function[_, _], param: Any) ⇒ try { reply(fun.asInstanceOf[Any ⇒ Any](param)) } finally { context.self.stop() } + case (fun: Function[_, _], param: Any) ⇒ try { channel ! fun.asInstanceOf[Any ⇒ Any](param) } finally { context.self.stop() } }).copy(dispatcher = computeGridDispatcher), Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]]) } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala index 70c9bf99ef..c3cfedfc0b 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/direct_routed/DirectRoutedRemoteActorMultiJvmSpec.scala @@ -10,9 +10,7 @@ object DirectRoutedRemoteActorMultiJvmSpec { class SomeActor extends Actor with Serializable { def receive = { - case "identify" ⇒ { - reply(app.nodename) - } + case "identify" ⇒ channel ! app.nodename } } } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala index c38009cbeb..375b380f8c 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/new_remote_actor/NewRemoteActorMultiJvmSpec.scala @@ -8,9 +8,7 @@ object NewRemoteActorMultiJvmSpec { class SomeActor extends Actor with Serializable { def receive = { - case "identify" ⇒ { - reply(app.nodename) - } + case "identify" ⇒ channel ! app.nodename } } } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala index e7b6139b7a..5001227865 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/random_routed/RandomRoutedRemoteActorMultiJvmSpec.scala @@ -9,7 +9,7 @@ object RandomRoutedRemoteActorMultiJvmSpec { val NrOfNodes = 4 class SomeActor extends Actor with Serializable { def receive = { - case "hit" ⇒ reply(app.nodename) + case "hit" ⇒ channel ! app.nodename case "end" ⇒ self.stop() } } diff --git a/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala b/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala index 7bc65fe7db..afc8fa13fa 100644 --- a/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala +++ b/akka-remote/src/multi-jvm/scala/akka/remote/round_robin_routed/RoundRobinRoutedRemoteActorMultiJvmSpec.scala @@ -9,7 +9,7 @@ object RoundRobinRoutedRemoteActorMultiJvmSpec { val NrOfNodes = 4 class SomeActor extends Actor with Serializable { def receive = { - case "hit" ⇒ reply(app.nodename) + case "hit" ⇒ channel ! app.nodename case "end" ⇒ self.stop() } } diff --git a/akka-remote/src/test/scala/akka/serialization/ActorSerializeSpec.scala b/akka-remote/src/test/scala/akka/serialization/ActorSerializeSpec.scala index 6a31c6eea3..7c87c5e570 100644 --- a/akka-remote/src/test/scala/akka/serialization/ActorSerializeSpec.scala +++ b/akka-remote/src/test/scala/akka/serialization/ActorSerializeSpec.scala @@ -128,7 +128,7 @@ class MyJavaSerializableActor extends Actor with scala.Serializable { def receive = { case "hello" ⇒ count = count + 1 - reply("world " + count) + channel ! "world " + count } } @@ -136,7 +136,7 @@ class MyStatelessActorWithMessagesInMailbox extends Actor with scala.Serializabl def receive = { case "hello" ⇒ Thread.sleep(500) - case "hello-reply" ⇒ reply("world") + case "hello-reply" ⇒ channel ! "world" } } @@ -144,7 +144,7 @@ class MyActorWithProtobufMessagesInMailbox extends Actor with scala.Serializable def receive = { case m: Message ⇒ Thread.sleep(500) - case "hello-reply" ⇒ reply("world") + case "hello-reply" ⇒ channel ! "world" } } @@ -152,6 +152,6 @@ class PersonActorWithMessagesInMailbox extends Actor with scala.Serializable { def receive = { case p: Person ⇒ Thread.sleep(500) - case "hello-reply" ⇒ reply("hello") + case "hello-reply" ⇒ channel ! "hello" } } diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/UntypedConsumer1.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/UntypedConsumer1.java index 8a09ed2c0f..dc89a22ceb 100644 --- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/UntypedConsumer1.java +++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/UntypedConsumer1.java @@ -15,6 +15,6 @@ public class UntypedConsumer1 extends UntypedConsumerActor { public void onReceive(Object message) { Message msg = (Message)message; String body = msg.getBodyAs(String.class); - tryReply(String.format("received %s", body)); + channel.tryTell(String.format("received %s", body)); } } diff --git a/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Actors.scala b/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Actors.scala index 1208e5fae0..35654bc264 100644 --- a/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Actors.scala +++ b/akka-samples/akka-sample-camel/src/main/scala/sample/camel/Actors.scala @@ -12,7 +12,7 @@ class RemoteActor1 extends Actor with Consumer { def endpointUri = "jetty:http://localhost:6644/camel/remote-actor-1" protected def receive = { - case msg: Message ⇒ reply(Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote1"))) + case msg: Message ⇒ channel ! Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote1")) } } @@ -23,7 +23,7 @@ class RemoteActor2 extends Actor with Consumer { def endpointUri = "jetty:http://localhost:6644/camel/remote-actor-2" protected def receive = { - case msg: Message ⇒ reply(Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote2"))) + case msg: Message ⇒ channel ! Message("hello %s" format msg.bodyAs[String], Map("sender" -> "remote2")) } } @@ -44,7 +44,7 @@ class Consumer2 extends Actor with Consumer { def endpointUri = "jetty:http://0.0.0.0:8877/camel/default" def receive = { - case msg: Message ⇒ reply("Hello %s" format msg.bodyAs[String]) + case msg: Message ⇒ channel ! ("Hello %s" format msg.bodyAs[String]) } } @@ -62,10 +62,10 @@ class Consumer4 extends Actor with Consumer { def receive = { case msg: Message ⇒ msg.bodyAs[String] match { case "stop" ⇒ { - reply("Consumer4 stopped") + channel ! "Consumer4 stopped" self.stop } - case body ⇒ reply(body) + case body ⇒ channel ! body } } } @@ -76,7 +76,7 @@ class Consumer5 extends Actor with Consumer { def receive = { case _ ⇒ { Actor.actorOf[Consumer4] - reply("Consumer4 started") + channel ! "Consumer4 started" } } } @@ -106,7 +106,7 @@ class PublisherBridge(uri: String, publisher: ActorRef) extends Actor with Consu protected def receive = { case msg: Message ⇒ { publisher ! msg.bodyAs[String] - reply("message published") + channel ! "message published" } } } @@ -135,8 +135,8 @@ class HttpProducer(transformer: ActorRef) extends Actor with Producer { class HttpTransformer extends Actor { protected def receive = { - case msg: Message ⇒ reply(msg.transformBody { body: String ⇒ body replaceAll ("Akka ", "AKKA ") }) - case msg: Failure ⇒ reply(msg) + case msg: Message ⇒ channel ! (msg.transformBody { body: String ⇒ body replaceAll ("Akka ", "AKKA ") }) + case msg: Failure ⇒ channel ! msg } } @@ -150,11 +150,11 @@ class FileConsumer extends Actor with Consumer { case msg: Message ⇒ { if (counter == 2) { println("received %s" format msg.bodyAs[String]) - reply(Ack) + channel ! Ack } else { println("rejected %s" format msg.bodyAs[String]) counter += 1 - reply(Failure(new Exception("message number %s not accepted" format counter))) + channel ! Failure(new Exception("message number %s not accepted" format counter)) } } } diff --git a/akka-samples/akka-sample-camel/src/test/java/sample/camel/SampleRemoteUntypedConsumer.java b/akka-samples/akka-sample-camel/src/test/java/sample/camel/SampleRemoteUntypedConsumer.java index b0f3486e9a..cc4c5d8c48 100644 --- a/akka-samples/akka-sample-camel/src/test/java/sample/camel/SampleRemoteUntypedConsumer.java +++ b/akka-samples/akka-sample-camel/src/test/java/sample/camel/SampleRemoteUntypedConsumer.java @@ -15,7 +15,7 @@ public class SampleRemoteUntypedConsumer extends UntypedConsumerActor { Message msg = (Message)message; String body = msg.getBodyAs(String.class); String header = msg.getHeaderAs("test", String.class); - tryReply(String.format("%s %s", body, header)); + channel.tryTell(String.format("%s %s", body, header)); } } diff --git a/akka-samples/akka-sample-camel/src/test/scala/sample/camel/HttpConcurrencyTestStress.scala b/akka-samples/akka-sample-camel/src/test/scala/sample/camel/HttpConcurrencyTestStress.scala index 5a19bf5cdf..f10f050633 100644 --- a/akka-samples/akka-sample-camel/src/test/scala/sample/camel/HttpConcurrencyTestStress.scala +++ b/akka-samples/akka-sample-camel/src/test/scala/sample/camel/HttpConcurrencyTestStress.scala @@ -66,7 +66,7 @@ object HttpConcurrencyTestStress { var correlationIds = Set[Any]() override protected def receive = { - case "getCorrelationIdCount" ⇒ reply(correlationIds.size) + case "getCorrelationIdCount" ⇒ channel ! correlationIds.size case msg ⇒ super.receive(msg) } @@ -93,7 +93,7 @@ object HttpConcurrencyTestStress { class HttpServerWorker extends Actor { protected def receive = { - case msg ⇒ reply(msg) + case msg ⇒ channel ! msg } } } diff --git a/akka-samples/akka-sample-camel/src/test/scala/sample/camel/RemoteConsumerTest.scala b/akka-samples/akka-sample-camel/src/test/scala/sample/camel/RemoteConsumerTest.scala index 79ddae9bbc..e5433b1096 100644 --- a/akka-samples/akka-sample-camel/src/test/scala/sample/camel/RemoteConsumerTest.scala +++ b/akka-samples/akka-sample-camel/src/test/scala/sample/camel/RemoteConsumerTest.scala @@ -94,8 +94,8 @@ object RemoteConsumerTest { def endpointUri = "direct:remote-consumer" protected def receive = { - case "init" ⇒ reply("done") - case m: Message ⇒ reply("remote actor: %s" format m.body) + case "init" ⇒ channel ! "done" + case m: Message ⇒ channel ! ("remote actor: %s" format m.body) } } } diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala index a9ae2a63c4..175bd87c5c 100644 --- a/akka-stm/src/main/scala/akka/agent/Agent.scala +++ b/akka-stm/src/main/scala/akka/agent/Agent.scala @@ -288,8 +288,8 @@ class AgentUpdater[T](agent: Agent[T]) extends Actor { def receive = { case update: Update[_] ⇒ - tryReply(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T ⇒ T] }) - case Get ⇒ reply(agent.get) + channel.tryTell(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T ⇒ T] }) + case Get ⇒ channel ! agent.get case _ ⇒ () } } @@ -302,7 +302,7 @@ class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor { def receive = { case update: Update[_] ⇒ try { - tryReply(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T ⇒ T] }) + channel.tryTell(atomic(txFactory) { agent.ref alter update.function.asInstanceOf[T ⇒ T] }) } finally { agent.resume() self.stop() diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java index 128a25bcf0..035c195c7e 100644 --- a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java +++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java @@ -32,7 +32,7 @@ public class UntypedCoordinatedCounter extends UntypedActor { } else if (incoming instanceof String) { String message = (String) incoming; if (message.equals("GetCount")) { - reply(count.get()); + getChannel().tell(count.get()); } } } diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCounter.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCounter.java index 5cad6e0e64..377e3560da 100644 --- a/akka-stm/src/test/java/akka/transactor/example/UntypedCounter.java +++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCounter.java @@ -26,7 +26,7 @@ public class UntypedCounter extends UntypedTransactor { @Override public boolean normally(Object message) { if ("GetCount".equals(message)) { - reply(count.get()); + getChannel().tell(count.get()); return true; } else return false; } diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java index c23d961055..90636b99f3 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java @@ -57,7 +57,7 @@ public class UntypedCoordinatedCounter extends UntypedActor { } else if (incoming instanceof String) { String message = (String) incoming; if (message.equals("GetCount")) { - reply(count.get()); + getChannel().tell(count.get()); } } } diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java index 1b561a2d25..4e3f3fde71 100644 --- a/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java +++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java @@ -70,7 +70,7 @@ public class UntypedCounter extends UntypedTransactor { @Override public boolean normally(Object message) { if ("GetCount".equals(message)) { - reply(count.get()); + getChannel().tell(count.get()); return true; } else return false; } diff --git a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala index 38d0496f43..35b6128fb1 100644 --- a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala +++ b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala @@ -34,7 +34,7 @@ object CoordinatedIncrement { } } - case GetCount ⇒ reply(count.get) + case GetCount ⇒ channel ! count.get } } diff --git a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala index 29a1bd82e3..18b08f920c 100644 --- a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala +++ b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala @@ -56,7 +56,7 @@ object FickleFriends { } } - case GetCount ⇒ reply(count.get) + case GetCount ⇒ channel ! count.get } } @@ -93,7 +93,7 @@ object FickleFriends { } } - case GetCount ⇒ reply(count.get) + case GetCount ⇒ channel ! count.get } } } diff --git a/akka-stm/src/test/scala/transactor/TransactorSpec.scala b/akka-stm/src/test/scala/transactor/TransactorSpec.scala index 32b5034847..52313ace13 100644 --- a/akka-stm/src/test/scala/transactor/TransactorSpec.scala +++ b/akka-stm/src/test/scala/transactor/TransactorSpec.scala @@ -49,7 +49,7 @@ object TransactorIncrement { } override def normally = { - case GetCount ⇒ reply(count.get) + case GetCount ⇒ channel ! count.get } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index f4541f97f6..42abee6c04 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -224,14 +224,6 @@ class TestKit(_app: AkkaApplication) { */ def within[T](max: Duration)(f: ⇒ T): T = within(0 seconds, max)(f) - /** - * Send reply to the last dequeued message. Will throw - * IllegalActorStateException if no message has been dequeued, yet. Dequeuing - * means reception of the message as part of an expect... or receive... call, - * not reception by the testActor. - */ - def reply(msg: AnyRef) { lastMessage.channel.!(msg)(testActor) } - /** * Same as `expectMsg(remaining, obj)`, but correctly treating the timeFactor. */ diff --git a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala index 4546c60a85..3f1cc7d1d1 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestActorRefSpec.scala @@ -48,14 +48,14 @@ object TestActorRefSpec { val worker = TestActorRef(Props[WorkerActor]) worker ! channel case "workDone" ⇒ replyTo ! "complexReply" - case "simpleRequest" ⇒ reply("simpleReply") + case "simpleRequest" ⇒ channel ! "simpleReply" } } class WorkerActor() extends TActor { def receiveT = { case "work" ⇒ { - reply("workDone") + channel ! "workDone" self.stop() } case replyTo: UntypedChannel ⇒ { @@ -109,7 +109,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach { "used with TestActorRef" in { val a = TestActorRef(Props(new Actor { val nested = TestActorRef(Props(self ⇒ { case _ ⇒ })) - def receive = { case _ ⇒ reply(nested) } + def receive = { case _ ⇒ channel ! nested } })) a must not be (null) val nested = (a ? "any").as[ActorRef].get @@ -120,7 +120,7 @@ class TestActorRefSpec extends AkkaSpec with BeforeAndAfterEach { "used with ActorRef" in { val a = TestActorRef(Props(new Actor { val nested = context.actorOf(Props(self ⇒ { case _ ⇒ })) - def receive = { case _ ⇒ reply(nested) } + def receive = { case _ ⇒ channel ! nested } })) a must not be (null) val nested = (a ? "any").as[ActorRef].get diff --git a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala index 6da2bf6439..e162cad834 100644 --- a/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala +++ b/akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala @@ -16,7 +16,7 @@ class TestProbeSpec extends AkkaSpec { val tk = TestProbe() val future = tk.ref ? "hello" tk.expectMsg(0 millis, "hello") // TestActor runs on CallingThreadDispatcher - tk.reply("world") + tk.lastMessage.channel ! "world" future must be('completed) future.get must equal("world") } @@ -26,7 +26,7 @@ class TestProbeSpec extends AkkaSpec { val tk2 = TestProbe() tk1.ref.!("hello")(tk2.ref) tk1.expectMsg(0 millis, "hello") - tk1.reply("world") + tk1.lastMessage.channel ! "world" tk2.expectMsg(0 millis, "world") } @@ -35,7 +35,7 @@ class TestProbeSpec extends AkkaSpec { val probe2 = TestProbe() probe1.send(probe2.ref, "hello") probe2.expectMsg(0 millis, "hello") - probe2.reply("world") + probe2.lastMessage.channel ! "world" probe1.expectMsg(0 millis, "world") } diff --git a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java index 8db364396f..1a179052ec 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java +++ b/akka-tutorials/akka-tutorial-first/src/main/java/akka/tutorial/first/java/Pi.java @@ -83,7 +83,7 @@ public class Pi { double result = calculatePiFor(work.getStart(), work.getNrOfElements()); // reply with the result - reply(new Result(result)); + getChannel().tell(new Result(result)); } else throw new IllegalArgumentException("Unknown message [" + message + "]"); } diff --git a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala index 1150538109..05e05abf20 100644 --- a/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-first/src/main/scala/Pi.scala @@ -43,7 +43,7 @@ object Pi extends App { def receive = { case Work(start, nrOfElements) ⇒ - reply(Result(calculatePiFor(start, nrOfElements))) // perform the work + channel ! Result(calculatePiFor(start, nrOfElements)) // perform the work } } diff --git a/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java b/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java index 8b1e924101..55cd4dbe66 100644 --- a/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java +++ b/akka-tutorials/akka-tutorial-second/src/main/java/akka/tutorial/java/second/Pi.java @@ -80,7 +80,7 @@ public class Pi { public void onReceive(Object message) { if (message instanceof Work) { Work work = (Work) message; - reply(new Result(calculatePiFor(work.getArg(), work.getNrOfElements()))); // perform the work + getChannel().tell(new Result(calculatePiFor(work.getArg(), work.getNrOfElements()))); // perform the work } else throw new IllegalArgumentException("Unknown message [" + message + "]"); } } diff --git a/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala b/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala index 84541abc8a..67841c7a60 100644 --- a/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala +++ b/akka-tutorials/akka-tutorial-second/src/main/scala/Pi.scala @@ -41,7 +41,7 @@ object Pi extends App { def receive = { case Work(arg, nrOfElements) ⇒ - reply(Result(calculatePiFor(arg, nrOfElements))) // perform the work + channel ! Result(calculatePiFor(arg, nrOfElements)) // perform the work } } From f68c170ba892dd5994aa961c77d12429e8fe7954 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 17:11:09 +0200 Subject: [PATCH 24/33] Removing senderFuture, in preparation for 'sender ! response' --- akka-actor/src/main/scala/akka/actor/Actor.scala | 6 ------ akka-actor/src/main/scala/akka/actor/ActorCell.scala | 8 -------- .../src/main/scala/akka/actor/UntypedActor.scala | 10 ---------- 3 files changed, 24 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 34b95174f6..5d72c77d60 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -267,12 +267,6 @@ trait Actor { */ def sender: Option[ActorRef] = context.sender - /** - * The reference sender future of the last received message. - * Is defined if the message was sent with sent with '?'/'ask', else None. - */ - def senderFuture(): Option[Promise[Any]] = context.senderFuture - /** * Abstraction for unification of sender and senderFuture for later reply */ diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 392e935b4a..d54c25d92c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -36,8 +36,6 @@ private[akka] trait ActorContext extends ActorRefFactory { def sender: Option[ActorRef] - def senderFuture(): Option[Promise[Any]] - def channel: UntypedChannel def children: Iterable[ActorRef] @@ -298,12 +296,6 @@ private[akka] class ActorCell( case _ ⇒ None } - def senderFuture(): Option[Promise[Any]] = currentMessage match { - case null ⇒ None - case msg if msg.channel.isInstanceOf[ActorPromise] ⇒ Some(msg.channel.asInstanceOf[ActorPromise]) - case _ ⇒ None - } - def channel: UntypedChannel = currentMessage match { case null ⇒ NullChannel case msg ⇒ msg.channel diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index 4a4237b59d..f1f3c93e8c 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -28,10 +28,6 @@ import akka.dispatch.{ MessageDispatcher, Promise } * // also passing along my own reference (the self) * getSender().get().tell(msg, getSelf()); * - * } else if (msg.equals("UseSenderFuture") && getSenderFuture().isDefined()) { - * // Reply to original sender of message using the sender future reference - * getSenderFuture().get().completeWithResult(msg); - * * } else if (msg.equals("SendToSelf")) { * // Send message to the actor itself recursively * getSelf().tell(msg) @@ -73,12 +69,6 @@ abstract class UntypedActor extends Actor { */ def getSender: Option[ActorRef] = sender - /** - * The reference sender future of the last received message. - * Is defined if the message was sent with sent with '?'/'ask', else None. - */ - def getSenderFuture: Option[Promise[Any]] = senderFuture - /** * Abstraction for unification of sender and senderFuture for later reply */ From 83e17aa8eade14aee691863eb3a5e1cd224bff7b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 17:48:27 +0200 Subject: [PATCH 25/33] Removing the 'def config', removing the null check for every message being processed and adding some TODOs --- akka-actor/src/main/scala/akka/actor/Actor.scala | 16 ++++++---------- .../scala/akka/dispatch/AbstractDispatcher.scala | 4 +++- 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 5d72c77d60..c61ec6f0b9 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -212,12 +212,10 @@ trait Actor { implicit def app = context.app - private def config = context.app.AkkaConfig - /** * The default timeout, based on the config setting 'akka.actor.timeout' */ - implicit def defaultTimeout = config.ActorTimeout + implicit def defaultTimeout = app.AkkaConfig.ActorTimeout /** * Wrap a Receive partial function in a logging enclosure, which sends a @@ -233,7 +231,7 @@ trait Actor { * This method does NOT modify the given Receive unless * akka.actor.debug.receive is set within akka.conf. */ - def loggable(self: AnyRef)(r: Receive): Receive = if (config.AddLoggingReceive) LoggingReceive(self, r) else r + def loggable(self: AnyRef)(r: Receive): Receive = if (app.AkkaConfig.AddLoggingReceive) LoggingReceive(self, r) else r //TODO FIXME Shouldn't this be in a Loggable-trait? /** * Some[ActorRef] representation of the 'self' ActorRef reference. @@ -241,7 +239,7 @@ trait Actor { * Mainly for internal use, functions as the implicit sender references when invoking * the 'forward' function. */ - def someSelf: Some[ActorRef with ScalaActorRef] = Some(context.self) + def someSelf: Some[ActorRef with ScalaActorRef] = Some(context.self) //TODO FIXME we might not need this when we switch to sender-in-scope-always /* * Option[ActorRef] representation of the 'self' ActorRef reference. @@ -249,7 +247,7 @@ trait Actor { * Mainly for internal use, functions as the implicit sender references when invoking * one of the message send functions ('!' and '?'). */ - def optionSelf: Option[ActorRef with ScalaActorRef] = someSelf + def optionSelf: Option[ActorRef with ScalaActorRef] = someSelf //TODO FIXME we might not need this when we switch to sender-in-scope-always /** * The 'self' field holds the ActorRef for this actor. @@ -272,7 +270,7 @@ trait Actor { */ def channel: UntypedChannel = context.channel - // just for current compatibility + // TODO FIXME REMOVE ME just for current compatibility implicit def forwardable: ForwardableChannel = ForwardableChannel(channel) /** @@ -387,11 +385,9 @@ trait Actor { // ========================================= private[akka] final def apply(msg: Any) = { - if (msg.isInstanceOf[AnyRef] && (msg.asInstanceOf[AnyRef] eq null)) - throw new InvalidMessageException("Message from [" + channel + "] to [" + self + "] is null") def autoReceiveMessage(msg: AutoReceivedMessage) { - if (config.DebugAutoReceive) app.eventHandler.debug(this, "received AutoReceiveMessage " + msg) + if (app.AkkaConfig.DebugAutoReceive) app.eventHandler.debug(this, "received AutoReceiveMessage " + msg) msg match { case HotSwap(code, discardOld) ⇒ become(code(self), discardOld) diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index d7bdb68229..7bcf72667a 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -17,7 +17,9 @@ import scala.annotation.tailrec /** * @author Jonas Bonér */ -final case class Envelope(val message: Any, val channel: UntypedChannel) +final case class Envelope(val message: Any, val channel: UntypedChannel) { + if (message.isInstanceOf[AnyRef] && (message.asInstanceOf[AnyRef] eq null)) throw new InvalidMessageException("Message is null") +} object SystemMessage { @tailrec From 70bacc4c3d130b30de266648d27038c70b6f730d Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 18:23:23 +0200 Subject: [PATCH 26/33] Fixing yet another potential race in the DeathWatchSpec --- .../src/test/scala/akka/actor/DeathWatchSpec.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index c4622b081f..b344b3f49a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -12,7 +12,7 @@ import java.util.concurrent.atomic._ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSender { "The Death Watch" must { - def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(2 seconds, actorRef + ": Stopped") { + def expectTerminationOf(actorRef: ActorRef) = expectMsgPF(5 seconds, actorRef + ": Stopped or Already terminated when linking") { case Terminated(`actorRef`, ex: ActorKilledException) if ex.getMessage == "Stopped" || ex.getMessage == "Already terminated when linking" ⇒ true } @@ -21,11 +21,12 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende testActor startsMonitoring terminal + testActor ! "ping" + expectMsg("ping") + terminal ! PoisonPill expectTerminationOf(terminal) - - terminal.stop() } "notify with all monitors with one Terminated message when an Actor is stopped" in { @@ -42,7 +43,6 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende expectTerminationOf(terminal) expectTerminationOf(terminal) - terminal.stop() monitor1.stop() monitor2.stop() } From adccc9b89b7a52f63afa1295089e6ce8e18be6c2 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 18:23:39 +0200 Subject: [PATCH 27/33] Adding possibility to specify Actor.address to TypedActor --- .../main/scala/akka/actor/TypedActor.scala | 92 +++++++++++++++---- 1 file changed, 75 insertions(+), 17 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-actor/src/main/scala/akka/actor/TypedActor.scala index 43d98a5941..5ac65fb354 100644 --- a/akka-actor/src/main/scala/akka/actor/TypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/TypedActor.scala @@ -147,7 +147,15 @@ class TypedActor(val app: AkkaApplication) { * all interfaces (Class.getInterfaces) if it's not an interface class */ def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props): R = - createProxyAndTypedActor(interface, impl.newInstance, props, interface.getClassLoader) + createProxyAndTypedActor(interface, impl.newInstance, props, Props.randomAddress, interface.getClassLoader) + + /** + * Creates a new TypedActor proxy using the supplied Props, + * the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or + * all interfaces (Class.getInterfaces) if it's not an interface class + */ + def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props, address: String): R = + createProxyAndTypedActor(interface, impl.newInstance, props, address, interface.getClassLoader) /** * Creates a new TypedActor proxy using the supplied Props, @@ -155,10 +163,31 @@ class TypedActor(val app: AkkaApplication) { * all interfaces (Class.getInterfaces) if it's not an interface class */ def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props): R = - createProxyAndTypedActor(interface, impl.create, props, interface.getClassLoader) + createProxyAndTypedActor(interface, impl.create, props, Props.randomAddress, interface.getClassLoader) + /** + * Creates a new TypedActor proxy using the supplied Props, + * the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or + * all interfaces (Class.getInterfaces) if it's not an interface class + */ + def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props, address: String): R = + createProxyAndTypedActor(interface, impl.create, props, address, interface.getClassLoader) + + /** + * Creates a new TypedActor proxy using the supplied Props, + * the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or + * all interfaces (Class.getInterfaces) if it's not an interface class + */ def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props, loader: ClassLoader): R = - createProxyAndTypedActor(interface, impl.newInstance, props, loader) + createProxyAndTypedActor(interface, impl.newInstance, props, Props.randomAddress, loader) + + /** + * Creates a new TypedActor proxy using the supplied Props, + * the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or + * all interfaces (Class.getInterfaces) if it's not an interface class + */ + def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Class[T], props: Props, address: String, loader: ClassLoader): R = + createProxyAndTypedActor(interface, impl.newInstance, props, address, loader) /** * Creates a new TypedActor proxy using the supplied Props, @@ -166,22 +195,37 @@ class TypedActor(val app: AkkaApplication) { * all interfaces (Class.getInterfaces) if it's not an interface class */ def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props, loader: ClassLoader): R = - createProxyAndTypedActor(interface, impl.create, props, loader) + createProxyAndTypedActor(interface, impl.create, props, Props.randomAddress, loader) + + /** + * Creates a new TypedActor proxy using the supplied Props, + * the interfaces usable by the returned proxy is the supplied interface class (if the class represents an interface) or + * all interfaces (Class.getInterfaces) if it's not an interface class + */ + def typedActorOf[R <: AnyRef, T <: R](interface: Class[R], impl: Creator[T], props: Props, address: String, loader: ClassLoader): R = + createProxyAndTypedActor(interface, impl.create, props, address, loader) /** * Creates a new TypedActor proxy using the supplied Props, * the interfaces usable by the returned proxy is the supplied implementation class' interfaces (Class.getInterfaces) */ def typedActorOf[R <: AnyRef, T <: R](impl: Class[T], props: Props, loader: ClassLoader): R = - createProxyAndTypedActor(impl, impl.newInstance, props, loader) + createProxyAndTypedActor(impl, impl.newInstance, props, Props.randomAddress, loader) /** * Creates a new TypedActor proxy using the supplied Props, * the interfaces usable by the returned proxy is the supplied implementation class' interfaces (Class.getInterfaces) */ - def typedActorOf[R <: AnyRef, T <: R](props: Props = Props(), loader: ClassLoader = null)(implicit m: Manifest[T]): R = { + def typedActorOf[R <: AnyRef, T <: R](impl: Class[T], props: Props, address: String, loader: ClassLoader): R = + createProxyAndTypedActor(impl, impl.newInstance, props, address, loader) + + /** + * Creates a new TypedActor proxy using the supplied Props, + * the interfaces usable by the returned proxy is the supplied implementation class' interfaces (Class.getInterfaces) + */ + def typedActorOf[R <: AnyRef, T <: R](props: Props = Props(), address: String = Props.randomAddress, loader: ClassLoader = null)(implicit m: Manifest[T]): R = { val clazz = m.erasure.asInstanceOf[Class[T]] - createProxyAndTypedActor(clazz, clazz.newInstance, props, if (loader eq null) clazz.getClassLoader else loader) + createProxyAndTypedActor(clazz, clazz.newInstance, props, address, if (loader eq null) clazz.getClassLoader else loader) } /** @@ -209,22 +253,36 @@ class TypedActor(val app: AkkaApplication) { * Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself, * to create TypedActor proxies, use typedActorOf */ - def createProxy[R <: AnyRef](constructor: ⇒ Actor, props: Props = Props(), loader: ClassLoader = null)(implicit m: Manifest[R]): R = - createProxy[R](extractInterfaces(m.erasure), (ref: AtomVar[R]) ⇒ constructor, props, if (loader eq null) m.erasure.getClassLoader else loader) + def createProxy[R <: AnyRef](constructor: ⇒ Actor, props: Props = Props(), address: String = Props.randomAddress, loader: ClassLoader = null)(implicit m: Manifest[R]): R = + createProxy[R](extractInterfaces(m.erasure), (ref: AtomVar[R]) ⇒ constructor, props, address, if (loader eq null) m.erasure.getClassLoader else loader) /** * Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself, * to create TypedActor proxies, use typedActorOf */ def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: Creator[Actor], props: Props, loader: ClassLoader): R = - createProxy(interfaces, (ref: AtomVar[R]) ⇒ constructor.create, props, loader) + createProxy(interfaces, (ref: AtomVar[R]) ⇒ constructor.create, props, Props.randomAddress, loader) + + /** + * Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself, + * to create TypedActor proxies, use typedActorOf + */ + def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: Creator[Actor], props: Props, address: String, loader: ClassLoader): R = + createProxy(interfaces, (ref: AtomVar[R]) ⇒ constructor.create, props, address, loader) /** * Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself, * to create TypedActor proxies, use typedActorOf */ def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: ⇒ Actor, props: Props, loader: ClassLoader): R = - createProxy[R](interfaces, (ref: AtomVar[R]) ⇒ constructor, props, loader) + createProxy[R](interfaces, (ref: AtomVar[R]) ⇒ constructor, props, Props.randomAddress, loader) + + /** + * Creates a proxy given the supplied Props, this is not a TypedActor, so you'll need to implement the MethodCall handling yourself, + * to create TypedActor proxies, use typedActorOf + */ + def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: ⇒ Actor, props: Props, address: String, loader: ClassLoader): R = + createProxy[R](interfaces, (ref: AtomVar[R]) ⇒ constructor, props, address, loader) /* Internal API */ @@ -239,15 +297,15 @@ class TypedActor(val app: AkkaApplication) { } else null - private[akka] def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: (AtomVar[R]) ⇒ Actor, props: Props, loader: ClassLoader): R = { + private[akka] def createProxy[R <: AnyRef](interfaces: Array[Class[_]], constructor: (AtomVar[R]) ⇒ Actor, props: Props, address: String, loader: ClassLoader): R = { val proxyVar = new AtomVar[R] - configureAndProxyLocalActorRef[R](interfaces, proxyVar, props.withCreator(constructor(proxyVar)), loader) + configureAndProxyLocalActorRef[R](interfaces, proxyVar, props.withCreator(constructor(proxyVar)), address, loader) } - private[akka] def createProxyAndTypedActor[R <: AnyRef, T <: R](interface: Class[_], constructor: ⇒ T, props: Props, loader: ClassLoader): R = - createProxy[R](extractInterfaces(interface), (ref: AtomVar[R]) ⇒ new TypedActor[R, T](ref, constructor), props, loader) + private[akka] def createProxyAndTypedActor[R <: AnyRef, T <: R](interface: Class[_], constructor: ⇒ T, props: Props, address: String, loader: ClassLoader): R = + createProxy[R](extractInterfaces(interface), (ref: AtomVar[R]) ⇒ new TypedActor[R, T](ref, constructor), props, address, loader) - private[akka] def configureAndProxyLocalActorRef[T <: AnyRef](interfaces: Array[Class[_]], proxyVar: AtomVar[T], props: Props, loader: ClassLoader): T = { + private[akka] def configureAndProxyLocalActorRef[T <: AnyRef](interfaces: Array[Class[_]], proxyVar: AtomVar[T], props: Props, address: String, loader: ClassLoader): T = { //Warning, do not change order of the following statements, it's some elaborate chicken-n-egg handling val actorVar = new AtomVar[ActorRef](null) val timeout = props.timeout match { @@ -256,7 +314,7 @@ class TypedActor(val app: AkkaApplication) { } val proxy: T = Proxy.newProxyInstance(loader, interfaces, new TypedActorInvocationHandler(actorVar)(timeout)).asInstanceOf[T] proxyVar.set(proxy) // Chicken and egg situation we needed to solve, set the proxy so that we can set the self-reference inside each receive - val ref = app.actorOf(props) + val ref = app.actorOf(props, address) actorVar.set(ref) //Make sure the InvocationHandler gets ahold of the actor reference, this is not a problem since the proxy hasn't escaped this method yet proxyVar.get } From 57e994386da48601fc177851eadee4ed27fb82b8 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 22:20:53 +0200 Subject: [PATCH 28/33] Making sender always return an ActorRef, which will be the DeadLetterActor if there is no real sender --- akka-actor-tests/src/test/scala/akka/actor/Bench.scala | 4 ++-- akka-actor/src/main/scala/akka/AkkaApplication.scala | 2 ++ akka-actor/src/main/scala/akka/actor/Actor.scala | 3 ++- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 10 +++++----- .../src/main/scala/akka/actor/UntypedActor.scala | 2 +- 5 files changed, 12 insertions(+), 9 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/Bench.scala b/akka-actor-tests/src/test/scala/akka/actor/Bench.scala index 9aa5ffc05d..492a3e6680 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/Bench.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/Bench.scala @@ -44,7 +44,7 @@ object Chameneos { case Exit ⇒ colour = FADED - sender.get ! MeetingCount(meetings) + sender ! MeetingCount(meetings) } def complement(otherColour: Colour): Colour = colour match { @@ -97,7 +97,7 @@ object Chameneos { n -= 1 chameneo ! msg waitingChameneo = None - case None ⇒ waitingChameneo = sender + case None ⇒ waitingChameneo = Some(sender) } } else { waitingChameneo.foreach(_ ! Exit) diff --git a/akka-actor/src/main/scala/akka/AkkaApplication.scala b/akka-actor/src/main/scala/akka/AkkaApplication.scala index ce67f66745..51bf5a3062 100644 --- a/akka-actor/src/main/scala/akka/AkkaApplication.scala +++ b/akka-actor/src/main/scala/akka/AkkaApplication.scala @@ -170,6 +170,8 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor val reflective = new ReflectiveAccess(this) + val deadLetterRecipient = new DeadLetterActorRef(this) + // TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor val deployer = new Deployer(this) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index c61ec6f0b9..d56a64ea0e 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -263,7 +263,8 @@ trait Actor { * The reference sender Actor of the last received message. * Is defined if the message was sent from another Actor, else None. */ - def sender: Option[ActorRef] = context.sender + @inline + final def sender: ActorRef = context.sender /** * Abstraction for unification of sender and senderFuture for later reply diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index d54c25d92c..a12771037c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -34,7 +34,7 @@ private[akka] trait ActorContext extends ActorRefFactory { def currentMessage_=(invocation: Envelope): Unit - def sender: Option[ActorRef] + def sender: ActorRef def channel: UntypedChannel @@ -290,10 +290,10 @@ private[akka] class ActorCell( future } - def sender: Option[ActorRef] = currentMessage match { - case null ⇒ None - case msg if msg.channel.isInstanceOf[ActorRef] ⇒ Some(msg.channel.asInstanceOf[ActorRef]) - case _ ⇒ None + def sender: ActorRef = currentMessage match { + case null ⇒ app.deadLetterRecipient + case msg if msg.channel.isInstanceOf[ActorRef] ⇒ msg.channel.asInstanceOf[ActorRef] + case _ ⇒ app.deadLetterRecipient } def channel: UntypedChannel = currentMessage match { diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala index f1f3c93e8c..ce7ce1a3f2 100644 --- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala +++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala @@ -67,7 +67,7 @@ abstract class UntypedActor extends Actor { * The reference sender Actor of the last received message. * Is defined if the message was sent from another Actor, else None. */ - def getSender: Option[ActorRef] = sender + def getSender: ActorRef = sender /** * Abstraction for unification of sender and senderFuture for later reply From bf4af15817672bc5400e1af6b94e37f7e765105b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 22:38:35 +0200 Subject: [PATCH 29/33] Making the DeadLetterActorRef push notifications to the EventHandler --- akka-actor/src/main/scala/akka/AkkaApplication.scala | 6 ++---- akka-actor/src/main/scala/akka/actor/ActorCell.scala | 4 ++-- akka-actor/src/main/scala/akka/actor/ActorRef.scala | 7 +++++-- 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/akka-actor/src/main/scala/akka/AkkaApplication.scala b/akka-actor/src/main/scala/akka/AkkaApplication.scala index 51bf5a3062..a7a9dfddc5 100644 --- a/akka-actor/src/main/scala/akka/AkkaApplication.scala +++ b/akka-actor/src/main/scala/akka/AkkaApplication.scala @@ -5,6 +5,7 @@ package akka import akka.config._ import akka.actor._ +import event._ import java.net.InetAddress import com.eaio.uuid.UUID import dispatch.{ Dispatcher, Dispatchers } @@ -16,9 +17,6 @@ import akka.dispatch.UnboundedMailbox import akka.routing.Routing import remote.RemoteSupport import akka.serialization.Serialization -import akka.event.EventHandler -import akka.event.EventHandlerLogging -import akka.event.Logging import java.net.InetSocketAddress object AkkaApplication { @@ -170,7 +168,7 @@ class AkkaApplication(val name: String, val config: Configuration) extends Actor val reflective = new ReflectiveAccess(this) - val deadLetterRecipient = new DeadLetterActorRef(this) + val deadLetters = new DeadLetterActorRef(this) // TODO think about memory consistency effects when doing funky stuff inside an ActorRefProvider's constructor val deployer = new Deployer(this) diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index a12771037c..bcc36406e2 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -291,9 +291,9 @@ private[akka] class ActorCell( } def sender: ActorRef = currentMessage match { - case null ⇒ app.deadLetterRecipient + case null ⇒ app.deadLetters case msg if msg.channel.isInstanceOf[ActorRef] ⇒ msg.channel.asInstanceOf[ActorRef] - case _ ⇒ app.deadLetterRecipient + case _ ⇒ app.deadLetters } def channel: UntypedChannel = currentMessage match { diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 188cb6b00f..89cb0cb207 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -9,6 +9,7 @@ import akka.util._ import scala.collection.immutable.Stack import java.lang.{ UnsupportedOperationException, IllegalStateException } import akka.AkkaApplication +import akka.event.ActorEventBus /** * ActorRef is an immutable and serializable handle to an Actor. @@ -342,6 +343,8 @@ trait UnsupportedActorRef extends ActorRef with ScalaActorRef { private def unsupported = throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName)) } +case class DeadLetter(message: Any, channel: UntypedChannel) + class DeadLetterActorRef(app: AkkaApplication) extends UnsupportedActorRef { val brokenPromise = new KeptPromise[Any](Left(new ActorKilledException("In DeadLetterActorRef, promises are always broken.")))(app.dispatcher) val address: String = "akka:internal:DeadLetterActorRef" @@ -356,10 +359,10 @@ class DeadLetterActorRef(app: AkkaApplication) extends UnsupportedActorRef { def stop(): Unit = () - protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = app.eventHandler.debug(this, message) + protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = app.eventHandler.notify(DeadLetter(message, channel)) protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( message: Any, timeout: Timeout, - channel: UntypedChannel): Future[Any] = { app.eventHandler.debug(this, message); brokenPromise } + channel: UntypedChannel): Future[Any] = { app.eventHandler.notify(DeadLetter(message, channel)); brokenPromise } } From 4fc108027eb51898f11b207458f4e64b95fd6149 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 19 Oct 2011 23:21:06 +0200 Subject: [PATCH 30/33] I've stopped hating Jenkins, fixed the pesky elusing DeathWatch bug --- akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala index 7133c82fcd..0e62699243 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRefProvider.scala @@ -185,7 +185,7 @@ class LocalDeathWatch extends DeathWatch with ActorClassification { override def subscribe(subscriber: Subscriber, to: Classifier): Boolean = { if (!super.subscribe(subscriber, to)) { - subscriber ! Terminated(subscriber, new ActorKilledException("Already terminated when linking")) + subscriber ! Terminated(to, new ActorKilledException("Already terminated when linking")) false } else true } From 6a2f203d26816c4ee564321541ef04fc8f15620b Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 20 Oct 2011 11:46:13 +0200 Subject: [PATCH 31/33] Rewriting DeathWatchSpec and FSMTransitionSpec to do the startsMonitoring inside the Actor --- .../scala/akka/actor/DeathWatchSpec.scala | 41 +++++++++++-------- .../scala/akka/actor/FSMTransitionSpec.scala | 8 +++- 2 files changed, 30 insertions(+), 19 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index b344b3f49a..62f31a6c48 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -30,12 +30,12 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende } "notify with all monitors with one Terminated message when an Actor is stopped" in { - val monitor1, monitor2 = actorOf(Props(context ⇒ { case t: Terminated ⇒ testActor ! t })) val terminal = actorOf(Props(context ⇒ { case _ ⇒ })) - - monitor1 startsMonitoring terminal - monitor2 startsMonitoring terminal - testActor startsMonitoring terminal + val monitor1, monitor2, monitor3 = + actorOf(Props(new Actor { + self startsMonitoring terminal + def receive = { case t: Terminated ⇒ testActor ! t } + })) terminal ! PoisonPill @@ -45,20 +45,24 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende monitor1.stop() monitor2.stop() + monitor3.stop() } "notify with _current_ monitors with one Terminated message when an Actor is stopped" in { - val monitor1, monitor2 = actorOf(Props(context ⇒ { - case t: Terminated ⇒ testActor ! t - case "ping" ⇒ context.channel ! "pong" - })) val terminal = actorOf(Props(context ⇒ { case _ ⇒ })) - - monitor1 startsMonitoring terminal - monitor2 startsMonitoring terminal - testActor startsMonitoring terminal - - monitor2 stopsMonitoring terminal + val monitor1, monitor3 = + actorOf(Props(new Actor { + self startsMonitoring terminal + def receive = { case t: Terminated ⇒ testActor ! t } + })) + val monitor2 = actorOf(Props(new Actor { + self startsMonitoring terminal + self stopsMonitoring terminal + def receive = { + case "ping" ⇒ sender ! "pong" + case t: Terminated ⇒ testActor ! t + } + })) monitor2 ! "ping" @@ -71,14 +75,17 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende monitor1.stop() monitor2.stop() + monitor3.stop() } "notify with a Terminated message once when an Actor is stopped but not when restarted" in { filterException[ActorKilledException] { val supervisor = actorOf(Props(context ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(2)))) val terminal = actorOf(Props(context ⇒ { case x ⇒ context.channel ! x }).withSupervisor(supervisor)) - - testActor startsMonitoring terminal + val monitor = actorOf(Props(new Actor { + self startsMonitoring terminal + def receive = { case t: Terminated ⇒ testActor ! t } + }).withSupervisor(supervisor)) terminal ! Kill terminal ! Kill diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala index 6a98ac5448..c72c095ce2 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTransitionSpec.scala @@ -55,8 +55,12 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender { "not fail when listener goes away" in { val forward = actorOf(new Forwarder(testActor)) - val sup = actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) - val fsm = sup startsMonitoring actorOf(new MyFSM(testActor)) + val fsm = actorOf(new MyFSM(testActor)) + val sup = actorOf(Props(new Actor { + self startsMonitoring fsm + def receive = { case _ ⇒ } + }).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) + within(300 millis) { fsm ! SubscribeTransitionCallBack(forward) expectMsg(CurrentState(fsm, 0)) From 25f436d521bb4a9bce6b5e20c372e02b98b940f7 Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 20 Oct 2011 13:43:19 +0200 Subject: [PATCH 32/33] add TestKit.fishForMessage --- .../src/main/scala/akka/testkit/TestKit.scala | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala index 42abee6c04..fa4896d0df 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestKit.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestKit.scala @@ -263,6 +263,27 @@ class TestKit(_app: AkkaApplication) { f(o) } + /** + * Hybrid of expectMsgPF and receiveWhile: receive messages while the + * partial function matches and returns false. Use it to ignore certain + * messages while waiting for a specific message. + * + * @return the last received messsage, i.e. the first one for which the + * partial function returned true + */ + def fishForMessage(max: Duration = Duration.MinusInf, hint: String = "")(f: PartialFunction[Any, Boolean]): Any = { + val _max = if (max eq Duration.MinusInf) remaining else max.dilated + val end = now + _max + @tailrec + def recv: Any = { + val o = receiveOne(end - now) + assert(o ne null, "timeout during fishForMessage, hint: " + hint) + assert(f.isDefinedAt(o), "fishForMessage(" + hint + ") found unexpected message " + o) + if (f(o)) o else recv + } + recv + } + /** * Same as `expectMsgType[T](remaining)`, but correctly treating the timeFactor. */ From e0a7b88fc56165fca3459c22ba4de1fa37eebe3c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 20 Oct 2011 13:56:57 +0200 Subject: [PATCH 33/33] Adding Actor.watch and Actor.unwatch - verrrry niiiice --- .../src/test/scala/akka/actor/DeathWatchSpec.scala | 10 +++++----- akka-actor/src/main/scala/akka/actor/Actor.scala | 12 ++++++++++++ 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala index 62f31a6c48..f3b35a1091 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/DeathWatchSpec.scala @@ -33,7 +33,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende val terminal = actorOf(Props(context ⇒ { case _ ⇒ })) val monitor1, monitor2, monitor3 = actorOf(Props(new Actor { - self startsMonitoring terminal + watch(terminal) def receive = { case t: Terminated ⇒ testActor ! t } })) @@ -52,12 +52,12 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende val terminal = actorOf(Props(context ⇒ { case _ ⇒ })) val monitor1, monitor3 = actorOf(Props(new Actor { - self startsMonitoring terminal + watch(terminal) def receive = { case t: Terminated ⇒ testActor ! t } })) val monitor2 = actorOf(Props(new Actor { - self startsMonitoring terminal - self stopsMonitoring terminal + watch(terminal) + unwatch(terminal) def receive = { case "ping" ⇒ sender ! "pong" case t: Terminated ⇒ testActor ! t @@ -83,7 +83,7 @@ class DeathWatchSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende val supervisor = actorOf(Props(context ⇒ { case _ ⇒ }).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(2)))) val terminal = actorOf(Props(context ⇒ { case x ⇒ context.channel ! x }).withSupervisor(supervisor)) val monitor = actorOf(Props(new Actor { - self startsMonitoring terminal + watch(terminal) def receive = { case t: Terminated ⇒ testActor ! t } }).withSupervisor(supervisor)) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index d56a64ea0e..896a89471c 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -381,6 +381,18 @@ trait Actor { if (h.nonEmpty) context.hotswap = h.pop } + /** + * Registers this actor as a Monitor for the provided ActorRef + * @returns the provided ActorRef + */ + def watch(subject: ActorRef): ActorRef = self startsMonitoring subject + + /** + * Unregisters this actor as Monitor for the provided ActorRef + * @returns the provided ActorRef + */ + def unwatch(subject: ActorRef): ActorRef = self stopsMonitoring subject + // ========================================= // ==== INTERNAL IMPLEMENTATION DETAILS ==== // =========================================