2018-10-29 17:19:37 +08:00
|
|
|
|
/*
|
2019-01-02 18:55:26 +08:00
|
|
|
|
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
|
2011-05-19 21:34:21 +02:00
|
|
|
|
*/
|
2018-03-13 23:45:55 +09:00
|
|
|
|
|
2011-03-06 22:45:44 +01:00
|
|
|
|
package akka.testkit
|
|
|
|
|
|
|
2012-06-21 16:09:14 +02:00
|
|
|
|
import language.postfixOps
|
|
|
|
|
|
|
2012-01-24 11:33:40 +01:00
|
|
|
|
import java.lang.ref.WeakReference
|
2011-03-06 22:45:44 +01:00
|
|
|
|
import java.util.concurrent.locks.ReentrantLock
|
|
|
|
|
|
import scala.annotation.tailrec
|
2011-12-20 21:08:27 +01:00
|
|
|
|
import com.typesafe.config.Config
|
2019-03-11 10:38:24 +01:00
|
|
|
|
import akka.actor.{
|
|
|
|
|
|
ActorCell,
|
|
|
|
|
|
ActorInitializationException,
|
|
|
|
|
|
ActorRef,
|
|
|
|
|
|
ExtendedActorSystem,
|
|
|
|
|
|
Extension,
|
|
|
|
|
|
ExtensionId,
|
|
|
|
|
|
ExtensionIdProvider
|
|
|
|
|
|
}
|
|
|
|
|
|
import akka.dispatch.{
|
|
|
|
|
|
DefaultSystemMessageQueue,
|
|
|
|
|
|
DispatcherPrerequisites,
|
|
|
|
|
|
Envelope,
|
|
|
|
|
|
Mailbox,
|
|
|
|
|
|
MailboxType,
|
|
|
|
|
|
MessageDispatcher,
|
|
|
|
|
|
MessageDispatcherConfigurator,
|
|
|
|
|
|
MessageQueue,
|
|
|
|
|
|
TaskInvocation
|
|
|
|
|
|
}
|
|
|
|
|
|
import akka.dispatch.sysmsg.{ Resume, Suspend, SystemMessage }
|
2012-09-21 14:50:06 +02:00
|
|
|
|
import scala.concurrent.duration._
|
2012-07-22 15:33:18 +02:00
|
|
|
|
import akka.util.Switch
|
2012-09-21 14:50:06 +02:00
|
|
|
|
import scala.concurrent.duration.Duration
|
2012-07-22 15:33:18 +02:00
|
|
|
|
import scala.util.control.NonFatal
|
2013-02-27 15:45:40 +01:00
|
|
|
|
import java.util.concurrent.TimeUnit
|
2012-01-24 11:33:40 +01:00
|
|
|
|
|
2011-03-06 22:45:44 +01:00
|
|
|
|
/*
|
|
|
|
|
|
* Locking rules:
|
|
|
|
|
|
*
|
2013-02-27 15:45:40 +01:00
|
|
|
|
* Normal messages are always queued thread locally.
|
|
|
|
|
|
* Processing a queue checks suspendSwitch before each invocation, not processing
|
|
|
|
|
|
* if suspendSwitch.
|
|
|
|
|
|
* When resuming an actor, all messages are atomically scooped from all threads and
|
|
|
|
|
|
* queued on the resuming thread's queue, to be processed immediately.
|
|
|
|
|
|
* Scooping up messages means replacing the ThreadLocal contents with an empty
|
|
|
|
|
|
* new MessageQueue.
|
2011-03-06 22:45:44 +01:00
|
|
|
|
*
|
2011-09-21 15:01:47 +02:00
|
|
|
|
* All accesses to the queue must be done under the suspendSwitch-switch's lock, so
|
2011-03-06 22:45:44 +01:00
|
|
|
|
* within one of its methods taking a closure argument.
|
2013-02-27 15:45:40 +01:00
|
|
|
|
*
|
|
|
|
|
|
* System messages always go directly to the actors SystemMessageQueue which isn't thread local.
|
2011-03-06 22:45:44 +01:00
|
|
|
|
*/
|
|
|
|
|
|
|
2019-03-11 10:38:24 +01:00
|
|
|
|
private[testkit] object CallingThreadDispatcherQueues
|
|
|
|
|
|
extends ExtensionId[CallingThreadDispatcherQueues]
|
|
|
|
|
|
with ExtensionIdProvider {
|
2011-11-30 15:16:20 +01:00
|
|
|
|
override def lookup = CallingThreadDispatcherQueues
|
2019-03-11 10:38:24 +01:00
|
|
|
|
override def createExtension(system: ExtendedActorSystem): CallingThreadDispatcherQueues =
|
|
|
|
|
|
new CallingThreadDispatcherQueues
|
2011-11-30 15:16:20 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
private[testkit] class CallingThreadDispatcherQueues extends Extension {
|
2011-04-16 22:20:04 +02:00
|
|
|
|
|
|
|
|
|
|
// PRIVATE DATA
|
|
|
|
|
|
|
2013-02-27 15:45:40 +01:00
|
|
|
|
private var queues = Map[CallingThreadMailbox, Set[WeakReference[MessageQueue]]]()
|
2019-03-11 10:38:24 +01:00
|
|
|
|
private var lastGC = 0L
|
2011-03-06 22:45:44 +01:00
|
|
|
|
|
|
|
|
|
|
// we have to forget about long-gone threads sometime
|
2013-03-28 23:45:48 +01:00
|
|
|
|
private def gc(): Unit = {
|
2019-03-11 10:38:24 +01:00
|
|
|
|
queues = queues
|
|
|
|
|
|
.foldLeft(Map.newBuilder[CallingThreadMailbox, Set[WeakReference[MessageQueue]]]) {
|
|
|
|
|
|
case (m, (k, v)) =>
|
|
|
|
|
|
val nv = v.filter(_.get ne null)
|
|
|
|
|
|
if (nv.isEmpty) m else m += (k -> nv)
|
|
|
|
|
|
}
|
|
|
|
|
|
.result
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2013-02-27 15:45:40 +01:00
|
|
|
|
protected[akka] def registerQueue(mbox: CallingThreadMailbox, q: MessageQueue): Unit = synchronized {
|
2011-03-06 22:45:44 +01:00
|
|
|
|
if (queues contains mbox) {
|
|
|
|
|
|
val newSet = queues(mbox) + new WeakReference(q)
|
2019-02-09 15:25:39 +01:00
|
|
|
|
queues += mbox -> newSet
|
2011-03-06 22:45:44 +01:00
|
|
|
|
} else {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
queues += mbox -> Set(new WeakReference(q))
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
2011-10-21 18:47:44 +02:00
|
|
|
|
val now = System.nanoTime
|
2019-03-11 10:38:24 +01:00
|
|
|
|
if (now - lastGC > 1000000000L) {
|
2011-10-21 18:47:44 +02:00
|
|
|
|
lastGC = now
|
2013-03-28 23:45:48 +01:00
|
|
|
|
gc()
|
2011-10-21 18:47:44 +02:00
|
|
|
|
}
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2013-01-24 16:36:26 +01:00
|
|
|
|
protected[akka] def unregisterQueues(mbox: CallingThreadMailbox): Unit = synchronized {
|
|
|
|
|
|
queues -= mbox
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2011-03-06 22:45:44 +01:00
|
|
|
|
/*
|
|
|
|
|
|
* This method must be called with "own" being this thread's queue for the
|
|
|
|
|
|
* given mailbox. When this method returns, the queue will be entered
|
|
|
|
|
|
* (active).
|
|
|
|
|
|
*/
|
2013-02-27 15:45:40 +01:00
|
|
|
|
protected[akka] def gatherFromAllOtherQueues(mbox: CallingThreadMailbox, own: MessageQueue): Unit = synchronized {
|
2011-03-06 22:45:44 +01:00
|
|
|
|
if (queues contains mbox) {
|
|
|
|
|
|
for {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
ref <- queues(mbox)
|
2012-06-15 13:04:10 +02:00
|
|
|
|
q = ref.get
|
2011-10-21 18:47:44 +02:00
|
|
|
|
if (q ne null) && (q ne own)
|
2011-03-06 22:45:44 +01:00
|
|
|
|
} {
|
2012-02-19 10:28:56 +01:00
|
|
|
|
val owner = mbox.actor.self
|
2013-02-27 15:45:40 +01:00
|
|
|
|
var msg = q.dequeue()
|
2012-02-19 10:28:56 +01:00
|
|
|
|
while (msg ne null) {
|
2011-10-21 18:47:44 +02:00
|
|
|
|
// this is safe because this method is only ever called while holding the suspendSwitch monitor
|
2013-02-27 15:45:40 +01:00
|
|
|
|
own.enqueue(owner, msg)
|
|
|
|
|
|
msg = q.dequeue()
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2011-12-20 21:08:27 +01:00
|
|
|
|
object CallingThreadDispatcher {
|
2011-12-21 19:02:06 +01:00
|
|
|
|
val Id = "akka.test.calling-thread-dispatcher"
|
2011-12-20 21:08:27 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2011-03-06 22:45:44 +01:00
|
|
|
|
/**
|
|
|
|
|
|
* Dispatcher which runs invocations on the current thread only. This
|
|
|
|
|
|
* dispatcher does not create any new threads, but it can be used from
|
|
|
|
|
|
* different threads concurrently for the same actor. The dispatch strategy is
|
2011-09-21 15:01:47 +02:00
|
|
|
|
* to run on the current thread unless the target actor is either suspendSwitch or
|
2011-03-06 22:45:44 +01:00
|
|
|
|
* already running on the current thread (if it is running on a different
|
|
|
|
|
|
* thread, then this thread will block until that other invocation is
|
|
|
|
|
|
* finished); if the invocation is not run, it is queued in a thread-local
|
|
|
|
|
|
* queue to be executed once the active invocation further up the call stack
|
|
|
|
|
|
* finishes. This leads to completely deterministic execution order if only one
|
|
|
|
|
|
* thread is used.
|
|
|
|
|
|
*
|
|
|
|
|
|
* Suspending and resuming are global actions for one actor, meaning they can
|
|
|
|
|
|
* affect different threads, which leads to complications. If messages are
|
2011-09-21 15:01:47 +02:00
|
|
|
|
* queued (thread-locally) during the suspendSwitch period, the only thread to run
|
2011-03-06 22:45:44 +01:00
|
|
|
|
* them upon resume is the thread actually calling the resume method. Hence,
|
|
|
|
|
|
* all thread-local queues which are not currently being drained (possible,
|
|
|
|
|
|
* since suspend-queue-resume might happen entirely during an invocation on a
|
|
|
|
|
|
* different thread) are scooped up into the current thread-local queue which
|
|
|
|
|
|
* is then executed. It is possible to suspend an actor from within its call
|
|
|
|
|
|
* stack.
|
|
|
|
|
|
*
|
|
|
|
|
|
* @since 1.1
|
|
|
|
|
|
*/
|
2013-06-01 21:58:34 +02:00
|
|
|
|
class CallingThreadDispatcher(_configurator: MessageDispatcherConfigurator) extends MessageDispatcher(_configurator) {
|
2011-03-06 22:45:44 +01:00
|
|
|
|
import CallingThreadDispatcher._
|
|
|
|
|
|
|
2019-10-14 17:55:12 +02:00
|
|
|
|
val log = akka.event.Logging(eventStream, getClass)
|
2011-11-18 11:59:43 +01:00
|
|
|
|
|
2011-12-21 19:02:06 +01:00
|
|
|
|
override def id: String = Id
|
2011-12-20 21:08:27 +01:00
|
|
|
|
|
2013-06-01 21:58:34 +02:00
|
|
|
|
protected[akka] override def createMailbox(actor: akka.actor.Cell, mailboxType: MailboxType) =
|
|
|
|
|
|
new CallingThreadMailbox(actor, mailboxType)
|
2011-03-06 22:45:44 +01:00
|
|
|
|
|
2018-07-25 20:38:27 +09:00
|
|
|
|
protected[akka] override def shutdown(): Unit = {}
|
2011-03-06 22:45:44 +01:00
|
|
|
|
|
2011-09-21 15:01:47 +02:00
|
|
|
|
protected[akka] override def throughput = 0
|
2011-11-21 10:48:21 +01:00
|
|
|
|
protected[akka] override def throughputDeadlineTime = Duration.Zero
|
2019-03-13 10:56:20 +01:00
|
|
|
|
protected[akka] override def registerForExecution(
|
|
|
|
|
|
mbox: Mailbox,
|
|
|
|
|
|
hasMessageHint: Boolean,
|
|
|
|
|
|
hasSystemMessageHint: Boolean): Boolean = false
|
2011-09-21 15:01:47 +02:00
|
|
|
|
|
2011-12-02 10:32:17 +01:00
|
|
|
|
protected[akka] override def shutdownTimeout = 1 second
|
2011-03-06 22:45:44 +01:00
|
|
|
|
|
2012-02-13 15:33:31 +01:00
|
|
|
|
protected[akka] override def register(actor: ActorCell): Unit = {
|
|
|
|
|
|
super.register(actor)
|
|
|
|
|
|
actor.mailbox match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case mbox: CallingThreadMailbox =>
|
2012-02-13 15:33:31 +01:00
|
|
|
|
val queue = mbox.queue
|
|
|
|
|
|
runQueue(mbox, queue)
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case x => throw ActorInitializationException("expected CallingThreadMailbox, got " + x.getClass)
|
2012-02-13 15:33:31 +01:00
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2013-01-24 16:36:26 +01:00
|
|
|
|
protected[akka] override def unregister(actor: ActorCell): Unit = {
|
|
|
|
|
|
val mbox = actor.mailbox match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case m: CallingThreadMailbox => Some(m)
|
|
|
|
|
|
case _ => None
|
2013-01-24 16:36:26 +01:00
|
|
|
|
}
|
|
|
|
|
|
super.unregister(actor)
|
2019-03-11 10:38:24 +01:00
|
|
|
|
mbox.foreach(CallingThreadDispatcherQueues(actor.system).unregisterQueues)
|
2013-01-24 16:36:26 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2018-07-25 20:38:27 +09:00
|
|
|
|
protected[akka] override def suspend(actor: ActorCell): Unit = {
|
2012-01-10 13:33:57 +01:00
|
|
|
|
actor.mailbox match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case m: CallingThreadMailbox => { m.suspendSwitch.switchOn; m.suspend() }
|
|
|
|
|
|
case m => m.systemEnqueue(actor.self, Suspend())
|
2012-01-10 13:33:57 +01:00
|
|
|
|
}
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2018-07-25 20:38:27 +09:00
|
|
|
|
protected[akka] override def resume(actor: ActorCell): Unit = {
|
2011-11-12 10:57:28 +01:00
|
|
|
|
actor.mailbox match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case mbox: CallingThreadMailbox =>
|
2011-11-12 10:57:28 +01:00
|
|
|
|
val queue = mbox.queue
|
|
|
|
|
|
val switched = mbox.suspendSwitch.switchOff {
|
2011-11-30 15:16:20 +01:00
|
|
|
|
CallingThreadDispatcherQueues(actor.system).gatherFromAllOtherQueues(mbox, queue)
|
2012-07-13 12:25:26 +02:00
|
|
|
|
mbox.resume()
|
2011-11-12 10:57:28 +01:00
|
|
|
|
}
|
2013-02-27 15:45:40 +01:00
|
|
|
|
if (switched)
|
2011-11-12 10:57:28 +01:00
|
|
|
|
runQueue(mbox, queue)
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case m => m.systemEnqueue(actor.self, Resume(causedByFailure = null))
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
|
|
|
|
|
}
|
|
|
|
|
|
|
2018-07-25 20:38:27 +09:00
|
|
|
|
protected[akka] override def systemDispatch(receiver: ActorCell, message: SystemMessage): Unit = {
|
2011-11-12 10:57:28 +01:00
|
|
|
|
receiver.mailbox match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case mbox: CallingThreadMailbox =>
|
2011-11-12 10:57:28 +01:00
|
|
|
|
mbox.systemEnqueue(receiver.self, message)
|
2013-02-27 15:45:40 +01:00
|
|
|
|
runQueue(mbox, mbox.queue)
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case m => m.systemEnqueue(receiver.self, message)
|
2011-09-21 08:25:08 +02:00
|
|
|
|
}
|
2011-09-20 18:34:21 +02:00
|
|
|
|
}
|
|
|
|
|
|
|
2018-07-25 20:38:27 +09:00
|
|
|
|
protected[akka] override def dispatch(receiver: ActorCell, handle: Envelope): Unit = {
|
2011-11-12 10:57:28 +01:00
|
|
|
|
receiver.mailbox match {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case mbox: CallingThreadMailbox =>
|
2011-11-12 10:57:28 +01:00
|
|
|
|
val queue = mbox.queue
|
|
|
|
|
|
val execute = mbox.suspendSwitch.fold {
|
2013-02-27 15:45:40 +01:00
|
|
|
|
queue.enqueue(receiver.self, handle)
|
2011-11-12 10:57:28 +01:00
|
|
|
|
false
|
|
|
|
|
|
} {
|
2013-02-27 15:45:40 +01:00
|
|
|
|
queue.enqueue(receiver.self, handle)
|
|
|
|
|
|
true
|
2011-11-12 10:57:28 +01:00
|
|
|
|
}
|
|
|
|
|
|
if (execute) runQueue(mbox, queue)
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case m => m.enqueue(receiver.self, handle)
|
2011-05-18 17:25:30 +02:00
|
|
|
|
}
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2018-07-25 20:38:27 +09:00
|
|
|
|
protected[akka] override def executeTask(invocation: TaskInvocation): Unit = { invocation.run }
|
2011-03-17 22:18:39 +01:00
|
|
|
|
|
2011-03-06 22:45:44 +01:00
|
|
|
|
/*
|
2013-02-27 15:45:40 +01:00
|
|
|
|
* This method must be called with this thread's queue.
|
2011-03-06 22:45:44 +01:00
|
|
|
|
*
|
|
|
|
|
|
* If the catch block is executed, then a non-empty mailbox may be stalled as
|
|
|
|
|
|
* there is no-one who cares to execute it before the next message is sent or
|
2011-09-21 15:01:47 +02:00
|
|
|
|
* it is suspendSwitch and resumed.
|
2011-03-06 22:45:44 +01:00
|
|
|
|
*/
|
2011-05-18 17:25:30 +02:00
|
|
|
|
@tailrec
|
2019-03-13 10:56:20 +01:00
|
|
|
|
private def runQueue(
|
|
|
|
|
|
mbox: CallingThreadMailbox,
|
|
|
|
|
|
queue: MessageQueue,
|
|
|
|
|
|
interruptedEx: InterruptedException = null): Unit = {
|
2013-02-27 15:45:40 +01:00
|
|
|
|
def checkThreadInterruption(intEx: InterruptedException): InterruptedException = {
|
|
|
|
|
|
if (Thread.interrupted()) { // clear interrupted flag before we continue, exception will be thrown later
|
|
|
|
|
|
val ie = new InterruptedException("Interrupted during message processing")
|
|
|
|
|
|
log.error(ie, "Interrupted during message processing")
|
|
|
|
|
|
ie
|
|
|
|
|
|
} else intEx
|
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
def throwInterruptionIfExistsOrSet(intEx: InterruptedException): Unit = {
|
|
|
|
|
|
val ie = checkThreadInterruption(intEx)
|
|
|
|
|
|
if (ie ne null) {
|
|
|
|
|
|
Thread.interrupted() // clear interrupted flag before throwing according to java convention
|
|
|
|
|
|
throw ie
|
2011-05-18 17:25:30 +02:00
|
|
|
|
}
|
2013-02-27 15:45:40 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
@tailrec
|
|
|
|
|
|
def process(intEx: InterruptedException): InterruptedException = {
|
|
|
|
|
|
var intex = intEx
|
|
|
|
|
|
val recurse = {
|
|
|
|
|
|
mbox.processAllSystemMessages()
|
|
|
|
|
|
val handle = mbox.suspendSwitch.fold[Envelope](null) {
|
|
|
|
|
|
if (mbox.isClosed) null else queue.dequeue()
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
2013-02-27 15:45:40 +01:00
|
|
|
|
if (handle ne null) {
|
|
|
|
|
|
try {
|
|
|
|
|
|
if (Mailbox.debug) println(mbox.actor.self + " processing message " + handle)
|
|
|
|
|
|
mbox.actor.invoke(handle)
|
|
|
|
|
|
intex = checkThreadInterruption(intex)
|
|
|
|
|
|
true
|
|
|
|
|
|
} catch {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case ie: InterruptedException =>
|
2013-02-27 15:45:40 +01:00
|
|
|
|
log.error(ie, "Interrupted during message processing")
|
|
|
|
|
|
Thread.interrupted() // clear interrupted flag before we continue, exception will be thrown later
|
|
|
|
|
|
intex = ie
|
|
|
|
|
|
true
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case NonFatal(e) =>
|
2013-02-27 15:45:40 +01:00
|
|
|
|
log.error(e, "Error during message processing")
|
|
|
|
|
|
false
|
|
|
|
|
|
}
|
|
|
|
|
|
} else false
|
|
|
|
|
|
}
|
|
|
|
|
|
if (recurse) process(intex)
|
|
|
|
|
|
else intex
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
2013-02-27 15:45:40 +01:00
|
|
|
|
|
|
|
|
|
|
// if we own the lock then we shouldn't do anything since we are processing
|
|
|
|
|
|
// this actors mailbox at some other level on our call stack
|
|
|
|
|
|
if (!mbox.ctdLock.isHeldByCurrentThread) {
|
|
|
|
|
|
var intex = interruptedEx
|
|
|
|
|
|
val gotLock = try {
|
|
|
|
|
|
mbox.ctdLock.tryLock(50, TimeUnit.MILLISECONDS)
|
|
|
|
|
|
} catch {
|
2019-02-09 15:25:39 +01:00
|
|
|
|
case ie: InterruptedException =>
|
2013-02-27 15:45:40 +01:00
|
|
|
|
Thread.interrupted() // clear interrupted flag before we continue, exception will be thrown later
|
|
|
|
|
|
intex = ie
|
|
|
|
|
|
false
|
|
|
|
|
|
}
|
|
|
|
|
|
if (gotLock) {
|
|
|
|
|
|
val ie = try {
|
|
|
|
|
|
process(intex)
|
|
|
|
|
|
} finally {
|
|
|
|
|
|
mbox.ctdLock.unlock
|
|
|
|
|
|
}
|
|
|
|
|
|
throwInterruptionIfExistsOrSet(ie)
|
|
|
|
|
|
} else {
|
|
|
|
|
|
// if we didn't get the lock and our mailbox still has messages, then we need to try again
|
|
|
|
|
|
if (mbox.hasSystemMessages || mbox.hasMessages) {
|
|
|
|
|
|
runQueue(mbox, queue, intex)
|
|
|
|
|
|
} else {
|
|
|
|
|
|
throwInterruptionIfExistsOrSet(intex)
|
|
|
|
|
|
}
|
2011-10-25 15:07:20 +02:00
|
|
|
|
}
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
|
|
|
|
|
}
|
2011-12-20 21:08:27 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
|
|
class CallingThreadDispatcherConfigurator(config: Config, prerequisites: DispatcherPrerequisites)
|
2019-03-11 10:38:24 +01:00
|
|
|
|
extends MessageDispatcherConfigurator(config, prerequisites) {
|
2012-02-19 10:28:56 +01:00
|
|
|
|
|
2013-06-01 21:58:34 +02:00
|
|
|
|
private val instance = new CallingThreadDispatcher(this)
|
2011-12-20 21:08:27 +01:00
|
|
|
|
|
|
|
|
|
|
override def dispatcher(): MessageDispatcher = instance
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2012-06-13 17:57:56 +02:00
|
|
|
|
class CallingThreadMailbox(_receiver: akka.actor.Cell, val mailboxType: MailboxType)
|
2019-03-11 10:38:24 +01:00
|
|
|
|
extends Mailbox(null)
|
|
|
|
|
|
with DefaultSystemMessageQueue {
|
2012-06-13 17:57:56 +02:00
|
|
|
|
|
|
|
|
|
|
val system = _receiver.system
|
|
|
|
|
|
val self = _receiver.self
|
2011-03-06 22:45:44 +01:00
|
|
|
|
|
2013-02-27 15:45:40 +01:00
|
|
|
|
private val q = new ThreadLocal[MessageQueue]() {
|
2011-10-21 18:47:44 +02:00
|
|
|
|
override def initialValue = {
|
2013-02-27 15:45:40 +01:00
|
|
|
|
val queue = mailboxType.create(Some(self), Some(system))
|
2012-06-13 17:57:56 +02:00
|
|
|
|
CallingThreadDispatcherQueues(system).registerQueue(CallingThreadMailbox.this, queue)
|
2011-10-21 18:47:44 +02:00
|
|
|
|
queue
|
|
|
|
|
|
}
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|
|
|
|
|
|
|
2014-01-17 09:12:44 +01:00
|
|
|
|
/**
|
|
|
|
|
|
* This is only a marker to be put in the messageQueue’s stead to make error
|
|
|
|
|
|
* messages pertaining to violated mailbox type requirements less cryptic.
|
|
|
|
|
|
*/
|
|
|
|
|
|
override val messageQueue: MessageQueue = q.get
|
|
|
|
|
|
|
2013-02-27 15:45:40 +01:00
|
|
|
|
override def enqueue(receiver: ActorRef, msg: Envelope): Unit = q.get.enqueue(receiver, msg)
|
2019-03-11 10:38:24 +01:00
|
|
|
|
override def dequeue(): Envelope =
|
|
|
|
|
|
throw new UnsupportedOperationException("CallingThreadMailbox cannot dequeue normally")
|
2013-02-27 15:45:40 +01:00
|
|
|
|
override def hasMessages: Boolean = q.get.hasMessages
|
2012-04-03 16:24:50 +02:00
|
|
|
|
override def numberOfMessages: Int = 0
|
2012-04-03 15:38:54 +02:00
|
|
|
|
|
2011-03-06 22:45:44 +01:00
|
|
|
|
def queue = q.get
|
|
|
|
|
|
|
2012-01-10 13:33:57 +01:00
|
|
|
|
val ctdLock = new ReentrantLock
|
2011-09-21 15:01:47 +02:00
|
|
|
|
val suspendSwitch = new Switch
|
2011-03-06 22:45:44 +01:00
|
|
|
|
|
2012-01-10 13:33:57 +01:00
|
|
|
|
override def cleanUp(): Unit = {
|
|
|
|
|
|
/*
|
2012-02-01 14:40:12 +01:00
|
|
|
|
* This is called from dispatcher.unregister, i.e. under this.lock. If
|
|
|
|
|
|
* another thread obtained a reference to this mailbox and enqueues after
|
2012-01-10 13:33:57 +01:00
|
|
|
|
* the gather operation, tough luck: no guaranteed delivery to deadLetters.
|
|
|
|
|
|
*/
|
|
|
|
|
|
suspendSwitch.locked {
|
2013-01-24 16:36:26 +01:00
|
|
|
|
val qq = queue
|
|
|
|
|
|
CallingThreadDispatcherQueues(actor.system).gatherFromAllOtherQueues(this, qq)
|
2012-01-10 13:33:57 +01:00
|
|
|
|
super.cleanUp()
|
2013-06-03 11:41:11 +02:00
|
|
|
|
qq.cleanUp(actor.self, actor.dispatcher.mailboxes.deadLetterMailbox.messageQueue)
|
2013-01-24 16:36:26 +01:00
|
|
|
|
q.remove()
|
2012-01-10 13:33:57 +01:00
|
|
|
|
}
|
|
|
|
|
|
}
|
2011-03-06 22:45:44 +01:00
|
|
|
|
}
|