+act #3246 Added control aware mailbox types
This commit is contained in:
parent
dfef14a590
commit
c3950a7525
8 changed files with 414 additions and 0 deletions
|
|
@ -0,0 +1,60 @@
|
||||||
|
package akka.dispatch
|
||||||
|
|
||||||
|
import akka.testkit.{ DefaultTimeout, AkkaSpec }
|
||||||
|
import akka.actor.{ Actor, Props }
|
||||||
|
|
||||||
|
object ControlAwareDispatcherSpec {
|
||||||
|
val config = """
|
||||||
|
unbounded-control-dispatcher {
|
||||||
|
mailbox-type = "akka.dispatch.UnboundedControlAwareMailbox"
|
||||||
|
}
|
||||||
|
bounded-control-dispatcher {
|
||||||
|
mailbox-type = "akka.dispatch.BoundedControlAwareMailbox"
|
||||||
|
}
|
||||||
|
"""
|
||||||
|
|
||||||
|
case object ImportantMessage extends ControlMessage
|
||||||
|
}
|
||||||
|
|
||||||
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
|
class ControlAwareDispatcherSpec extends AkkaSpec(ControlAwareDispatcherSpec.config) with DefaultTimeout {
|
||||||
|
import ControlAwareDispatcherSpec.ImportantMessage
|
||||||
|
|
||||||
|
"A ControlAwareDispatcher" must {
|
||||||
|
"deliver control messages first using an unbounded mailbox" in {
|
||||||
|
val dispatcherKey = "unbounded-control-dispatcher"
|
||||||
|
testControl(dispatcherKey)
|
||||||
|
}
|
||||||
|
|
||||||
|
"deliver control messages first using a bounded mailbox" in {
|
||||||
|
val dispatcherKey = "bounded-control-dispatcher"
|
||||||
|
testControl(dispatcherKey)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def testControl(dispatcherKey: String) = {
|
||||||
|
// It's important that the actor under test is not a top level actor
|
||||||
|
// with RepointableActorRef, since messages might be queued in
|
||||||
|
// UnstartedCell and the sent to the PriorityQueue and consumed immediately
|
||||||
|
// without the ordering taking place.
|
||||||
|
val actor = system.actorOf(Props(new Actor {
|
||||||
|
context.actorOf(Props(new Actor {
|
||||||
|
|
||||||
|
self ! "test"
|
||||||
|
self ! "test2"
|
||||||
|
self ! ImportantMessage
|
||||||
|
|
||||||
|
def receive = {
|
||||||
|
case x ⇒ testActor ! x
|
||||||
|
}
|
||||||
|
}).withDispatcher(dispatcherKey))
|
||||||
|
|
||||||
|
def receive = Actor.emptyBehavior
|
||||||
|
|
||||||
|
}))
|
||||||
|
|
||||||
|
expectMsg(ImportantMessage)
|
||||||
|
expectMsg("test")
|
||||||
|
expectMsg("test2")
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -194,6 +194,14 @@ class PriorityMailboxSpec extends MailboxSpec {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
class ControlAwareMailboxSpec extends MailboxSpec {
|
||||||
|
lazy val name = "The control aware mailbox implementation"
|
||||||
|
def factory = {
|
||||||
|
case UnboundedMailbox() ⇒ new UnboundedControlAwareMailbox().create(None, None)
|
||||||
|
case BoundedMailbox(capacity, pushTimeOut) ⇒ new BoundedControlAwareMailbox(capacity, pushTimeOut).create(None, None)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
object CustomMailboxSpec {
|
object CustomMailboxSpec {
|
||||||
val config = """
|
val config = """
|
||||||
my-dispatcher {
|
my-dispatcher {
|
||||||
|
|
|
||||||
|
|
@ -377,6 +377,12 @@ akka {
|
||||||
akka.actor.mailbox.bounded-deque-based
|
akka.actor.mailbox.bounded-deque-based
|
||||||
"akka.dispatch.MultipleConsumerSemantics" =
|
"akka.dispatch.MultipleConsumerSemantics" =
|
||||||
akka.actor.mailbox.unbounded-queue-based
|
akka.actor.mailbox.unbounded-queue-based
|
||||||
|
"akka.dispatch.ControlAwareMessageQueueSemantics" =
|
||||||
|
akka.actor.mailbox.unbounded-control-aware-queue-based
|
||||||
|
"akka.dispatch.UnboundedControlAwareMessageQueueSemantics" =
|
||||||
|
akka.actor.mailbox.unbounded-control-aware-queue-based
|
||||||
|
"akka.dispatch.BoundedControlAwareMessageQueueSemantics" =
|
||||||
|
akka.actor.mailbox.bounded-control-aware-queue-based
|
||||||
}
|
}
|
||||||
|
|
||||||
unbounded-queue-based {
|
unbounded-queue-based {
|
||||||
|
|
@ -406,6 +412,20 @@ akka {
|
||||||
# com.typesafe.config.Config) parameters.
|
# com.typesafe.config.Config) parameters.
|
||||||
mailbox-type = "akka.dispatch.BoundedDequeBasedMailbox"
|
mailbox-type = "akka.dispatch.BoundedDequeBasedMailbox"
|
||||||
}
|
}
|
||||||
|
|
||||||
|
unbounded-control-aware-queue-based {
|
||||||
|
# FQCN of the MailboxType, The Class of the FQCN must have a public
|
||||||
|
# constructor with (akka.actor.ActorSystem.Settings,
|
||||||
|
# com.typesafe.config.Config) parameters.
|
||||||
|
mailbox-type = "akka.dispatch.UnboundedControlAwareMailbox"
|
||||||
|
}
|
||||||
|
|
||||||
|
bounded-control-aware-queue-based {
|
||||||
|
# FQCN of the MailboxType, The Class of the FQCN must have a public
|
||||||
|
# constructor with (akka.actor.ActorSystem.Settings,
|
||||||
|
# com.typesafe.config.Config) parameters.
|
||||||
|
mailbox-type = "akka.dispatch.BoundedControlAwareMailbox"
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
debug {
|
debug {
|
||||||
|
|
|
||||||
|
|
@ -16,6 +16,9 @@ import scala.concurrent.duration.FiniteDuration
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import scala.util.control.NonFatal
|
import scala.util.control.NonFatal
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
|
import java.util.concurrent.locks.ReentrantLock
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
|
|
@ -676,6 +679,153 @@ object BoundedDequeBasedMailbox {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* ControlAwareMessageQueue handles messages that extend [[akka.dispatch.ControlMessage]] with priority.
|
||||||
|
*/
|
||||||
|
trait ControlAwareMessageQueueSemantics extends QueueBasedMessageQueue {
|
||||||
|
def controlQueue: Queue[Envelope]
|
||||||
|
def queue: Queue[Envelope]
|
||||||
|
|
||||||
|
def enqueue(receiver: ActorRef, handle: Envelope): Unit = handle match {
|
||||||
|
case envelope @ Envelope(_: ControlMessage, _) ⇒ controlQueue add envelope
|
||||||
|
case envelope ⇒ queue add envelope
|
||||||
|
}
|
||||||
|
|
||||||
|
def dequeue(): Envelope = {
|
||||||
|
val controlMsg = controlQueue.poll()
|
||||||
|
|
||||||
|
if (controlMsg ne null) controlMsg
|
||||||
|
else queue.poll()
|
||||||
|
}
|
||||||
|
|
||||||
|
override def numberOfMessages: Int = controlQueue.size() + queue.size()
|
||||||
|
|
||||||
|
override def hasMessages: Boolean = !(queue.isEmpty && controlQueue.isEmpty)
|
||||||
|
}
|
||||||
|
|
||||||
|
trait UnboundedControlAwareMessageQueueSemantics extends UnboundedMessageQueueSemantics with ControlAwareMessageQueueSemantics
|
||||||
|
trait BoundedControlAwareMessageQueueSemantics extends BoundedMessageQueueSemantics with ControlAwareMessageQueueSemantics
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Messages that extend this trait will be handled with priority by control aware mailboxes.
|
||||||
|
*/
|
||||||
|
trait ControlMessage
|
||||||
|
|
||||||
|
/**
|
||||||
|
* UnboundedControlAwareMailbox is an unbounded MailboxType, that maintains two queues
|
||||||
|
* to allow messages that extend [[akka.dispatch.ControlMessage]] to be delivered with priority.
|
||||||
|
*/
|
||||||
|
final case class UnboundedControlAwareMailbox() extends MailboxType with ProducesMessageQueue[UnboundedControlAwareMailbox.MessageQueue] {
|
||||||
|
|
||||||
|
// this constructor will be called via reflection when this mailbox type
|
||||||
|
// is used in the application config
|
||||||
|
def this(settings: ActorSystem.Settings, config: Config) = this()
|
||||||
|
|
||||||
|
def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new UnboundedControlAwareMailbox.MessageQueue
|
||||||
|
}
|
||||||
|
|
||||||
|
object UnboundedControlAwareMailbox {
|
||||||
|
class MessageQueue extends ControlAwareMessageQueueSemantics with UnboundedMessageQueueSemantics {
|
||||||
|
val controlQueue: Queue[Envelope] = new ConcurrentLinkedQueue[Envelope]()
|
||||||
|
val queue: Queue[Envelope] = new ConcurrentLinkedQueue[Envelope]()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* BoundedControlAwareMailbox is a bounded MailboxType, that maintains two queues
|
||||||
|
* to allow messages that extend [[akka.dispatch.ControlMessage]] to be delivered with priority.
|
||||||
|
*/
|
||||||
|
final case class BoundedControlAwareMailbox(capacity: Int, pushTimeOut: FiniteDuration) extends MailboxType with ProducesMessageQueue[BoundedControlAwareMailbox.MessageQueue] {
|
||||||
|
def this(settings: ActorSystem.Settings, config: Config) = this(config.getInt("mailbox-capacity"),
|
||||||
|
config.getNanosDuration("mailbox-push-timeout-time"))
|
||||||
|
|
||||||
|
def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = new BoundedControlAwareMailbox.MessageQueue(capacity, pushTimeOut)
|
||||||
|
}
|
||||||
|
|
||||||
|
object BoundedControlAwareMailbox {
|
||||||
|
class MessageQueue(val capacity: Int, val pushTimeOut: FiniteDuration) extends BoundedControlAwareMessageQueueSemantics {
|
||||||
|
|
||||||
|
private final val size = new AtomicInteger(0)
|
||||||
|
private final val putLock = new ReentrantLock()
|
||||||
|
private final val notFull = putLock.newCondition()
|
||||||
|
|
||||||
|
// no need to use blocking queues here, as blocking is being handled in `enqueueWithTimeout`
|
||||||
|
val controlQueue = new ConcurrentLinkedQueue[Envelope]()
|
||||||
|
val queue = new ConcurrentLinkedQueue[Envelope]()
|
||||||
|
|
||||||
|
override def enqueue(receiver: ActorRef, handle: Envelope): Unit = handle match {
|
||||||
|
case envelope @ Envelope(_: ControlMessage, _) ⇒ enqueueWithTimeout(controlQueue, receiver, envelope)
|
||||||
|
case envelope ⇒ enqueueWithTimeout(queue, receiver, envelope)
|
||||||
|
}
|
||||||
|
|
||||||
|
override def numberOfMessages: Int = size.get()
|
||||||
|
override def hasMessages: Boolean = numberOfMessages > 0
|
||||||
|
|
||||||
|
@tailrec
|
||||||
|
final override def dequeue(): Envelope = {
|
||||||
|
val count = size.get()
|
||||||
|
|
||||||
|
// if both queues are empty return null
|
||||||
|
if (count > 0) {
|
||||||
|
// if there are messages try to fetch the current head
|
||||||
|
// or retry if other consumer dequeued in the mean time
|
||||||
|
if (size.compareAndSet(count, count - 1)) {
|
||||||
|
val item = super.dequeue()
|
||||||
|
|
||||||
|
if (size.get < capacity) signalNotFull()
|
||||||
|
|
||||||
|
item
|
||||||
|
} else {
|
||||||
|
dequeue()
|
||||||
|
}
|
||||||
|
} else {
|
||||||
|
null
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private def signalNotFull() {
|
||||||
|
putLock.lock()
|
||||||
|
|
||||||
|
try {
|
||||||
|
notFull.signal()
|
||||||
|
} finally {
|
||||||
|
putLock.unlock()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private final def enqueueWithTimeout(q: Queue[Envelope], receiver: ActorRef, envelope: Envelope) {
|
||||||
|
var remaining = pushTimeOut.toNanos
|
||||||
|
|
||||||
|
putLock.lockInterruptibly()
|
||||||
|
val inserted = try {
|
||||||
|
var stop = false
|
||||||
|
while (size.get() == capacity && !stop) {
|
||||||
|
remaining = notFull.awaitNanos(remaining)
|
||||||
|
stop = remaining <= 0
|
||||||
|
}
|
||||||
|
|
||||||
|
if (stop) {
|
||||||
|
false
|
||||||
|
} else {
|
||||||
|
q.add(envelope)
|
||||||
|
val c = size.incrementAndGet()
|
||||||
|
|
||||||
|
if (c < capacity) notFull.signal()
|
||||||
|
|
||||||
|
true
|
||||||
|
}
|
||||||
|
} finally {
|
||||||
|
putLock.unlock()
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!inserted) {
|
||||||
|
receiver.asInstanceOf[InternalActorRef].provider.deadLetters.tell(
|
||||||
|
DeadLetter(envelope.message, envelope.sender, receiver), envelope.sender)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Trait to signal that an Actor requires a certain type of message queue semantics.
|
* Trait to signal that an Actor requires a certain type of message queue semantics.
|
||||||
*
|
*
|
||||||
|
|
|
||||||
|
|
@ -3,6 +3,7 @@
|
||||||
*/
|
*/
|
||||||
package docs.dispatcher;
|
package docs.dispatcher;
|
||||||
|
|
||||||
|
import akka.dispatch.ControlMessage;
|
||||||
import akka.dispatch.RequiresMessageQueue;
|
import akka.dispatch.RequiresMessageQueue;
|
||||||
import akka.testkit.AkkaSpec;
|
import akka.testkit.AkkaSpec;
|
||||||
import com.typesafe.config.ConfigFactory;
|
import com.typesafe.config.ConfigFactory;
|
||||||
|
|
@ -150,6 +151,41 @@ public class DispatcherDocTest {
|
||||||
probe.expectMsgClass(Terminated.class);
|
probe.expectMsgClass(Terminated.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void controlAwareDispatcher() throws Exception {
|
||||||
|
JavaTestKit probe = new JavaTestKit(system);
|
||||||
|
//#control-aware-dispatcher
|
||||||
|
|
||||||
|
class Demo extends UntypedActor {
|
||||||
|
LoggingAdapter log = Logging.getLogger(getContext().system(), this);
|
||||||
|
{
|
||||||
|
for (Object msg : new Object[] { "foo", "bar", new MyControlMessage(),
|
||||||
|
PoisonPill.getInstance() }) {
|
||||||
|
getSelf().tell(msg, getSelf());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public void onReceive(Object message) {
|
||||||
|
log.info(message.toString());
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// We create a new Actor that just prints out what it processes
|
||||||
|
ActorRef myActor = system.actorOf(Props.create(Demo.class, this)
|
||||||
|
.withDispatcher("control-aware-dispatcher"));
|
||||||
|
|
||||||
|
/*
|
||||||
|
Logs:
|
||||||
|
'MyControlMessage
|
||||||
|
'foo
|
||||||
|
'bar
|
||||||
|
*/
|
||||||
|
//#control-aware-dispatcher
|
||||||
|
|
||||||
|
probe.watch(myActor);
|
||||||
|
probe.expectMsgClass(Terminated.class);
|
||||||
|
}
|
||||||
|
|
||||||
static
|
static
|
||||||
//#prio-mailbox
|
//#prio-mailbox
|
||||||
public class MyPrioMailbox extends UnboundedPriorityMailbox {
|
public class MyPrioMailbox extends UnboundedPriorityMailbox {
|
||||||
|
|
@ -173,6 +209,11 @@ public class DispatcherDocTest {
|
||||||
}
|
}
|
||||||
//#prio-mailbox
|
//#prio-mailbox
|
||||||
|
|
||||||
|
static
|
||||||
|
//#control-aware-mailbox-messages
|
||||||
|
public class MyControlMessage implements ControlMessage {}
|
||||||
|
//#control-aware-mailbox-messages
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void requiredMailboxDispatcher() throws Exception {
|
public void requiredMailboxDispatcher() throws Exception {
|
||||||
ActorRef myActor = system.actorOf(Props.create(MyUntypedActor.class)
|
ActorRef myActor = system.actorOf(Props.create(MyUntypedActor.class)
|
||||||
|
|
|
||||||
|
|
@ -161,9 +161,36 @@ Akka comes shipped with a number of mailbox implementations:
|
||||||
|
|
||||||
- Configuration name: "akka.dispatch.BoundedPriorityMailbox"
|
- Configuration name: "akka.dispatch.BoundedPriorityMailbox"
|
||||||
|
|
||||||
|
* UnboundedControlAwareMailbox
|
||||||
|
|
||||||
|
- Delivers messages that extend ``akka.dispatch.ControlMessage`` with higher priority
|
||||||
|
|
||||||
|
- Backed by two ``java.util.concurrent.ConcurrentLinkedQueue``
|
||||||
|
|
||||||
|
- Blocking: No
|
||||||
|
|
||||||
|
- Bounded: No
|
||||||
|
|
||||||
|
- Configuration name: "akka.dispatch.UnboundedControlAwareMailbox"
|
||||||
|
|
||||||
|
* BoundedControlAwareMailbox
|
||||||
|
|
||||||
|
- Delivers messages that extend ``akka.dispatch.ControlMessage`` with higher priority
|
||||||
|
|
||||||
|
- Backed by two ``java.util.concurrent.ConcurrentLinkedQueue`` and blocking on enqueue if capacity has been reached
|
||||||
|
|
||||||
|
- Blocking: Yes
|
||||||
|
|
||||||
|
- Bounded: Yes
|
||||||
|
|
||||||
|
- Configuration name: "akka.dispatch.BoundedControlAwareMailbox"
|
||||||
|
|
||||||
Mailbox configuration examples
|
Mailbox configuration examples
|
||||||
==============================
|
==============================
|
||||||
|
|
||||||
|
PriorityMailbox
|
||||||
|
---------------
|
||||||
|
|
||||||
How to create a PriorityMailbox:
|
How to create a PriorityMailbox:
|
||||||
|
|
||||||
.. includecode:: ../java/code/docs/dispatcher/DispatcherDocTest.java#prio-mailbox
|
.. includecode:: ../java/code/docs/dispatcher/DispatcherDocTest.java#prio-mailbox
|
||||||
|
|
@ -189,6 +216,23 @@ Or code like this:
|
||||||
|
|
||||||
.. includecode:: code/docs/dispatcher/DispatcherDocTest.java#defining-mailbox-in-code
|
.. includecode:: code/docs/dispatcher/DispatcherDocTest.java#defining-mailbox-in-code
|
||||||
|
|
||||||
|
ControlAwareMailbox
|
||||||
|
-------------------
|
||||||
|
|
||||||
|
A ``ControlAwareMailbox`` can be very useful if an actor needs to be able to receive control messages
|
||||||
|
immediately no matter how many other messages are already in its mailbox.
|
||||||
|
|
||||||
|
It can be configured like this:
|
||||||
|
|
||||||
|
.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#control-aware-mailbox-config
|
||||||
|
|
||||||
|
Control messages need to extend the ``ControlMessage`` trait:
|
||||||
|
|
||||||
|
.. includecode:: ../java/code/docs/dispatcher/DispatcherDocTest.java#control-aware-mailbox-messages
|
||||||
|
|
||||||
|
And then an example on how you would use it:
|
||||||
|
|
||||||
|
.. includecode:: ../java/code/docs/dispatcher/DispatcherDocTest.java#control-aware-dispatcher
|
||||||
|
|
||||||
Creating your own Mailbox type
|
Creating your own Mailbox type
|
||||||
==============================
|
==============================
|
||||||
|
|
|
||||||
|
|
@ -175,6 +175,14 @@ object DispatcherDocSpec {
|
||||||
mailbox-type = "docs.dispatcher.MyUnboundedMailbox"
|
mailbox-type = "docs.dispatcher.MyUnboundedMailbox"
|
||||||
}
|
}
|
||||||
//#custom-mailbox-config
|
//#custom-mailbox-config
|
||||||
|
|
||||||
|
//#control-aware-mailbox-config
|
||||||
|
control-aware-dispatcher {
|
||||||
|
mailbox-type = "akka.dispatch.UnboundedControlAwareMailbox"
|
||||||
|
//Other dispatcher configuration goes here
|
||||||
|
}
|
||||||
|
//#control-aware-mailbox-config
|
||||||
|
|
||||||
"""
|
"""
|
||||||
|
|
||||||
//#prio-mailbox
|
//#prio-mailbox
|
||||||
|
|
@ -202,6 +210,12 @@ object DispatcherDocSpec {
|
||||||
})
|
})
|
||||||
//#prio-mailbox
|
//#prio-mailbox
|
||||||
|
|
||||||
|
//#control-aware-mailbox-messages
|
||||||
|
import akka.dispatch.ControlMessage
|
||||||
|
|
||||||
|
case object MyControlMessage extends ControlMessage
|
||||||
|
//#control-aware-mailbox-messages
|
||||||
|
|
||||||
class MyActor extends Actor {
|
class MyActor extends Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
case x =>
|
case x =>
|
||||||
|
|
@ -331,6 +345,39 @@ class DispatcherDocSpec extends AkkaSpec(DispatcherDocSpec.config) {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
"defining control aware dispatcher" in {
|
||||||
|
new AnyRef {
|
||||||
|
//#control-aware-dispatcher
|
||||||
|
|
||||||
|
// We create a new Actor that just prints out what it processes
|
||||||
|
class Logger extends Actor {
|
||||||
|
val log: LoggingAdapter = Logging(context.system, this)
|
||||||
|
|
||||||
|
self ! 'foo
|
||||||
|
self ! 'bar
|
||||||
|
self ! MyControlMessage
|
||||||
|
self ! PoisonPill
|
||||||
|
|
||||||
|
def receive = {
|
||||||
|
case x => log.info(x.toString)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
val a = system.actorOf(Props(classOf[Logger], this).withDispatcher(
|
||||||
|
"control-aware-dispatcher"))
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Logs:
|
||||||
|
* MyControlMessage
|
||||||
|
* 'foo
|
||||||
|
* 'bar
|
||||||
|
*/
|
||||||
|
//#control-aware-dispatcher
|
||||||
|
|
||||||
|
watch(a)
|
||||||
|
expectMsgPF() { case Terminated(`a`) => () }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
"require custom mailbox on dispatcher" in {
|
"require custom mailbox on dispatcher" in {
|
||||||
val myActor = system.actorOf(Props[MyActor].withDispatcher(
|
val myActor = system.actorOf(Props[MyActor].withDispatcher(
|
||||||
"custom-dispatcher"))
|
"custom-dispatcher"))
|
||||||
|
|
|
||||||
|
|
@ -161,9 +161,36 @@ Akka comes shipped with a number of mailbox implementations:
|
||||||
|
|
||||||
- Configuration name: "akka.dispatch.BoundedPriorityMailbox"
|
- Configuration name: "akka.dispatch.BoundedPriorityMailbox"
|
||||||
|
|
||||||
|
* UnboundedControlAwareMailbox
|
||||||
|
|
||||||
|
- Delivers messages that extend ``akka.dispatch.ControlMessage`` with higher priority
|
||||||
|
|
||||||
|
- Backed by two ``java.util.concurrent.ConcurrentLinkedQueue``
|
||||||
|
|
||||||
|
- Blocking: No
|
||||||
|
|
||||||
|
- Bounded: No
|
||||||
|
|
||||||
|
- Configuration name: "akka.dispatch.UnboundedControlAwareMailbox"
|
||||||
|
|
||||||
|
* BoundedControlAwareMailbox
|
||||||
|
|
||||||
|
- Delivers messages that extend ``akka.dispatch.ControlMessage`` with higher priority
|
||||||
|
|
||||||
|
- Backed by two ``java.util.concurrent.ConcurrentLinkedQueue`` and blocking on enqueue if capacity has been reached
|
||||||
|
|
||||||
|
- Blocking: Yes
|
||||||
|
|
||||||
|
- Bounded: Yes
|
||||||
|
|
||||||
|
- Configuration name: "akka.dispatch.BoundedControlAwareMailbox"
|
||||||
|
|
||||||
Mailbox configuration examples
|
Mailbox configuration examples
|
||||||
==============================
|
==============================
|
||||||
|
|
||||||
|
PriorityMailbox
|
||||||
|
---------------
|
||||||
|
|
||||||
How to create a PriorityMailbox:
|
How to create a PriorityMailbox:
|
||||||
|
|
||||||
.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#prio-mailbox
|
.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#prio-mailbox
|
||||||
|
|
@ -189,6 +216,23 @@ Or code like this:
|
||||||
|
|
||||||
.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#defining-mailbox-in-code
|
.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#defining-mailbox-in-code
|
||||||
|
|
||||||
|
ControlAwareMailbox
|
||||||
|
-------------------
|
||||||
|
|
||||||
|
A ``ControlAwareMailbox`` can be very useful if an actor needs to be able to receive control messages
|
||||||
|
immediately no matter how many other messages are already in its mailbox.
|
||||||
|
|
||||||
|
It can be configured like this:
|
||||||
|
|
||||||
|
.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#control-aware-mailbox-config
|
||||||
|
|
||||||
|
Control messages need to extend the ``ControlMessage`` trait:
|
||||||
|
|
||||||
|
.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#control-aware-mailbox-messages
|
||||||
|
|
||||||
|
And then an example on how you would use it:
|
||||||
|
|
||||||
|
.. includecode:: ../scala/code/docs/dispatcher/DispatcherDocSpec.scala#control-aware-dispatcher
|
||||||
|
|
||||||
Creating your own Mailbox type
|
Creating your own Mailbox type
|
||||||
==============================
|
==============================
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue