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)