Changing Mailbox.Status to be an Int

This commit is contained in:
Viktor Klang 2011-09-26 11:39:07 +02:00
parent 1b90a467bc
commit 29a327aba4
3 changed files with 27 additions and 28 deletions

View file

@ -122,7 +122,7 @@ class Dispatcher(
protected[akka] def shutdown { protected[akka] def shutdown {
val old = executorService.getAndSet(new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService)) val old = executorService.getAndSet(new LazyExecutorServiceWrapper(executorServiceFactory.createExecutorService))
if (old ne null) if (old ne null)
old.shutdownNow() old.shutdown()
} }
/** /**

View file

@ -15,11 +15,10 @@ import atomic.AtomicReferenceFieldUpdater
class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause) class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
object Mailbox { object Mailbox {
sealed trait Status type Status = Int
case object OPEN extends Status val Open = 0
case object SUSPENDED extends Status val Suspended = 1
case object CLOSED extends Status val Closed = 2
//private[Mailbox] val mailboxStatusUpdater = AtomicReferenceFieldUpdater.newUpdater[Mailbox, Status](classOf[Mailbox], classOf[Status], "_status") //private[Mailbox] val mailboxStatusUpdater = AtomicReferenceFieldUpdater.newUpdater[Mailbox, Status](classOf[Mailbox], classOf[Status], "_status")
} }
@ -33,20 +32,20 @@ abstract class Mailbox extends MessageQueue with SystemMessageQueue with Runnabl
*/ */
final val dispatcherLock = new SimpleLock(startLocked = false) final val dispatcherLock = new SimpleLock(startLocked = false)
@volatile @volatile
var _status: Status = OPEN //Must be named _status because of the updater var _status: Status = Open //Must be named _status because of the updater
final def status: Mailbox.Status = _status //mailboxStatusUpdater.get(this) final def status: Mailbox.Status = _status //mailboxStatusUpdater.get(this)
final def isSuspended: Boolean = status == SUSPENDED final def isSuspended: Boolean = status == Suspended
final def isClosed: Boolean = status == CLOSED final def isClosed: Boolean = status == Closed
final def isOpen: Boolean = status == OPEN final def isOpen: Boolean = status == Open
def become(newStatus: Status) = _status = newStatus //mailboxStatusUpdater.set(this, newStatus) def become(newStatus: Status) = _status = newStatus //mailboxStatusUpdater.set(this, newStatus)
def shouldBeRegisteredForExecution(hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = status match { def shouldBeRegisteredForExecution(hasMessageHint: Boolean, hasSystemMessageHint: Boolean): Boolean = status match {
case CLOSED false case `Open` hasMessageHint || hasSystemMessageHint || hasSystemMessages || hasMessages
case OPEN hasMessageHint || hasSystemMessageHint || hasSystemMessages || hasMessages case `Closed` false
case SUSPENDED hasSystemMessageHint || hasSystemMessages case `Suspended` hasSystemMessageHint || hasSystemMessages
} }
final def run = { final def run = {
@ -66,7 +65,7 @@ abstract class Mailbox extends MessageQueue with SystemMessageQueue with Runnabl
final def processMailbox() { final def processMailbox() {
processAllSystemMessages() processAllSystemMessages()
if (status == OPEN) { if (isOpen) {
var nextMessage = dequeue() var nextMessage = dequeue()
if (nextMessage ne null) { //If we have a message if (nextMessage ne null) { //If we have a message
if (dispatcher.throughput <= 1) { //If we only run one message per process { if (dispatcher.throughput <= 1) { //If we only run one message per process {
@ -82,7 +81,7 @@ abstract class Mailbox extends MessageQueue with SystemMessageQueue with Runnabl
processAllSystemMessages() processAllSystemMessages()
nextMessage = if (status == OPEN) { // If we aren't suspended, we need to make sure we're not overstepping our boundaries nextMessage = if (isOpen) { // If we aren't suspended, we need to make sure we're not overstepping our boundaries
processedMessages += 1 processedMessages += 1
if ((processedMessages >= dispatcher.throughput) || (isDeadlineEnabled && System.nanoTime >= deadlineNs)) // If we're throttled, break out if ((processedMessages >= dispatcher.throughput) || (isDeadlineEnabled && System.nanoTime >= deadlineNs)) // If we're throttled, break out
null //We reached our boundaries, abort null //We reached our boundaries, abort

View file

@ -86,8 +86,8 @@ abstract class MessageDispatcher extends Serializable {
object DeadLetterMailbox extends Mailbox { object DeadLetterMailbox extends Mailbox {
dispatcherLock.tryLock() dispatcherLock.tryLock()
become(Mailbox.CLOSED) become(Mailbox.Closed)
override def become(newStatus: Mailbox.Status) { super.become(Mailbox.CLOSED) } //Always transcend to CLOSED to preserve the volatile write override def become(newStatus: Mailbox.Status) { super.become(Mailbox.Closed) } //Always transcend to CLOSED to preserve the volatile write
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
@ -184,7 +184,7 @@ abstract class MessageDispatcher extends Serializable {
protected[akka] def unregister(actor: ActorCell): Unit = { protected[akka] def unregister(actor: ActorCell): Unit = {
if (uuids remove actor.uuid) { if (uuids remove actor.uuid) {
val mailBox = actor.mailbox val mailBox = actor.mailbox
mailBox.become(Mailbox.CLOSED) mailBox.become(Mailbox.Closed)
actor.mailbox = deadLetterMailbox //FIXME getAndSet would be preferrable here actor.mailbox = deadLetterMailbox //FIXME getAndSet would be preferrable here
cleanUpMailboxFor(actor, mailBox) cleanUpMailboxFor(actor, mailBox)
if (uuids.isEmpty && _tasks.get == 0) { if (uuids.isEmpty && _tasks.get == 0) {
@ -205,21 +205,20 @@ abstract class MessageDispatcher extends Serializable {
* called when an actor is unregistered. * called when an actor is unregistered.
*/ */
protected def cleanUpMailboxFor(actor: ActorCell, mailBox: Mailbox): Unit = { protected def cleanUpMailboxFor(actor: ActorCell, mailBox: Mailbox): Unit = {
val m = mailBox
if (m.hasSystemMessages) { if (mailBox.hasSystemMessages) {
var envelope = m.systemDequeue() var envelope = mailBox.systemDequeue()
while (envelope ne null) { while (envelope ne null) {
deadLetterMailbox.systemEnqueue(envelope) deadLetterMailbox.systemEnqueue(envelope)
envelope = m.systemDequeue() envelope = mailBox.systemDequeue()
} }
} }
if (m.hasMessages) { if (mailBox.hasMessages) {
var envelope = m.dequeue var envelope = mailBox.dequeue
while (envelope ne null) { while (envelope ne null) {
deadLetterMailbox.enqueue(envelope) deadLetterMailbox.enqueue(envelope)
envelope = m.dequeue envelope = mailBox.dequeue
} }
} }
} }
@ -248,7 +247,8 @@ abstract class MessageDispatcher extends Serializable {
case SCHEDULED case SCHEDULED
if (uuids.isEmpty && _tasks.get == 0) { if (uuids.isEmpty && _tasks.get == 0) {
active switchOff { active switchOff {
shutdown() // shut down in the dispatcher's references is zero if (uuids.isEmpty && _tasks.get == 0)
shutdown() // shut down in the dispatcher's references is zero
} }
} }
shutdownSchedule = UNSCHEDULED shutdownSchedule = UNSCHEDULED
@ -268,14 +268,14 @@ abstract class MessageDispatcher extends Serializable {
* After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference * After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference
*/ */
def suspend(actor: ActorCell): Unit = def suspend(actor: ActorCell): Unit =
if (uuids.contains(actor.uuid)) actor.mailbox.become(Mailbox.SUSPENDED) if (uuids.contains(actor.uuid)) actor.mailbox.become(Mailbox.Suspended)
/* /*
* After the call to this method, the dispatcher must begin any new message processing for the specified reference * After the call to this method, the dispatcher must begin any new message processing for the specified reference
*/ */
def resume(actor: ActorCell): Unit = if (uuids.contains(actor.uuid)) { def resume(actor: ActorCell): Unit = if (uuids.contains(actor.uuid)) {
val mbox = actor.mailbox val mbox = actor.mailbox
mbox.become(Mailbox.OPEN) mbox.become(Mailbox.Open)
registerForExecution(mbox, false, false) registerForExecution(mbox, false, false)
} }