pekko/akka-actor/src/main/scala/akka/dispatch/MailboxHandling.scala

194 lines
6.9 KiB
Scala
Raw Normal View History

2010-09-21 18:52:41 +02:00
/**
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
2010-09-21 18:52:41 +02:00
*/
package akka.dispatch
2010-09-21 18:52:41 +02:00
import akka.AkkaException
import java.util.{ Comparator, PriorityQueue }
import akka.util._
import java.util.Queue
import akka.actor.ActorContext
import java.util.concurrent._
2010-09-21 18:52:41 +02:00
2011-04-29 17:15:00 +02:00
class MessageQueueAppendFailedException(message: String, cause: Throwable = null) extends AkkaException(message, cause)
2010-09-21 18:52:41 +02:00
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
2011-09-23 09:33:53 +02:00
abstract class Mailbox extends MessageQueue with SystemMessageQueue with Runnable {
/*
* Internal implementation of MessageDispatcher uses these, don't touch or rely on
*/
final val dispatcherLock = new SimpleLock(startLocked = false)
final val suspended = new SimpleLock(startLocked = false) //(startLocked = true)
final def run = {
try { processMailbox() } catch {
case ie: InterruptedException Thread.currentThread().interrupt() //Restore interrupt
} finally {
dispatcherLock.unlock()
if (hasMessages || hasSystemMessages)
dispatcher.reRegisterForExecution(this)
}
}
/**
* Process the messages in the mailbox
*
* @return true if the processing finished before the mailbox was empty, due to the throughput constraint
*/
final def processMailbox() {
if (hasSystemMessages)
processAllSystemMessages()
else if (!suspended.locked) {
var nextMessage = dequeue()
if (nextMessage ne null) { //If we have a message
if (dispatcher.throughput <= 1) //If we only run one message per process
nextMessage.invoke //Just run it
else { //But otherwise, if we are throttled, we need to do some book-keeping
var processedMessages = 0
val isDeadlineEnabled = dispatcher.throughputDeadlineTime > 0
val deadlineNs = if (isDeadlineEnabled) System.nanoTime + TimeUnit.MILLISECONDS.toNanos(dispatcher.throughputDeadlineTime)
else 0
do {
nextMessage.invoke
nextMessage = if (hasSystemMessages) {
processAllSystemMessages()
null
} else if (suspended.locked) {
null // If we are suspended, abort
} else { // If we aren't suspended, we need to make sure we're not overstepping our boundaries
processedMessages += 1
if ((processedMessages >= dispatcher.throughput) || (isDeadlineEnabled && System.nanoTime >= deadlineNs)) // If we're throttled, break out
null //We reached our boundaries, abort
else dequeue //Dequeue the next message
}
} while (nextMessage ne null)
}
}
}
}
def processAllSystemMessages(): Unit = {
var nextMessage = systemDequeue()
while (nextMessage ne null) {
nextMessage.invoke()
nextMessage = systemDequeue()
}
}
2011-09-23 09:33:53 +02:00
def dispatcher: MessageDispatcher
}
trait MessageQueue {
/*
* These method need to be implemented in subclasses; they should not rely on the internal stuff above.
*/
def enqueue(handle: Envelope)
2011-09-23 09:33:53 +02:00
def dequeue(): Envelope
def numberOfMessages: Int
2011-09-23 09:33:53 +02:00
def hasMessages: Boolean
}
trait SystemMessageQueue {
def systemEnqueue(handle: SystemEnvelope): Unit
2011-09-23 09:33:53 +02:00
def systemDequeue(): SystemEnvelope
def hasSystemMessages: Boolean
}
2011-09-23 09:33:53 +02:00
trait DefaultSystemMessageQueue { self: SystemMessageQueue
val systemMessages = new ConcurrentLinkedQueue[SystemEnvelope]()
def systemEnqueue(handle: SystemEnvelope): Unit = systemMessages offer handle
2011-09-23 09:33:53 +02:00
def systemDequeue(): SystemEnvelope = systemMessages.poll()
def hasSystemMessages: Boolean = !systemMessages.isEmpty
}
2011-09-23 09:33:53 +02:00
trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue {
final def enqueue(handle: Envelope): Unit = queue add handle
final def dequeue(): Envelope = queue.poll()
}
2011-09-23 09:33:53 +02:00
trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
def pushTimeOut: Duration
2011-09-23 09:33:53 +02:00
override def queue: BlockingQueue[Envelope]
final def enqueue(handle: Envelope) {
if (pushTimeOut.length > 0) {
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + toString)
}
} else queue put handle
}
final def dequeue(): Envelope = queue.poll()
}
2011-09-23 09:33:53 +02:00
trait QueueBasedMessageQueue extends MessageQueue {
def queue: Queue[Envelope]
final def numberOfMessages = queue.size
final def hasMessages = !queue.isEmpty
}
2010-09-21 18:52:41 +02:00
/**
* Mailbox configuration.
*/
trait MailboxType {
def create(dispatcher: MessageDispatcher): Mailbox
}
2011-09-23 09:33:53 +02:00
/**
* It's a case class for Java (new UnboundedMailbox)
*/
case class UnboundedMailbox() extends MailboxType {
2011-09-23 09:33:53 +02:00
override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue {
2011-09-22 12:08:00 +02:00
final val queue = new ConcurrentLinkedQueue[Envelope]()
2011-09-23 09:33:53 +02:00
final val dispatcher = _dispatcher
}
}
2010-09-21 18:52:41 +02:00
case class BoundedMailbox(
val capacity: Int = { if (Dispatchers.MAILBOX_CAPACITY < 0) Int.MaxValue else Dispatchers.MAILBOX_CAPACITY },
2011-01-20 17:16:44 +01:00
val pushTimeOut: Duration = Dispatchers.MAILBOX_PUSH_TIME_OUT) extends MailboxType {
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
2010-09-21 18:52:41 +02:00
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null")
2011-09-23 09:33:53 +02:00
override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue {
2011-09-22 12:08:00 +02:00
final val queue = new LinkedBlockingQueue[Envelope](capacity)
final val pushTimeOut = BoundedMailbox.this.pushTimeOut
2011-09-23 09:33:53 +02:00
final val dispatcher = _dispatcher
2010-09-21 18:52:41 +02:00
}
}
2010-09-21 18:52:41 +02:00
case class UnboundedPriorityMailbox(cmp: Comparator[Envelope]) extends MailboxType {
2011-09-23 09:33:53 +02:00
override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with UnboundedMessageQueueSemantics with DefaultSystemMessageQueue {
val queue = new PriorityBlockingQueue[Envelope](11, cmp)
2011-09-23 09:33:53 +02:00
final val dispatcher = _dispatcher
}
}
case class BoundedPriorityMailbox(
val cmp: Comparator[Envelope],
val capacity: Int = { if (Dispatchers.MAILBOX_CAPACITY < 0) Int.MaxValue else Dispatchers.MAILBOX_CAPACITY },
val pushTimeOut: Duration = Dispatchers.MAILBOX_PUSH_TIME_OUT) extends MailboxType {
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")
2011-09-23 09:33:53 +02:00
override def create(_dispatcher: MessageDispatcher) = new Mailbox with QueueBasedMessageQueue with BoundedMessageQueueSemantics with DefaultSystemMessageQueue {
2011-09-22 12:08:00 +02:00
final val queue = new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp))
final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut
2011-09-23 09:33:53 +02:00
final val dispatcher = _dispatcher
}
}