make system.actorOf() non-blocking (and working), see #2031
- introducing RepointableActorRef, which starts out with an UnstartedActorCell which can cheaply be created; the Supervise() message will trigger child.activate() in the supervisor, which means that the actual creation (now with normal ActorCell) happens exactly in the right place and with the right semantics. Messages which were enqueued to the dummy cell are transferred atomically into the ActorCell (using normal .tell()), so message sends keep working exactly as they used to - this enables getting rid of the brittle synchronization around RoutedActorRef by replacing that one with a RepointableActorRef subclass which creates RoutedActorCells upon activate(), with the nice benefit that there is no hurry then to get it right because the new cell is constructed “on the side” misc fixes: - InvalidMessageException is now actually enforced when trying to send “null” - Mailboxes may be created without having an ActorCell, which can come in handy later, because the cell is only needed when this mailbox is going to be scheduled on some executor - remove occurrences of Props(), which is equivalent to Props[Nothing], which is equivalent to «bug» - add test case which verifies that context.actorOf is still synchronous - plus all the stuff I have forgotten.
This commit is contained in:
parent
8cd11550fa
commit
b60210362e
37 changed files with 772 additions and 310 deletions
|
|
@ -227,7 +227,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
||||||
contextStackMustBeEmpty
|
contextStackMustBeEmpty
|
||||||
}
|
}
|
||||||
|
|
||||||
filterException[java.lang.IllegalStateException] {
|
EventFilter[ActorInitializationException](occurrences = 1) intercept {
|
||||||
(intercept[java.lang.IllegalStateException] {
|
(intercept[java.lang.IllegalStateException] {
|
||||||
wrap(result ⇒
|
wrap(result ⇒
|
||||||
actorOf(Props(new OuterActor(actorOf(Props(promiseIntercept({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor })(result)))))))
|
actorOf(Props(new OuterActor(actorOf(Props(promiseIntercept({ throw new IllegalStateException("Ur state be b0rked"); new InnerActor })(result)))))))
|
||||||
|
|
@ -257,14 +257,14 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
||||||
val in = new ObjectInputStream(new ByteArrayInputStream(bytes))
|
val in = new ObjectInputStream(new ByteArrayInputStream(bytes))
|
||||||
val readA = in.readObject
|
val readA = in.readObject
|
||||||
|
|
||||||
a.isInstanceOf[LocalActorRef] must be === true
|
a.isInstanceOf[ActorRefWithCell] must be === true
|
||||||
readA.isInstanceOf[LocalActorRef] must be === true
|
readA.isInstanceOf[ActorRefWithCell] must be === true
|
||||||
(readA eq a) must be === true
|
(readA eq a) must be === true
|
||||||
}
|
}
|
||||||
|
|
||||||
val ser = new JavaSerializer(esys)
|
val ser = new JavaSerializer(esys)
|
||||||
val readA = ser.fromBinary(bytes, None)
|
val readA = ser.fromBinary(bytes, None)
|
||||||
readA.isInstanceOf[LocalActorRef] must be === true
|
readA.isInstanceOf[ActorRefWithCell] must be === true
|
||||||
(readA eq a) must be === true
|
(readA eq a) must be === true
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -362,13 +362,13 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
|
||||||
val timeout = Timeout(20000)
|
val timeout = Timeout(20000)
|
||||||
val ref = system.actorOf(Props(new Actor {
|
val ref = system.actorOf(Props(new Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
case 5 ⇒ sender.tell("five")
|
case 5 ⇒ sender.tell("five")
|
||||||
case null ⇒ sender.tell("null")
|
case 0 ⇒ sender.tell("null")
|
||||||
}
|
}
|
||||||
}))
|
}))
|
||||||
|
|
||||||
val ffive = (ref.ask(5)(timeout)).mapTo[String]
|
val ffive = (ref.ask(5)(timeout)).mapTo[String]
|
||||||
val fnull = (ref.ask(null)(timeout)).mapTo[String]
|
val fnull = (ref.ask(0)(timeout)).mapTo[String]
|
||||||
ref ! PoisonPill
|
ref ! PoisonPill
|
||||||
|
|
||||||
Await.result(ffive, timeout.duration) must be("five")
|
Await.result(ffive, timeout.duration) must be("five")
|
||||||
|
|
|
||||||
|
|
@ -165,17 +165,21 @@ class ActorSystemSpec extends AkkaSpec("""akka.extensions = ["akka.actor.TestExt
|
||||||
system.scheduler.scheduleOnce(200 millis) { system.shutdown() }
|
system.scheduler.scheduleOnce(200 millis) { system.shutdown() }
|
||||||
var failing = false
|
var failing = false
|
||||||
var created = Vector.empty[ActorRef]
|
var created = Vector.empty[ActorRef]
|
||||||
while (!system.isTerminated) {
|
while (!system.isTerminated && system.uptime < 5) {
|
||||||
try {
|
try {
|
||||||
val t = system.actorOf(Props[ActorSystemSpec.Terminater])
|
val t = system.actorOf(Props[ActorSystemSpec.Terminater])
|
||||||
failing must not be true // because once failing => always failing (it’s due to shutdown)
|
failing must not be true // because once failing => always failing (it’s due to shutdown)
|
||||||
created :+= t
|
created :+= t
|
||||||
} catch {
|
} catch {
|
||||||
case e: Exception ⇒ failing = true
|
case _: IllegalStateException ⇒ failing = true
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
println(created.last)
|
if (system.uptime >= 5) {
|
||||||
created filter (!_.isTerminated) must be(Seq())
|
println(created.last)
|
||||||
|
println(system.asInstanceOf[ExtendedActorSystem].printTree)
|
||||||
|
system.uptime must be < 5L
|
||||||
|
}
|
||||||
|
created filter (ref ⇒ !ref.isTerminated && !ref.asInstanceOf[ActorRefWithCell].underlying.isInstanceOf[UnstartedCell]) must be(Seq())
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -140,13 +140,13 @@ class FSMTimingSpec extends AkkaSpec with ImplicitSender {
|
||||||
object FSMTimingSpec {
|
object FSMTimingSpec {
|
||||||
|
|
||||||
def suspend(actorRef: ActorRef): Unit = actorRef match {
|
def suspend(actorRef: ActorRef): Unit = actorRef match {
|
||||||
case l: LocalActorRef ⇒ l.suspend()
|
case l: ActorRefWithCell ⇒ l.suspend()
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
}
|
}
|
||||||
|
|
||||||
def resume(actorRef: ActorRef): Unit = actorRef match {
|
def resume(actorRef: ActorRef): Unit = actorRef match {
|
||||||
case l: LocalActorRef ⇒ l.resume()
|
case l: ActorRefWithCell ⇒ l.resume()
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
}
|
}
|
||||||
|
|
||||||
trait State
|
trait State
|
||||||
|
|
|
||||||
|
|
@ -3,24 +3,23 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor.dispatch
|
package akka.actor.dispatch
|
||||||
|
|
||||||
import org.scalatest.Assertions._
|
|
||||||
import akka.testkit._
|
|
||||||
import akka.dispatch._
|
|
||||||
import akka.util.Timeout
|
|
||||||
import java.util.concurrent.atomic.AtomicLong
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
|
||||||
import java.util.concurrent.{ ConcurrentHashMap, CountDownLatch, TimeUnit }
|
|
||||||
import akka.util.Switch
|
|
||||||
import java.rmi.RemoteException
|
import java.rmi.RemoteException
|
||||||
import org.junit.{ After, Test }
|
import java.util.concurrent.{ TimeUnit, CountDownLatch, ConcurrentHashMap }
|
||||||
import akka.actor._
|
import java.util.concurrent.atomic.{ AtomicLong, AtomicInteger }
|
||||||
import util.control.NoStackTrace
|
|
||||||
import akka.actor.ActorSystem
|
import org.junit.runner.RunWith
|
||||||
import akka.util.duration._
|
import org.scalatest.Assertions.{ fail, assert }
|
||||||
import akka.event.Logging.Error
|
import org.scalatest.junit.JUnitRunner
|
||||||
|
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.util.Duration
|
|
||||||
|
import akka.actor._
|
||||||
|
import akka.dispatch._
|
||||||
|
import akka.event.Logging.Error
|
||||||
import akka.pattern.ask
|
import akka.pattern.ask
|
||||||
|
import akka.testkit._
|
||||||
|
import akka.util.{ Timeout, Switch, Duration }
|
||||||
|
import akka.util.duration._
|
||||||
|
|
||||||
object ActorModelSpec {
|
object ActorModelSpec {
|
||||||
|
|
||||||
|
|
@ -201,7 +200,7 @@ object ActorModelSpec {
|
||||||
msgsReceived: Long = statsFor(actorRef, dispatcher).msgsReceived.get(),
|
msgsReceived: Long = statsFor(actorRef, dispatcher).msgsReceived.get(),
|
||||||
msgsProcessed: Long = statsFor(actorRef, dispatcher).msgsProcessed.get(),
|
msgsProcessed: Long = statsFor(actorRef, dispatcher).msgsProcessed.get(),
|
||||||
restarts: Long = statsFor(actorRef, dispatcher).restarts.get())(implicit system: ActorSystem) {
|
restarts: Long = statsFor(actorRef, dispatcher).restarts.get())(implicit system: ActorSystem) {
|
||||||
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[LocalActorRef].underlying.dispatcher))
|
val stats = statsFor(actorRef, Option(dispatcher).getOrElse(actorRef.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].dispatcher))
|
||||||
val deadline = System.currentTimeMillis + 1000
|
val deadline = System.currentTimeMillis + 1000
|
||||||
try {
|
try {
|
||||||
await(deadline)(stats.suspensions.get() == suspensions)
|
await(deadline)(stats.suspensions.get() == suspensions)
|
||||||
|
|
@ -241,6 +240,13 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
||||||
|
|
||||||
def newTestActor(dispatcher: String) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
|
def newTestActor(dispatcher: String) = system.actorOf(Props[DispatcherActor].withDispatcher(dispatcher))
|
||||||
|
|
||||||
|
def awaitStarted(ref: ActorRef): Unit = {
|
||||||
|
awaitCond(ref match {
|
||||||
|
case r: RepointableRef ⇒ r.isStarted
|
||||||
|
case _ ⇒ true
|
||||||
|
}, 1 second, 10 millis)
|
||||||
|
}
|
||||||
|
|
||||||
protected def interceptedDispatcher(): MessageDispatcherInterceptor
|
protected def interceptedDispatcher(): MessageDispatcherInterceptor
|
||||||
protected def dispatcherType: String
|
protected def dispatcherType: String
|
||||||
|
|
||||||
|
|
@ -328,7 +334,8 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
||||||
|
|
||||||
"not process messages for a suspended actor" in {
|
"not process messages for a suspended actor" in {
|
||||||
implicit val dispatcher = interceptedDispatcher()
|
implicit val dispatcher = interceptedDispatcher()
|
||||||
val a = newTestActor(dispatcher.id).asInstanceOf[LocalActorRef]
|
val a = newTestActor(dispatcher.id).asInstanceOf[InternalActorRef]
|
||||||
|
awaitStarted(a)
|
||||||
val done = new CountDownLatch(1)
|
val done = new CountDownLatch(1)
|
||||||
a.suspend
|
a.suspend
|
||||||
a ! CountDown(done)
|
a ! CountDown(done)
|
||||||
|
|
@ -436,6 +443,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
||||||
|
|
||||||
"not double-deregister" in {
|
"not double-deregister" in {
|
||||||
implicit val dispatcher = interceptedDispatcher()
|
implicit val dispatcher = interceptedDispatcher()
|
||||||
|
for (i ← 1 to 1000) system.actorOf(Props.empty)
|
||||||
val a = newTestActor(dispatcher.id)
|
val a = newTestActor(dispatcher.id)
|
||||||
a ! DoubleStop
|
a ! DoubleStop
|
||||||
awaitCond(statsFor(a, dispatcher).registers.get == 1)
|
awaitCond(statsFor(a, dispatcher).registers.get == 1)
|
||||||
|
|
|
||||||
|
|
@ -1,8 +1,12 @@
|
||||||
package akka.actor.dispatch
|
package akka.actor.dispatch
|
||||||
|
|
||||||
import java.util.concurrent.{ TimeUnit, CountDownLatch }
|
import java.util.concurrent.{ TimeUnit, CountDownLatch }
|
||||||
import akka.dispatch.{ Mailbox, Dispatchers }
|
|
||||||
import akka.actor.{ LocalActorRef, IllegalActorStateException, Actor, Props }
|
import org.junit.runner.RunWith
|
||||||
|
import org.scalatest.junit.JUnitRunner
|
||||||
|
|
||||||
|
import akka.actor.{ Props, ActorRefWithCell, ActorCell, Actor }
|
||||||
|
import akka.dispatch.Mailbox
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
|
|
||||||
object BalancingDispatcherSpec {
|
object BalancingDispatcherSpec {
|
||||||
|
|
@ -51,8 +55,8 @@ class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) {
|
||||||
"have fast actor stealing work from slow actor" in {
|
"have fast actor stealing work from slow actor" in {
|
||||||
val finishedCounter = new CountDownLatch(110)
|
val finishedCounter = new CountDownLatch(110)
|
||||||
|
|
||||||
val slow = system.actorOf(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef]
|
val slow = system.actorOf(Props(new DelayableActor(50, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[ActorRefWithCell]
|
||||||
val fast = system.actorOf(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[LocalActorRef]
|
val fast = system.actorOf(Props(new DelayableActor(10, finishedCounter)).withDispatcher(delayableActorDispatcher)).asInstanceOf[ActorRefWithCell]
|
||||||
|
|
||||||
var sentToFast = 0
|
var sentToFast = 0
|
||||||
|
|
||||||
|
|
@ -76,11 +80,11 @@ class BalancingDispatcherSpec extends AkkaSpec(BalancingDispatcherSpec.config) {
|
||||||
}
|
}
|
||||||
|
|
||||||
finishedCounter.await(5, TimeUnit.SECONDS)
|
finishedCounter.await(5, TimeUnit.SECONDS)
|
||||||
fast.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false)
|
fast.underlying.asInstanceOf[ActorCell].mailbox.asInstanceOf[Mailbox].hasMessages must be(false)
|
||||||
slow.underlying.mailbox.asInstanceOf[Mailbox].hasMessages must be(false)
|
slow.underlying.asInstanceOf[ActorCell].mailbox.asInstanceOf[Mailbox].hasMessages must be(false)
|
||||||
fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast
|
fast.underlying.asInstanceOf[ActorCell].actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast
|
||||||
fast.underlying.actor.asInstanceOf[DelayableActor].invocationCount must be >
|
fast.underlying.asInstanceOf[ActorCell].actor.asInstanceOf[DelayableActor].invocationCount must be >
|
||||||
(slow.underlying.actor.asInstanceOf[DelayableActor].invocationCount)
|
(slow.underlying.asInstanceOf[ActorCell].actor.asInstanceOf[DelayableActor].invocationCount)
|
||||||
system.stop(slow)
|
system.stop(slow)
|
||||||
system.stop(fast)
|
system.stop(fast)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,14 +1,17 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
package akka.dispatch
|
package akka.dispatch
|
||||||
|
|
||||||
import org.scalatest.{ BeforeAndAfterAll, BeforeAndAfterEach }
|
import java.util.concurrent.{ ConcurrentLinkedQueue, BlockingQueue }
|
||||||
import java.util.concurrent.{ TimeUnit, BlockingQueue }
|
|
||||||
import java.util.concurrent.ConcurrentLinkedQueue
|
import org.scalatest.{ BeforeAndAfterEach, BeforeAndAfterAll }
|
||||||
import akka.util._
|
|
||||||
import akka.util.duration._
|
|
||||||
import akka.testkit.AkkaSpec
|
|
||||||
import akka.actor.{ ActorRef, ActorContext, Props, LocalActorRef }
|
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.actor.ActorSystem
|
|
||||||
|
import akka.actor.{ RepointableRef, Props, ActorSystem, ActorRefWithCell, ActorRef, ActorCell }
|
||||||
|
import akka.testkit.AkkaSpec
|
||||||
|
import akka.util.duration._
|
||||||
|
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach {
|
abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAndAfterEach {
|
||||||
|
|
@ -75,7 +78,7 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
|
||||||
result
|
result
|
||||||
}
|
}
|
||||||
|
|
||||||
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters)(system)
|
def createMessageInvocation(msg: Any): Envelope = Envelope(msg, system.deadLetters, system)
|
||||||
|
|
||||||
def ensureInitialMailboxState(config: MailboxType, q: MessageQueue) {
|
def ensureInitialMailboxState(config: MailboxType, q: MessageQueue) {
|
||||||
q must not be null
|
q must not be null
|
||||||
|
|
@ -136,8 +139,8 @@ abstract class MailboxSpec extends AkkaSpec with BeforeAndAfterAll with BeforeAn
|
||||||
class DefaultMailboxSpec extends MailboxSpec {
|
class DefaultMailboxSpec extends MailboxSpec {
|
||||||
lazy val name = "The default mailbox implementation"
|
lazy val name = "The default mailbox implementation"
|
||||||
def factory = {
|
def factory = {
|
||||||
case u: UnboundedMailbox ⇒ u.create(None)
|
case u: UnboundedMailbox ⇒ u.create(None, None)
|
||||||
case b: BoundedMailbox ⇒ b.create(None)
|
case b: BoundedMailbox ⇒ b.create(None, None)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -145,8 +148,8 @@ class PriorityMailboxSpec extends MailboxSpec {
|
||||||
val comparator = PriorityGenerator(_.##)
|
val comparator = PriorityGenerator(_.##)
|
||||||
lazy val name = "The priority mailbox implementation"
|
lazy val name = "The priority mailbox implementation"
|
||||||
def factory = {
|
def factory = {
|
||||||
case UnboundedMailbox() ⇒ new UnboundedPriorityMailbox(comparator).create(None)
|
case UnboundedMailbox() ⇒ new UnboundedPriorityMailbox(comparator).create(None, None)
|
||||||
case BoundedMailbox(capacity, pushTimeOut) ⇒ new BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(None)
|
case BoundedMailbox(capacity, pushTimeOut) ⇒ new BoundedPriorityMailbox(comparator, capacity, pushTimeOut).create(None, None)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -158,13 +161,13 @@ object CustomMailboxSpec {
|
||||||
"""
|
"""
|
||||||
|
|
||||||
class MyMailboxType(settings: ActorSystem.Settings, config: Config) extends MailboxType {
|
class MyMailboxType(settings: ActorSystem.Settings, config: Config) extends MailboxType {
|
||||||
override def create(owner: Option[ActorContext]) = owner match {
|
override def create(owner: Option[ActorRef], system: Option[ActorSystem]) = owner match {
|
||||||
case Some(o) ⇒ new MyMailbox(o)
|
case Some(o) ⇒ new MyMailbox(o)
|
||||||
case None ⇒ throw new Exception("no mailbox owner given")
|
case None ⇒ throw new Exception("no mailbox owner given")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class MyMailbox(owner: ActorContext) extends QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
class MyMailbox(owner: ActorRef) extends QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
||||||
final val queue = new ConcurrentLinkedQueue[Envelope]()
|
final val queue = new ConcurrentLinkedQueue[Envelope]()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -174,7 +177,11 @@ class CustomMailboxSpec extends AkkaSpec(CustomMailboxSpec.config) {
|
||||||
"Dispatcher configuration" must {
|
"Dispatcher configuration" must {
|
||||||
"support custom mailboxType" in {
|
"support custom mailboxType" in {
|
||||||
val actor = system.actorOf(Props.empty.withDispatcher("my-dispatcher"))
|
val actor = system.actorOf(Props.empty.withDispatcher("my-dispatcher"))
|
||||||
val queue = actor.asInstanceOf[LocalActorRef].underlying.mailbox.messageQueue
|
awaitCond(actor match {
|
||||||
|
case r: RepointableRef ⇒ r.isStarted
|
||||||
|
case _ ⇒ true
|
||||||
|
}, 1 second, 10 millis)
|
||||||
|
val queue = actor.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].mailbox.messageQueue
|
||||||
queue.getClass must be(classOf[CustomMailboxSpec.MyMailbox])
|
queue.getClass must be(classOf[CustomMailboxSpec.MyMailbox])
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -1,12 +1,14 @@
|
||||||
package akka.dispatch
|
package akka.dispatch
|
||||||
|
|
||||||
import akka.actor.{ Props, LocalActorRef, Actor }
|
import org.junit.runner.RunWith
|
||||||
import akka.testkit.AkkaSpec
|
import org.scalatest.junit.JUnitRunner
|
||||||
import akka.pattern.ask
|
|
||||||
import akka.util.duration._
|
|
||||||
import akka.testkit.DefaultTimeout
|
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.actor.ActorSystem
|
|
||||||
|
import akka.actor.{ Props, InternalActorRef, ActorSystem, Actor }
|
||||||
|
import akka.pattern.ask
|
||||||
|
import akka.testkit.{ DefaultTimeout, AkkaSpec }
|
||||||
|
import akka.util.duration.intToDurationInt
|
||||||
|
|
||||||
object PriorityDispatcherSpec {
|
object PriorityDispatcherSpec {
|
||||||
val config = """
|
val config = """
|
||||||
|
|
@ -54,7 +56,7 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit
|
||||||
case i: Int ⇒ acc = i :: acc
|
case i: Int ⇒ acc = i :: acc
|
||||||
case 'Result ⇒ sender.tell(acc)
|
case 'Result ⇒ sender.tell(acc)
|
||||||
}
|
}
|
||||||
}).withDispatcher(dispatcherKey)).asInstanceOf[LocalActorRef]
|
}).withDispatcher(dispatcherKey)).asInstanceOf[InternalActorRef]
|
||||||
|
|
||||||
actor.suspend //Make sure the actor isn't treating any messages, let it buffer the incoming messages
|
actor.suspend //Make sure the actor isn't treating any messages, let it buffer the incoming messages
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,15 +4,14 @@
|
||||||
package akka.routing
|
package akka.routing
|
||||||
|
|
||||||
import java.util.concurrent.atomic.AtomicInteger
|
import java.util.concurrent.atomic.AtomicInteger
|
||||||
|
|
||||||
import org.junit.runner.RunWith
|
import org.junit.runner.RunWith
|
||||||
|
import akka.actor.{ Props, Deploy, Actor, ActorRef }
|
||||||
import akka.actor.{ Props, LocalActorRef, Deploy, Actor, ActorRef }
|
|
||||||
import akka.ConfigurationException
|
import akka.ConfigurationException
|
||||||
import akka.dispatch.Await
|
import akka.dispatch.Await
|
||||||
import akka.pattern.{ ask, gracefulStop }
|
import akka.pattern.{ ask, gracefulStop }
|
||||||
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
|
import akka.testkit.{ TestLatch, ImplicitSender, DefaultTimeout, AkkaSpec }
|
||||||
import akka.util.duration.intToDurationInt
|
import akka.util.duration.intToDurationInt
|
||||||
|
import akka.actor.UnstartedCell
|
||||||
|
|
||||||
object ConfiguredLocalRoutingSpec {
|
object ConfiguredLocalRoutingSpec {
|
||||||
val config = """
|
val config = """
|
||||||
|
|
@ -47,6 +46,14 @@ object ConfiguredLocalRoutingSpec {
|
||||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||||
class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.config) with DefaultTimeout with ImplicitSender {
|
class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.config) with DefaultTimeout with ImplicitSender {
|
||||||
|
|
||||||
|
def routerConfig(ref: ActorRef): RouterConfig = ref match {
|
||||||
|
case r: RoutedActorRef ⇒
|
||||||
|
r.underlying match {
|
||||||
|
case c: RoutedActorCell ⇒ c.routerConfig
|
||||||
|
case _: UnstartedCell ⇒ awaitCond(r.isStarted, 1 second, 10 millis); routerConfig(ref)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
"RouterConfig" must {
|
"RouterConfig" must {
|
||||||
|
|
||||||
"be picked up from Props" in {
|
"be picked up from Props" in {
|
||||||
|
|
@ -55,7 +62,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con
|
||||||
case "get" ⇒ sender ! context.props
|
case "get" ⇒ sender ! context.props
|
||||||
}
|
}
|
||||||
}).withRouter(RoundRobinRouter(12)), "someOther")
|
}).withRouter(RoundRobinRouter(12)), "someOther")
|
||||||
actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RoundRobinRouter(12)
|
routerConfig(actor) must be === RoundRobinRouter(12)
|
||||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -65,7 +72,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con
|
||||||
case "get" ⇒ sender ! context.props
|
case "get" ⇒ sender ! context.props
|
||||||
}
|
}
|
||||||
}).withRouter(RoundRobinRouter(12)), "config")
|
}).withRouter(RoundRobinRouter(12)), "config")
|
||||||
actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RandomRouter(4)
|
routerConfig(actor) must be === RandomRouter(4)
|
||||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -75,7 +82,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con
|
||||||
case "get" ⇒ sender ! context.props
|
case "get" ⇒ sender ! context.props
|
||||||
}
|
}
|
||||||
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "someOther")
|
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "someOther")
|
||||||
actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RoundRobinRouter(12)
|
routerConfig(actor) must be === RoundRobinRouter(12)
|
||||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -85,7 +92,7 @@ class ConfiguredLocalRoutingSpec extends AkkaSpec(ConfiguredLocalRoutingSpec.con
|
||||||
case "get" ⇒ sender ! context.props
|
case "get" ⇒ sender ! context.props
|
||||||
}
|
}
|
||||||
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "config")
|
}).withRouter(FromConfig).withDeploy(Deploy(routerConfig = RoundRobinRouter(12))), "config")
|
||||||
actor.asInstanceOf[LocalActorRef].underlying.props.routerConfig must be === RandomRouter(4)
|
routerConfig(actor) must be === RandomRouter(4)
|
||||||
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
Await.result(gracefulStop(actor, 3 seconds), 3 seconds)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,10 +12,11 @@ import akka.dispatch.Await
|
||||||
import akka.util.Duration
|
import akka.util.Duration
|
||||||
import akka.ConfigurationException
|
import akka.ConfigurationException
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import akka.pattern.ask
|
import akka.pattern.{ ask, pipe }
|
||||||
import java.util.concurrent.ConcurrentHashMap
|
import java.util.concurrent.ConcurrentHashMap
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import akka.dispatch.Dispatchers
|
import akka.dispatch.Dispatchers
|
||||||
|
import akka.util.Timeout
|
||||||
|
|
||||||
object RoutingSpec {
|
object RoutingSpec {
|
||||||
|
|
||||||
|
|
@ -171,6 +172,18 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
||||||
expectMsg("restarted")
|
expectMsg("restarted")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
"must start in-line for context.actorOf()" in {
|
||||||
|
system.actorOf(Props(new Actor {
|
||||||
|
def receive = {
|
||||||
|
case "start" ⇒
|
||||||
|
context.actorOf(Props(new Actor {
|
||||||
|
def receive = { case x ⇒ sender ! x }
|
||||||
|
}).withRouter(RoundRobinRouter(2))) ? "hello" pipeTo sender
|
||||||
|
}
|
||||||
|
})) ! "start"
|
||||||
|
expectMsg("hello")
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
"no router" must {
|
"no router" must {
|
||||||
|
|
@ -528,7 +541,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
"support custom router" in {
|
"support custom router" in {
|
||||||
val myrouter = system.actorOf(Props().withRouter(FromConfig), "myrouter")
|
val myrouter = system.actorOf(Props.empty.withRouter(FromConfig), "myrouter")
|
||||||
myrouter.isTerminated must be(false)
|
myrouter.isTerminated must be(false)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -540,7 +553,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
|
||||||
}
|
}
|
||||||
|
|
||||||
"count votes as intended - not as in Florida" in {
|
"count votes as intended - not as in Florida" in {
|
||||||
val routedActor = system.actorOf(Props().withRouter(VoteCountRouter()))
|
val routedActor = system.actorOf(Props.empty.withRouter(VoteCountRouter()))
|
||||||
routedActor ! DemocratVote
|
routedActor ! DemocratVote
|
||||||
routedActor ! DemocratVote
|
routedActor ! DemocratVote
|
||||||
routedActor ! RepublicanVote
|
routedActor ! RepublicanVote
|
||||||
|
|
|
||||||
|
|
@ -9,11 +9,13 @@ import akka.util.Unsafe;
|
||||||
final class AbstractActorCell {
|
final class AbstractActorCell {
|
||||||
final static long mailboxOffset;
|
final static long mailboxOffset;
|
||||||
final static long childrenOffset;
|
final static long childrenOffset;
|
||||||
|
final static long nextNameOffset;
|
||||||
|
|
||||||
static {
|
static {
|
||||||
try {
|
try {
|
||||||
mailboxOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_mailboxDoNotCallMeDirectly"));
|
mailboxOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_mailboxDoNotCallMeDirectly"));
|
||||||
childrenOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_childrenRefsDoNotCallMeDirectly"));
|
childrenOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_childrenRefsDoNotCallMeDirectly"));
|
||||||
|
nextNameOffset = Unsafe.instance.objectFieldOffset(ActorCell.class.getDeclaredField("_nextNameDoNotCallMeDirectly"));
|
||||||
} catch(Throwable t){
|
} catch(Throwable t){
|
||||||
throw new ExceptionInInitializerError(t);
|
throw new ExceptionInInitializerError(t);
|
||||||
}
|
}
|
||||||
|
|
|
||||||
19
akka-actor/src/main/java/akka/actor/AbstractActorRef.java
Normal file
19
akka-actor/src/main/java/akka/actor/AbstractActorRef.java
Normal file
|
|
@ -0,0 +1,19 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.actor;
|
||||||
|
|
||||||
|
import akka.util.Unsafe;
|
||||||
|
|
||||||
|
final class AbstractActorRef {
|
||||||
|
final static long cellOffset;
|
||||||
|
|
||||||
|
static {
|
||||||
|
try {
|
||||||
|
cellOffset = Unsafe.instance.objectFieldOffset(RepointableActorRef.class.getDeclaredField("_cellDoNotCallMeDirectly"));
|
||||||
|
} catch(Throwable t){
|
||||||
|
throw new ExceptionInInitializerError(t);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
@ -135,8 +135,7 @@ class ActorInitializationException private[akka] (actor: ActorRef, message: Stri
|
||||||
* there might be more of them in the future, or not.
|
* there might be more of them in the future, or not.
|
||||||
*/
|
*/
|
||||||
class InvalidMessageException private[akka] (message: String, cause: Throwable = null)
|
class InvalidMessageException private[akka] (message: String, cause: Throwable = null)
|
||||||
extends AkkaException(message, cause)
|
extends AkkaException(message, cause) {
|
||||||
with NoStackTrace {
|
|
||||||
def this(msg: String) = this(msg, null)
|
def this(msg: String) = this(msg, null)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -168,6 +168,78 @@ trait UntypedActorContext extends ActorContext {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* INTERNAL API
|
||||||
|
*/
|
||||||
|
private[akka] trait Cell {
|
||||||
|
/**
|
||||||
|
* The “self” reference which this Cell is attached to.
|
||||||
|
*/
|
||||||
|
def self: ActorRef
|
||||||
|
/**
|
||||||
|
* The system within which this Cell lives.
|
||||||
|
*/
|
||||||
|
def system: ActorSystem
|
||||||
|
/**
|
||||||
|
* The system internals where this Cell lives.
|
||||||
|
*/
|
||||||
|
def systemImpl: ActorSystemImpl
|
||||||
|
/**
|
||||||
|
* Recursively suspend this actor and all its children.
|
||||||
|
*/
|
||||||
|
def suspend(): Unit
|
||||||
|
/**
|
||||||
|
* Recursively resume this actor and all its children.
|
||||||
|
*/
|
||||||
|
def resume(): Unit
|
||||||
|
/**
|
||||||
|
* Restart this actor (will recursively restart or stop all children).
|
||||||
|
*/
|
||||||
|
def restart(cause: Throwable): Unit
|
||||||
|
/**
|
||||||
|
* Recursively terminate this actor and all its children.
|
||||||
|
*/
|
||||||
|
def stop(): Unit
|
||||||
|
/**
|
||||||
|
* Returns “true” if the actor is locally known to be terminated, “false” if
|
||||||
|
* alive or uncertain.
|
||||||
|
*/
|
||||||
|
def isTerminated: Boolean
|
||||||
|
/**
|
||||||
|
* The supervisor of this actor.
|
||||||
|
*/
|
||||||
|
def parent: InternalActorRef
|
||||||
|
/**
|
||||||
|
* All children of this actor, including only reserved-names.
|
||||||
|
*/
|
||||||
|
def childrenRefs: ActorCell.ChildrenContainer
|
||||||
|
/**
|
||||||
|
* Enqueue a message to be sent to the actor; may or may not actually
|
||||||
|
* schedule the actor to run, depending on which type of cell it is.
|
||||||
|
*/
|
||||||
|
def tell(message: Any, sender: ActorRef): Unit
|
||||||
|
/**
|
||||||
|
* Enqueue a message to be sent to the actor; may or may not actually
|
||||||
|
* schedule the actor to run, depending on which type of cell it is.
|
||||||
|
*/
|
||||||
|
def sendSystemMessage(msg: SystemMessage): Unit
|
||||||
|
/**
|
||||||
|
* Returns true if the actor is local, i.e. if it is actually scheduled
|
||||||
|
* on a Thread in the current JVM when run.
|
||||||
|
*/
|
||||||
|
def isLocal: Boolean
|
||||||
|
/**
|
||||||
|
* If the actor isLocal, returns whether messages are currently queued,
|
||||||
|
* “false” otherwise.
|
||||||
|
*/
|
||||||
|
def hasMessages: Boolean
|
||||||
|
/**
|
||||||
|
* If the actor isLocal, returns the number of messages currently queued,
|
||||||
|
* which may be a costly operation, 0 otherwise.
|
||||||
|
*/
|
||||||
|
def numberOfMessages: Int
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Everything in here is completely Akka PRIVATE. You will not find any
|
* Everything in here is completely Akka PRIVATE. You will not find any
|
||||||
* supported APIs in this place. This is not the API you were looking
|
* supported APIs in this place. This is not the API you were looking
|
||||||
|
|
@ -359,11 +431,13 @@ private[akka] class ActorCell(
|
||||||
val system: ActorSystemImpl,
|
val system: ActorSystemImpl,
|
||||||
val self: InternalActorRef,
|
val self: InternalActorRef,
|
||||||
val props: Props,
|
val props: Props,
|
||||||
@volatile var parent: InternalActorRef) extends UntypedActorContext {
|
@volatile var parent: InternalActorRef) extends UntypedActorContext with Cell {
|
||||||
|
|
||||||
import AbstractActorCell.{ mailboxOffset, childrenOffset }
|
import AbstractActorCell.{ mailboxOffset, childrenOffset, nextNameOffset }
|
||||||
import ActorCell._
|
import ActorCell._
|
||||||
|
|
||||||
|
final def isLocal = true
|
||||||
|
|
||||||
final def systemImpl = system
|
final def systemImpl = system
|
||||||
|
|
||||||
protected final def guardian = self
|
protected final def guardian = self
|
||||||
|
|
@ -448,7 +522,7 @@ private[akka] class ActorCell(
|
||||||
case _ ⇒ true
|
case _ ⇒ true
|
||||||
}
|
}
|
||||||
|
|
||||||
private def _actorOf(props: Props, name: String): ActorRef = {
|
private def _actorOf(props: Props, name: String, async: Boolean): ActorRef = {
|
||||||
if (system.settings.SerializeAllCreators && !props.creator.isInstanceOf[NoSerializationVerificationNeeded]) {
|
if (system.settings.SerializeAllCreators && !props.creator.isInstanceOf[NoSerializationVerificationNeeded]) {
|
||||||
val ser = SerializationExtension(system)
|
val ser = SerializationExtension(system)
|
||||||
ser.serialize(props.creator) match {
|
ser.serialize(props.creator) match {
|
||||||
|
|
@ -459,14 +533,19 @@ private[akka] class ActorCell(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// in case we are currently terminating, swallow creation requests and return EmptyLocalActorRef
|
/*
|
||||||
if (isTerminating) provider.actorFor(self, Seq(name))
|
* in case we are currently terminating, fail external attachChild requests
|
||||||
|
* (internal calls cannot happen anyway because we are suspended)
|
||||||
|
*/
|
||||||
|
if (isTerminating) throw new IllegalStateException("cannot create children while terminating or terminated")
|
||||||
else {
|
else {
|
||||||
reserveChild(name)
|
reserveChild(name)
|
||||||
// this name will either be unreserved or overwritten with a real child below
|
// this name will either be unreserved or overwritten with a real child below
|
||||||
val actor =
|
val actor =
|
||||||
try provider.actorOf(systemImpl, props, self, self.path / name, false, None, true)
|
try {
|
||||||
catch {
|
provider.actorOf(systemImpl, props, self, self.path / name,
|
||||||
|
systemService = false, deploy = None, lookupDeploy = true, async = async)
|
||||||
|
} catch {
|
||||||
case NonFatal(e) ⇒
|
case NonFatal(e) ⇒
|
||||||
unreserveChild(name)
|
unreserveChild(name)
|
||||||
throw e
|
throw e
|
||||||
|
|
@ -476,36 +555,53 @@ private[akka] class ActorCell(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def actorOf(props: Props): ActorRef = _actorOf(props, randomName())
|
def actorOf(props: Props): ActorRef = _actorOf(props, randomName(), async = false)
|
||||||
|
|
||||||
def actorOf(props: Props, name: String): ActorRef = {
|
def actorOf(props: Props, name: String): ActorRef = _actorOf(props, checkName(name), async = false)
|
||||||
|
|
||||||
|
private def checkName(name: String): String = {
|
||||||
import ActorPath.ElementRegex
|
import ActorPath.ElementRegex
|
||||||
name match {
|
name match {
|
||||||
case null ⇒ throw new InvalidActorNameException("actor name must not be null")
|
case null ⇒ throw new InvalidActorNameException("actor name must not be null")
|
||||||
case "" ⇒ throw new InvalidActorNameException("actor name must not be empty")
|
case "" ⇒ throw new InvalidActorNameException("actor name must not be empty")
|
||||||
case ElementRegex() ⇒ // this is fine
|
case ElementRegex() ⇒ name
|
||||||
case _ ⇒ throw new InvalidActorNameException("illegal actor name '" + name + "', must conform to " + ElementRegex)
|
case _ ⇒ throw new InvalidActorNameException("illegal actor name '" + name + "', must conform to " + ElementRegex)
|
||||||
}
|
}
|
||||||
_actorOf(props, name)
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private[akka] def attachChild(props: Props, name: String): ActorRef =
|
||||||
|
_actorOf(props, checkName(name), async = true)
|
||||||
|
|
||||||
|
private[akka] def attachChild(props: Props): ActorRef =
|
||||||
|
_actorOf(props, randomName(), async = true)
|
||||||
|
|
||||||
final def stop(actor: ActorRef): Unit = {
|
final def stop(actor: ActorRef): Unit = {
|
||||||
if (childrenRefs.getByRef(actor).isDefined) shallDie(actor)
|
val started = actor match {
|
||||||
|
case r: RepointableRef ⇒ r.isStarted
|
||||||
|
case _ ⇒ true
|
||||||
|
}
|
||||||
|
if (childrenRefs.getByRef(actor).isDefined && started) shallDie(actor)
|
||||||
actor.asInstanceOf[InternalActorRef].stop()
|
actor.asInstanceOf[InternalActorRef].stop()
|
||||||
}
|
}
|
||||||
|
|
||||||
var currentMessage: Envelope = _
|
var currentMessage: Envelope = _
|
||||||
var actor: Actor = _
|
var actor: Actor = _
|
||||||
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
|
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
|
||||||
@volatile var _mailboxDoNotCallMeDirectly: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status
|
|
||||||
var watching: Set[ActorRef] = emptyActorRefSet
|
var watching: Set[ActorRef] = emptyActorRefSet
|
||||||
var watchedBy: Set[ActorRef] = emptyActorRefSet
|
var watchedBy: Set[ActorRef] = emptyActorRefSet
|
||||||
|
|
||||||
val nextNameSequence = new AtomicLong
|
@volatile private var _nextNameDoNotCallMeDirectly = 0L
|
||||||
final protected def randomName(): String = Helpers.base64(nextNameSequence.getAndIncrement())
|
final protected def randomName(): String = {
|
||||||
|
@tailrec def inc(): Long = {
|
||||||
|
val current = Unsafe.instance.getLongVolatile(this, nextNameOffset)
|
||||||
|
if (Unsafe.instance.compareAndSwapLong(this, nextNameOffset, current, current + 1)) current
|
||||||
|
else inc()
|
||||||
|
}
|
||||||
|
Helpers.base64(inc())
|
||||||
|
}
|
||||||
|
|
||||||
@inline
|
@volatile private var _mailboxDoNotCallMeDirectly: Mailbox = _ //This must be volatile since it isn't protected by the mailbox status
|
||||||
final val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
@ -525,6 +621,12 @@ private[akka] class ActorCell(
|
||||||
else oldMailbox
|
else oldMailbox
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final def hasMessages: Boolean = mailbox.hasMessages
|
||||||
|
|
||||||
|
final def numberOfMessages: Int = mailbox.numberOfMessages
|
||||||
|
|
||||||
|
val dispatcher: MessageDispatcher = system.dispatchers.lookup(props.dispatcher)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* UntypedActorContext impl
|
* UntypedActorContext impl
|
||||||
*/
|
*/
|
||||||
|
|
@ -532,20 +634,22 @@ private[akka] class ActorCell(
|
||||||
|
|
||||||
final def isTerminated: Boolean = mailbox.isClosed
|
final def isTerminated: Boolean = mailbox.isClosed
|
||||||
|
|
||||||
final def start(): Unit = {
|
final def start(): this.type = {
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Create the mailbox and enqueue the Create() message to ensure that
|
* Create the mailbox and enqueue the Create() message to ensure that
|
||||||
* this is processed before anything else.
|
* this is processed before anything else.
|
||||||
*/
|
*/
|
||||||
swapMailbox(dispatcher.createMailbox(this))
|
swapMailbox(dispatcher.createMailbox(this))
|
||||||
|
mailbox.setActor(this)
|
||||||
|
|
||||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||||
mailbox.systemEnqueue(self, Create())
|
mailbox.systemEnqueue(self, Create())
|
||||||
|
|
||||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
|
||||||
parent.sendSystemMessage(akka.dispatch.Supervise(self))
|
|
||||||
|
|
||||||
// This call is expected to start off the actor by scheduling its mailbox.
|
// This call is expected to start off the actor by scheduling its mailbox.
|
||||||
dispatcher.attach(this)
|
dispatcher.attach(this)
|
||||||
|
|
||||||
|
this
|
||||||
}
|
}
|
||||||
|
|
||||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||||
|
|
@ -583,8 +687,10 @@ private[akka] class ActorCell(
|
||||||
final def getChildren(): java.lang.Iterable[ActorRef] =
|
final def getChildren(): java.lang.Iterable[ActorRef] =
|
||||||
scala.collection.JavaConverters.asJavaIterableConverter(children).asJava
|
scala.collection.JavaConverters.asJavaIterableConverter(children).asJava
|
||||||
|
|
||||||
final def tell(message: Any, sender: ActorRef): Unit =
|
def tell(message: Any, sender: ActorRef): Unit =
|
||||||
dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender)(system))
|
dispatcher.dispatch(this, Envelope(message, if (sender eq null) system.deadLetters else sender, system))
|
||||||
|
|
||||||
|
override def sendSystemMessage(message: SystemMessage): Unit = dispatcher.systemDispatch(this, message)
|
||||||
|
|
||||||
final def sender: ActorRef = currentMessage match {
|
final def sender: ActorRef = currentMessage match {
|
||||||
case null ⇒ system.deadLetters
|
case null ⇒ system.deadLetters
|
||||||
|
|
@ -719,6 +825,7 @@ private[akka] class ActorCell(
|
||||||
|
|
||||||
def supervise(child: ActorRef): Unit = if (!isTerminating) {
|
def supervise(child: ActorRef): Unit = if (!isTerminating) {
|
||||||
if (childrenRefs.getByRef(child).isEmpty) addChild(child)
|
if (childrenRefs.getByRef(child).isEmpty) addChild(child)
|
||||||
|
handleSupervise(child)
|
||||||
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now supervising " + child))
|
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now supervising " + child))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -904,6 +1011,11 @@ private[akka] class ActorCell(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
protected def handleSupervise(child: ActorRef): Unit = child match {
|
||||||
|
case r: RepointableActorRef ⇒ r.activate()
|
||||||
|
case _ ⇒
|
||||||
|
}
|
||||||
|
|
||||||
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||||
final def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause))
|
final def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -167,6 +167,10 @@ private[akka] trait LocalRef extends ActorRefScope {
|
||||||
final def isLocal = true
|
final def isLocal = true
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private[akka] trait RepointableRef extends ActorRefScope {
|
||||||
|
def isStarted: Boolean
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Internal trait for assembling all the functionality needed internally on
|
* Internal trait for assembling all the functionality needed internally on
|
||||||
* ActorRefs. NOTE THAT THIS IS NOT A STABLE EXTERNAL INTERFACE!
|
* ActorRefs. NOTE THAT THIS IS NOT A STABLE EXTERNAL INTERFACE!
|
||||||
|
|
@ -210,6 +214,10 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe
|
||||||
def isLocal: Boolean
|
def isLocal: Boolean
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private[akka] abstract class ActorRefWithCell extends InternalActorRef { this: ActorRefScope ⇒
|
||||||
|
def underlying: Cell
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This is an internal look-up failure token, not useful for anything else.
|
* This is an internal look-up failure token, not useful for anything else.
|
||||||
*/
|
*/
|
||||||
|
|
@ -228,21 +236,21 @@ private[akka] class LocalActorRef private[akka] (
|
||||||
_props: Props,
|
_props: Props,
|
||||||
_supervisor: InternalActorRef,
|
_supervisor: InternalActorRef,
|
||||||
override val path: ActorPath)
|
override val path: ActorPath)
|
||||||
extends InternalActorRef with LocalRef {
|
extends ActorRefWithCell with LocalRef {
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* actorCell.start() publishes actorCell & this to the dispatcher, which
|
* Safe publication of this class’s fields is guaranteed by mailbox.setActor()
|
||||||
* means that messages may be processed theoretically before the constructor
|
* which is called indirectly from actorCell.start() (if you’re wondering why
|
||||||
* ends. The JMM guarantees visibility for final fields only after the end
|
* this is at all important, remember that under the JMM final fields are only
|
||||||
* of the constructor, so publish the actorCell safely by making it a
|
* frozen at the _end_ of the constructor, but we are publishing “this” before
|
||||||
* @volatile var which is NOT TO BE WRITTEN TO. The alternative would be to
|
* that is reached).
|
||||||
* move start() outside of the constructor, which would basically require
|
|
||||||
* us to use purely factory methods for creating LocalActorRefs.
|
|
||||||
*/
|
*/
|
||||||
@volatile
|
private val actorCell: ActorCell = newActorCell(_system, this, _props, _supervisor)
|
||||||
private var actorCell = newActorCell(_system, this, _props, _supervisor)
|
|
||||||
actorCell.start()
|
actorCell.start()
|
||||||
|
|
||||||
|
// ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅
|
||||||
|
_supervisor.sendSystemMessage(akka.dispatch.Supervise(this))
|
||||||
|
|
||||||
protected def newActorCell(system: ActorSystemImpl, ref: InternalActorRef, props: Props, supervisor: InternalActorRef): ActorCell =
|
protected def newActorCell(system: ActorSystemImpl, ref: InternalActorRef, props: Props, supervisor: InternalActorRef): ActorCell =
|
||||||
new ActorCell(system, ref, props, supervisor)
|
new ActorCell(system, ref, props, supervisor)
|
||||||
|
|
||||||
|
|
@ -313,9 +321,9 @@ private[akka] class LocalActorRef private[akka] (
|
||||||
|
|
||||||
// ========= AKKA PROTECTED FUNCTIONS =========
|
// ========= AKKA PROTECTED FUNCTIONS =========
|
||||||
|
|
||||||
protected[akka] def underlying: ActorCell = actorCell
|
def underlying: ActorCell = actorCell
|
||||||
|
|
||||||
override def sendSystemMessage(message: SystemMessage): Unit = underlying.dispatcher.systemDispatch(underlying, message)
|
override def sendSystemMessage(message: SystemMessage): Unit = actorCell.sendSystemMessage(message)
|
||||||
|
|
||||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = actorCell.tell(message, sender)
|
override def !(message: Any)(implicit sender: ActorRef = null): Unit = actorCell.tell(message, sender)
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -104,7 +104,8 @@ trait ActorRefProvider {
|
||||||
path: ActorPath,
|
path: ActorPath,
|
||||||
systemService: Boolean,
|
systemService: Boolean,
|
||||||
deploy: Option[Deploy],
|
deploy: Option[Deploy],
|
||||||
lookupDeploy: Boolean): InternalActorRef
|
lookupDeploy: Boolean,
|
||||||
|
async: Boolean): InternalActorRef
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Create actor reference for a specified local or remote path. If no such
|
* Create actor reference for a specified local or remote path. If no such
|
||||||
|
|
@ -482,10 +483,12 @@ class LocalActorRefProvider(
|
||||||
}
|
}
|
||||||
|
|
||||||
lazy val guardian: InternalActorRef =
|
lazy val guardian: InternalActorRef =
|
||||||
actorOf(system, guardianProps, rootGuardian, rootPath / "user", systemService = true, None, false)
|
actorOf(system, guardianProps, rootGuardian, rootPath / "user",
|
||||||
|
systemService = true, deploy = None, lookupDeploy = false, async = false)
|
||||||
|
|
||||||
lazy val systemGuardian: InternalActorRef =
|
lazy val systemGuardian: InternalActorRef =
|
||||||
actorOf(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, rootPath / "system", systemService = true, None, false)
|
actorOf(system, guardianProps.withCreator(new SystemGuardian), rootGuardian, rootPath / "system",
|
||||||
|
systemService = true, deploy = None, lookupDeploy = false, async = false)
|
||||||
|
|
||||||
lazy val tempContainer = new VirtualPathContainer(system.provider, tempNode, rootGuardian, log)
|
lazy val tempContainer = new VirtualPathContainer(system.provider, tempNode, rootGuardian, log)
|
||||||
|
|
||||||
|
|
@ -539,14 +542,17 @@ class LocalActorRefProvider(
|
||||||
}
|
}
|
||||||
|
|
||||||
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
|
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
|
||||||
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean): InternalActorRef = {
|
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = {
|
||||||
props.routerConfig match {
|
props.routerConfig match {
|
||||||
case NoRouter ⇒ new LocalActorRef(system, props, supervisor, path) // create a local actor
|
case NoRouter ⇒
|
||||||
|
if (async) new RepointableActorRef(system, props, supervisor, path).initialize()
|
||||||
|
else new LocalActorRef(system, props, supervisor, path)
|
||||||
case router ⇒
|
case router ⇒
|
||||||
val lookup = if (lookupDeploy) deployer.lookup(path) else None
|
val lookup = if (lookupDeploy) deployer.lookup(path) else None
|
||||||
val fromProps = Iterator(props.deploy.copy(routerConfig = props.deploy.routerConfig withFallback router))
|
val fromProps = Iterator(props.deploy.copy(routerConfig = props.deploy.routerConfig withFallback router))
|
||||||
val d = fromProps ++ deploy.iterator ++ lookup.iterator reduce ((a, b) ⇒ b withFallback a)
|
val d = fromProps ++ deploy.iterator ++ lookup.iterator reduce ((a, b) ⇒ b withFallback a)
|
||||||
new RoutedActorRef(system, props.withRouter(d.routerConfig), supervisor, path)
|
val ref = new RoutedActorRef(system, props.withRouter(d.routerConfig), supervisor, path).initialize()
|
||||||
|
if (async) ref else ref.activate()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -423,6 +423,11 @@ abstract class ExtendedActorSystem extends ActorSystem {
|
||||||
* creation.
|
* creation.
|
||||||
*/
|
*/
|
||||||
def dynamicAccess: DynamicAccess
|
def dynamicAccess: DynamicAccess
|
||||||
|
|
||||||
|
/**
|
||||||
|
* For debugging: traverse actor hierarchy and make string representation.
|
||||||
|
*/
|
||||||
|
def printTree: String
|
||||||
}
|
}
|
||||||
|
|
||||||
private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, classLoader: ClassLoader) extends ExtendedActorSystem {
|
private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config, classLoader: ClassLoader) extends ExtendedActorSystem {
|
||||||
|
|
@ -482,21 +487,21 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
||||||
|
|
||||||
private[akka] def systemActorOf(props: Props, name: String): ActorRef = {
|
private[akka] def systemActorOf(props: Props, name: String): ActorRef = {
|
||||||
systemGuardian match {
|
systemGuardian match {
|
||||||
case g: LocalActorRef ⇒ g.underlying.actorOf(props, name)
|
case g: LocalActorRef ⇒ g.underlying.attachChild(props, name)
|
||||||
case s ⇒ throw new UnsupportedOperationException("unknown systemGuardian type " + s.getClass)
|
case s ⇒ throw new UnsupportedOperationException("unknown systemGuardian type " + s.getClass)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def actorOf(props: Props, name: String): ActorRef = {
|
def actorOf(props: Props, name: String): ActorRef = {
|
||||||
guardian match {
|
guardian match {
|
||||||
case g: LocalActorRef ⇒ g.underlying.actorOf(props, name)
|
case g: LocalActorRef ⇒ g.underlying.attachChild(props, name)
|
||||||
case s ⇒ throw new UnsupportedOperationException("unknown guardian type " + s.getClass)
|
case s ⇒ throw new UnsupportedOperationException("unknown guardian type " + s.getClass)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def actorOf(props: Props): ActorRef = {
|
def actorOf(props: Props): ActorRef = {
|
||||||
guardian match {
|
guardian match {
|
||||||
case g: LocalActorRef ⇒ g.underlying.actorOf(props)
|
case g: LocalActorRef ⇒ g.underlying.attachChild(props)
|
||||||
case s ⇒ throw new UnsupportedOperationException("unknown guardian type " + s.getClass)
|
case s ⇒ throw new UnsupportedOperationException("unknown guardian type " + s.getClass)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -546,10 +551,10 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
||||||
def dequeue() = null
|
def dequeue() = null
|
||||||
def hasMessages = false
|
def hasMessages = false
|
||||||
def numberOfMessages = 0
|
def numberOfMessages = 0
|
||||||
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = ()
|
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = ()
|
||||||
}
|
}
|
||||||
//FIXME Why do we need this at all?
|
//FIXME Why do we need this at all?
|
||||||
val deadLetterMailbox: Mailbox = new Mailbox(null, deadLetterQueue) {
|
val deadLetterMailbox: Mailbox = new Mailbox(deadLetterQueue) {
|
||||||
becomeClosed()
|
becomeClosed()
|
||||||
def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit =
|
def systemEnqueue(receiver: ActorRef, handle: SystemMessage): Unit =
|
||||||
deadLetters ! DeadLetter(handle, receiver, receiver)
|
deadLetters ! DeadLetter(handle, receiver, receiver)
|
||||||
|
|
@ -689,6 +694,31 @@ private[akka] class ActorSystemImpl(val name: String, applicationConfig: Config,
|
||||||
|
|
||||||
override def toString: String = lookupRoot.path.root.address.toString
|
override def toString: String = lookupRoot.path.root.address.toString
|
||||||
|
|
||||||
|
override def printTree: String = {
|
||||||
|
def printNode(node: ActorRef, indent: String): String = {
|
||||||
|
node match {
|
||||||
|
case wc: ActorRefWithCell ⇒
|
||||||
|
val cell = wc.underlying
|
||||||
|
indent + "-> " + node.path.name + " " + Logging.simpleName(node) + " " +
|
||||||
|
(cell match {
|
||||||
|
case real: ActorCell ⇒ if (real.actor ne null) real.actor.getClass else "null"
|
||||||
|
case _ ⇒ Logging.simpleName(cell)
|
||||||
|
}) +
|
||||||
|
" " + (cell.childrenRefs match {
|
||||||
|
case ActorCell.TerminatingChildrenContainer(_, toDie, reason) ⇒
|
||||||
|
"Terminating(" + reason + ")" +
|
||||||
|
(toDie.toSeq.sorted mkString ("\n" + indent + " toDie: ", "\n" + indent + " ", ""))
|
||||||
|
case x ⇒ Logging.simpleName(x)
|
||||||
|
}) +
|
||||||
|
(if (cell.childrenRefs.children.isEmpty) "" else "\n") +
|
||||||
|
(cell.childrenRefs.children.toSeq.sorted map (printNode(_, indent + " |")) mkString ("\n"))
|
||||||
|
case _ ⇒
|
||||||
|
indent + node.path.name + " " + Logging.simpleName(node)
|
||||||
|
}
|
||||||
|
}
|
||||||
|
printNode(actorFor("/"), "")
|
||||||
|
}
|
||||||
|
|
||||||
final class TerminationCallbacks extends Runnable with Awaitable[Unit] {
|
final class TerminationCallbacks extends Runnable with Awaitable[Unit] {
|
||||||
private val lock = new ReentrantGuard
|
private val lock = new ReentrantGuard
|
||||||
private var callbacks: Stack[Runnable] = _ //non-volatile since guarded by the lock
|
private var callbacks: Stack[Runnable] = _ //non-volatile since guarded by the lock
|
||||||
|
|
|
||||||
201
akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala
Normal file
201
akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala
Normal file
|
|
@ -0,0 +1,201 @@
|
||||||
|
/**
|
||||||
|
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
|
||||||
|
*/
|
||||||
|
|
||||||
|
package akka.actor
|
||||||
|
|
||||||
|
import akka.util.Unsafe
|
||||||
|
import scala.annotation.tailrec
|
||||||
|
import akka.dispatch.SystemMessage
|
||||||
|
import akka.dispatch.Mailbox
|
||||||
|
import akka.dispatch.Terminate
|
||||||
|
import akka.dispatch.Envelope
|
||||||
|
import akka.dispatch.Supervise
|
||||||
|
import akka.dispatch.Create
|
||||||
|
import akka.dispatch.MessageDispatcher
|
||||||
|
import java.util.concurrent.locks.ReentrantLock
|
||||||
|
import akka.event.Logging.Warning
|
||||||
|
import scala.collection.mutable.Queue
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This actor ref starts out with some dummy cell (by default just enqueuing
|
||||||
|
* messages into vectors protected by ReentrantLock), it must be initialize()’d
|
||||||
|
* before it can be sent to, and it will be activate()’d by its supervisor in
|
||||||
|
* response to the Supervise() message, which will replace the contained Cell
|
||||||
|
* with a fully functional one, transfer all messages from dummy to real queue
|
||||||
|
* and swap out the cell ref.
|
||||||
|
*/
|
||||||
|
private[akka] class RepointableActorRef(
|
||||||
|
val system: ActorSystemImpl,
|
||||||
|
val props: Props,
|
||||||
|
val supervisor: InternalActorRef,
|
||||||
|
val path: ActorPath)
|
||||||
|
extends ActorRefWithCell with RepointableRef {
|
||||||
|
|
||||||
|
import AbstractActorRef.cellOffset
|
||||||
|
|
||||||
|
@volatile private var _cellDoNotCallMeDirectly: Cell = _
|
||||||
|
|
||||||
|
def underlying: Cell = Unsafe.instance.getObjectVolatile(this, cellOffset).asInstanceOf[Cell]
|
||||||
|
|
||||||
|
@tailrec final def swapCell(next: Cell): Cell = {
|
||||||
|
val old = underlying
|
||||||
|
if (Unsafe.instance.compareAndSwapObject(this, cellOffset, old, next)) old else swapCell(next)
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Initialize: make a dummy cell which holds just a mailbox, then tell our
|
||||||
|
* supervisor that we exist so that he can create the real Cell in
|
||||||
|
* handleSupervise().
|
||||||
|
*
|
||||||
|
* This is protected so that others can have different initialization.
|
||||||
|
*/
|
||||||
|
def initialize(): this.type = {
|
||||||
|
swapCell(new UnstartedCell(system, this, props, supervisor))
|
||||||
|
supervisor.sendSystemMessage(Supervise(this))
|
||||||
|
this
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* This method is supposedly called by the supervisor in handleSupervise()
|
||||||
|
* to replace the UnstartedCell with the real one. It assumes no concurrent
|
||||||
|
* modification of the underlying Cell.
|
||||||
|
*/
|
||||||
|
def activate(): this.type = {
|
||||||
|
underlying match {
|
||||||
|
case u: UnstartedCell ⇒ u.replaceWith(newCell())
|
||||||
|
case _ ⇒ // this happens routinely for things which were created async=false
|
||||||
|
}
|
||||||
|
this
|
||||||
|
}
|
||||||
|
|
||||||
|
def newCell(): Cell = new ActorCell(system, this, props, supervisor).start()
|
||||||
|
|
||||||
|
def suspend(): Unit = underlying.suspend()
|
||||||
|
|
||||||
|
def resume(): Unit = underlying.resume()
|
||||||
|
|
||||||
|
def stop(): Unit = underlying.stop()
|
||||||
|
|
||||||
|
def restart(cause: Throwable): Unit = underlying.restart(cause)
|
||||||
|
|
||||||
|
def isStarted: Boolean = !underlying.isInstanceOf[UnstartedCell]
|
||||||
|
|
||||||
|
def isTerminated: Boolean = underlying.isTerminated
|
||||||
|
|
||||||
|
def provider: ActorRefProvider = system.provider
|
||||||
|
|
||||||
|
def isLocal: Boolean = underlying.isLocal
|
||||||
|
|
||||||
|
def getParent: InternalActorRef = underlying.parent
|
||||||
|
|
||||||
|
def getChild(name: Iterator[String]): InternalActorRef =
|
||||||
|
if (name.hasNext) {
|
||||||
|
name.next match {
|
||||||
|
case ".." ⇒ getParent.getChild(name)
|
||||||
|
case "" ⇒ getChild(name)
|
||||||
|
case other ⇒
|
||||||
|
underlying.childrenRefs.getByName(other) match {
|
||||||
|
case Some(crs) ⇒ crs.child.asInstanceOf[InternalActorRef].getChild(name)
|
||||||
|
case None ⇒ Nobody
|
||||||
|
}
|
||||||
|
}
|
||||||
|
} else this
|
||||||
|
|
||||||
|
def !(message: Any)(implicit sender: ActorRef = null) = underlying.tell(message, sender)
|
||||||
|
|
||||||
|
def sendSystemMessage(message: SystemMessage) = underlying.sendSystemMessage(message)
|
||||||
|
|
||||||
|
@throws(classOf[java.io.ObjectStreamException])
|
||||||
|
protected def writeReplace(): AnyRef = SerializedActorRef(path)
|
||||||
|
}
|
||||||
|
|
||||||
|
private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, val self: RepointableActorRef, val props: Props, val supervisor: InternalActorRef)
|
||||||
|
extends Cell {
|
||||||
|
|
||||||
|
/*
|
||||||
|
* This lock protects all accesses to this cell’s queues. It also ensures
|
||||||
|
* safe switching to the started ActorCell.
|
||||||
|
*/
|
||||||
|
val lock = new ReentrantLock
|
||||||
|
|
||||||
|
// use Envelope to keep on-send checks in the same place
|
||||||
|
val queue: Queue[Envelope] = Queue()
|
||||||
|
val systemQueue: Queue[SystemMessage] = Queue()
|
||||||
|
|
||||||
|
def replaceWith(cell: Cell): Unit = {
|
||||||
|
lock.lock()
|
||||||
|
try {
|
||||||
|
/*
|
||||||
|
* The CallingThreadDispatcher nicely dives under the ReentrantLock and
|
||||||
|
* breaks things by enqueueing into stale queues from within the message
|
||||||
|
* processing which happens in-line for sendSystemMessage() and tell().
|
||||||
|
* Since this is the only possible way to f*ck things up within this
|
||||||
|
* lock, double-tap (well, N-tap, really); concurrent modification is
|
||||||
|
* still not possible because we’re the only thread accessing the queues.
|
||||||
|
*/
|
||||||
|
var interrupted = false
|
||||||
|
while (systemQueue.nonEmpty || queue.nonEmpty) {
|
||||||
|
while (systemQueue.nonEmpty) {
|
||||||
|
val msg = systemQueue.dequeue()
|
||||||
|
try cell sendSystemMessage msg catch { case _: InterruptedException ⇒ interrupted = true }
|
||||||
|
}
|
||||||
|
if (queue.nonEmpty) {
|
||||||
|
val envelope = queue.dequeue()
|
||||||
|
try cell tell (envelope.message, envelope.sender) catch { case _: InterruptedException ⇒ interrupted = true }
|
||||||
|
}
|
||||||
|
}
|
||||||
|
if (interrupted) throw new InterruptedException
|
||||||
|
} finally try
|
||||||
|
self.swapCell(cell)
|
||||||
|
finally
|
||||||
|
lock.unlock()
|
||||||
|
}
|
||||||
|
|
||||||
|
def system: ActorSystem = systemImpl
|
||||||
|
def suspend(): Unit = {}
|
||||||
|
def resume(): Unit = {}
|
||||||
|
def restart(cause: Throwable): Unit = {}
|
||||||
|
def stop(): Unit = sendSystemMessage(Terminate())
|
||||||
|
def isTerminated: Boolean = false
|
||||||
|
def parent: InternalActorRef = supervisor
|
||||||
|
def childrenRefs: ActorCell.ChildrenContainer = ActorCell.EmptyChildrenContainer
|
||||||
|
def tell(message: Any, sender: ActorRef): Unit = {
|
||||||
|
lock.lock()
|
||||||
|
try {
|
||||||
|
if (self.underlying eq this) queue enqueue Envelope(message, sender, system)
|
||||||
|
else self.underlying.tell(message, sender)
|
||||||
|
} finally {
|
||||||
|
lock.unlock()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
def sendSystemMessage(msg: SystemMessage): Unit = {
|
||||||
|
lock.lock()
|
||||||
|
try {
|
||||||
|
if (self.underlying eq this) systemQueue enqueue msg
|
||||||
|
else self.underlying.sendSystemMessage(msg)
|
||||||
|
} finally {
|
||||||
|
lock.unlock()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
def isLocal = true
|
||||||
|
def hasMessages: Boolean = {
|
||||||
|
lock.lock()
|
||||||
|
try {
|
||||||
|
if (self.underlying eq this) !queue.isEmpty
|
||||||
|
else self.underlying.hasMessages
|
||||||
|
} finally {
|
||||||
|
lock.unlock()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
def numberOfMessages: Int = {
|
||||||
|
lock.lock()
|
||||||
|
try {
|
||||||
|
if (self.underlying eq this) queue.size
|
||||||
|
else self.underlying.numberOfMessages
|
||||||
|
} finally {
|
||||||
|
lock.unlock()
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -592,7 +592,7 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] (
|
||||||
/**
|
/**
|
||||||
* Returns the akka.actor.Props representation of this TypedProps
|
* Returns the akka.actor.Props representation of this TypedProps
|
||||||
*/
|
*/
|
||||||
def actorProps(): Props = if (dispatcher == Props().dispatcher) Props() else Props(dispatcher = dispatcher)
|
def actorProps(): Props = if (dispatcher == Props.default.dispatcher) Props.default else Props(dispatcher = dispatcher)
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -16,8 +16,10 @@ import akka.event.Logging.LogEventException
|
||||||
import akka.jsr166y.{ ForkJoinTask, ForkJoinPool }
|
import akka.jsr166y.{ ForkJoinTask, ForkJoinPool }
|
||||||
import akka.util.{ Unsafe, Duration, NonFatal, Index }
|
import akka.util.{ Unsafe, Duration, NonFatal, Index }
|
||||||
|
|
||||||
final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorSystem) {
|
final case class Envelope private (val message: Any, val sender: ActorRef)
|
||||||
if (message.isInstanceOf[AnyRef]) {
|
|
||||||
|
object Envelope {
|
||||||
|
def apply(message: Any, sender: ActorRef, system: ActorSystem): Envelope = {
|
||||||
val msg = message.asInstanceOf[AnyRef]
|
val msg = message.asInstanceOf[AnyRef]
|
||||||
if (msg eq null) throw new InvalidMessageException("Message is null")
|
if (msg eq null) throw new InvalidMessageException("Message is null")
|
||||||
if (system.settings.SerializeAllMessages && !msg.isInstanceOf[NoSerializationVerificationNeeded]) {
|
if (system.settings.SerializeAllMessages && !msg.isInstanceOf[NoSerializationVerificationNeeded]) {
|
||||||
|
|
@ -30,6 +32,7 @@ final case class Envelope(val message: Any, val sender: ActorRef)(system: ActorS
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
new Envelope(message, sender)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -228,8 +231,8 @@ private[akka] object MessageDispatcher {
|
||||||
} {
|
} {
|
||||||
val status = if (a.isTerminated) " (terminated)" else " (alive)"
|
val status = if (a.isTerminated) " (terminated)" else " (alive)"
|
||||||
val messages = a match {
|
val messages = a match {
|
||||||
case l: LocalActorRef ⇒ " " + l.underlying.mailbox.numberOfMessages + " messages"
|
case r: ActorRefWithCell ⇒ " " + r.underlying.numberOfMessages + " messages"
|
||||||
case _ ⇒ " " + a.getClass
|
case _ ⇒ " " + a.getClass
|
||||||
}
|
}
|
||||||
val parent = a match {
|
val parent = a match {
|
||||||
case i: InternalActorRef ⇒ ", parent: " + i.getParent
|
case i: InternalActorRef ⇒ ", parent: " + i.getParent
|
||||||
|
|
@ -265,7 +268,7 @@ abstract class MessageDispatcher(val prerequisites: DispatcherPrerequisites) ext
|
||||||
/**
|
/**
|
||||||
* Creates and returns a mailbox for the given actor.
|
* Creates and returns a mailbox for the given actor.
|
||||||
*/
|
*/
|
||||||
protected[akka] def createMailbox(actor: ActorCell): Mailbox //FIXME should this really be private[akka]?
|
protected[akka] def createMailbox(actor: Cell): Mailbox //FIXME should this really be private[akka]?
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Identifier of this dispatcher, corresponds to the full key
|
* Identifier of this dispatcher, corresponds to the full key
|
||||||
|
|
|
||||||
|
|
@ -9,6 +9,7 @@ import annotation.tailrec
|
||||||
import akka.util.{ Duration, Helpers }
|
import akka.util.{ Duration, Helpers }
|
||||||
import java.util.{ Comparator, Iterator }
|
import java.util.{ Comparator, Iterator }
|
||||||
import java.util.concurrent.{ Executor, LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
|
import java.util.concurrent.{ Executor, LinkedBlockingQueue, ConcurrentLinkedQueue, ConcurrentSkipListSet }
|
||||||
|
import akka.actor.ActorSystemImpl
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
|
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
|
||||||
|
|
@ -46,24 +47,25 @@ class BalancingDispatcher(
|
||||||
/**
|
/**
|
||||||
* INTERNAL USE ONLY
|
* INTERNAL USE ONLY
|
||||||
*/
|
*/
|
||||||
private[akka] val messageQueue: MessageQueue = mailboxType.create(None)
|
private[akka] val messageQueue: MessageQueue = mailboxType.create(None, None)
|
||||||
|
|
||||||
private class SharingMailbox(_actor: ActorCell, _messageQueue: MessageQueue) extends Mailbox(_actor, _messageQueue) with DefaultSystemMessageQueue {
|
private class SharingMailbox(val system: ActorSystemImpl, _messageQueue: MessageQueue)
|
||||||
|
extends Mailbox(_messageQueue) with DefaultSystemMessageQueue {
|
||||||
override def cleanUp(): Unit = {
|
override def cleanUp(): Unit = {
|
||||||
val dlq = actor.systemImpl.deadLetterMailbox
|
val dlq = system.deadLetterMailbox
|
||||||
//Don't call the original implementation of this since it scraps all messages, and we don't want to do that
|
//Don't call the original implementation of this since it scraps all messages, and we don't want to do that
|
||||||
var message = systemDrain(NoMessage)
|
var message = systemDrain(NoMessage)
|
||||||
while (message ne null) {
|
while (message ne null) {
|
||||||
// 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
|
||||||
dlq.systemEnqueue(actor.self, message)
|
dlq.systemEnqueue(system.deadLetters, message)
|
||||||
message = next
|
message = next
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
protected[akka] override def createMailbox(actor: ActorCell): Mailbox = new SharingMailbox(actor, messageQueue)
|
protected[akka] override def createMailbox(actor: akka.actor.Cell): Mailbox = new SharingMailbox(actor.systemImpl, messageQueue)
|
||||||
|
|
||||||
protected[akka] override def register(actor: ActorCell): Unit = {
|
protected[akka] override def register(actor: ActorCell): Unit = {
|
||||||
super.register(actor)
|
super.register(actor)
|
||||||
|
|
|
||||||
|
|
@ -82,7 +82,8 @@ class Dispatcher(
|
||||||
/**
|
/**
|
||||||
* INTERNAL USE ONLY
|
* INTERNAL USE ONLY
|
||||||
*/
|
*/
|
||||||
protected[akka] def createMailbox(actor: ActorCell): Mailbox = new Mailbox(actor, mailboxType.create(Some(actor))) with DefaultSystemMessageQueue
|
protected[akka] def createMailbox(actor: akka.actor.Cell): Mailbox =
|
||||||
|
new Mailbox(mailboxType.create(Some(actor.self), Some(actor.system))) with DefaultSystemMessageQueue
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL USE ONLY
|
* INTERNAL USE ONLY
|
||||||
|
|
|
||||||
|
|
@ -6,7 +6,7 @@ package akka.dispatch
|
||||||
import akka.AkkaException
|
import akka.AkkaException
|
||||||
import java.util.{ Comparator, PriorityQueue, Queue, Deque }
|
import java.util.{ Comparator, PriorityQueue, Queue, Deque }
|
||||||
import akka.util._
|
import akka.util._
|
||||||
import akka.actor.{ ActorCell, ActorRef }
|
import akka.actor.{ ActorCell, ActorRef, Cell }
|
||||||
import java.util.concurrent._
|
import java.util.concurrent._
|
||||||
import annotation.tailrec
|
import annotation.tailrec
|
||||||
import akka.event.Logging.Error
|
import akka.event.Logging.Error
|
||||||
|
|
@ -48,11 +48,32 @@ private[akka] object Mailbox {
|
||||||
*
|
*
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
*/
|
*/
|
||||||
private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: MessageQueue)
|
private[akka] abstract class Mailbox(val messageQueue: MessageQueue)
|
||||||
extends SystemMessageQueue with Runnable {
|
extends SystemMessageQueue with Runnable {
|
||||||
|
|
||||||
import Mailbox._
|
import Mailbox._
|
||||||
|
|
||||||
|
/*
|
||||||
|
* This is needed for actually executing the mailbox, i.e. invoking the
|
||||||
|
* ActorCell. There are situations (e.g. RepointableActorRef) where a Mailbox
|
||||||
|
* is constructed but we know that we will not execute it, in which case this
|
||||||
|
* will be null. It must be a var to support switching into an “active”
|
||||||
|
* mailbox, should the owning ActorRef turn local.
|
||||||
|
*
|
||||||
|
* ANOTHER THING, IMPORTANT:
|
||||||
|
*
|
||||||
|
* actorCell.start() publishes actorCell & self to the dispatcher, which
|
||||||
|
* means that messages may be processed theoretically before self’s constructor
|
||||||
|
* ends. The JMM guarantees visibility for final fields only after the end
|
||||||
|
* of the constructor, so safe publication requires that THIS WRITE BELOW
|
||||||
|
* stay as it is.
|
||||||
|
*/
|
||||||
|
@volatile
|
||||||
|
var actor: ActorCell = _
|
||||||
|
def setActor(cell: ActorCell): Unit = actor = cell
|
||||||
|
|
||||||
|
def dispatcher: MessageDispatcher = actor.dispatcher
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Try to enqueue the message to this queue, or throw an exception.
|
* Try to enqueue the message to this queue, or throw an exception.
|
||||||
*/
|
*/
|
||||||
|
|
@ -237,11 +258,12 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
||||||
* if we closed the mailbox, we must dump the remaining system messages
|
* if we closed the mailbox, we must dump the remaining system messages
|
||||||
* to deadLetters (this is essential for DeathWatch)
|
* to deadLetters (this is essential for DeathWatch)
|
||||||
*/
|
*/
|
||||||
|
val dlm = actor.systemImpl.deadLetterMailbox
|
||||||
while (nextMessage ne null) {
|
while (nextMessage ne null) {
|
||||||
val msg = nextMessage
|
val msg = nextMessage
|
||||||
nextMessage = nextMessage.next
|
nextMessage = nextMessage.next
|
||||||
msg.next = null
|
msg.next = null
|
||||||
try actor.systemImpl.deadLetterMailbox.systemEnqueue(actor.self, msg)
|
try dlm.systemEnqueue(actor.self, msg)
|
||||||
catch {
|
catch {
|
||||||
case NonFatal(e) ⇒ actor.system.eventStream.publish(
|
case NonFatal(e) ⇒ actor.system.eventStream.publish(
|
||||||
Error(e, actor.self.path.toString, this.getClass, "error while enqueuing " + msg + " to deadLetters: " + e.getMessage))
|
Error(e, actor.self.path.toString, this.getClass, "error while enqueuing " + msg + " to deadLetters: " + e.getMessage))
|
||||||
|
|
@ -251,9 +273,6 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
||||||
if (failure ne null) actor.handleInvokeFailure(failure, failure.getMessage)
|
if (failure ne null) actor.handleInvokeFailure(failure, failure.getMessage)
|
||||||
}
|
}
|
||||||
|
|
||||||
@inline
|
|
||||||
final def dispatcher: MessageDispatcher = actor.dispatcher
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Overridable callback to clean up the mailbox,
|
* Overridable callback to clean up the mailbox,
|
||||||
* called when an actor is unregistered.
|
* called when an actor is unregistered.
|
||||||
|
|
@ -272,7 +291,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell, val messageQueue: Mes
|
||||||
}
|
}
|
||||||
|
|
||||||
if (messageQueue ne null) // needed for CallingThreadDispatcher, which never calls Mailbox.run()
|
if (messageQueue ne null) // needed for CallingThreadDispatcher, which never calls Mailbox.run()
|
||||||
messageQueue.cleanUp(actor, actor.systemImpl.deadLetterQueue)
|
messageQueue.cleanUp(actor.self, actor.systemImpl.deadLetterQueue)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -310,7 +329,7 @@ trait MessageQueue {
|
||||||
* which is passed in. The owner of this MessageQueue is passed in if
|
* which is passed in. The owner of this MessageQueue is passed in if
|
||||||
* available (e.g. for creating DeadLetters()), “/deadletters” otherwise.
|
* available (e.g. for creating DeadLetters()), “/deadletters” otherwise.
|
||||||
*/
|
*/
|
||||||
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit
|
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -338,10 +357,11 @@ private[akka] trait DefaultSystemMessageQueue { self: Mailbox ⇒
|
||||||
@tailrec
|
@tailrec
|
||||||
final def systemEnqueue(receiver: ActorRef, 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.self + " having enqueued " + message)
|
if (Mailbox.debug) println(receiver + " having enqueued " + message)
|
||||||
val head = systemQueueGet
|
val head = systemQueueGet
|
||||||
if (head == NoMessage) actor.system.deadLetterMailbox.systemEnqueue(receiver, message)
|
if (head == NoMessage) {
|
||||||
else {
|
if (actor ne null) actor.systemImpl.deadLetterMailbox.systemEnqueue(receiver, message)
|
||||||
|
} else {
|
||||||
/*
|
/*
|
||||||
* this write is safely published by the compareAndSet contained within
|
* this write is safely published by the compareAndSet contained within
|
||||||
* systemQueuePut; “Intra-Thread Semantics” on page 12 of the JSR133 spec
|
* systemQueuePut; “Intra-Thread Semantics” on page 12 of the JSR133 spec
|
||||||
|
|
@ -373,11 +393,11 @@ trait QueueBasedMessageQueue extends MessageQueue {
|
||||||
def queue: Queue[Envelope]
|
def queue: Queue[Envelope]
|
||||||
def numberOfMessages = queue.size
|
def numberOfMessages = queue.size
|
||||||
def hasMessages = !queue.isEmpty
|
def hasMessages = !queue.isEmpty
|
||||||
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = {
|
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = {
|
||||||
if (hasMessages) {
|
if (hasMessages) {
|
||||||
var envelope = dequeue
|
var envelope = dequeue
|
||||||
while (envelope ne null) {
|
while (envelope ne null) {
|
||||||
deadLetters.enqueue(owner.self, envelope)
|
deadLetters.enqueue(owner, envelope)
|
||||||
envelope = dequeue
|
envelope = dequeue
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -459,7 +479,7 @@ trait BoundedDequeBasedMessageQueueSemantics extends DequeBasedMessageQueue {
|
||||||
* MailboxType is a factory to create MessageQueues for an optionally provided ActorContext
|
* MailboxType is a factory to create MessageQueues for an optionally provided ActorContext
|
||||||
*/
|
*/
|
||||||
trait MailboxType {
|
trait MailboxType {
|
||||||
def create(owner: Option[ActorContext]): MessageQueue
|
def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -469,7 +489,7 @@ case class UnboundedMailbox() extends MailboxType {
|
||||||
|
|
||||||
def this(settings: ActorSystem.Settings, config: Config) = this()
|
def this(settings: ActorSystem.Settings, config: Config) = this()
|
||||||
|
|
||||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||||
new ConcurrentLinkedQueue[Envelope]() with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
new ConcurrentLinkedQueue[Envelope]() with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
||||||
final def queue: Queue[Envelope] = this
|
final def queue: Queue[Envelope] = this
|
||||||
}
|
}
|
||||||
|
|
@ -486,7 +506,7 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat
|
||||||
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
|
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")
|
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null")
|
||||||
|
|
||||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||||
new LinkedBlockingQueue[Envelope](capacity) with QueueBasedMessageQueue with BoundedMessageQueueSemantics {
|
new LinkedBlockingQueue[Envelope](capacity) with QueueBasedMessageQueue with BoundedMessageQueueSemantics {
|
||||||
final def queue: BlockingQueue[Envelope] = this
|
final def queue: BlockingQueue[Envelope] = this
|
||||||
final val pushTimeOut = BoundedMailbox.this.pushTimeOut
|
final val pushTimeOut = BoundedMailbox.this.pushTimeOut
|
||||||
|
|
@ -499,7 +519,7 @@ case class BoundedMailbox( final val capacity: Int, final val pushTimeOut: Durat
|
||||||
*/
|
*/
|
||||||
class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope], final val initialCapacity: Int) extends MailboxType {
|
class UnboundedPriorityMailbox( final val cmp: Comparator[Envelope], final val initialCapacity: Int) extends MailboxType {
|
||||||
def this(cmp: Comparator[Envelope]) = this(cmp, 11)
|
def this(cmp: Comparator[Envelope]) = this(cmp, 11)
|
||||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||||
new PriorityBlockingQueue[Envelope](initialCapacity, cmp) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
new PriorityBlockingQueue[Envelope](initialCapacity, cmp) with QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
||||||
final def queue: Queue[Envelope] = this
|
final def queue: Queue[Envelope] = this
|
||||||
}
|
}
|
||||||
|
|
@ -514,7 +534,7 @@ class BoundedPriorityMailbox( final val cmp: Comparator[Envelope], final val cap
|
||||||
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
|
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")
|
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null")
|
||||||
|
|
||||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||||
new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) with QueueBasedMessageQueue with BoundedMessageQueueSemantics {
|
new BoundedBlockingQueue[Envelope](capacity, new PriorityQueue[Envelope](11, cmp)) with QueueBasedMessageQueue with BoundedMessageQueueSemantics {
|
||||||
final def queue: BlockingQueue[Envelope] = this
|
final def queue: BlockingQueue[Envelope] = this
|
||||||
final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut
|
final val pushTimeOut = BoundedPriorityMailbox.this.pushTimeOut
|
||||||
|
|
@ -528,7 +548,7 @@ case class UnboundedDequeBasedMailbox() extends MailboxType {
|
||||||
|
|
||||||
def this(settings: ActorSystem.Settings, config: Config) = this()
|
def this(settings: ActorSystem.Settings, config: Config) = this()
|
||||||
|
|
||||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||||
new LinkedBlockingDeque[Envelope]() with DequeBasedMessageQueue with UnboundedDequeBasedMessageQueueSemantics {
|
new LinkedBlockingDeque[Envelope]() with DequeBasedMessageQueue with UnboundedDequeBasedMessageQueueSemantics {
|
||||||
final val queue = this
|
final val queue = this
|
||||||
}
|
}
|
||||||
|
|
@ -545,7 +565,7 @@ case class BoundedDequeBasedMailbox( final val capacity: Int, final val pushTime
|
||||||
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedDequeBasedMailbox can not be negative")
|
if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedDequeBasedMailbox can not be negative")
|
||||||
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedDequeBasedMailbox can not be null")
|
if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedDequeBasedMailbox can not be null")
|
||||||
|
|
||||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||||
new LinkedBlockingDeque[Envelope](capacity) with DequeBasedMessageQueue with BoundedDequeBasedMessageQueueSemantics {
|
new LinkedBlockingDeque[Envelope](capacity) with DequeBasedMessageQueue with BoundedDequeBasedMessageQueueSemantics {
|
||||||
final val queue = this
|
final val queue = this
|
||||||
final val pushTimeOut = BoundedDequeBasedMailbox.this.pushTimeOut
|
final val pushTimeOut = BoundedDequeBasedMailbox.this.pushTimeOut
|
||||||
|
|
|
||||||
|
|
@ -23,42 +23,28 @@ import scala.runtime.ScalaRunTime
|
||||||
* send a message to on (or more) of these actors.
|
* send a message to on (or more) of these actors.
|
||||||
*/
|
*/
|
||||||
private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath)
|
private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _supervisor: InternalActorRef, _path: ActorPath)
|
||||||
extends LocalActorRef(
|
extends RepointableActorRef(_system, _props, _supervisor, _path) {
|
||||||
_system,
|
|
||||||
_props.copy(creator = () ⇒ _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher),
|
|
||||||
_supervisor,
|
|
||||||
_path) {
|
|
||||||
|
|
||||||
/*
|
// verify that a BalancingDispatcher is not used with a Router
|
||||||
* CAUTION: RoutedActorRef is PROBLEMATIC
|
if (_props.routerConfig != NoRouter && _system.dispatchers.isBalancingDispatcher(_props.routerConfig.routerDispatcher)) {
|
||||||
* ======================================
|
throw new ConfigurationException(
|
||||||
*
|
"Configuration for " + this +
|
||||||
* We are constructing/assembling the children outside of the scope of the
|
" is invalid - you can not use a 'BalancingDispatcher' as a Router's dispatcher, you can however use it for the routees.")
|
||||||
* Router actor, inserting them in its childrenRef list, which is not at all
|
|
||||||
* synchronized. This is done exactly once at start-up, all other accesses
|
|
||||||
* are done from the Router actor. This means that the only thing which is
|
|
||||||
* really hairy is making sure that the Router does not touch its childrenRefs
|
|
||||||
* before we are done with them: lock the monitor of the actor cell (hence the
|
|
||||||
* override of newActorCell) and use that to block the Router constructor for
|
|
||||||
* as long as it takes to setup the RoutedActorRef itself.
|
|
||||||
*
|
|
||||||
* ===> I M P O R T A N T N O T I C E <===
|
|
||||||
*
|
|
||||||
* DO NOT THROW ANY EXCEPTIONS BEFORE THE FOLLOWING TRY-BLOCK WITHOUT
|
|
||||||
* EXITING THE MONITOR OF THE actorCell!
|
|
||||||
*
|
|
||||||
* This is important, just don’t do it! No kidding.
|
|
||||||
*/
|
|
||||||
override def newActorCell(
|
|
||||||
system: ActorSystemImpl,
|
|
||||||
ref: InternalActorRef,
|
|
||||||
props: Props,
|
|
||||||
supervisor: InternalActorRef): ActorCell = {
|
|
||||||
val cell = super.newActorCell(system, ref, props, supervisor)
|
|
||||||
Unsafe.instance.monitorEnter(cell)
|
|
||||||
cell
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
_props.routerConfig.verifyConfig()
|
||||||
|
|
||||||
|
override def newCell(): Cell = new RoutedActorCell(system, this, props, supervisor)
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActorRef, _props: Props, _supervisor: InternalActorRef)
|
||||||
|
extends ActorCell(
|
||||||
|
_system,
|
||||||
|
_ref,
|
||||||
|
_props.copy(creator = () ⇒ _props.routerConfig.createActor(), dispatcher = _props.routerConfig.routerDispatcher),
|
||||||
|
_supervisor) {
|
||||||
|
|
||||||
private[akka] val routerConfig = _props.routerConfig
|
private[akka] val routerConfig = _props.routerConfig
|
||||||
private[akka] val routeeProps = _props.copy(routerConfig = NoRouter)
|
private[akka] val routeeProps = _props.copy(routerConfig = NoRouter)
|
||||||
private[akka] val resizeInProgress = new AtomicBoolean
|
private[akka] val resizeInProgress = new AtomicBoolean
|
||||||
|
|
@ -72,39 +58,29 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
||||||
private var _routeeProvider: RouteeProvider = _
|
private var _routeeProvider: RouteeProvider = _
|
||||||
def routeeProvider = _routeeProvider
|
def routeeProvider = _routeeProvider
|
||||||
|
|
||||||
val route =
|
val route = {
|
||||||
try {
|
_routeeProvider = routerConfig.createRouteeProvider(this)
|
||||||
// verify that a BalancingDispatcher is not used with a Router
|
val r = routerConfig.createRoute(routeeProps, routeeProvider)
|
||||||
if (_props.routerConfig != NoRouter && _system.dispatchers.isBalancingDispatcher(_props.routerConfig.routerDispatcher)) {
|
// initial resize, before message send
|
||||||
actorContext.stop(actorContext.self)
|
routerConfig.resizer foreach { r ⇒
|
||||||
throw new ConfigurationException(
|
if (r.isTimeForResize(resizeCounter.getAndIncrement()))
|
||||||
"Configuration for actor [" + _path.toString +
|
r.resize(routeeProps, routeeProvider)
|
||||||
"] is invalid - you can not use a 'BalancingDispatcher' as a Router's dispatcher, you can however use it for the routees.")
|
|
||||||
}
|
|
||||||
|
|
||||||
_routeeProvider = routerConfig.createRouteeProvider(actorContext)
|
|
||||||
val r = routerConfig.createRoute(routeeProps, routeeProvider)
|
|
||||||
// initial resize, before message send
|
|
||||||
routerConfig.resizer foreach { r ⇒
|
|
||||||
if (r.isTimeForResize(resizeCounter.getAndIncrement()))
|
|
||||||
r.resize(routeeProps, routeeProvider)
|
|
||||||
}
|
|
||||||
r
|
|
||||||
} finally {
|
|
||||||
assert(Thread.holdsLock(actorContext))
|
|
||||||
Unsafe.instance.monitorExit(actorContext) // unblock Router’s constructor
|
|
||||||
}
|
}
|
||||||
|
r
|
||||||
|
}
|
||||||
|
|
||||||
if (routerConfig.resizer.isEmpty && _routees.isEmpty)
|
if (routerConfig.resizer.isEmpty && _routees.isEmpty)
|
||||||
throw new ActorInitializationException("router " + routerConfig + " did not register routees!")
|
throw new ActorInitializationException("router " + routerConfig + " did not register routees!")
|
||||||
|
|
||||||
|
start()
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* end of construction
|
* end of construction
|
||||||
*/
|
*/
|
||||||
|
|
||||||
def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match {
|
def applyRoute(sender: ActorRef, message: Any): Iterable[Destination] = message match {
|
||||||
case _: AutoReceivedMessage ⇒ Destination(this, this) :: Nil
|
case _: AutoReceivedMessage ⇒ Destination(self, self) :: Nil
|
||||||
case Terminated(_) ⇒ Destination(this, this) :: Nil
|
case Terminated(_) ⇒ Destination(self, self) :: Nil
|
||||||
case CurrentRoutees ⇒
|
case CurrentRoutees ⇒
|
||||||
sender ! RouterRoutees(_routees)
|
sender ! RouterRoutees(_routees)
|
||||||
Nil
|
Nil
|
||||||
|
|
@ -122,7 +98,7 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
||||||
private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]): Unit = {
|
private[akka] def addRoutees(newRoutees: IndexedSeq[ActorRef]): Unit = {
|
||||||
_routees = _routees ++ newRoutees
|
_routees = _routees ++ newRoutees
|
||||||
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
|
// subscribe to Terminated messages for all route destinations, to be handled by Router actor
|
||||||
newRoutees foreach underlying.watch
|
newRoutees foreach watch
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -133,13 +109,13 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
||||||
*/
|
*/
|
||||||
private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]): Unit = {
|
private[akka] def removeRoutees(abandonedRoutees: IndexedSeq[ActorRef]): Unit = {
|
||||||
_routees = _routees diff abandonedRoutees
|
_routees = _routees diff abandonedRoutees
|
||||||
abandonedRoutees foreach underlying.unwatch
|
abandonedRoutees foreach unwatch
|
||||||
}
|
}
|
||||||
|
|
||||||
override def !(message: Any)(implicit sender: ActorRef = null): Unit = {
|
override def tell(message: Any, sender: ActorRef): Unit = {
|
||||||
resize()
|
resize()
|
||||||
|
|
||||||
val s = if (sender eq null) underlying.system.deadLetters else sender
|
val s = if (sender eq null) system.deadLetters else sender
|
||||||
|
|
||||||
val msg = message match {
|
val msg = message match {
|
||||||
case Broadcast(m) ⇒ m
|
case Broadcast(m) ⇒ m
|
||||||
|
|
@ -147,15 +123,18 @@ private[akka] class RoutedActorRef(_system: ActorSystemImpl, _props: Props, _sup
|
||||||
}
|
}
|
||||||
|
|
||||||
applyRoute(s, message) match {
|
applyRoute(s, message) match {
|
||||||
case Destination(_, x) :: Nil if x eq this ⇒ super.!(message)(s)
|
case Destination(_, x) :: Nil if x == self ⇒ super.tell(message, s)
|
||||||
case refs ⇒ refs foreach (p ⇒ p.recipient.!(msg)(p.sender))
|
case refs ⇒
|
||||||
|
refs foreach (p ⇒
|
||||||
|
if (p.recipient == self) super.tell(msg, p.sender)
|
||||||
|
else p.recipient.!(msg)(p.sender))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
def resize(): Unit = {
|
def resize(): Unit = {
|
||||||
for (r ← routerConfig.resizer) {
|
for (r ← routerConfig.resizer) {
|
||||||
if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeInProgress.compareAndSet(false, true))
|
if (r.isTimeForResize(resizeCounter.getAndIncrement()) && resizeInProgress.compareAndSet(false, true))
|
||||||
super.!(Router.Resize)
|
super.tell(Router.Resize, self)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -212,6 +191,11 @@ trait RouterConfig {
|
||||||
*/
|
*/
|
||||||
def resizer: Option[Resizer] = None
|
def resizer: Option[Resizer] = None
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Check that everything is there which is needed. Called in constructor of RoutedActorRef to fail early.
|
||||||
|
*/
|
||||||
|
def verifyConfig(): Unit = {}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -227,7 +211,7 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
|
||||||
* Not thread safe, but intended to be called from protected points, such as
|
* Not thread safe, but intended to be called from protected points, such as
|
||||||
* `RouterConfig.createRoute` and `Resizer.resize`.
|
* `RouterConfig.createRoute` and `Resizer.resize`.
|
||||||
*/
|
*/
|
||||||
def registerRoutees(routees: IndexedSeq[ActorRef]): Unit = routedRef.addRoutees(routees)
|
def registerRoutees(routees: IndexedSeq[ActorRef]): Unit = routedCell.addRoutees(routees)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Adds the routees to the router.
|
* Adds the routees to the router.
|
||||||
|
|
@ -247,7 +231,7 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
|
||||||
* Not thread safe, but intended to be called from protected points, such as
|
* Not thread safe, but intended to be called from protected points, such as
|
||||||
* `Resizer.resize`.
|
* `Resizer.resize`.
|
||||||
*/
|
*/
|
||||||
def unregisterRoutees(routees: IndexedSeq[ActorRef]): Unit = routedRef.removeRoutees(routees)
|
def unregisterRoutees(routees: IndexedSeq[ActorRef]): Unit = routedCell.removeRoutees(routees)
|
||||||
|
|
||||||
def createRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): IndexedSeq[ActorRef] =
|
def createRoutees(props: Props, nrOfInstances: Int, routees: Iterable[String]): IndexedSeq[ActorRef] =
|
||||||
(nrOfInstances, routees) match {
|
(nrOfInstances, routees) match {
|
||||||
|
|
@ -264,9 +248,9 @@ class RouteeProvider(val context: ActorContext, val resizer: Option[Resizer]) {
|
||||||
/**
|
/**
|
||||||
* All routees of the router
|
* All routees of the router
|
||||||
*/
|
*/
|
||||||
def routees: IndexedSeq[ActorRef] = routedRef.routees
|
def routees: IndexedSeq[ActorRef] = routedCell.routees
|
||||||
|
|
||||||
private def routedRef = context.self.asInstanceOf[RoutedActorRef]
|
private def routedCell = context.asInstanceOf[RoutedActorCell]
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -298,12 +282,9 @@ trait CustomRoute {
|
||||||
*/
|
*/
|
||||||
trait Router extends Actor {
|
trait Router extends Actor {
|
||||||
|
|
||||||
// make sure that we synchronize properly to get the childrenRefs into our CPU cache
|
val ref = context match {
|
||||||
val ref = context.synchronized {
|
case x: RoutedActorCell ⇒ x
|
||||||
self match {
|
case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef, not in " + context.getClass)
|
||||||
case x: RoutedActorRef ⇒ x
|
|
||||||
case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef")
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final def receive = ({
|
final def receive = ({
|
||||||
|
|
@ -417,8 +398,10 @@ class FromConfig(val routerDispatcher: String = Dispatchers.DefaultDispatcherId)
|
||||||
|
|
||||||
def this() = this(Dispatchers.DefaultDispatcherId)
|
def this() = this(Dispatchers.DefaultDispatcherId)
|
||||||
|
|
||||||
def createRoute(props: Props, routeeProvider: RouteeProvider): Route =
|
override def verifyConfig(): Unit =
|
||||||
throw new ConfigurationException("router " + routeeProvider.context.self + " needs external configuration from file (e.g. application.conf)")
|
throw new ConfigurationException("router needs external configuration from file (e.g. application.conf)")
|
||||||
|
|
||||||
|
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = null
|
||||||
|
|
||||||
def supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy
|
def supervisorStrategy: SupervisorStrategy = Router.defaultSupervisorStrategy
|
||||||
}
|
}
|
||||||
|
|
@ -774,9 +757,11 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||||
* routers based on mailbox and actor internal state.
|
* routers based on mailbox and actor internal state.
|
||||||
*/
|
*/
|
||||||
protected def isProcessingMessage(a: ActorRef): Boolean = a match {
|
protected def isProcessingMessage(a: ActorRef): Boolean = a match {
|
||||||
case x: LocalActorRef ⇒
|
case x: ActorRefWithCell ⇒
|
||||||
val cell = x.underlying
|
x.underlying match {
|
||||||
cell.mailbox.isScheduled && cell.currentMessage != null
|
case cell: ActorCell ⇒ cell.mailbox.isScheduled && cell.currentMessage != null
|
||||||
|
case _ ⇒ false
|
||||||
|
}
|
||||||
case _ ⇒ false
|
case _ ⇒ false
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -788,8 +773,8 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||||
* routers based on mailbox and actor internal state.
|
* routers based on mailbox and actor internal state.
|
||||||
*/
|
*/
|
||||||
protected def hasMessages(a: ActorRef): Boolean = a match {
|
protected def hasMessages(a: ActorRef): Boolean = a match {
|
||||||
case x: LocalActorRef ⇒ x.underlying.mailbox.hasMessages
|
case x: ActorRefWithCell ⇒ x.underlying.hasMessages
|
||||||
case _ ⇒ false
|
case _ ⇒ false
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -799,8 +784,12 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||||
* routers based on mailbox and actor internal state.
|
* routers based on mailbox and actor internal state.
|
||||||
*/
|
*/
|
||||||
protected def isSuspended(a: ActorRef): Boolean = a match {
|
protected def isSuspended(a: ActorRef): Boolean = a match {
|
||||||
case x: LocalActorRef ⇒ x.underlying.mailbox.isSuspended
|
case x: ActorRefWithCell ⇒
|
||||||
case _ ⇒ false
|
x.underlying match {
|
||||||
|
case cell: ActorCell ⇒ cell.mailbox.isSuspended
|
||||||
|
case _ ⇒ true
|
||||||
|
}
|
||||||
|
case _ ⇒ false
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -810,8 +799,8 @@ trait SmallestMailboxLike { this: RouterConfig ⇒
|
||||||
* routers based on mailbox and actor internal state.
|
* routers based on mailbox and actor internal state.
|
||||||
*/
|
*/
|
||||||
protected def numberOfMessages(a: ActorRef): Int = a match {
|
protected def numberOfMessages(a: ActorRef): Int = a match {
|
||||||
case x: LocalActorRef ⇒ x.underlying.mailbox.numberOfMessages
|
case x: ActorRefWithCell ⇒ x.underlying.numberOfMessages
|
||||||
case _ ⇒ 0
|
case _ ⇒ 0
|
||||||
}
|
}
|
||||||
|
|
||||||
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = {
|
def createRoute(props: Props, routeeProvider: RouteeProvider): Route = {
|
||||||
|
|
@ -1283,12 +1272,20 @@ case class DefaultResizer(
|
||||||
*/
|
*/
|
||||||
def pressure(routees: IndexedSeq[ActorRef]): Int = {
|
def pressure(routees: IndexedSeq[ActorRef]): Int = {
|
||||||
routees count {
|
routees count {
|
||||||
case a: LocalActorRef ⇒
|
case a: ActorRefWithCell ⇒
|
||||||
val cell = a.underlying
|
a.underlying match {
|
||||||
pressureThreshold match {
|
case cell: ActorCell ⇒
|
||||||
case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.hasMessages
|
pressureThreshold match {
|
||||||
case i if i < 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null
|
case 1 ⇒ cell.mailbox.isScheduled && cell.mailbox.hasMessages
|
||||||
case threshold ⇒ cell.mailbox.numberOfMessages >= threshold
|
case i if i < 1 ⇒ cell.mailbox.isScheduled && cell.currentMessage != null
|
||||||
|
case threshold ⇒ cell.mailbox.numberOfMessages >= threshold
|
||||||
|
}
|
||||||
|
case cell ⇒
|
||||||
|
pressureThreshold match {
|
||||||
|
case 1 ⇒ cell.hasMessages
|
||||||
|
case i if i < 1 ⇒ true // unstarted cells are always busy, for example
|
||||||
|
case threshold ⇒ cell.numberOfMessages >= threshold
|
||||||
|
}
|
||||||
}
|
}
|
||||||
case x ⇒
|
case x ⇒
|
||||||
false
|
false
|
||||||
|
|
|
||||||
|
|
@ -97,7 +97,7 @@ object Agent {
|
||||||
*/
|
*/
|
||||||
class Agent[T](initialValue: T, system: ActorSystem) {
|
class Agent[T](initialValue: T, system: ActorSystem) {
|
||||||
private val ref = Ref(initialValue)
|
private val ref = Ref(initialValue)
|
||||||
private val updater = system.actorOf(Props(new AgentUpdater(this, ref))).asInstanceOf[LocalActorRef] //TODO can we avoid this somehow?
|
private val updater = system.actorOf(Props(new AgentUpdater(this, ref))).asInstanceOf[InternalActorRef] //TODO can we avoid this somehow?
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Read the internal state of the agent.
|
* Read the internal state of the agent.
|
||||||
|
|
|
||||||
|
|
@ -11,7 +11,7 @@ import akka.actor.Props
|
||||||
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
|
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
|
||||||
import org.scalatest.matchers.MustMatchers
|
import org.scalatest.matchers.MustMatchers
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import akka.actor.Actor
|
import akka.actor.{ Actor, ExtendedActorSystem }
|
||||||
|
|
||||||
class MyActor extends Actor {
|
class MyActor extends Actor {
|
||||||
def receive = {
|
def receive = {
|
||||||
|
|
@ -56,20 +56,20 @@ import akka.util.duration._
|
||||||
class MyMailboxType(systemSettings: ActorSystem.Settings, config: Config)
|
class MyMailboxType(systemSettings: ActorSystem.Settings, config: Config)
|
||||||
extends MailboxType {
|
extends MailboxType {
|
||||||
|
|
||||||
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = owner zip system headOption match {
|
||||||
case Some(o) ⇒ new MyMessageQueue(o)
|
case Some((o, s: ExtendedActorSystem)) ⇒ new MyMessageQueue(o, s)
|
||||||
case None ⇒ throw new IllegalArgumentException(
|
case None ⇒ throw new IllegalArgumentException(
|
||||||
"requires an owner (i.e. does not work with BalancingDispatcher)")
|
"requires an owner (i.e. does not work with BalancingDispatcher)")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class MyMessageQueue(_owner: ActorContext)
|
class MyMessageQueue(_owner: ActorRef, _system: ExtendedActorSystem)
|
||||||
extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
extends DurableMessageQueue(_owner, _system) with DurableMessageSerialization {
|
||||||
|
|
||||||
val storage = new QueueStorage
|
val storage = new QueueStorage
|
||||||
// A real-world implmentation would use configuration to set the last
|
// A real-world implmentation would use configuration to set the last
|
||||||
// three parameters below
|
// three parameters below
|
||||||
val breaker = CircuitBreaker(_owner.system.scheduler, 5, 30.seconds, 1.minute)
|
val breaker = CircuitBreaker(system.scheduler, 5, 30.seconds, 1.minute)
|
||||||
|
|
||||||
def enqueue(receiver: ActorRef, envelope: Envelope): Unit = breaker.withSyncCircuitBreaker {
|
def enqueue(receiver: ActorRef, envelope: Envelope): Unit = breaker.withSyncCircuitBreaker {
|
||||||
val data: Array[Byte] = serialize(envelope)
|
val data: Array[Byte] = serialize(envelope)
|
||||||
|
|
@ -91,7 +91,7 @@ class MyMessageQueue(_owner: ActorContext)
|
||||||
* but the purpose of a durable mailbox is to continue
|
* but the purpose of a durable mailbox is to continue
|
||||||
* with the same message queue when the actor is started again.
|
* with the same message queue when the actor is started again.
|
||||||
*/
|
*/
|
||||||
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = ()
|
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = ()
|
||||||
|
|
||||||
}
|
}
|
||||||
//#custom-mailbox
|
//#custom-mailbox
|
||||||
|
|
|
||||||
|
|
@ -135,7 +135,7 @@ object DispatcherDocSpec {
|
||||||
|
|
||||||
//#mailbox-implementation-example
|
//#mailbox-implementation-example
|
||||||
case class MyUnboundedMailbox() extends akka.dispatch.MailboxType {
|
case class MyUnboundedMailbox() extends akka.dispatch.MailboxType {
|
||||||
import akka.actor.ActorContext
|
import akka.actor.{ ActorRef, ActorSystem }
|
||||||
import com.typesafe.config.Config
|
import com.typesafe.config.Config
|
||||||
import java.util.concurrent.ConcurrentLinkedQueue
|
import java.util.concurrent.ConcurrentLinkedQueue
|
||||||
import akka.dispatch.{
|
import akka.dispatch.{
|
||||||
|
|
@ -149,7 +149,7 @@ object DispatcherDocSpec {
|
||||||
def this(settings: ActorSystem.Settings, config: Config) = this()
|
def this(settings: ActorSystem.Settings, config: Config) = this()
|
||||||
|
|
||||||
// The create method is called to create the MessageQueue
|
// The create method is called to create the MessageQueue
|
||||||
final override def create(owner: Option[ActorContext]): MessageQueue =
|
final override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue =
|
||||||
new QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
new QueueBasedMessageQueue with UnboundedMessageQueueSemantics {
|
||||||
final val queue = new ConcurrentLinkedQueue[Envelope]()
|
final val queue = new ConcurrentLinkedQueue[Envelope]()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -13,20 +13,22 @@ import akka.actor.ActorSystem
|
||||||
import akka.dispatch._
|
import akka.dispatch._
|
||||||
import akka.util.{ Duration, NonFatal }
|
import akka.util.{ Duration, NonFatal }
|
||||||
import akka.pattern.{ CircuitBreakerOpenException, CircuitBreaker }
|
import akka.pattern.{ CircuitBreakerOpenException, CircuitBreaker }
|
||||||
|
import akka.actor.ExtendedActorSystem
|
||||||
|
|
||||||
class FileBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType {
|
class FileBasedMailboxType(systemSettings: ActorSystem.Settings, config: Config) extends MailboxType {
|
||||||
private val settings = new FileBasedMailboxSettings(systemSettings, config)
|
private val settings = new FileBasedMailboxSettings(systemSettings, config)
|
||||||
override def create(owner: Option[ActorContext]): MessageQueue = owner match {
|
override def create(owner: Option[ActorRef], system: Option[ActorSystem]): MessageQueue = owner zip system headOption match {
|
||||||
case Some(o) ⇒ new FileBasedMessageQueue(o, settings)
|
case Some((o, s: ExtendedActorSystem)) ⇒ new FileBasedMessageQueue(o, s, settings)
|
||||||
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
case None ⇒ throw new ConfigurationException("creating a durable mailbox requires an owner (i.e. does not work with BalancingDispatcher)")
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class FileBasedMessageQueue(_owner: ActorContext, val settings: FileBasedMailboxSettings) extends DurableMessageQueue(_owner) with DurableMessageSerialization {
|
class FileBasedMessageQueue(_owner: ActorRef, _system: ExtendedActorSystem, val settings: FileBasedMailboxSettings)
|
||||||
|
extends DurableMessageQueue(_owner, _system) with DurableMessageSerialization {
|
||||||
// TODO Is it reasonable for all FileBasedMailboxes to have their own logger?
|
// TODO Is it reasonable for all FileBasedMailboxes to have their own logger?
|
||||||
private val log = Logging(system, "FileBasedMessageQueue")
|
private val log = Logging(system, "FileBasedMessageQueue")
|
||||||
|
|
||||||
val breaker = CircuitBreaker(_owner.system.scheduler, settings.CircuitBreakerMaxFailures, settings.CircuitBreakerCallTimeout, settings.CircuitBreakerResetTimeout)
|
val breaker = CircuitBreaker(system.scheduler, settings.CircuitBreakerMaxFailures, settings.CircuitBreakerCallTimeout, settings.CircuitBreakerResetTimeout)
|
||||||
|
|
||||||
private val queue = try {
|
private val queue = try {
|
||||||
(new java.io.File(settings.QueuePath)) match {
|
(new java.io.File(settings.QueuePath)) match {
|
||||||
|
|
@ -79,5 +81,5 @@ class FileBasedMessageQueue(_owner: ActorContext, val settings: FileBasedMailbox
|
||||||
case NonFatal(_) ⇒ false
|
case NonFatal(_) ⇒ false
|
||||||
}
|
}
|
||||||
|
|
||||||
def cleanUp(owner: ActorContext, deadLetters: MessageQueue): Unit = ()
|
def cleanUp(owner: ActorRef, deadLetters: MessageQueue): Unit = ()
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -13,11 +13,10 @@ private[akka] object DurableExecutableMailboxConfig {
|
||||||
val Name = "[\\.\\/\\$\\s]".r
|
val Name = "[\\.\\/\\$\\s]".r
|
||||||
}
|
}
|
||||||
|
|
||||||
abstract class DurableMessageQueue(val owner: ActorContext) extends MessageQueue {
|
abstract class DurableMessageQueue(val owner: ActorRef, val system: ExtendedActorSystem) extends MessageQueue {
|
||||||
import DurableExecutableMailboxConfig._
|
import DurableExecutableMailboxConfig._
|
||||||
|
|
||||||
def system: ExtendedActorSystem = owner.system.asInstanceOf[ExtendedActorSystem]
|
def ownerPath: ActorPath = owner.path
|
||||||
def ownerPath: ActorPath = owner.self.path
|
|
||||||
val ownerPathString: String = ownerPath.elements.mkString("/")
|
val ownerPathString: String = ownerPath.elements.mkString("/")
|
||||||
val name: String = "mailbox_" + Name.replaceAllIn(ownerPathString, "_")
|
val name: String = "mailbox_" + Name.replaceAllIn(ownerPathString, "_")
|
||||||
|
|
||||||
|
|
@ -42,7 +41,7 @@ trait DurableMessageSerialization { this: DurableMessageQueue ⇒
|
||||||
val message = MessageSerializer.serialize(system, durableMessage.message.asInstanceOf[AnyRef])
|
val message = MessageSerializer.serialize(system, durableMessage.message.asInstanceOf[AnyRef])
|
||||||
val builder = RemoteMessageProtocol.newBuilder
|
val builder = RemoteMessageProtocol.newBuilder
|
||||||
.setMessage(message)
|
.setMessage(message)
|
||||||
.setRecipient(serializeActorRef(owner.self))
|
.setRecipient(serializeActorRef(owner))
|
||||||
.setSender(serializeActorRef(durableMessage.sender))
|
.setSender(serializeActorRef(durableMessage.sender))
|
||||||
|
|
||||||
builder.build.toByteArray
|
builder.build.toByteArray
|
||||||
|
|
@ -60,7 +59,7 @@ trait DurableMessageSerialization { this: DurableMessageQueue ⇒
|
||||||
val message = MessageSerializer.deserialize(system, durableMessage.getMessage)
|
val message = MessageSerializer.deserialize(system, durableMessage.getMessage)
|
||||||
val sender = deserializeActorRef(durableMessage.getSender)
|
val sender = deserializeActorRef(durableMessage.getSender)
|
||||||
|
|
||||||
Envelope(message, sender)(system)
|
Envelope(message, sender, system)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -3,25 +3,21 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor.mailbox
|
package akka.actor.mailbox
|
||||||
|
|
||||||
import DurableMailboxSpecActorFactory.AccumulatorActor
|
import java.io.InputStream
|
||||||
import DurableMailboxSpecActorFactory.MailboxTestActor
|
import java.util.concurrent.TimeoutException
|
||||||
import akka.actor.Actor
|
|
||||||
import akka.actor.ActorRef
|
import scala.annotation.tailrec
|
||||||
import akka.actor.ActorSystem
|
|
||||||
import akka.actor.LocalActorRef
|
import org.scalatest.{ WordSpec, BeforeAndAfterAll }
|
||||||
import akka.actor.Props
|
import org.scalatest.matchers.MustMatchers
|
||||||
import akka.actor.actorRef2Scala
|
|
||||||
|
import com.typesafe.config.{ ConfigFactory, Config }
|
||||||
|
|
||||||
|
import DurableMailboxSpecActorFactory.{ MailboxTestActor, AccumulatorActor }
|
||||||
|
import akka.actor.{ RepointableRef, Props, ActorSystem, ActorRefWithCell, ActorRef, ActorCell, Actor }
|
||||||
import akka.dispatch.Mailbox
|
import akka.dispatch.Mailbox
|
||||||
import akka.testkit.TestKit
|
import akka.testkit.TestKit
|
||||||
import akka.util.duration.intToDurationInt
|
import akka.util.duration.intToDurationInt
|
||||||
import com.typesafe.config.Config
|
|
||||||
import com.typesafe.config.ConfigFactory
|
|
||||||
import java.io.InputStream
|
|
||||||
import java.util.concurrent.TimeoutException
|
|
||||||
import org.scalatest.BeforeAndAfterAll
|
|
||||||
import org.scalatest.WordSpec
|
|
||||||
import org.scalatest.matchers.MustMatchers
|
|
||||||
import scala.annotation.tailrec
|
|
||||||
|
|
||||||
object DurableMailboxSpecActorFactory {
|
object DurableMailboxSpecActorFactory {
|
||||||
|
|
||||||
|
|
@ -115,9 +111,15 @@ abstract class DurableMailboxSpec(system: ActorSystem, val backendName: String)
|
||||||
if (!result.contains(words)) throw new Exception("stream did not contain '" + words + "':\n" + result)
|
if (!result.contains(words)) throw new Exception("stream did not contain '" + words + "':\n" + result)
|
||||||
}
|
}
|
||||||
|
|
||||||
def createMailboxTestActor(props: Props = Props[MailboxTestActor], id: String = ""): ActorRef = id match {
|
def createMailboxTestActor(props: Props = Props[MailboxTestActor], id: String = ""): ActorRef = {
|
||||||
case null | "" ⇒ system.actorOf(props.withDispatcher(backendName + "-dispatcher"))
|
val ref = id match {
|
||||||
case some ⇒ system.actorOf(props.withDispatcher(backendName + "-dispatcher"), some)
|
case null | "" ⇒ system.actorOf(props.withDispatcher(backendName + "-dispatcher"))
|
||||||
|
case some ⇒ system.actorOf(props.withDispatcher(backendName + "-dispatcher"), some)
|
||||||
|
}
|
||||||
|
awaitCond(ref match {
|
||||||
|
case r: RepointableRef ⇒ r.isStarted
|
||||||
|
}, 1 second, 10 millis)
|
||||||
|
ref
|
||||||
}
|
}
|
||||||
|
|
||||||
private def isDurableMailbox(m: Mailbox): Boolean =
|
private def isDurableMailbox(m: Mailbox): Boolean =
|
||||||
|
|
@ -127,9 +129,11 @@ abstract class DurableMailboxSpec(system: ActorSystem, val backendName: String)
|
||||||
|
|
||||||
"get a new, unique, durable mailbox" in {
|
"get a new, unique, durable mailbox" in {
|
||||||
val a1, a2 = createMailboxTestActor()
|
val a1, a2 = createMailboxTestActor()
|
||||||
isDurableMailbox(a1.asInstanceOf[LocalActorRef].underlying.mailbox) must be(true)
|
val mb1 = a1.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].mailbox
|
||||||
isDurableMailbox(a2.asInstanceOf[LocalActorRef].underlying.mailbox) must be(true)
|
val mb2 = a2.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].mailbox
|
||||||
(a1.asInstanceOf[LocalActorRef].underlying.mailbox ne a2.asInstanceOf[LocalActorRef].underlying.mailbox) must be(true)
|
isDurableMailbox(mb1) must be(true)
|
||||||
|
isDurableMailbox(mb2) must be(true)
|
||||||
|
(mb1 ne mb2) must be(true)
|
||||||
}
|
}
|
||||||
|
|
||||||
"deliver messages at most once" in {
|
"deliver messages at most once" in {
|
||||||
|
|
@ -148,7 +152,7 @@ abstract class DurableMailboxSpec(system: ActorSystem, val backendName: String)
|
||||||
"support having multiple actors at the same time" in {
|
"support having multiple actors at the same time" in {
|
||||||
val actors = Vector.fill(3)(createMailboxTestActor(Props[AccumulatorActor]))
|
val actors = Vector.fill(3)(createMailboxTestActor(Props[AccumulatorActor]))
|
||||||
|
|
||||||
actors foreach { a ⇒ isDurableMailbox(a.asInstanceOf[LocalActorRef].underlying.mailbox) must be(true) }
|
actors foreach { a ⇒ isDurableMailbox(a.asInstanceOf[ActorRefWithCell].underlying.asInstanceOf[ActorCell].mailbox) must be(true) }
|
||||||
|
|
||||||
val msgs = 1 to 3
|
val msgs = 1 to 3
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -96,8 +96,8 @@ private[akka] class RemoteActorRefProvider(
|
||||||
}
|
}
|
||||||
|
|
||||||
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
|
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
|
||||||
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean): InternalActorRef = {
|
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = {
|
||||||
if (systemService) local.actorOf(system, props, supervisor, path, systemService, deploy, lookupDeploy)
|
if (systemService) local.actorOf(system, props, supervisor, path, systemService, deploy, lookupDeploy, async)
|
||||||
else {
|
else {
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
|
@ -155,14 +155,14 @@ private[akka] class RemoteActorRefProvider(
|
||||||
Iterator(props.deploy) ++ deployment.iterator reduce ((a, b) ⇒ b withFallback a) match {
|
Iterator(props.deploy) ++ deployment.iterator reduce ((a, b) ⇒ b withFallback a) match {
|
||||||
case d @ Deploy(_, _, _, RemoteScope(addr)) ⇒
|
case d @ Deploy(_, _, _, RemoteScope(addr)) ⇒
|
||||||
if (addr == rootPath.address || addr == transport.address) {
|
if (addr == rootPath.address || addr == transport.address) {
|
||||||
local.actorOf(system, props, supervisor, path, false, deployment.headOption, false)
|
local.actorOf(system, props, supervisor, path, false, deployment.headOption, false, async)
|
||||||
} else {
|
} else {
|
||||||
val rpath = RootActorPath(addr) / "remote" / transport.address.hostPort / path.elements
|
val rpath = RootActorPath(addr) / "remote" / transport.address.hostPort / path.elements
|
||||||
useActorOnNode(rpath, props, d, supervisor)
|
useActorOnNode(rpath, props, d, supervisor)
|
||||||
new RemoteActorRef(this, transport, rpath, supervisor)
|
new RemoteActorRef(this, transport, rpath, supervisor)
|
||||||
}
|
}
|
||||||
|
|
||||||
case _ ⇒ local.actorOf(system, props, supervisor, path, systemService, deployment.headOption, false)
|
case _ ⇒ local.actorOf(system, props, supervisor, path, systemService, deployment.headOption, false, async)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -5,10 +5,11 @@
|
||||||
package akka.remote
|
package akka.remote
|
||||||
|
|
||||||
import scala.annotation.tailrec
|
import scala.annotation.tailrec
|
||||||
|
|
||||||
import akka.actor.{ VirtualPathContainer, Terminated, Deploy, Props, Nobody, LocalActorRef, InternalActorRef, Address, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor }
|
import akka.actor.{ VirtualPathContainer, Terminated, Deploy, Props, Nobody, LocalActorRef, InternalActorRef, Address, ActorSystemImpl, ActorRef, ActorPathExtractor, ActorPath, Actor }
|
||||||
import akka.event.LoggingAdapter
|
import akka.event.LoggingAdapter
|
||||||
import akka.dispatch.Watch
|
import akka.dispatch.Watch
|
||||||
|
import akka.actor.ActorRefWithCell
|
||||||
|
import akka.actor.ActorRefScope
|
||||||
|
|
||||||
private[akka] sealed trait DaemonMsg
|
private[akka] sealed trait DaemonMsg
|
||||||
private[akka] case class DaemonMsgCreate(props: Props, deploy: Deploy, path: String, supervisor: ActorRef) extends DaemonMsg
|
private[akka] case class DaemonMsgCreate(props: Props, deploy: Deploy, path: String, supervisor: ActorRef) extends DaemonMsg
|
||||||
|
|
@ -60,7 +61,7 @@ private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath
|
||||||
val subpath = elems.drop(1)
|
val subpath = elems.drop(1)
|
||||||
val path = this.path / subpath
|
val path = this.path / subpath
|
||||||
val actor = system.provider.actorOf(system, props, supervisor.asInstanceOf[InternalActorRef],
|
val actor = system.provider.actorOf(system, props, supervisor.asInstanceOf[InternalActorRef],
|
||||||
path, false, Some(deploy), true)
|
path, systemService = false, Some(deploy), lookupDeploy = true, async = false)
|
||||||
addChild(subpath.mkString("/"), actor)
|
addChild(subpath.mkString("/"), actor)
|
||||||
this.sendSystemMessage(Watch(actor, this))
|
this.sendSystemMessage(Watch(actor, this))
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
|
|
@ -68,11 +69,12 @@ private[akka] class RemoteSystemDaemon(system: ActorSystemImpl, _path: ActorPath
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
case Terminated(child: LocalActorRef) ⇒ removeChild(child.path.elements.drop(1).mkString("/"))
|
case Terminated(child: ActorRefWithCell) if child.asInstanceOf[ActorRefScope].isLocal ⇒
|
||||||
|
removeChild(child.path.elements.drop(1).mkString("/"))
|
||||||
|
|
||||||
case t: Terminated ⇒
|
case t: Terminated ⇒
|
||||||
|
|
||||||
case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this)
|
case unknown ⇒ log.warning("Unknown message {} received by {}", unknown, this)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -269,14 +269,14 @@ abstract class RemoteTransport(val system: ExtendedActorSystem, val provider: Re
|
||||||
}
|
}
|
||||||
case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender)
|
case x ⇒ log.warning("remoteDaemon received illegal message {} from {}", x, remoteMessage.sender)
|
||||||
}
|
}
|
||||||
case l: LocalRef ⇒
|
case l @ (_: LocalRef | _: RepointableRef) if l.isLocal ⇒
|
||||||
if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage)
|
if (provider.remoteSettings.LogReceive) log.debug("received local message {}", remoteMessage)
|
||||||
remoteMessage.payload match {
|
remoteMessage.payload match {
|
||||||
case msg: PossiblyHarmful if useUntrustedMode ⇒ log.warning("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type {}", msg.getClass)
|
case msg: PossiblyHarmful if useUntrustedMode ⇒ log.warning("operating in UntrustedMode, dropping inbound PossiblyHarmful message of type {}", msg.getClass)
|
||||||
case msg: SystemMessage ⇒ l.sendSystemMessage(msg)
|
case msg: SystemMessage ⇒ l.sendSystemMessage(msg)
|
||||||
case msg ⇒ l.!(msg)(remoteMessage.sender)
|
case msg ⇒ l.!(msg)(remoteMessage.sender)
|
||||||
}
|
}
|
||||||
case r: RemoteRef ⇒
|
case r @ (_: RemoteRef | _: RepointableRef) if !r.isLocal ⇒
|
||||||
if (provider.remoteSettings.LogReceive) log.debug("received remote-destined message {}", remoteMessage)
|
if (provider.remoteSettings.LogReceive) log.debug("received remote-destined message {}", remoteMessage)
|
||||||
remoteMessage.originalReceiver match {
|
remoteMessage.originalReceiver match {
|
||||||
case AddressFromURIString(address) if address == provider.transport.address ⇒
|
case AddressFromURIString(address) if address == provider.transport.address ⇒
|
||||||
|
|
@ -284,7 +284,7 @@ abstract class RemoteTransport(val system: ExtendedActorSystem, val provider: Re
|
||||||
r.!(remoteMessage.payload)(remoteMessage.sender)
|
r.!(remoteMessage.payload)(remoteMessage.sender)
|
||||||
case r ⇒ log.error("dropping message {} for non-local recipient {} arriving at {} inbound address is {}", remoteMessage.payload, r, address, provider.transport.address)
|
case r ⇒ log.error("dropping message {} for non-local recipient {} arriving at {} inbound address is {}", remoteMessage.payload, r, address, provider.transport.address)
|
||||||
}
|
}
|
||||||
case r ⇒ log.error("dropping message {} for non-local recipient {} arriving at {} inbound address is {}", remoteMessage.payload, r, address, provider.transport.address)
|
case r ⇒ log.error("dropping message {} for unknown recipient {} arriving at {} inbound address is {}", remoteMessage.payload, r, address, provider.transport.address)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -71,7 +71,8 @@ class RemoteRouteeProvider(nodes: Iterable[Address], _context: ActorContext, _re
|
||||||
IndexedSeq.empty[ActorRef] ++ (for (i ← 1 to nrOfInstances) yield {
|
IndexedSeq.empty[ActorRef] ++ (for (i ← 1 to nrOfInstances) yield {
|
||||||
val name = "c" + i
|
val name = "c" + i
|
||||||
val deploy = Deploy("", ConfigFactory.empty(), props.routerConfig, RemoteScope(nodeAddressIter.next))
|
val deploy = Deploy("", ConfigFactory.empty(), props.routerConfig, RemoteScope(nodeAddressIter.next))
|
||||||
impl.provider.actorOf(impl, props, context.self.asInstanceOf[InternalActorRef], context.self.path / name, false, Some(deploy), false)
|
impl.provider.actorOf(impl, props, context.self.asInstanceOf[InternalActorRef], context.self.path / name,
|
||||||
|
systemService = false, Some(deploy), lookupDeploy = false, async = false)
|
||||||
})
|
})
|
||||||
|
|
||||||
case (_, xs, _) ⇒ throw new ConfigurationException("Remote target.nodes can not be combined with routees for [%s]"
|
case (_, xs, _) ⇒ throw new ConfigurationException("Remote target.nodes can not be combined with routees for [%s]"
|
||||||
|
|
|
||||||
|
|
@ -118,7 +118,7 @@ akka {
|
||||||
val r = expectMsgType[ActorRef]
|
val r = expectMsgType[ActorRef]
|
||||||
r ! (Props[Echo], "grandchild")
|
r ! (Props[Echo], "grandchild")
|
||||||
val remref = expectMsgType[ActorRef]
|
val remref = expectMsgType[ActorRef]
|
||||||
remref.isInstanceOf[LocalActorRef] must be(true)
|
remref.asInstanceOf[ActorRefScope].isLocal must be(true)
|
||||||
val myref = system.actorFor(system / "looker" / "child" / "grandchild")
|
val myref = system.actorFor(system / "looker" / "child" / "grandchild")
|
||||||
myref.isInstanceOf[RemoteActorRef] must be(true)
|
myref.isInstanceOf[RemoteActorRef] must be(true)
|
||||||
myref ! 43
|
myref ! 43
|
||||||
|
|
|
||||||
|
|
@ -128,7 +128,7 @@ class CallingThreadDispatcher(
|
||||||
|
|
||||||
override def id: String = Id
|
override def id: String = Id
|
||||||
|
|
||||||
protected[akka] override def createMailbox(actor: ActorCell) = new CallingThreadMailbox(actor, mailboxType)
|
protected[akka] override def createMailbox(actor: akka.actor.Cell) = new CallingThreadMailbox(actor, mailboxType)
|
||||||
|
|
||||||
protected[akka] override def shutdown() {}
|
protected[akka] override def shutdown() {}
|
||||||
|
|
||||||
|
|
@ -281,17 +281,21 @@ class NestingQueue(val q: MessageQueue) {
|
||||||
def isActive = active
|
def isActive = active
|
||||||
}
|
}
|
||||||
|
|
||||||
class CallingThreadMailbox(_receiver: ActorCell, val mailboxType: MailboxType) extends Mailbox(_receiver, null) with DefaultSystemMessageQueue {
|
class CallingThreadMailbox(_receiver: akka.actor.Cell, val mailboxType: MailboxType)
|
||||||
|
extends Mailbox(null) with DefaultSystemMessageQueue {
|
||||||
|
|
||||||
|
val system = _receiver.system
|
||||||
|
val self = _receiver.self
|
||||||
|
|
||||||
private val q = new ThreadLocal[NestingQueue]() {
|
private val q = new ThreadLocal[NestingQueue]() {
|
||||||
override def initialValue = {
|
override def initialValue = {
|
||||||
val queue = new NestingQueue(mailboxType.create(Some(actor)))
|
val queue = new NestingQueue(mailboxType.create(Some(self), Some(system)))
|
||||||
CallingThreadDispatcherQueues(actor.system).registerQueue(CallingThreadMailbox.this, queue)
|
CallingThreadDispatcherQueues(system).registerQueue(CallingThreadMailbox.this, queue)
|
||||||
queue
|
queue
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
override def enqueue(receiver: ActorRef, msg: Envelope): Unit = throw new UnsupportedOperationException("CallingThreadMailbox cannot enqueue normally")
|
override def enqueue(receiver: ActorRef, msg: Envelope): Unit = q.get.q.enqueue(receiver, msg)
|
||||||
override def dequeue(): Envelope = throw new UnsupportedOperationException("CallingThreadMailbox cannot dequeue normally")
|
override def dequeue(): Envelope = throw new UnsupportedOperationException("CallingThreadMailbox cannot dequeue normally")
|
||||||
override def hasMessages: Boolean = q.get.q.hasMessages
|
override def hasMessages: Boolean = q.get.q.hasMessages
|
||||||
override def numberOfMessages: Int = 0
|
override def numberOfMessages: Int = 0
|
||||||
|
|
@ -311,7 +315,7 @@ class CallingThreadMailbox(_receiver: ActorCell, val mailboxType: MailboxType) e
|
||||||
val q = queue
|
val q = queue
|
||||||
CallingThreadDispatcherQueues(actor.system).gatherFromAllOtherQueues(this, q)
|
CallingThreadDispatcherQueues(actor.system).gatherFromAllOtherQueues(this, q)
|
||||||
super.cleanUp()
|
super.cleanUp()
|
||||||
q.q.cleanUp(actor, actor.systemImpl.deadLetterQueue)
|
q.q.cleanUp(actor.self, actor.systemImpl.deadLetterQueue)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -58,7 +58,7 @@ class TestActorRef[T <: Actor](
|
||||||
* become/unbecome.
|
* become/unbecome.
|
||||||
*/
|
*/
|
||||||
def receive(o: Any, sender: ActorRef): Unit = try {
|
def receive(o: Any, sender: ActorRef): Unit = try {
|
||||||
underlying.currentMessage = Envelope(o, if (sender eq null) underlying.system.deadLetters else sender)(underlying.system)
|
underlying.currentMessage = Envelope(o, if (sender eq null) underlying.system.deadLetters else sender, underlying.system)
|
||||||
underlying.receiveMessage(o)
|
underlying.receiveMessage(o)
|
||||||
} finally underlying.currentMessage = null
|
} finally underlying.currentMessage = null
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -97,9 +97,14 @@ trait TestKitBase {
|
||||||
*/
|
*/
|
||||||
lazy val testActor: ActorRef = {
|
lazy val testActor: ActorRef = {
|
||||||
val impl = system.asInstanceOf[ActorSystemImpl] //TODO ticket #1559
|
val impl = system.asInstanceOf[ActorSystemImpl] //TODO ticket #1559
|
||||||
impl.systemActorOf(Props(new TestActor(queue))
|
val ref = impl.systemActorOf(Props(new TestActor(queue))
|
||||||
.withDispatcher(CallingThreadDispatcher.Id),
|
.withDispatcher(CallingThreadDispatcher.Id),
|
||||||
"testActor" + TestKit.testActorId.incrementAndGet)
|
"testActor" + TestKit.testActorId.incrementAndGet)
|
||||||
|
awaitCond(ref match {
|
||||||
|
case r: RepointableRef ⇒ r.isStarted
|
||||||
|
case _ ⇒ true
|
||||||
|
}, 1 second, 10 millis)
|
||||||
|
ref
|
||||||
}
|
}
|
||||||
|
|
||||||
private var end: Duration = Duration.Undefined
|
private var end: Duration = Duration.Undefined
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue