improve DeadLetter reporting
(since I know now what’s causing these Jenkins failures ;-) ) - include recipient in DeadLetter - include recipient in calls to enqueue/systemEnqueue - move DeadLetterMailbox to ActorSystem (saves some space, too) - hook up DeadLetterMailbox so it sends DeadLetters to app.deadLetters, which publishes them on the eventStream - subscribe TestEventListener to DeadLetter and turn it into Warning The generated warnings about ChildTerminated are very much correct, they remind us that we still need to fix supervisor.stop() to await all children’s death before actually committing suicide.
This commit is contained in:
parent
85e37ea8ef
commit
1ba168774f
11 changed files with 96 additions and 78 deletions
|
|
@ -39,13 +39,13 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
|
||||||
|
|
||||||
val exampleMessage = createMessageInvocation("test")
|
val exampleMessage = createMessageInvocation("test")
|
||||||
|
|
||||||
for (i ← 1 to config.capacity) q.enqueue(exampleMessage)
|
for (i ← 1 to config.capacity) q.enqueue(null, exampleMessage)
|
||||||
|
|
||||||
q.numberOfMessages must be === config.capacity
|
q.numberOfMessages must be === config.capacity
|
||||||
q.hasMessages must be === true
|
q.hasMessages must be === true
|
||||||
|
|
||||||
intercept[MessageQueueAppendFailedException] {
|
intercept[MessageQueueAppendFailedException] {
|
||||||
q.enqueue(exampleMessage)
|
q.enqueue(null, exampleMessage)
|
||||||
}
|
}
|
||||||
|
|
||||||
q.dequeue must be === exampleMessage
|
q.dequeue must be === exampleMessage
|
||||||
|
|
@ -103,7 +103,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
|
||||||
|
|
||||||
def createProducer(fromNum: Int, toNum: Int): Future[Vector[Envelope]] = spawn {
|
def createProducer(fromNum: Int, toNum: Int): Future[Vector[Envelope]] = spawn {
|
||||||
val messages = Vector() ++ (for (i ← fromNum to toNum) yield createMessageInvocation(i))
|
val messages = Vector() ++ (for (i ← fromNum to toNum) yield createMessageInvocation(i))
|
||||||
for (i ← messages) q.enqueue(i)
|
for (i ← messages) q.enqueue(null, i)
|
||||||
messages
|
messages
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -364,7 +364,7 @@ trait MinimalActorRef extends ActorRef with ScalaActorRef {
|
||||||
throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName))
|
throw new UnsupportedOperationException("Not supported for %s".format(getClass.getName))
|
||||||
}
|
}
|
||||||
|
|
||||||
case class DeadLetter(message: Any, sender: ActorRef)
|
case class DeadLetter(message: Any, sender: ActorRef, recipient: ActorRef)
|
||||||
|
|
||||||
object DeadLetterActorRef {
|
object DeadLetterActorRef {
|
||||||
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
|
class SerializedDeadLetterActorRef extends Serializable { //TODO implement as Protobuf for performance?
|
||||||
|
|
@ -387,11 +387,13 @@ class DeadLetterActorRef(val app: ActorSystem) extends MinimalActorRef {
|
||||||
|
|
||||||
override def isShutdown(): Boolean = true
|
override def isShutdown(): Boolean = true
|
||||||
|
|
||||||
override def tell(msg: Any, sender: ActorRef): Unit =
|
override def tell(msg: Any, sender: ActorRef): Unit = msg match {
|
||||||
app.eventStream.publish(DeadLetter(msg, sender))
|
case d: DeadLetter ⇒ app.eventStream.publish(d)
|
||||||
|
case _ ⇒ app.eventStream.publish(DeadLetter(msg, sender, this))
|
||||||
|
}
|
||||||
|
|
||||||
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = {
|
override def ?(message: Any)(implicit timeout: Timeout): Future[Any] = {
|
||||||
app.eventStream.publish(DeadLetter(message, this))
|
app.eventStream.publish(DeadLetter(message, app.provider.dummyAskSender, this))
|
||||||
brokenPromise
|
brokenPromise
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -49,6 +49,7 @@ trait ActorRefProvider {
|
||||||
|
|
||||||
private[akka] def terminationFuture: Future[ActorSystem.ExitStatus]
|
private[akka] def terminationFuture: Future[ActorSystem.ExitStatus]
|
||||||
|
|
||||||
|
private[akka] def dummyAskSender: ActorRef
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -277,6 +278,8 @@ class LocalActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
||||||
a.result
|
a.result
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private[akka] val dummyAskSender = new DeadLetterActorRef(app)
|
||||||
}
|
}
|
||||||
|
|
||||||
class LocalDeathWatch extends DeathWatch with ActorClassification {
|
class LocalDeathWatch extends DeathWatch with ActorClassification {
|
||||||
|
|
|
||||||
|
|
@ -9,7 +9,7 @@ import akka.event._
|
||||||
import akka.util.duration._
|
import akka.util.duration._
|
||||||
import java.net.InetAddress
|
import java.net.InetAddress
|
||||||
import com.eaio.uuid.UUID
|
import com.eaio.uuid.UUID
|
||||||
import akka.dispatch.{ Dispatchers, Future }
|
import akka.dispatch.{ Dispatchers, Future, Mailbox, Envelope, SystemMessage }
|
||||||
import akka.util.Duration
|
import akka.util.Duration
|
||||||
import akka.util.ReflectiveAccess
|
import akka.util.ReflectiveAccess
|
||||||
import akka.serialization.Serialization
|
import akka.serialization.Serialization
|
||||||
|
|
@ -210,6 +210,17 @@ class ActorSystem(val name: String, val config: Configuration) extends ActorRefF
|
||||||
|
|
||||||
// TODO think about memory consistency effects when doing funky stuff inside constructor
|
// TODO think about memory consistency effects when doing funky stuff inside constructor
|
||||||
val deadLetters = new DeadLetterActorRef(this)
|
val deadLetters = new DeadLetterActorRef(this)
|
||||||
|
val deadLetterMailbox = new Mailbox(null) {
|
||||||
|
becomeClosed()
|
||||||
|
override def dispatcher = null //MessageDispatcher.this
|
||||||
|
override def enqueue(receiver: ActorRef, envelope: Envelope) { deadLetters ! DeadLetter(envelope.message, envelope.sender, receiver) }
|
||||||
|
override def dequeue() = null
|
||||||
|
override def systemEnqueue(receiver: ActorRef, handle: SystemMessage) { deadLetters ! DeadLetter(handle, receiver, receiver) }
|
||||||
|
override def systemDrain(): SystemMessage = null
|
||||||
|
override def hasMessages = false
|
||||||
|
override def hasSystemMessages = false
|
||||||
|
override def numberOfMessages = 0
|
||||||
|
}
|
||||||
|
|
||||||
val deathWatch = provider.createDeathWatch()
|
val deathWatch = provider.createDeathWatch()
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -100,21 +100,9 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
|
||||||
protected[akka] def createMailbox(actor: ActorCell): Mailbox
|
protected[akka] def createMailbox(actor: ActorCell): Mailbox
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create a blackhole mailbox for the purpose of replacing the real one upon actor termination
|
* a blackhole mailbox for the purpose of replacing the real one upon actor termination
|
||||||
*/
|
*/
|
||||||
protected[akka] val deadLetterMailbox: Mailbox = DeadLetterMailbox
|
import app.deadLetterMailbox
|
||||||
|
|
||||||
object DeadLetterMailbox extends Mailbox(null) {
|
|
||||||
becomeClosed()
|
|
||||||
override def dispatcher = null //MessageDispatcher.this
|
|
||||||
override def enqueue(envelope: Envelope) = ()
|
|
||||||
override def dequeue() = null
|
|
||||||
override def systemEnqueue(handle: SystemMessage): Unit = ()
|
|
||||||
override def systemDrain(): SystemMessage = null
|
|
||||||
override def hasMessages = false
|
|
||||||
override def hasSystemMessages = false
|
|
||||||
override def numberOfMessages = 0
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Name of this dispatcher.
|
* Name of this dispatcher.
|
||||||
|
|
@ -225,7 +213,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
|
||||||
// message must be “virgin” before being able to systemEnqueue again
|
// message must be “virgin” before being able to systemEnqueue again
|
||||||
val next = message.next
|
val next = message.next
|
||||||
message.next = null
|
message.next = null
|
||||||
deadLetterMailbox.systemEnqueue(message)
|
deadLetterMailbox.systemEnqueue(actor.self, message)
|
||||||
message = next
|
message = next
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -233,7 +221,7 @@ abstract class MessageDispatcher(val app: ActorSystem) extends Serializable {
|
||||||
if (mailBox.hasMessages) {
|
if (mailBox.hasMessages) {
|
||||||
var envelope = mailBox.dequeue
|
var envelope = mailBox.dequeue
|
||||||
while (envelope ne null) {
|
while (envelope ne null) {
|
||||||
deadLetterMailbox.enqueue(envelope)
|
deadLetterMailbox.enqueue(actor.self, envelope)
|
||||||
envelope = mailBox.dequeue
|
envelope = mailBox.dequeue
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -5,7 +5,7 @@
|
||||||
package akka.dispatch
|
package akka.dispatch
|
||||||
|
|
||||||
import util.DynamicVariable
|
import util.DynamicVariable
|
||||||
import akka.actor.{ ActorCell, Actor, IllegalActorStateException }
|
import akka.actor.{ ActorCell, Actor, IllegalActorStateException, ActorRef }
|
||||||
import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
|
import java.util.concurrent.{ LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
|
||||||
import java.util.{ Comparator, Queue }
|
import java.util.{ Comparator, Queue }
|
||||||
import annotation.tailrec
|
import annotation.tailrec
|
||||||
|
|
@ -37,6 +37,8 @@ class BalancingDispatcher(
|
||||||
_timeoutMs: Long)
|
_timeoutMs: Long)
|
||||||
extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
|
extends Dispatcher(_app, _name, throughput, throughputDeadlineTime, mailboxType, config, _timeoutMs) {
|
||||||
|
|
||||||
|
import app.deadLetterMailbox
|
||||||
|
|
||||||
private val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator)
|
private val buddies = new ConcurrentSkipListSet[ActorCell](akka.util.Helpers.IdentityHashComparator)
|
||||||
|
|
||||||
protected val messageQueue: MessageQueue = mailboxType match {
|
protected val messageQueue: MessageQueue = mailboxType match {
|
||||||
|
|
@ -55,7 +57,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(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue {
|
class SharingMailbox(_actor: ActorCell) extends Mailbox(_actor) with DefaultSystemMessageQueue {
|
||||||
final def enqueue(handle: Envelope) = messageQueue.enqueue(handle)
|
final def enqueue(receiver: ActorRef, handle: Envelope) = messageQueue.enqueue(receiver, handle)
|
||||||
|
|
||||||
final def dequeue(): Envelope = messageQueue.dequeue()
|
final def dequeue(): Envelope = messageQueue.dequeue()
|
||||||
|
|
||||||
|
|
@ -86,7 +88,7 @@ class BalancingDispatcher(
|
||||||
if (mailBox.hasSystemMessages) {
|
if (mailBox.hasSystemMessages) {
|
||||||
var messages = mailBox.systemDrain()
|
var messages = mailBox.systemDrain()
|
||||||
while (messages ne null) {
|
while (messages ne null) {
|
||||||
deadLetterMailbox.systemEnqueue(messages) //Send to dead letter queue
|
deadLetterMailbox.systemEnqueue(actor.self, messages) //Send to dead letter queue
|
||||||
messages = messages.next
|
messages = messages.next
|
||||||
if (messages eq null) //Make sure that any system messages received after the current drain are also sent to the dead letter mbox
|
if (messages eq null) //Make sure that any system messages received after the current drain are also sent to the dead letter mbox
|
||||||
messages = mailBox.systemDrain()
|
messages = mailBox.systemDrain()
|
||||||
|
|
@ -112,7 +114,7 @@ class BalancingDispatcher(
|
||||||
}
|
}
|
||||||
|
|
||||||
override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
|
override protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
|
||||||
messageQueue enqueue invocation
|
messageQueue.enqueue(receiver.self, invocation)
|
||||||
|
|
||||||
intoTheFray(except = receiver)
|
intoTheFray(except = receiver)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -78,13 +78,13 @@ class Dispatcher(
|
||||||
|
|
||||||
protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
|
protected[akka] def dispatch(receiver: ActorCell, invocation: Envelope) = {
|
||||||
val mbox = receiver.mailbox
|
val mbox = receiver.mailbox
|
||||||
mbox enqueue invocation
|
mbox.enqueue(receiver.self, invocation)
|
||||||
registerForExecution(mbox, true, false)
|
registerForExecution(mbox, true, false)
|
||||||
}
|
}
|
||||||
|
|
||||||
protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage) = {
|
protected[akka] def systemDispatch(receiver: ActorCell, invocation: SystemMessage) = {
|
||||||
val mbox = receiver.mailbox
|
val mbox = receiver.mailbox
|
||||||
mbox systemEnqueue invocation
|
mbox.systemEnqueue(receiver.self, invocation)
|
||||||
registerForExecution(mbox, false, true)
|
registerForExecution(mbox, false, true)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -7,7 +7,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, ActorCell }
|
import akka.actor.{ ActorContext, ActorCell, ActorRef }
|
||||||
import java.util.concurrent._
|
import java.util.concurrent._
|
||||||
import atomic.{ AtomicInteger, AtomicReferenceFieldUpdater }
|
import atomic.{ AtomicInteger, AtomicReferenceFieldUpdater }
|
||||||
import annotation.tailrec
|
import annotation.tailrec
|
||||||
|
|
@ -217,7 +217,7 @@ trait MessageQueue {
|
||||||
/*
|
/*
|
||||||
* These method need to be implemented in subclasses; they should not rely on the internal stuff above.
|
* These method need to be implemented in subclasses; they should not rely on the internal stuff above.
|
||||||
*/
|
*/
|
||||||
def enqueue(handle: Envelope)
|
def enqueue(receiver: ActorRef, handle: Envelope)
|
||||||
|
|
||||||
def dequeue(): Envelope
|
def dequeue(): Envelope
|
||||||
|
|
||||||
|
|
@ -230,7 +230,7 @@ trait SystemMessageQueue {
|
||||||
/**
|
/**
|
||||||
* Enqueue a new system message, e.g. by prepending atomically as new head of a single-linked list.
|
* Enqueue a new system message, e.g. by prepending atomically as new head of a single-linked list.
|
||||||
*/
|
*/
|
||||||
def systemEnqueue(message: SystemMessage): Unit
|
def systemEnqueue(receiver: ActorRef, message: SystemMessage): Unit
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Dequeue all messages from system queue and return them as single-linked list.
|
* Dequeue all messages from system queue and return them as single-linked list.
|
||||||
|
|
@ -243,7 +243,7 @@ trait SystemMessageQueue {
|
||||||
trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
||||||
|
|
||||||
@tailrec
|
@tailrec
|
||||||
final def systemEnqueue(message: SystemMessage): Unit = {
|
final def systemEnqueue(receiver: ActorRef, message: SystemMessage): Unit = {
|
||||||
assert(message.next eq null)
|
assert(message.next eq null)
|
||||||
if (Mailbox.debug) println(actor + " having enqueued " + message)
|
if (Mailbox.debug) println(actor + " having enqueued " + message)
|
||||||
val head = systemQueueGet
|
val head = systemQueueGet
|
||||||
|
|
@ -256,7 +256,7 @@ trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
||||||
message.next = head
|
message.next = head
|
||||||
if (!systemQueuePut(head, message)) {
|
if (!systemQueuePut(head, message)) {
|
||||||
message.next = null
|
message.next = null
|
||||||
systemEnqueue(message)
|
systemEnqueue(receiver, message)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -270,7 +270,7 @@ trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
||||||
}
|
}
|
||||||
|
|
||||||
trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
trait UnboundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
||||||
final def enqueue(handle: Envelope): Unit = queue add handle
|
final def enqueue(receiver: ActorRef, handle: Envelope): Unit = queue add handle
|
||||||
final def dequeue(): Envelope = queue.poll()
|
final def dequeue(): Envelope = queue.poll()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -278,7 +278,7 @@ trait BoundedMessageQueueSemantics extends QueueBasedMessageQueue {
|
||||||
def pushTimeOut: Duration
|
def pushTimeOut: Duration
|
||||||
override def queue: BlockingQueue[Envelope]
|
override def queue: BlockingQueue[Envelope]
|
||||||
|
|
||||||
final def enqueue(handle: Envelope) {
|
final def enqueue(receiver: ActorRef, handle: Envelope) {
|
||||||
if (pushTimeOut.length > 0) {
|
if (pushTimeOut.length > 0) {
|
||||||
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
|
queue.offer(handle, pushTimeOut.length, pushTimeOut.unit) || {
|
||||||
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + toString)
|
throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + toString)
|
||||||
|
|
|
||||||
|
|
@ -241,6 +241,8 @@ class RemoteActorRefProvider(val app: ActorSystem) extends ActorRefProvider {
|
||||||
private[akka] def createDeathWatch(): DeathWatch = local.createDeathWatch() //FIXME Implement Remote DeathWatch
|
private[akka] def createDeathWatch(): DeathWatch = local.createDeathWatch() //FIXME Implement Remote DeathWatch
|
||||||
|
|
||||||
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = local.ask(message, recipient, within)
|
private[akka] def ask(message: Any, recipient: ActorRef, within: Timeout): Future[Any] = local.ask(message, recipient, within)
|
||||||
|
|
||||||
|
private[akka] def dummyAskSender = local.dummyAskSender
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -10,9 +10,8 @@ import java.util.concurrent.RejectedExecutionException
|
||||||
import akka.util.Switch
|
import akka.util.Switch
|
||||||
import java.lang.ref.WeakReference
|
import java.lang.ref.WeakReference
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
import akka.actor.ActorCell
|
import akka.actor.{ ActorCell, ActorRef, ActorSystem }
|
||||||
import akka.dispatch._
|
import akka.dispatch._
|
||||||
import akka.actor.ActorSystem
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Locking rules:
|
* Locking rules:
|
||||||
|
|
@ -129,16 +128,17 @@ class CallingThreadDispatcher(_app: ActorSystem, val name: String = "calling-thr
|
||||||
}
|
}
|
||||||
|
|
||||||
override def resume(actor: ActorCell) {
|
override def resume(actor: ActorCell) {
|
||||||
val mboxopt = getMailbox(actor)
|
actor.mailbox match {
|
||||||
if (mboxopt.isEmpty) return
|
case mbox: CallingThreadMailbox ⇒
|
||||||
val mbox = mboxopt.get
|
val queue = mbox.queue
|
||||||
val queue = mbox.queue
|
val wasActive = queue.isActive
|
||||||
val wasActive = queue.isActive
|
val switched = mbox.suspendSwitch.switchOff {
|
||||||
val switched = mbox.suspendSwitch.switchOff {
|
gatherFromAllOtherQueues(mbox, queue)
|
||||||
gatherFromAllOtherQueues(mbox, queue)
|
}
|
||||||
}
|
if (switched && !wasActive) {
|
||||||
if (switched && !wasActive) {
|
runQueue(mbox, queue)
|
||||||
runQueue(mbox, queue)
|
}
|
||||||
|
case m ⇒ m.systemEnqueue(actor.self, Resume())
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -147,35 +147,37 @@ class CallingThreadDispatcher(_app: ActorSystem, val name: String = "calling-thr
|
||||||
override def mailboxIsEmpty(actor: ActorCell): Boolean = getMailbox(actor) map (_.queue.isEmpty) getOrElse true
|
override def mailboxIsEmpty(actor: ActorCell): Boolean = getMailbox(actor) map (_.queue.isEmpty) getOrElse true
|
||||||
|
|
||||||
protected[akka] override def systemDispatch(receiver: ActorCell, message: SystemMessage) {
|
protected[akka] override def systemDispatch(receiver: ActorCell, message: SystemMessage) {
|
||||||
val mboxopt = getMailbox(receiver)
|
receiver.mailbox match {
|
||||||
if (mboxopt.isEmpty) return
|
case mbox: CallingThreadMailbox ⇒
|
||||||
val mbox = mboxopt.get
|
mbox.systemEnqueue(receiver.self, message)
|
||||||
mbox.systemEnqueue(message)
|
val queue = mbox.queue
|
||||||
val queue = mbox.queue
|
if (!queue.isActive) {
|
||||||
if (!queue.isActive) {
|
queue.enter
|
||||||
queue.enter
|
runQueue(mbox, queue)
|
||||||
runQueue(mbox, queue)
|
}
|
||||||
|
case m ⇒ m.systemEnqueue(receiver.self, message)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected[akka] override def dispatch(receiver: ActorCell, handle: Envelope) {
|
protected[akka] override def dispatch(receiver: ActorCell, handle: Envelope) {
|
||||||
val mboxopt = getMailbox(receiver)
|
receiver.mailbox match {
|
||||||
if (mboxopt.isEmpty) return
|
case mbox: CallingThreadMailbox ⇒
|
||||||
val mbox = mboxopt.get
|
val queue = mbox.queue
|
||||||
val queue = mbox.queue
|
val execute = mbox.suspendSwitch.fold {
|
||||||
val execute = mbox.suspendSwitch.fold {
|
queue.push(handle)
|
||||||
queue.push(handle)
|
false
|
||||||
false
|
} {
|
||||||
} {
|
queue.push(handle)
|
||||||
queue.push(handle)
|
if (queue.isActive)
|
||||||
if (queue.isActive)
|
false
|
||||||
false
|
else {
|
||||||
else {
|
queue.enter
|
||||||
queue.enter
|
true
|
||||||
true
|
}
|
||||||
}
|
}
|
||||||
|
if (execute) runQueue(mbox, queue)
|
||||||
|
case m ⇒ m.enqueue(receiver.self, handle)
|
||||||
}
|
}
|
||||||
if (execute) runQueue(mbox, queue)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
protected[akka] override def executeTask(invocation: TaskInvocation) { invocation.run }
|
protected[akka] override def executeTask(invocation: TaskInvocation) { invocation.run }
|
||||||
|
|
@ -270,7 +272,7 @@ class CallingThreadMailbox(val dispatcher: MessageDispatcher, _receiver: ActorCe
|
||||||
val lock = new ReentrantLock
|
val lock = new ReentrantLock
|
||||||
val suspendSwitch = new Switch
|
val suspendSwitch = new Switch
|
||||||
|
|
||||||
override def enqueue(msg: Envelope) {}
|
override def enqueue(receiver: ActorRef, msg: Envelope) {}
|
||||||
override def dequeue() = null
|
override def dequeue() = null
|
||||||
override def hasMessages = true
|
override def hasMessages = true
|
||||||
override def numberOfMessages = 0
|
override def numberOfMessages = 0
|
||||||
|
|
|
||||||
|
|
@ -3,13 +3,15 @@
|
||||||
*/
|
*/
|
||||||
package akka.testkit
|
package akka.testkit
|
||||||
|
|
||||||
|
import scala.annotation.tailrec
|
||||||
import scala.util.matching.Regex
|
import scala.util.matching.Regex
|
||||||
|
|
||||||
import akka.actor.Actor
|
import akka.actor.{ DeadLetter, ActorSystem }
|
||||||
import akka.event.Logging._
|
import akka.dispatch.SystemMessage
|
||||||
|
import akka.event.Logging.{ Warning, LogEvent, InitializeLogger, Info, Error, Debug }
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
|
import akka.testkit.TestEvent.{ UnMute, Mute }
|
||||||
import akka.util.Duration
|
import akka.util.Duration
|
||||||
import akka.actor.ActorSystem
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Implementation helpers of the EventFilter facilities: send `Mute`
|
* Implementation helpers of the EventFilter facilities: send `Mute`
|
||||||
|
|
@ -443,10 +445,16 @@ class TestEventListener extends Logging.DefaultLogger {
|
||||||
var filters: List[EventFilter] = Nil
|
var filters: List[EventFilter] = Nil
|
||||||
|
|
||||||
override def receive = {
|
override def receive = {
|
||||||
case InitializeLogger(bus) ⇒ Seq(classOf[Mute], classOf[UnMute]) foreach (bus.subscribe(context.self, _))
|
case InitializeLogger(bus) ⇒ Seq(classOf[Mute], classOf[UnMute], classOf[DeadLetter]) foreach (bus.subscribe(context.self, _))
|
||||||
case Mute(filters) ⇒ filters foreach addFilter
|
case Mute(filters) ⇒ filters foreach addFilter
|
||||||
case UnMute(filters) ⇒ filters foreach removeFilter
|
case UnMute(filters) ⇒ filters foreach removeFilter
|
||||||
case event: LogEvent ⇒ if (!filter(event)) print(event)
|
case event: LogEvent ⇒ if (!filter(event)) print(event)
|
||||||
|
case DeadLetter(msg: SystemMessage, null, rcp) ⇒
|
||||||
|
val event = Warning(rcp, "received dead system message: " + msg)
|
||||||
|
if (!filter(event)) print(event)
|
||||||
|
case DeadLetter(msg, snd, rcp) ⇒
|
||||||
|
val event = Warning(rcp, "received dead letter from " + snd + ": " + msg)
|
||||||
|
if (!filter(event)) print(event)
|
||||||
}
|
}
|
||||||
|
|
||||||
def filter(event: LogEvent): Boolean = filters exists (f ⇒ try { f(event) } catch { case e: Exception ⇒ false })
|
def filter(event: LogEvent): Boolean = filters exists (f ⇒ try { f(event) } catch { case e: Exception ⇒ false })
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue