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
This commit is contained in:
Roland 2011-10-18 16:44:35 +02:00
parent 65868d7c96
commit 183dfb4d7f
7 changed files with 78 additions and 83 deletions

View file

@ -146,8 +146,8 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
class DefaultMailboxSpec extends MailboxSpec { class DefaultMailboxSpec extends MailboxSpec {
lazy val name = "The default mailbox implementation" lazy val name = "The default mailbox implementation"
def factory = { def factory = {
case u: UnboundedMailbox u.create(null) case u: UnboundedMailbox u.create(null, null)
case b: BoundedMailbox b.create(null) case b: BoundedMailbox b.create(null, null)
} }
} }
@ -155,7 +155,7 @@ class PriorityMailboxSpec extends MailboxSpec {
val comparator = PriorityGenerator(_.##) val comparator = PriorityGenerator(_.##)
lazy val name = "The priority mailbox implementation" lazy val name = "The priority mailbox implementation"
def factory = { def factory = {
case UnboundedMailbox() UnboundedPriorityMailbox(comparator).create(null) case UnboundedMailbox() UnboundedPriorityMailbox(comparator).create(null, null)
case BoundedMailbox(capacity, pushTimeOut) BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(null) case BoundedMailbox(capacity, pushTimeOut) BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(null, null)
} }
} }

View file

@ -245,7 +245,7 @@ private[akka] class ActorCell(
if (props.supervisor.isDefined) { if (props.supervisor.isDefined) {
props.supervisor.get match { props.supervisor.get match {
case l: LocalActorRef 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") 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) 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 = private[akka] def stop(): Unit =
dispatcher.systemDispatch(SystemEnvelope(this, Terminate, NullChannel)) dispatcher.systemDispatch(this, Terminate())
def startsMonitoring(subject: ActorRef): ActorRef = { def startsMonitoring(subject: ActorRef): ActorRef = {
dispatcher.systemDispatch(SystemEnvelope(this, Link(subject), NullChannel)) dispatcher.systemDispatch(this, Link(subject))
subject subject
} }
def stopsMonitoring(subject: ActorRef): ActorRef = { def stopsMonitoring(subject: ActorRef): ActorRef = {
dispatcher.systemDispatch(SystemEnvelope(this, Unlink(subject), NullChannel)) dispatcher.systemDispatch(this, Unlink(subject))
subject subject
} }
@ -324,7 +324,7 @@ private[akka] class ActorCell(
} }
} }
def systemInvoke(envelope: SystemEnvelope) { def systemInvoke(message: SystemMessage) {
def create(): Unit = try { def create(): Unit = try {
val created = newActor() val created = newActor()
@ -337,7 +337,6 @@ private[akka] class ActorCell(
app.eventHandler.error(e, self, "error while creating actor") app.eventHandler.error(e, self, "error while creating actor")
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
envelope.channel.sendException(e)
} finally { } finally {
if (supervisor.isDefined) supervisor.get ! Failed(self, e) else self.stop() 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") app.eventHandler.error(e, self, "error while creating actor")
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
envelope.channel.sendException(e)
} finally { } finally {
if (supervisor.isDefined) supervisor.get ! Failed(self, e) else self.stop() if (supervisor.isDefined) supervisor.get ! Failed(self, e) else self.stop()
} }
@ -424,24 +422,24 @@ private[akka] class ActorCell(
try { try {
val isClosed = mailbox.isClosed //Fence plus volatile read val isClosed = mailbox.isClosed //Fence plus volatile read
if (!isClosed) { if (!isClosed) {
envelope.message match { message match {
case Create create() case Create(_) create()
case Recreate(cause) recreate(cause) case Recreate(cause, _) recreate(cause)
case Link(subject) case Link(subject, _)
app.deathWatch.subscribe(self, subject) app.deathWatch.subscribe(self, subject)
if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "now monitoring " + subject) if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "now monitoring " + subject)
case Unlink(subject) case Unlink(subject, _)
app.deathWatch.unsubscribe(self, subject) app.deathWatch.unsubscribe(self, subject)
if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopped monitoring " + subject) if (app.AkkaConfig.DebugLifecycle) app.eventHandler.debug(self, "stopped monitoring " + subject)
case Suspend suspend() case Suspend(_) suspend()
case Resume resume() case Resume(_) resume()
case Terminate terminate() case Terminate(_) terminate()
case Supervise(child) supervise(child) case Supervise(child, _) supervise(child)
} }
} }
} catch { } catch {
case e //Should we really catch everything here? 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? //TODO FIXME How should problems here be handled?
throw e throw e
} }
@ -495,7 +493,7 @@ private[akka] class ActorCell(
def handleChildTerminated(child: ActorRef): Unit = _children = props.faultHandler.handleChildTerminated(child, _children) 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() { def checkReceiveTimeout() {
cancelReceiveTimeout() cancelReceiveTimeout()

View file

@ -24,25 +24,17 @@ final case class Envelope(val receiver: ActorCell, val message: Any, val channel
} }
} }
sealed trait SystemMessage extends PossiblyHarmful sealed trait SystemMessage extends PossiblyHarmful {
case object Create extends SystemMessage def next: 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
}
} }
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 { final case class TaskInvocation(app: AkkaApplication, function: () Unit, cleanup: () Unit) extends Runnable {
def run() { def run() {
@ -87,13 +79,13 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable
*/ */
protected[akka] val deadLetterMailbox: Mailbox = DeadLetterMailbox protected[akka] val deadLetterMailbox: Mailbox = DeadLetterMailbox
object DeadLetterMailbox extends Mailbox { object DeadLetterMailbox extends Mailbox(null) {
becomeClosed() becomeClosed()
override def dispatcher = null //MessageDispatcher.this override def dispatcher = null //MessageDispatcher.this
override def enqueue(envelope: Envelope) { envelope.channel sendException new ActorKilledException("Actor has been stopped") } override def enqueue(envelope: Envelope) { envelope.channel sendException new ActorKilledException("Actor has been stopped") }
override def dequeue() = null override def dequeue() = null
override def systemEnqueue(handle: SystemEnvelope): Unit = () override def systemEnqueue(handle: SystemMessage): Unit = ()
override def systemDequeue(): SystemEnvelope = null override def systemDequeue(): SystemMessage = null
override def hasMessages = false override def hasMessages = false
override def hasSystemMessages = false override def hasSystemMessages = false
override def numberOfMessages = 0 override def numberOfMessages = 0
@ -174,7 +166,7 @@ abstract class MessageDispatcher(val app: AkkaApplication) extends Serializable
*/ */
protected[akka] def register(actor: ActorCell) { protected[akka] def register(actor: ActorCell) {
if (uuids add actor.uuid) { 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) } 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 * 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 * Will be called when the dispatcher is to queue an invocation for execution

View file

@ -54,7 +54,7 @@ class BalancingDispatcher(
protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor) 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 enqueue(handle: Envelope) = messageQueue.enqueue(handle)
final def dequeue(): Envelope = { final def dequeue(): Envelope = {

View file

@ -84,8 +84,8 @@ class Dispatcher(
registerForExecution(mbox, true, false) registerForExecution(mbox, true, false)
} }
protected[akka] def systemDispatch(invocation: SystemEnvelope) = { protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage) = {
val mbox = invocation.receiver.mailbox val mbox = receiver.mailbox
mbox systemEnqueue invocation mbox systemEnqueue invocation
registerForExecution(mbox, false, true) 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 {} protected[akka] def start {}

View file

@ -8,7 +8,7 @@ import akka.AkkaException
import java.util.{ Comparator, PriorityQueue } import java.util.{ Comparator, PriorityQueue }
import akka.util._ import akka.util._
import java.util.Queue import java.util.Queue
import akka.actor.ActorContext import akka.actor.{ ActorContext, ActorCell }
import java.util.concurrent._ import java.util.concurrent._
import atomic.{ AtomicInteger, AtomicReferenceFieldUpdater } import atomic.{ AtomicInteger, AtomicReferenceFieldUpdater }
import annotation.tailrec import annotation.tailrec
@ -34,7 +34,7 @@ private[dispatch] object Mailbox {
/** /**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
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._ import Mailbox._
@inline @inline
@ -186,7 +186,7 @@ abstract class Mailbox extends AbstractMailbox with MessageQueue with SystemMess
def processAllSystemMessages() { def processAllSystemMessages() {
var nextMessage = systemDequeue() var nextMessage = systemDequeue()
while (nextMessage ne null) { while (nextMessage ne null) {
nextMessage.invoke() actor systemInvoke nextMessage
nextMessage = systemDequeue() nextMessage = systemDequeue()
} }
} }
@ -208,19 +208,20 @@ trait MessageQueue {
} }
trait SystemMessageQueue { trait SystemMessageQueue {
def systemEnqueue(handle: SystemEnvelope): Unit def systemEnqueue(message: SystemMessage): Unit
def systemDequeue(): SystemEnvelope def systemDequeue(): SystemMessage
def hasSystemMessages: Boolean def hasSystemMessages: Boolean
} }
trait DefaultSystemMessageQueue { self: SystemMessageQueue 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 def hasSystemMessages: Boolean = !systemMessages.isEmpty
} }
@ -255,17 +256,18 @@ trait QueueBasedMessageQueue extends MessageQueue {
* Mailbox configuration. * Mailbox configuration.
*/ */
trait MailboxType { trait MailboxType {
def create(dispatcher: MessageDispatcher): Mailbox def create(dispatcher: MessageDispatcher, receiver: ActorCell): Mailbox
} }
/** /**
* It's a case class for Java (new UnboundedMailbox) * It's a case class for Java (new UnboundedMailbox)
*/ */
case class UnboundedMailbox() extends MailboxType { case class UnboundedMailbox() extends MailboxType {
override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { override def create(_dispatcher: MessageDispatcher, receiver: ActorCell) =
final val queue = new ConcurrentLinkedQueue[Envelope]() new Mailbox(receiver) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue {
final val dispatcher = _dispatcher final val queue = new ConcurrentLinkedQueue[Envelope]()
} final val dispatcher = _dispatcher
}
} }
case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Duration) extends MailboxType { 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 (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") 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 { override def create(_dispatcher: MessageDispatcher, receiver: ActorCell) =
final val queue = new LinkedBlockingQueue[Envelope](capacity) new Mailbox(receiver) with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue {
final val pushTimeOut = BoundedMailbox.this.pushTimeOut final val queue = new LinkedBlockingQueue[Envelope](capacity)
final val dispatcher = _dispatcher final val pushTimeOut = BoundedMailbox.this.pushTimeOut
} final val dispatcher = _dispatcher
}
} }
case class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType { case class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope]) extends MailboxType {
override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue { override def create(_dispatcher: MessageDispatcher, receiver: ActorCell) =
final val queue = new PriorityBlockingQueue[Envelope](11, cmp) new Mailbox(receiver) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue {
final val dispatcher = _dispatcher 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 { 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 (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") 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 { override def create(_dispatcher: MessageDispatcher, receiver: ActorCell) =
final val queue = new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) new Mailbox(receiver) with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue {
final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut final val queue = new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp))
final val dispatcher = _dispatcher final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut
} final val dispatcher = _dispatcher
}
} }

View file

@ -106,7 +106,7 @@ object CallingThreadDispatcher {
class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling-thread", val warnings: Boolean = true) extends MessageDispatcher(_app) { class CallingThreadDispatcher(_app: AkkaApplication, val name: String = "calling-thread", val warnings: Boolean = true) extends MessageDispatcher(_app) {
import CallingThreadDispatcher._ 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] 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 override def mailboxIsEmpty(actor: ActorCell): Boolean = getMailbox(actor).queue.isEmpty
protected[akka] override def systemDispatch(handle: SystemEnvelope) { protected[akka] override def systemDispatch(receiver: ActorCell, message: SystemMessage) {
val mbox = getMailbox(handle.receiver) val mbox = getMailbox(receiver)
mbox.lock.lock mbox.lock.lock
try { try {
handle.invoke() receiver systemInvoke message
} finally { } finally {
mbox.lock.unlock mbox.lock.unlock
} }
@ -241,7 +241,7 @@ class NestingQueue {
def isActive = active 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]() { private val q = new ThreadLocal[NestingQueue]() {
override def initialValue = new NestingQueue override def initialValue = new NestingQueue