Merge branch 'master' into wip-2212-recursive-resume-∂π
also split out ChildrenContainer into its own file and add suspendCounter to UnstartedActorCell
This commit is contained in:
commit
1cb204da49
194 changed files with 9705 additions and 2308 deletions
|
|
@ -6,6 +6,7 @@ package akka.dispatch
|
|||
import akka.AkkaException
|
||||
import java.util.{ Comparator, PriorityQueue, Queue, Deque }
|
||||
import akka.util._
|
||||
import akka.actor.{ ActorCell, ActorRef, Cell }
|
||||
import java.util.concurrent._
|
||||
import annotation.tailrec
|
||||
import akka.event.Logging.Error
|
||||
|
|
@ -45,11 +46,32 @@ private[akka] object Mailbox {
|
|||
*
|
||||
* INTERNAL API
|
||||
*/
|
||||
private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: MessageQueue)
|
||||
private[akka] abstract class Mailbox(val messageQueue: MessageQueue)
|
||||
extends SystemMessageQueue with Runnable {
|
||||
|
||||
import Mailbox._
|
||||
|
||||
/*
|
||||
* This is needed for actually executing the mailbox, i.e. invoking the
|
||||
* ActorCell. There are situations (e.g. RepointableActorRef) where a Mailbox
|
||||
* is constructed but we know that we will not execute it, in which case this
|
||||
* will be null. It must be a var to support switching into an “active”
|
||||
* mailbox, should the owning ActorRef turn local.
|
||||
*
|
||||
* ANOTHER THING, IMPORTANT:
|
||||
*
|
||||
* actorCell.start() publishes actorCell & self to the dispatcher, which
|
||||
* means that messages may be processed theoretically before self’s constructor
|
||||
* ends. The JMM guarantees visibility for final fields only after the end
|
||||
* of the constructor, so safe publication requires that THIS WRITE BELOW
|
||||
* stay as it is.
|
||||
*/
|
||||
@volatile
|
||||
var actor: ActorCell = _
|
||||
def setActor(cell: ActorCell): Unit = actor = cell
|
||||
|
||||
def dispatcher: MessageDispatcher = actor.dispatcher
|
||||
|
||||
/**
|
||||
* Try to enqueue the message to this queue, or throw an exception.
|
||||
*/
|
||||
|
|
@ -236,11 +258,12 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
|||
* if we closed the mailbox, we must dump the remaining system messages
|
||||
* to deadLetters (this is essential for DeathWatch)
|
||||
*/
|
||||
val dlm = actor.systemImpl.deadLetterMailbox
|
||||
while (nextMessage ne null) {
|
||||
val msg = nextMessage
|
||||
nextMessage = nextMessage.next
|
||||
msg.next = null
|
||||
try actor.systemImpl.deadLetterMailbox.systemEnqueue(actor.self, msg)
|
||||
try dlm.systemEnqueue(actor.self, msg)
|
||||
catch {
|
||||
case NonFatal(e) ⇒ actor.system.eventStream.publish(
|
||||
Error(e, actor.self.path.toString, this.getClass, "error while enqueuing " + msg + " to deadLetters: " + e.getMessage))
|
||||
|
|
@ -250,9 +273,6 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
|||
if (failure ne null) actor.handleInvokeFailure(failure, failure.getMessage)
|
||||
}
|
||||
|
||||
@inline
|
||||
final def dispatcher: MessageDispatcher = actor.dispatcher
|
||||
|
||||
/**
|
||||
* Overridable callback to clean up the mailbox,
|
||||
* called when an actor is unregistered.
|
||||
|
|
@ -271,7 +291,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
|||
}
|
||||
|
||||
if (messageQueue ne null) // needed for CallingThreadDispatcher, which never calls Mailbox.run()
|
||||
messageQueue.cleanUp(actor, actor.systemImpl.deadLetterQueue)
|
||||
messageQueue.cleanUp(actor.self, actor.systemImpl.deadLetterQueue)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -309,7 +329,7 @@ trait MessageQueue {
|
|||
* which is passed in. The owner of this MessageQueue is passed in if
|
||||
* available (e.g. for creating DeadLetters()), “/deadletters” otherwise.
|
||||
*/
|
||||
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit
|
||||
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -337,10 +357,11 @@ private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
|||
@tailrec
|
||||
final def systemEnqueue(receiver: ActorRef, message: SystemMessage): Unit = {
|
||||
assert(message.next eq null)
|
||||
if (Mailbox.debug) println(actor.self + " having enqueued " + message)
|
||||
if (Mailbox.debug) println(receiver + " having enqueued " + message)
|
||||
val head = systemQueueGet
|
||||
if (head == NoMessage) actor.system.deadLetterMailbox.systemEnqueue(receiver, message)
|
||||
else {
|
||||
if (head == NoMessage) {
|
||||
if (actor ne null) actor.systemImpl.deadLetterMailbox.systemEnqueue(receiver, message)
|
||||
} else {
|
||||
/*
|
||||
* this write is safely published by the compareAndSet contained within
|
||||
* systemQueuePut; “Intra-Thread Semantics” on page 12 of the JSR133 spec
|
||||
|
|
@ -372,11 +393,11 @@ trait QueueBasedMessageQueue extends MessageQueue {
|
|||
def queue: Queue[Envelope]
|
||||
def numberOfMessages = queue.size
|
||||
def hasMessages = !queue.isEmpty
|
||||
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = {
|
||||
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = {
|
||||
if (hasMessages) {
|
||||
var envelope = dequeue
|
||||
while (envelope ne null) {
|
||||
deadLetters.enqueue(owner.self, envelope)
|
||||
deadLetters.enqueue(owner, envelope)
|
||||
envelope = dequeue
|
||||
}
|
||||
}
|
||||
|
|
@ -451,10 +472,20 @@ trait BoundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue {
|
|||
}
|
||||
|
||||
/**
|
||||
* MailboxType is a factory to create MessageQueues for an optionally provided ActorContext
|
||||
* MailboxType is a factory to create MessageQueues for an optionally
|
||||
* provided ActorContext.
|
||||
*
|
||||
* <b>Possibly Important Notice</b>
|
||||
*
|
||||
* When implementing a custom mailbox type, be aware that there is special
|
||||
* semantics attached to `system.actorOf()` in that sending to the returned
|
||||
* ActorRef may—for a short period of time—enqueue the messages first in a
|
||||
* dummy queue. Top-level actors are created in two steps, and only after the
|
||||
* guardian actor has performed that second step will all previously sent
|
||||
* messages be transferred from the dummy queue into the real mailbox.
|
||||
*/
|
||||
trait MailboxType {
|
||||
def create(owner: Option[ActorContext]): MessageQueue
|
||||
def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -464,7 +495,7 @@ case class UnboundedMailbox() extends MailboxType {
|
|||
|
||||
def this(settings: ActorSystem.Settings, config: Config) = this()
|
||||
|
||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
||||
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||
new ConcurrentLinkedQueue[Envelope]() with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
||||
final def queue: Queue[Envelope] = this
|
||||
}
|
||||
|
|
@ -481,7 +512,7 @@ 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")
|
||||
|
||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
||||
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||
new LinkedBlockingQueue[Envelope](capacity) with QueueBasedMessageQueue with BoundedMessageQueueSemantics {
|
||||
final def queue: BlockingQueue[Envelope] = this
|
||||
final val pushTimeOut = BoundedMailbox.this.pushTimeOut
|
||||
|
|
@ -494,7 +525,7 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat
|
|||
*/
|
||||
class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope], final val initialCapacity: Int) extends MailboxType {
|
||||
def this(cmp: Comparator[Envelope]) = this(cmp, 11)
|
||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
||||
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||
new PriorityBlockingQueue[Envelope](initialCapacity, cmp) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
||||
final def queue: Queue[Envelope] = this
|
||||
}
|
||||
|
|
@ -509,7 +540,7 @@ class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val cap
|
|||
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")
|
||||
|
||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
||||
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||
new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) with QueueBasedMessageQueue with BoundedMessageQueueSemantics {
|
||||
final def queue: BlockingQueue[Envelope] = this
|
||||
final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut
|
||||
|
|
@ -523,7 +554,7 @@ case class UnboundedDequeBasedMailbox() extends MailboxType {
|
|||
|
||||
def this(settings: ActorSystem.Settings, config: Config) = this()
|
||||
|
||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
||||
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||
new LinkedBlockingDeque[Envelope]() with DequeBasedMessageQueue with UnboundedDequeBasedMessageQueueSemantics {
|
||||
final val queue = this
|
||||
}
|
||||
|
|
@ -540,7 +571,7 @@ case class BoundedDequeBasedMailbox( final val capacity: Int, final val pushTime
|
|||
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedDequeBasedMailbox can not be negative")
|
||||
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedDequeBasedMailbox can not be null")
|
||||
|
||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
||||
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||
new LinkedBlockingDeque[Envelope](capacity) with DequeBasedMessageQueue with BoundedDequeBasedMessageQueueSemantics {
|
||||
final val queue = this
|
||||
final val pushTimeOut = BoundedDequeBasedMailbox.this.pushTimeOut
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue