decoupled the mailbox implementation from the actor. The implementation is now controled by dispatcher associated with the actor.
This commit is contained in:
parent
bc29b0ef2e
commit
cc7da99bea
9 changed files with 61 additions and 46 deletions
|
|
@ -247,10 +247,10 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
|
||||||
def spawnLink[T <: Actor: Manifest]: ActorRef = unsupported
|
def spawnLink[T <: Actor: Manifest]: ActorRef = unsupported
|
||||||
def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef = unsupported
|
def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef = unsupported
|
||||||
def shutdownLinkedActors: Unit = unsupported
|
def shutdownLinkedActors: Unit = unsupported
|
||||||
def mailboxSize: Int = unsupported
|
|
||||||
def supervisor: Option[ActorRef] = unsupported
|
def supervisor: Option[ActorRef] = unsupported
|
||||||
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T](message: Any, timeout: Long, senderOption: Option[ActorRef], senderFuture: Option[CompletableFuture[T]]) = unsupported
|
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T](message: Any, timeout: Long, senderOption: Option[ActorRef], senderFuture: Option[CompletableFuture[T]]) = unsupported
|
||||||
protected[akka] def mailbox: Deque[MessageInvocation] = unsupported
|
protected[akka] def mailbox: AnyRef = unsupported
|
||||||
|
protected[akka] def mailbox_=(msg: AnyRef):AnyRef = unsupported
|
||||||
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
|
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
|
||||||
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
|
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
|
||||||
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
|
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
|
||||||
|
|
|
||||||
|
|
@ -514,7 +514,7 @@ trait ActorRef extends TransactionManagement {
|
||||||
/**
|
/**
|
||||||
* Returns the mailbox size.
|
* Returns the mailbox size.
|
||||||
*/
|
*/
|
||||||
def mailboxSize: Int
|
def mailboxSize = dispatcher.mailboxSize(this)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns the supervisor, if there is one.
|
* Returns the supervisor, if there is one.
|
||||||
|
|
@ -542,7 +542,8 @@ trait ActorRef extends TransactionManagement {
|
||||||
|
|
||||||
protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit
|
protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit
|
||||||
|
|
||||||
protected[akka] def mailbox: Deque[MessageInvocation]
|
protected[akka] def mailbox: AnyRef
|
||||||
|
protected[akka] def mailbox_=(value: AnyRef): AnyRef
|
||||||
|
|
||||||
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit
|
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit
|
||||||
|
|
||||||
|
|
@ -599,8 +600,8 @@ sealed class LocalActorRef private[akka](
|
||||||
@volatile private var loader: Option[ClassLoader] = None
|
@volatile private var loader: Option[ClassLoader] = None
|
||||||
@volatile private var maxNrOfRetriesCount: Int = 0
|
@volatile private var maxNrOfRetriesCount: Int = 0
|
||||||
@volatile private var restartsWithinTimeRangeTimestamp: Long = 0L
|
@volatile private var restartsWithinTimeRangeTimestamp: Long = 0L
|
||||||
|
@volatile private var _mailbox: AnyRef = _
|
||||||
|
|
||||||
protected[akka] val _mailbox: Deque[MessageInvocation] = new ConcurrentLinkedDeque[MessageInvocation]
|
|
||||||
protected[this] val actorInstance = guard.withGuard { new AtomicReference[Actor](newActor) }
|
protected[this] val actorInstance = guard.withGuard { new AtomicReference[Actor](newActor) }
|
||||||
|
|
||||||
// Needed to be able to null out the 'val self: ActorRef' member variables to make the Actor
|
// Needed to be able to null out the 'val self: ActorRef' member variables to make the Actor
|
||||||
|
|
@ -890,17 +891,9 @@ sealed class LocalActorRef private[akka](
|
||||||
/**
|
/**
|
||||||
* Returns the mailbox.
|
* Returns the mailbox.
|
||||||
*/
|
*/
|
||||||
def mailbox: Deque[MessageInvocation] = _mailbox
|
def mailbox: AnyRef = _mailbox
|
||||||
|
|
||||||
/**
|
protected[akka] def mailbox_=(value: AnyRef):AnyRef = { _mailbox = value; value }
|
||||||
* Returns the mailbox size.
|
|
||||||
*/
|
|
||||||
def mailboxSize: Int = _mailbox.size
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns a copy of all the messages, put into a List[MessageInvocation].
|
|
||||||
*/
|
|
||||||
def messagesInMailbox: List[MessageInvocation] = _mailbox.toArray.toList.asInstanceOf[List[MessageInvocation]]
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Shuts down and removes all linked actors.
|
* Shuts down and removes all linked actors.
|
||||||
|
|
@ -927,10 +920,7 @@ sealed class LocalActorRef private[akka](
|
||||||
createRemoteRequestProtocolBuilder(this, message, true, senderOption).build, None)
|
createRemoteRequestProtocolBuilder(this, message, true, senderOption).build, None)
|
||||||
} else {
|
} else {
|
||||||
val invocation = new MessageInvocation(this, message, senderOption, None, transactionSet.get)
|
val invocation = new MessageInvocation(this, message, senderOption, None, transactionSet.get)
|
||||||
if (dispatcher.usesActorMailbox) {
|
invocation.send
|
||||||
_mailbox.add(invocation)
|
|
||||||
invocation.send
|
|
||||||
} else invocation.send
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -951,7 +941,6 @@ sealed class LocalActorRef private[akka](
|
||||||
else new DefaultCompletableFuture[T](timeout)
|
else new DefaultCompletableFuture[T](timeout)
|
||||||
val invocation = new MessageInvocation(
|
val invocation = new MessageInvocation(
|
||||||
this, message, senderOption, Some(future.asInstanceOf[CompletableFuture[Any]]), transactionSet.get)
|
this, message, senderOption, Some(future.asInstanceOf[CompletableFuture[Any]]), transactionSet.get)
|
||||||
if (dispatcher.usesActorMailbox) _mailbox.add(invocation)
|
|
||||||
invocation.send
|
invocation.send
|
||||||
future
|
future
|
||||||
}
|
}
|
||||||
|
|
@ -1338,10 +1327,10 @@ private[akka] case class RemoteActorRef private[akka] (
|
||||||
def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef = unsupported
|
def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef = unsupported
|
||||||
def spawnLink[T <: Actor: Manifest]: ActorRef = unsupported
|
def spawnLink[T <: Actor: Manifest]: ActorRef = unsupported
|
||||||
def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef = unsupported
|
def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef = unsupported
|
||||||
def mailboxSize: Int = unsupported
|
|
||||||
def supervisor: Option[ActorRef] = unsupported
|
def supervisor: Option[ActorRef] = unsupported
|
||||||
def shutdownLinkedActors: Unit = unsupported
|
def shutdownLinkedActors: Unit = unsupported
|
||||||
protected[akka] def mailbox: Deque[MessageInvocation] = unsupported
|
protected[akka] def mailbox: AnyRef = unsupported
|
||||||
|
protected[akka] def mailbox_=(value: AnyRef):AnyRef = unsupported
|
||||||
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
|
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
|
||||||
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
|
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
|
||||||
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
|
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
|
||||||
|
|
|
||||||
|
|
@ -5,6 +5,7 @@
|
||||||
package se.scalablesolutions.akka.dispatch
|
package se.scalablesolutions.akka.dispatch
|
||||||
|
|
||||||
import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException}
|
import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException}
|
||||||
|
import jsr166x.ConcurrentLinkedDeque
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Default settings are:
|
* Default settings are:
|
||||||
|
|
@ -67,15 +68,34 @@ class ExecutorBasedEventDrivenDispatcher(_name: String, throughput: Int = Dispat
|
||||||
val name = "akka:event-driven:dispatcher:" + _name
|
val name = "akka:event-driven:dispatcher:" + _name
|
||||||
init
|
init
|
||||||
|
|
||||||
def dispatch(invocation: MessageInvocation) = dispatch(invocation.receiver)
|
def dispatch(invocation: MessageInvocation) = {
|
||||||
|
getMailbox(invocation.receiver).add(invocation)
|
||||||
|
dispatch(invocation.receiver)
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the mailbox associated with the actor
|
||||||
|
*/
|
||||||
|
private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[ConcurrentLinkedDeque[MessageInvocation]]
|
||||||
|
|
||||||
|
override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
|
||||||
|
|
||||||
|
override def register(actorRef: ActorRef) = {
|
||||||
|
// The actor will need a ConcurrentLinkedDeque based mailbox
|
||||||
|
if( actorRef.mailbox == null ) {
|
||||||
|
actorRef.mailbox = new ConcurrentLinkedDeque[MessageInvocation]()
|
||||||
|
}
|
||||||
|
super.register(actorRef)
|
||||||
|
}
|
||||||
|
|
||||||
def dispatch(receiver: ActorRef): Unit = if (active) {
|
def dispatch(receiver: ActorRef): Unit = if (active) {
|
||||||
|
|
||||||
executor.execute(new Runnable() {
|
executor.execute(new Runnable() {
|
||||||
def run = {
|
def run = {
|
||||||
var lockAcquiredOnce = false
|
var lockAcquiredOnce = false
|
||||||
var finishedBeforeMailboxEmpty = false
|
var finishedBeforeMailboxEmpty = false
|
||||||
val lock = receiver.dispatcherLock
|
val lock = receiver.dispatcherLock
|
||||||
val mailbox = receiver.mailbox
|
val mailbox = getMailbox(receiver)
|
||||||
// this do-while loop is required to prevent missing new messages between the end of the inner while
|
// this do-while loop is required to prevent missing new messages between the end of the inner while
|
||||||
// loop and releasing the lock
|
// loop and releasing the lock
|
||||||
do {
|
do {
|
||||||
|
|
@ -102,15 +122,16 @@ class ExecutorBasedEventDrivenDispatcher(_name: String, throughput: Int = Dispat
|
||||||
*/
|
*/
|
||||||
def processMailbox(receiver: ActorRef): Boolean = {
|
def processMailbox(receiver: ActorRef): Boolean = {
|
||||||
var processedMessages = 0
|
var processedMessages = 0
|
||||||
var messageInvocation = receiver.mailbox.poll
|
val mailbox = getMailbox(receiver)
|
||||||
|
var messageInvocation = mailbox.poll
|
||||||
while (messageInvocation != null) {
|
while (messageInvocation != null) {
|
||||||
messageInvocation.invoke
|
messageInvocation.invoke
|
||||||
processedMessages += 1
|
processedMessages += 1
|
||||||
// check if we simply continue with other messages, or reached the throughput limit
|
// check if we simply continue with other messages, or reached the throughput limit
|
||||||
if (throughput <= 0 || processedMessages < throughput) messageInvocation = receiver.mailbox.poll
|
if (throughput <= 0 || processedMessages < throughput) messageInvocation = mailbox.poll
|
||||||
else {
|
else {
|
||||||
messageInvocation = null
|
messageInvocation = null
|
||||||
return !receiver.mailbox.isEmpty
|
return !mailbox.isEmpty
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
false
|
false
|
||||||
|
|
@ -128,8 +149,6 @@ class ExecutorBasedEventDrivenDispatcher(_name: String, throughput: Int = Dispat
|
||||||
references.clear
|
references.clear
|
||||||
}
|
}
|
||||||
|
|
||||||
def usesActorMailbox = true
|
|
||||||
|
|
||||||
def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
|
def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
|
||||||
"Can't build a new thread pool for a dispatcher that is already up and running")
|
"Can't build a new thread pool for a dispatcher that is already up and running")
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,7 @@ package se.scalablesolutions.akka.dispatch
|
||||||
import java.util.concurrent.CopyOnWriteArrayList
|
import java.util.concurrent.CopyOnWriteArrayList
|
||||||
|
|
||||||
import se.scalablesolutions.akka.actor.{Actor, ActorRef, IllegalActorStateException}
|
import se.scalablesolutions.akka.actor.{Actor, ActorRef, IllegalActorStateException}
|
||||||
|
import jsr166x.ConcurrentLinkedDeque
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
|
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
|
||||||
|
|
@ -44,7 +45,16 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
|
||||||
val name = "akka:event-driven-work-stealing:dispatcher:" + _name
|
val name = "akka:event-driven-work-stealing:dispatcher:" + _name
|
||||||
init
|
init
|
||||||
|
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return the mailbox associated with the actor
|
||||||
|
*/
|
||||||
|
private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[ConcurrentLinkedDeque[MessageInvocation]]
|
||||||
|
|
||||||
|
override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
|
||||||
|
|
||||||
def dispatch(invocation: MessageInvocation) = if (active) {
|
def dispatch(invocation: MessageInvocation) = if (active) {
|
||||||
|
getMailbox(invocation.receiver).add(invocation)
|
||||||
executor.execute(new Runnable() {
|
executor.execute(new Runnable() {
|
||||||
def run = {
|
def run = {
|
||||||
if (!tryProcessMailbox(invocation.receiver)) {
|
if (!tryProcessMailbox(invocation.receiver)) {
|
||||||
|
|
@ -76,7 +86,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
|
||||||
lock.unlock
|
lock.unlock
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} while ((lockAcquiredOnce && !receiver.mailbox.isEmpty))
|
} while ((lockAcquiredOnce && !getMailbox(receiver).isEmpty))
|
||||||
|
|
||||||
return lockAcquiredOnce
|
return lockAcquiredOnce
|
||||||
}
|
}
|
||||||
|
|
@ -85,10 +95,11 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
|
||||||
* Process the messages in the mailbox of the given actor.
|
* Process the messages in the mailbox of the given actor.
|
||||||
*/
|
*/
|
||||||
private def processMailbox(receiver: ActorRef) = {
|
private def processMailbox(receiver: ActorRef) = {
|
||||||
var messageInvocation = receiver.mailbox.poll
|
val mailbox = getMailbox(receiver)
|
||||||
|
var messageInvocation = mailbox.poll
|
||||||
while (messageInvocation != null) {
|
while (messageInvocation != null) {
|
||||||
messageInvocation.invoke
|
messageInvocation.invoke
|
||||||
messageInvocation = receiver.mailbox.poll
|
messageInvocation = mailbox.poll
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -116,7 +127,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
|
||||||
for (i <- 0 to actors.length) {
|
for (i <- 0 to actors.length) {
|
||||||
val index = (i + startIndex) % actors.length
|
val index = (i + startIndex) % actors.length
|
||||||
val actor = actors(index)
|
val actor = actors(index)
|
||||||
if (actor != receiver && actor.mailbox.isEmpty) return (Some(actor), index)
|
if (actor != receiver && getMailbox(actor).isEmpty) return (Some(actor), index)
|
||||||
}
|
}
|
||||||
(None, startIndex) // nothing found, reuse same start index next time
|
(None, startIndex) // nothing found, reuse same start index next time
|
||||||
}
|
}
|
||||||
|
|
@ -139,7 +150,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
|
||||||
* Steal a message from the receiver and give it to the thief.
|
* Steal a message from the receiver and give it to the thief.
|
||||||
*/
|
*/
|
||||||
private def donateMessage(receiver: ActorRef, thief: ActorRef): Boolean = {
|
private def donateMessage(receiver: ActorRef, thief: ActorRef): Boolean = {
|
||||||
val donated = receiver.mailbox.pollLast
|
val donated = getMailbox(receiver).pollLast
|
||||||
if (donated ne null) {
|
if (donated ne null) {
|
||||||
if (donated.senderFuture.isDefined) thief.self.postMessageToMailboxAndCreateFutureResultWithTimeout[Any](
|
if (donated.senderFuture.isDefined) thief.self.postMessageToMailboxAndCreateFutureResultWithTimeout[Any](
|
||||||
donated.message, receiver.timeout, donated.sender, donated.senderFuture)
|
donated.message, receiver.timeout, donated.sender, donated.senderFuture)
|
||||||
|
|
@ -169,6 +180,10 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
|
||||||
|
|
||||||
override def register(actorRef: ActorRef) = {
|
override def register(actorRef: ActorRef) = {
|
||||||
verifyActorsAreOfSameType(actorRef)
|
verifyActorsAreOfSameType(actorRef)
|
||||||
|
// The actor will need a ConcurrentLinkedDeque based mailbox
|
||||||
|
if( actorRef.mailbox == null ) {
|
||||||
|
actorRef.mailbox = new ConcurrentLinkedDeque[MessageInvocation]()
|
||||||
|
}
|
||||||
pooledActors.add(actorRef)
|
pooledActors.add(actorRef)
|
||||||
super.register(actorRef)
|
super.register(actorRef)
|
||||||
}
|
}
|
||||||
|
|
@ -178,8 +193,6 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
|
||||||
super.unregister(actorRef)
|
super.unregister(actorRef)
|
||||||
}
|
}
|
||||||
|
|
||||||
def usesActorMailbox = true
|
|
||||||
|
|
||||||
private def verifyActorsAreOfSameType(actorOfId: ActorRef) = {
|
private def verifyActorsAreOfSameType(actorOfId: ActorRef) = {
|
||||||
actorType match {
|
actorType match {
|
||||||
case None => actorType = Some(actorOfId.actor.getClass)
|
case None => actorType = Some(actorOfId.actor.getClass)
|
||||||
|
|
|
||||||
|
|
@ -79,7 +79,7 @@ trait MessageDispatcher extends Logging {
|
||||||
}
|
}
|
||||||
def canBeShutDown: Boolean = references.isEmpty
|
def canBeShutDown: Boolean = references.isEmpty
|
||||||
def isShutdown: Boolean
|
def isShutdown: Boolean
|
||||||
def usesActorMailbox : Boolean
|
def mailboxSize(actorRef: ActorRef):Int = 0
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -41,8 +41,6 @@ class ReactorBasedSingleThreadEventDrivenDispatcher(_name: String)
|
||||||
|
|
||||||
def isShutdown = !active
|
def isShutdown = !active
|
||||||
|
|
||||||
def usesActorMailbox = false
|
|
||||||
|
|
||||||
override def toString = "ReactorBasedSingleThreadEventDrivenDispatcher[" + name + "]"
|
override def toString = "ReactorBasedSingleThreadEventDrivenDispatcher[" + name + "]"
|
||||||
|
|
||||||
class Demultiplexer(private val messageQueue: ReactiveMessageQueue) extends MessageDemultiplexer {
|
class Demultiplexer(private val messageQueue: ReactiveMessageQueue) extends MessageDemultiplexer {
|
||||||
|
|
|
||||||
|
|
@ -139,8 +139,6 @@ class ReactorBasedThreadPoolEventDrivenDispatcher(_name: String)
|
||||||
else nrOfBusyMessages < 100
|
else nrOfBusyMessages < 100
|
||||||
}
|
}
|
||||||
|
|
||||||
def usesActorMailbox = false
|
|
||||||
|
|
||||||
def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
|
def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
|
||||||
"Can't build a new thread pool for a dispatcher that is already up and running")
|
"Can't build a new thread pool for a dispatcher that is already up and running")
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -40,8 +40,6 @@ class ThreadBasedDispatcher(private val actor: ActorRef) extends MessageDispatch
|
||||||
|
|
||||||
def isShutdown = !active
|
def isShutdown = !active
|
||||||
|
|
||||||
def usesActorMailbox = false
|
|
||||||
|
|
||||||
def shutdown = if (active) {
|
def shutdown = if (active) {
|
||||||
log.debug("Shutting down %s", toString)
|
log.debug("Shutting down %s", toString)
|
||||||
active = false
|
active = false
|
||||||
|
|
|
||||||
|
|
@ -166,7 +166,7 @@ class MyStatelessActor extends Actor {
|
||||||
class MyStatelessActorWithMessagesInMailbox extends Actor {
|
class MyStatelessActorWithMessagesInMailbox extends Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
case "hello" =>
|
case "hello" =>
|
||||||
println("# messages in mailbox " + self.mailbox.size)
|
println("# messages in mailbox " + self.mailboxSize)
|
||||||
Thread.sleep(500)
|
Thread.sleep(500)
|
||||||
case "hello-reply" => self.reply("world")
|
case "hello-reply" => self.reply("world")
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue