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 f08dd47425..dcca865b24 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/ActorRefSpec.scala @@ -129,7 +129,7 @@ class ActorRefSpec extends WordSpec with MustMatchers { }) } - def contextStackMustBeEmpty = ActorInstance.contextStack.get.headOption must be === None + def contextStackMustBeEmpty = ActorCell.contextStack.get.headOption must be === None contextStackMustBeEmpty 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 ffe6c7b831..f1520cac4c 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 @@ -103,22 +103,22 @@ object ActorModelSpec { stats.get(actorRef) } - abstract override def suspend(actor: ActorInstance) { + abstract override def suspend(actor: ActorCell) { super.suspend(actor) getStats(actor.ref).suspensions.incrementAndGet() } - abstract override def resume(actor: ActorInstance) { + abstract override def resume(actor: ActorCell) { super.resume(actor) getStats(actor.ref).resumes.incrementAndGet() } - protected[akka] abstract override def register(actor: ActorInstance) { + protected[akka] abstract override def register(actor: ActorCell) { super.register(actor) getStats(actor.ref).registers.incrementAndGet() } - protected[akka] abstract override def unregister(actor: ActorInstance) { + protected[akka] abstract override def unregister(actor: ActorCell) { super.unregister(actor) getStats(actor.ref).unregisters.incrementAndGet() } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index a9eee30c9d..235972ec61 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -427,7 +427,7 @@ trait Actor { */ @transient private[akka] val context: ActorContext = { - val contextStack = ActorInstance.contextStack.get + val contextStack = ActorCell.contextStack.get def noContextError = { throw new ActorInitializationException( @@ -440,7 +440,7 @@ trait Actor { if (contextStack.isEmpty) noContextError val context = contextStack.head if (context eq null) noContextError - ActorInstance.contextStack.set(contextStack.push(null)) + ActorCell.contextStack.set(contextStack.push(null)) context } diff --git a/akka-actor/src/main/scala/akka/actor/ActorInstance.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala similarity index 98% rename from akka-actor/src/main/scala/akka/actor/ActorInstance.scala rename to akka-actor/src/main/scala/akka/actor/ActorCell.scala index 8a9b77a3ae..2d17abf2d5 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorInstance.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -15,7 +15,7 @@ import scala.annotation.tailrec import scala.collection.immutable.Stack /** - * The actor context - the view into the actor instance from the actor. + * The actor context - the view of the actor cell from the actor. * Exposes contextual information for the actor and the current message. * TODO: everything here for current compatibility - could be limited more */ @@ -48,7 +48,7 @@ private[akka] trait ActorContext { def handleDeath(death: Death) } -private[akka] object ActorInstance { +private[akka] object ActorCell { sealed trait Status object Status { object Running extends Status @@ -60,14 +60,14 @@ private[akka] object ActorInstance { } } -private[akka] class ActorInstance( +private[akka] class ActorCell( val self: ActorRef with ScalaActorRef, props: Props, _receiveTimeout: Option[Long], _hotswap: Stack[PartialFunction[Any, Unit]]) extends ActorContext { - import ActorInstance._ + import ActorCell._ val guard = new ReentrantGuard // TODO: remove this last synchronization point @@ -471,9 +471,9 @@ private[akka] class ActorInstance( override def hashCode: Int = HashCode.hash(HashCode.SEED, uuid) override def equals(that: Any): Boolean = { - that.isInstanceOf[ActorInstance] && that.asInstanceOf[ActorInstance].uuid == uuid + that.isInstanceOf[ActorCell] && that.asInstanceOf[ActorCell].uuid == uuid } - override def toString = "ActorInstance[%s]".format(uuid) + override def toString = "ActorCell[%s]".format(uuid) } diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 4401013b18..a5e761d86f 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -205,36 +205,36 @@ class LocalActorRef private[akka] ( this(__props, __address, false, __uuid, __receiveTimeout, __hotswap) - actorInstance.setActorContext(actorInstance) // this is needed for deserialization - why? + actorCell.setActorContext(actorCell) // this is needed for deserialization - why? } - private[this] val actorInstance = new ActorInstance(this, props, receiveTimeout, hotswap) - actorInstance.start() + private[this] val actorCell = new ActorCell(this, props, receiveTimeout, hotswap) + actorCell.start() /** * Is the actor running? */ - def isRunning: Boolean = actorInstance.isRunning + def isRunning: Boolean = actorCell.isRunning /** * Is the actor shut down? */ - def isShutdown: Boolean = actorInstance.isShutdown + def isShutdown: Boolean = actorCell.isShutdown /** * Suspends the actor. It will not process messages while suspended. */ - def suspend(): Unit = actorInstance.suspend() + def suspend(): Unit = actorCell.suspend() /** * Resumes a suspended actor. */ - def resume(): Unit = actorInstance.resume() + def resume(): Unit = actorCell.resume() /** * Shuts down the actor and its message queue */ - def stop(): Unit = actorInstance.stop() + def stop(): Unit = actorCell.stop() /** * Links an other actor to this actor. Links are unidirectional and means that a the linking actor will @@ -247,7 +247,7 @@ class LocalActorRef private[akka] ( * To be invoked from within the actor itself. * Returns the ref that was passed into it */ - def link(actorRef: ActorRef): ActorRef = actorInstance.link(actorRef) + def link(actorRef: ActorRef): ActorRef = actorCell.link(actorRef) /** * Unlink the actor. @@ -255,40 +255,40 @@ class LocalActorRef private[akka] ( * To be invoked from within the actor itself. * Returns the ref that was passed into it */ - def unlink(actorRef: ActorRef): ActorRef = actorInstance.unlink(actorRef) + def unlink(actorRef: ActorRef): ActorRef = actorCell.unlink(actorRef) /** * Returns the supervisor, if there is one. */ - def supervisor: Option[ActorRef] = actorInstance.supervisor + def supervisor: Option[ActorRef] = actorCell.supervisor // ========= AKKA PROTECTED FUNCTIONS ========= - protected[akka] def actorClass: Class[_] = actorInstance.actorClass + protected[akka] def actorClass: Class[_] = actorCell.actorClass - protected[akka] def underlying: ActorInstance = actorInstance + protected[akka] def underlying: ActorCell = actorCell - protected[akka] def underlyingActorInstance: Actor = actorInstance.actor.get + protected[akka] def underlyingActorInstance: Actor = actorCell.actor.get protected[akka] override def timeout: Long = props.timeout.duration.toMillis // TODO: remove this if possible protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = - actorInstance.supervisor = sup + actorCell.supervisor = sup protected[akka] def postMessageToMailbox(message: Any, channel: UntypedChannel): Unit = - actorInstance.postMessageToMailbox(message, channel) + actorCell.postMessageToMailbox(message, channel) protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout( message: Any, timeout: Timeout, channel: UntypedChannel): Future[Any] = { - actorInstance.postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, channel) + actorCell.postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, channel) } - protected[akka] def handleDeath(death: Death): Unit = actorInstance.handleDeath(death) + protected[akka] def handleDeath(death: Death): Unit = actorCell.handleDeath(death) protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = - actorInstance.restart(reason, maxNrOfRetries, withinTimeRange) + actorCell.restart(reason, maxNrOfRetries, withinTimeRange) // ========= PRIVATE FUNCTIONS ========= diff --git a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala index 6370780156..5997d922cd 100644 --- a/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/BalancingDispatcher.scala @@ -5,7 +5,7 @@ package akka.dispatch import util.DynamicVariable -import akka.actor.{ ActorInstance, Actor, IllegalActorStateException } +import akka.actor.{ ActorCell, Actor, IllegalActorStateException } /** * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed @@ -50,15 +50,15 @@ class BalancingDispatcher( this(_name, Dispatchers.THROUGHPUT, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType) // Needed for Java API usage @volatile - private var members = Vector[ActorInstance]() + private var members = Vector[ActorCell]() private val donationInProgress = new DynamicVariable(false) - protected[akka] override def register(actor: ActorInstance) = { + protected[akka] override def register(actor: ActorCell) = { members :+= actor //Update members, doesn't need synchronized, is guarded in attach super.register(actor) } - protected[akka] override def unregister(actor: ActorInstance) = { + protected[akka] override def unregister(actor: ActorCell) = { members = members.filterNot(actor eq) //Update members, doesn't need synchronized, is guarded in detach super.unregister(actor) } @@ -114,7 +114,7 @@ class BalancingDispatcher( * Rewrites the message and adds that message to the recipients mailbox * returns true if the message is non-null */ - protected def donate(organ: MessageInvocation, recipient: ActorInstance): Boolean = { + protected def donate(organ: MessageInvocation, recipient: ActorCell): Boolean = { if (organ ne null) { recipient.postMessageToMailbox(organ.message, organ.channel) true @@ -124,10 +124,10 @@ class BalancingDispatcher( /** * Returns an available recipient for the message, if any */ - protected def doFindDonorRecipient(donorMbox: MessageQueue with ExecutableMailbox, potentialRecipients: Vector[ActorInstance], startIndex: Int): ActorInstance = { + protected def doFindDonorRecipient(donorMbox: MessageQueue with ExecutableMailbox, potentialRecipients: Vector[ActorCell], startIndex: Int): ActorCell = { val prSz = potentialRecipients.size var i = 0 - var recipient: ActorInstance = null + var recipient: ActorCell = null while ((i < prSz) && (recipient eq null)) { val actor = potentialRecipients((i + startIndex) % prSz) //Wrap-around, one full lap diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala index 49aecc70a4..21542f064f 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Dispatcher.scala @@ -7,7 +7,7 @@ package akka.dispatch import akka.event.EventHandler import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.{ TimeUnit, ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue } -import akka.actor.{ ActorInstance, ActorKilledException } +import akka.actor.{ ActorCell, ActorKilledException } /** * Default settings are: @@ -108,13 +108,13 @@ class Dispatcher( /** * @return the mailbox associated with the actor */ - protected def getMailbox(receiver: ActorInstance) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox] + protected def getMailbox(receiver: ActorCell) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox] - override def mailboxIsEmpty(actor: ActorInstance): Boolean = getMailbox(actor).isEmpty + override def mailboxIsEmpty(actor: ActorCell): Boolean = getMailbox(actor).isEmpty - override def mailboxSize(actor: ActorInstance): Int = getMailbox(actor).size + override def mailboxSize(actor: ActorCell): Int = getMailbox(actor).size - def createMailbox(actor: ActorInstance): AnyRef = mailboxType match { + def createMailbox(actor: ActorCell): AnyRef = mailboxType match { case b: UnboundedMailbox ⇒ new ConcurrentLinkedQueue[MessageInvocation] with MessageQueue with ExecutableMailbox { @inline @@ -160,7 +160,7 @@ class Dispatcher( protected[akka] def reRegisterForExecution(mbox: MessageQueue with ExecutableMailbox): Unit = registerForExecution(mbox) - protected override def cleanUpMailboxFor(actor: ActorInstance) { + protected override def cleanUpMailboxFor(actor: ActorCell) { val m = getMailbox(actor) if (!m.isEmpty) { var invocation = m.dequeue @@ -174,10 +174,10 @@ class Dispatcher( override val toString = getClass.getSimpleName + "[" + name + "]" - def suspend(actor: ActorInstance): Unit = + def suspend(actor: ActorCell): Unit = getMailbox(actor).suspended.tryLock - def resume(actor: ActorInstance): Unit = { + def resume(actor: ActorCell): Unit = { val mbox = getMailbox(actor) mbox.suspended.tryUnlock reRegisterForExecution(mbox) @@ -296,7 +296,7 @@ class PriorityDispatcher( trait PriorityMailbox { self: Dispatcher ⇒ def comparator: java.util.Comparator[MessageInvocation] - override def createMailbox(actor: ActorInstance): AnyRef = self.mailboxType match { + override def createMailbox(actor: ActorCell): AnyRef = self.mailboxType match { case b: UnboundedMailbox ⇒ new UnboundedPriorityMessageQueue(comparator) with ExecutableMailbox { @inline diff --git a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala index c8af167d2c..2156c06a6b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala +++ b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala @@ -16,7 +16,7 @@ import akka.actor._ /** * @author Jonas Bonér */ -final case class MessageInvocation(val receiver: ActorInstance, +final case class MessageInvocation(val receiver: ActorCell, val message: Any, val channel: UntypedChannel) { if (receiver eq null) throw new IllegalArgumentException("Receiver can't be null") @@ -62,7 +62,7 @@ abstract class MessageDispatcher extends Serializable { /** * Creates and returns a mailbox for the given actor. */ - protected[akka] def createMailbox(actor: ActorInstance): AnyRef + protected[akka] def createMailbox(actor: ActorCell): AnyRef /** * Name of this dispatcher. @@ -72,7 +72,7 @@ abstract class MessageDispatcher extends Serializable { /** * Attaches the specified actor instance to this dispatcher */ - final def attach(actor: ActorInstance): Unit = { + final def attach(actor: ActorCell): Unit = { val promise = new ActorPromise(Timeout.never)(this) guard.lock.lock() try { @@ -87,7 +87,7 @@ abstract class MessageDispatcher extends Serializable { /** * Detaches the specified actor instance from this dispatcher */ - final def detach(actor: ActorInstance): Unit = { + final def detach(actor: ActorCell): Unit = { guard withGuard { unregister(actor) } @@ -132,7 +132,7 @@ abstract class MessageDispatcher extends Serializable { * Only "private[akka] for the sake of intercepting calls, DO NOT CALL THIS OUTSIDE OF THE DISPATCHER, * and only call it under the dispatcher-guard, see "attach" for the only invocation */ - protected[akka] def register(actor: ActorInstance) { + protected[akka] def register(actor: ActorCell) { if (actor.mailbox eq null) actor.mailbox = createMailbox(actor) @@ -148,7 +148,7 @@ abstract class MessageDispatcher extends Serializable { * Only "private[akka] for the sake of intercepting calls, DO NOT CALL THIS OUTSIDE OF THE DISPATCHER, * and only call it under the dispatcher-guard, see "detach" for the only invocation */ - protected[akka] def unregister(actor: ActorInstance) = { + protected[akka] def unregister(actor: ActorCell) = { if (uuids remove actor.uuid) { cleanUpMailboxFor(actor) actor.mailbox = null @@ -169,7 +169,7 @@ abstract class MessageDispatcher extends Serializable { * Overridable callback to clean up the mailbox for a given actor, * called when an actor is unregistered. */ - protected def cleanUpMailboxFor(actor: ActorInstance) {} + protected def cleanUpMailboxFor(actor: ActorCell) {} /** * Traverses the list of actors (uuids) currently being attached to this dispatcher and stops those actors @@ -214,12 +214,12 @@ abstract class MessageDispatcher extends Serializable { /** * After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference */ - def suspend(actor: ActorInstance) + def suspend(actor: ActorCell) /* * After the call to this method, the dispatcher must begin any new message processing for the specified reference */ - def resume(actor: ActorInstance) + def resume(actor: ActorCell) /** * Will be called when the dispatcher is to queue an invocation for execution @@ -241,12 +241,12 @@ abstract class MessageDispatcher extends Serializable { /** * Returns the size of the mailbox for the specified actor */ - def mailboxSize(actor: ActorInstance): Int + def mailboxSize(actor: ActorCell): Int /** * Returns the "current" emptiness status of the mailbox for the specified actor */ - def mailboxIsEmpty(actor: ActorInstance): Boolean + def mailboxIsEmpty(actor: ActorCell): Boolean /** * Returns the amount of tasks queued for execution diff --git a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala index befc1cb1ec..cbb76baeea 100644 --- a/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/PinnedDispatcher.scala @@ -5,42 +5,42 @@ package akka.dispatch import java.util.concurrent.atomic.AtomicReference -import akka.actor.ActorInstance +import akka.actor.ActorCell /** * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue. * * @author Jonas Bonér */ -class PinnedDispatcher(_actor: ActorInstance, _name: String, _mailboxType: MailboxType) +class PinnedDispatcher(_actor: ActorCell, _name: String, _mailboxType: MailboxType) extends Dispatcher( _name, Dispatchers.THROUGHPUT, -1, _mailboxType, PinnedDispatcher.oneThread) { def this(_name: String, _mailboxType: MailboxType) = this(null, _name, _mailboxType) - def this(_actor: ActorInstance, _name: String) = this(_actor, _name, Dispatchers.MAILBOX_TYPE) + def this(_actor: ActorCell, _name: String) = this(_actor, _name, Dispatchers.MAILBOX_TYPE) def this(_name: String) = this(null, _name, Dispatchers.MAILBOX_TYPE) def this(_mailboxType: MailboxType) = this(null, "anon", _mailboxType) - def this(_actor: ActorInstance, _mailboxType: MailboxType) = this(_actor, _actor.uuid.toString, _mailboxType) + def this(_actor: ActorCell, _mailboxType: MailboxType) = this(_actor, _actor.uuid.toString, _mailboxType) - def this(_actor: ActorInstance) = this(_actor, _actor.uuid.toString, Dispatchers.MAILBOX_TYPE) + def this(_actor: ActorCell) = this(_actor, _actor.uuid.toString, Dispatchers.MAILBOX_TYPE) def this() = this(Dispatchers.MAILBOX_TYPE) - protected[akka] val owner = new AtomicReference[ActorInstance](_actor) + protected[akka] val owner = new AtomicReference[ActorCell](_actor) //Relies on an external lock provided by MessageDispatcher.attach - protected[akka] override def register(actorInstance: ActorInstance) = { + protected[akka] override def register(actorCell: ActorCell) = { val actor = owner.get() - if ((actor ne null) && actorInstance != actor) throw new IllegalArgumentException("Cannot register to anyone but " + actor) - owner.compareAndSet(null, actorInstance) //Register if unregistered - super.register(actorInstance) + if ((actor ne null) && actorCell != actor) throw new IllegalArgumentException("Cannot register to anyone but " + actor) + owner.compareAndSet(null, actorCell) //Register if unregistered + super.register(actorCell) } //Relies on an external lock provided by MessageDispatcher.detach - protected[akka] override def unregister(actor: ActorInstance) = { + protected[akka] override def unregister(actor: ActorCell) = { super.unregister(actor) owner.compareAndSet(actor, null) //Unregister (prevent memory leak) } diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 780e56abe6..340d44c364 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -11,7 +11,7 @@ import java.util.concurrent.RejectedExecutionException import akka.util.Switch import java.lang.ref.WeakReference import scala.annotation.tailrec -import akka.actor.ActorInstance +import akka.actor.ActorCell /* * Locking rules: @@ -107,9 +107,9 @@ object CallingThreadDispatcher { class CallingThreadDispatcher(val name: String = "calling-thread", val warnings: Boolean = true) extends MessageDispatcher { import CallingThreadDispatcher._ - protected[akka] override def createMailbox(actor: ActorInstance) = new CallingThreadMailbox + protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox - private def getMailbox(actor: ActorInstance) = actor.mailbox.asInstanceOf[CallingThreadMailbox] + private def getMailbox(actor: ActorCell) = actor.mailbox.asInstanceOf[CallingThreadMailbox] protected[akka] override def start() {} @@ -117,11 +117,11 @@ class CallingThreadDispatcher(val name: String = "calling-thread", val warnings: protected[akka] override def timeoutMs = 100L - override def suspend(actor: ActorInstance) { + override def suspend(actor: ActorCell) { getMailbox(actor).suspended.switchOn } - override def resume(actor: ActorInstance) { + override def resume(actor: ActorCell) { val mbox = getMailbox(actor) val queue = mbox.queue val wasActive = queue.isActive @@ -133,9 +133,9 @@ class CallingThreadDispatcher(val name: String = "calling-thread", val warnings: } } - override def mailboxSize(actor: ActorInstance) = getMailbox(actor).queue.size + override def mailboxSize(actor: ActorCell) = getMailbox(actor).queue.size - override def mailboxIsEmpty(actor: ActorInstance): Boolean = getMailbox(actor).queue.isEmpty + override def mailboxIsEmpty(actor: ActorCell): Boolean = getMailbox(actor).queue.isEmpty protected[akka] override def dispatch(handle: MessageInvocation) { val mbox = getMailbox(handle.receiver)