HUGE CHANGE - moving behaviorStack into ActorCell

This commit is contained in:
Viktor Klang 2012-05-21 13:47:48 +02:00
parent e14f9d0130
commit 26f6c48ae1
31 changed files with 71 additions and 90 deletions

View file

@ -393,7 +393,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
override def postRestart(reason: Throwable) = latch.countDown()
}))
protected def receive = { case "sendKill" ref ! Kill }
def receive = { case "sendKill" ref ! Kill }
}))
boss ! "sendKill"

View file

@ -131,7 +131,7 @@ class ActorWithStashSpec extends AkkaSpec(ActorWithStashSpec.testConf) with Defa
val hasMsgLatch = new TestLatch
val slaveProps = myProps(new Actor with Stash {
protected def receive = {
def receive = {
case "crash"
throw new Exception("Crashing...")

View file

@ -147,7 +147,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
object Hello
object Bye
val tester = system.actorOf(Props(new Actor {
protected def receive = {
def receive = {
case Hello lock ! "hello"
case "world" answerLatch.open
case Bye lock ! "bye"

View file

@ -22,7 +22,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
val timeoutActor = system.actorOf(Props(new Actor {
context.setReceiveTimeout(500 milliseconds)
protected def receive = {
def receive = {
case ReceiveTimeout timeoutLatch.open
}
}))
@ -38,7 +38,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
val timeoutActor = system.actorOf(Props(new Actor {
context.setReceiveTimeout(500 milliseconds)
protected def receive = {
def receive = {
case Tick ()
case ReceiveTimeout timeoutLatch.open
}
@ -58,7 +58,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
val timeoutActor = system.actorOf(Props(new Actor {
context.setReceiveTimeout(500 milliseconds)
protected def receive = {
def receive = {
case Tick ()
case ReceiveTimeout
count.incrementAndGet
@ -78,7 +78,7 @@ class ReceiveTimeoutSpec extends AkkaSpec {
val timeoutLatch = TestLatch()
val timeoutActor = system.actorOf(Props(new Actor {
protected def receive = {
def receive = {
case ReceiveTimeout timeoutLatch.open
}
}))

View file

@ -40,7 +40,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
val slaveProps = Props(new Actor {
protected def receive = {
def receive = {
case Ping countDownLatch.countDown()
case Crash throw new Exception("Crashing...")
}
@ -83,7 +83,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
val slaveProps = Props(new Actor {
protected def receive = {
def receive = {
case Crash throw new Exception("Crashing...")
}
@ -110,7 +110,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
val slaveProps = Props(new Actor {
protected def receive = {
def receive = {
case Ping
if (!pingLatch.isOpen) pingLatch.open else secondPingLatch.open
case Crash throw new Exception("Crashing...")
@ -166,7 +166,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
val slaveProps = Props(new Actor {
protected def receive = {
def receive = {
case Ping countDownLatch.countDown()
case Crash throw new Exception("Crashing...")
}
@ -221,7 +221,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
val slaveProps = Props(new Actor {
protected def receive = {
def receive = {
case Ping countDownLatch.countDown()
case Crash throw new Exception("Crashing...")
}

View file

@ -20,7 +20,7 @@ object SupervisorHierarchySpec {
*/
class CountDownActor(countDown: CountDownLatch, override val supervisorStrategy: SupervisorStrategy) extends Actor {
protected def receive = {
def receive = {
case p: Props sender ! context.actorOf(p)
}
// test relies on keeping children around during restart
@ -67,7 +67,7 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout {
val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, SupervisorStrategy.defaultStrategy))))
protected def receive = {
def receive = {
case "killCrasher" crasher ! Kill
case Terminated(_) countDownMax.countDown()
}

View file

@ -37,7 +37,7 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul
val workerProps = Props(new Actor {
override def postRestart(cause: Throwable) { countDownLatch.countDown() }
protected def receive = {
def receive = {
case "status" this.sender ! "OK"
case _ this.context.stop(self)
}

View file

@ -400,9 +400,9 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
val a = newTestActor(dispatcher.id)
val f1 = a ? Reply("foo")
val f2 = a ? Reply("bar")
val f3 = try { a ? Interrupt } catch { case ie: InterruptedException Promise.failed(ActorInterruptedException(ie)) }
val f3 = try { a ? Interrupt } catch { case ie: InterruptedException Promise.failed(new ActorInterruptedException(ie)) }
val f4 = a ? Reply("foo2")
val f5 = try { a ? Interrupt } catch { case ie: InterruptedException Promise.failed(ActorInterruptedException(ie)) }
val f5 = try { a ? Interrupt } catch { case ie: InterruptedException Promise.failed(new ActorInterruptedException(ie)) }
val f6 = a ? Reply("bar2")
assert(Await.result(f1, timeout.duration) === "foo")

View file

@ -152,7 +152,7 @@ case class DeathPactException private[akka] (dead: ActorRef)
* When an InterruptedException is thrown inside an Actor, it is wrapped as an ActorInterruptedException as to
* avoid cascading interrupts to other threads than the originally interrupted one.
*/
case class ActorInterruptedException private[akka] (cause: Throwable) extends AkkaException(cause.getMessage, cause) with NoStackTrace
class ActorInterruptedException private[akka] (cause: Throwable) extends AkkaException(cause.getMessage, cause) with NoStackTrace
/**
* This message is published to the EventStream whenever an Actor receives a message it doesn't understand
@ -173,6 +173,7 @@ object Status {
/**
* This class/message type is preferably used to indicate failure of some operation performed.
* As an example, it is used to signal failure with AskSupport is used (ask/?).
*/
case class Failure(cause: Throwable) extends Status
}
@ -317,7 +318,7 @@ trait Actor {
* This defines the initial actor behavior, it must return a partial function
* with the actor logic.
*/
protected def receive: Receive
def receive: Receive
/**
* User overridable definition the strategy to use for supervising
@ -377,45 +378,5 @@ trait Actor {
case _ context.system.eventStream.publish(UnhandledMessage(message, sender, self))
}
}
// =========================================
// ==== INTERNAL IMPLEMENTATION DETAILS ====
// =========================================
/**
* For Akka internal use only.
*/
private[akka] final def apply(msg: Any) = {
//FIXME replace with behaviorStack.head.applyOrElse(msg, unhandled) + "-optimize"
val head = behaviorStack.head
if (head.isDefinedAt(msg)) head.apply(msg) else unhandled(msg)
}
/**
* For Akka internal use only.
*/
private[akka] def pushBehavior(behavior: Receive): Unit = {
behaviorStack = behaviorStack.push(behavior)
}
/**
* For Akka internal use only.
*/
private[akka] def popBehavior(): Unit = {
val original = behaviorStack
val popped = original.pop
behaviorStack = if (popped.isEmpty) original else popped
}
/**
* For Akka internal use only.
*/
private[akka] def clearBehaviorStack(): Unit =
behaviorStack = Stack.empty[Receive].push(behaviorStack.last)
/**
* For Akka internal use only.
*/
private var behaviorStack: Stack[Receive] = Stack.empty[Receive].push(receive)
}

View file

@ -409,6 +409,8 @@ private[akka] class ActorCell(
var actor: Actor = _
private var behaviorStack: Stack[Actor.Receive] = Stack.empty
@volatile //This must be volatile since it isn't protected by the mailbox status
var mailbox: Mailbox = _
@ -489,8 +491,7 @@ private[akka] class ActorCell(
//This method is in charge of setting up the contextStack and create a new instance of the Actor
protected def newActor(): Actor = {
val stackBefore = contextStack.get
contextStack.set(stackBefore.push(this))
contextStack.set(contextStack.get.push(this))
try {
val instance = props.creator()
@ -511,6 +512,7 @@ private[akka] class ActorCell(
def create(): Unit = if (isNormal) {
try {
val created = newActor()
behaviorStack = Stack.empty.push(created.receive)
actor = created
created.preStart()
checkReceiveTimeout
@ -612,7 +614,7 @@ private[akka] class ActorCell(
cancelReceiveTimeout() // FIXME: leave this here???
messageHandle.message match {
case msg: AutoReceivedMessage autoReceiveMessage(messageHandle)
case msg actor(msg)
case msg receiveMessage(msg)
}
currentMessage = null // reset current message after successful invocation
} catch {
@ -628,14 +630,14 @@ private[akka] class ActorCell(
if (actor ne null) actor.supervisorStrategy.handleSupervisorFailing(self, children)
} finally {
t match { // Wrap InterruptedExceptions and rethrow
case _: InterruptedException parent.tell(Failed(ActorInterruptedException(t)), self); throw t
case _: InterruptedException parent.tell(Failed(new ActorInterruptedException(t)), self); throw t
case _ parent.tell(Failed(t), self)
}
}
def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit = {
if (discardOld) unbecome()
actor.pushBehavior(behavior)
behaviorStack = behaviorStack.push(behavior)
}
/**
@ -651,9 +653,13 @@ private[akka] class ActorCell(
become(newReceive, discardOld)
}
def unbecome(): Unit = actor.popBehavior()
def unbecome(): Unit = {
val original = behaviorStack
val popped = original.pop
behaviorStack = if (popped.isEmpty) original else popped
}
def autoReceiveMessage(msg: Envelope) {
def autoReceiveMessage(msg: Envelope): Unit = {
if (system.settings.DebugAutoReceive)
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "received AutoReceiveMessage " + msg))
@ -667,6 +673,12 @@ private[akka] class ActorCell(
}
}
final def receiveMessage(msg: Any): Unit = {
//FIXME replace with behaviorStack.head.applyOrElse(msg, unhandled) + "-optimize"
val head = behaviorStack.head
if (head.isDefinedAt(msg)) head.apply(msg) else actor.unhandled(msg)
}
private def doTerminate() {
val a = actor
try {
@ -682,7 +694,7 @@ private[akka] class ActorCell(
if (system.settings.DebugLifecycle)
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped"))
} finally {
if (a ne null) a.clearBehaviorStack()
behaviorStack = Stack.empty
clearActorFields(a)
actor = null
}
@ -694,6 +706,7 @@ private[akka] class ActorCell(
actor.supervisorStrategy.handleSupervisorRestarted(cause, self, children)
val freshActor = newActor()
behaviorStack = Stack.empty.push(freshActor.receive)
actor = freshActor // this must happen before postRestart has a chance to fail
if (freshActor eq failedActor) setActorFields(freshActor, this, self) // If the creator returns the same instance, we need to restore our nulled out fields.

View file

@ -15,7 +15,9 @@ object ActorPath {
}
/**
* This Regular Expression is used to validate a path element (Actor Name)
* This Regular Expression is used to validate a path element (Actor Name).
* Since Actors form a tree, it is addressable using an URL, therefor an Actor Name has to conform to:
* http://www.ietf.org/rfc/rfc2396.txt
*/
val ElementRegex = """[-\w:@&=+,.!~*'_;][-\w:@&=+,.!~*'$_;]*""".r
}

View file

@ -335,9 +335,10 @@ private[akka] class LocalActorRef private[akka] (
/**
* Memento pattern for serializing ActorRefs transparently
* INTERNAL API
*/
//TODO add @SerialVersionUID(1L) when SI-4804 is fixed
case class SerializedActorRef private (path: String) {
private[akka] case class SerializedActorRef private (path: String) {
import akka.serialization.JavaSerializer.currentSystem
@throws(classOf[java.io.ObjectStreamException])
@ -350,8 +351,11 @@ case class SerializedActorRef private (path: String) {
someSystem.actorFor(path)
}
}
//FIXME: Should SerializedActorRef be private[akka] ?
object SerializedActorRef {
/**
* INTERNAL API
*/
private[akka] object SerializedActorRef {
def apply(path: ActorPath): SerializedActorRef = {
Serialization.currentTransportAddress.value match {
case null new SerializedActorRef(path.toString)

View file

@ -437,7 +437,7 @@ trait FSM[S, D] extends Listeners with ActorLogging {
* Main actor receive() method
* *******************************************
*/
override final protected def receive: Receive = {
override final def receive: Receive = {
case TimeoutMarker(gen)
if (generation == gen) {
processMsg(StateTimeout, "state timeout")

View file

@ -176,9 +176,10 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
/**
* Sort so that subtypes always precede their supertypes, but without
* obeying any order between unrelated subtypes (insert sort).
*
* INTERNAL API
*/
//FIXME Should this really be public API?
def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] =
private[akka] def sort(in: Iterable[CauseDirective]): Seq[CauseDirective] =
(new ArrayBuffer[CauseDirective](in.size) /: in) { (buf, ca)
buf.indexWhere(_._1 isAssignableFrom ca._1) match {
case -1 buf append ca
@ -195,7 +196,7 @@ object SupervisorStrategy extends SupervisorStrategyLowPriorityImplicits {
}
/**
* An Akka SupervisorStrategy is
* An Akka SupervisorStrategy is the policy to apply for crashing children
*/
abstract class SupervisorStrategy {

View file

@ -987,7 +987,7 @@ final class IOManagerActor extends Actor with ActorLogging {
}
}
protected def receive = {
def receive = {
case Select
select()
if (running) self ! Select

View file

@ -236,7 +236,7 @@ private[akka] class ContinuousCancellable extends AtomicReference[HWTimeout](Con
def isCancelled(): Boolean = get match {
case null true
case some isCancelled()
case some some.isCancelled()
}
def cancel(): Unit =

View file

@ -156,7 +156,7 @@ abstract class UntypedActor extends Actor {
*/
override def postRestart(reason: Throwable): Unit = super.postRestart(reason)
final protected def receive = { case msg onReceive(msg) }
final def receive = { case msg onReceive(msg) }
}
/**

View file

@ -134,7 +134,7 @@ trait Producer extends ProducerSupport { this: Actor ⇒
* Default implementation of Actor.receive. Any messages received by this actors
* will be produced to the endpoint specified by <code>endpointUri</code>.
*/
protected def receive = produce
def receive = produce
}
/**

View file

@ -19,7 +19,7 @@ private[camelexamples] class SysOutConsumer extends Consumer {
override def activationTimeout = 10 seconds
def endpointUri = "file://data/input/CamelConsumer"
protected def receive = {
def receive = {
case msg: CamelMessage {
printf("Received '%s'\n", msg.bodyAs[String])
}
@ -30,12 +30,12 @@ private[camelexamples] class TroubleMaker extends Consumer {
def endpointUri = "WRONG URI"
println("Trying to instantiate conumer with uri: " + endpointUri)
protected def receive = { case _ }
def receive = { case _ }
}
private[camelexamples] class SysOutActor(implicit camel: Camel) extends Actor {
implicit val camelContext = camel.context
protected def receive = {
def receive = {
case msg: CamelMessage {
printf("Received '%s'\n", msg.bodyAs[String])
}

View file

@ -14,7 +14,7 @@ private[camelexamples] object SupervisedConsumersExample extends App {
system.actorOf(Props(new Actor {
context.watch(context.actorOf(Props[EndpointManager]))
protected def receive = {
def receive = {
case Terminated(ref) system.shutdown()
}
}))
@ -30,7 +30,7 @@ private[camelexamples] class EndpointManager extends Actor {
watch(actorOf(Props[SysOutConsumer]))
watch(actorOf(Props[TroubleMaker]))
protected def receive = {
def receive = {
case Terminated(ref) {
printf("Hey! One of the endpoints has died: %s. I am doing sepuku...\n", ref)
self ! PoisonPill

View file

@ -244,7 +244,7 @@ object ProducerFeatureTest {
}
class TestResponder extends Actor {
protected def receive = {
def receive = {
case msg: CamelMessage msg.body match {
case "fail" context.sender ! akka.actor.Status.Failure(new AkkaCamelException(new Exception("failure"), msg.headers))
case _
@ -256,7 +256,7 @@ object ProducerFeatureTest {
}
class ReplyingForwardTarget extends Actor {
protected def receive = {
def receive = {
case msg: CamelMessage
context.sender ! (msg.addHeader("test" -> "result"))
case msg: akka.actor.Status.Failure

View file

@ -334,7 +334,7 @@ trait ActorProducerFixture extends MockitoSugar with BeforeAndAfterAll with Befo
}
def echoActor = system.actorOf(Props(new Actor {
protected def receive = {
def receive = {
case msg sender ! "received " + msg
}
}))

View file

@ -443,7 +443,7 @@ An Actor has to implement the ``receive`` method to receive messages:
.. code-block:: scala
protected def receive: PartialFunction[Any, Unit]
def receive: PartialFunction[Any, Unit]
Note: Akka has an alias to the ``PartialFunction[Any, Unit]`` type called
``Receive`` (``akka.actor.Actor.Receive``), so you can use this type instead for

View file

@ -56,7 +56,7 @@ class TestActorRef[T <: Actor](
* thrown will be available to you, while still being able to use
* become/unbecome.
*/
def receive(o: Any) { underlyingActor.apply(o) }
def receive(o: Any): Unit = underlying.receiveMessage(o)
/**
* Retrieve reference to the underlying actor, where the static type matches the factory used inside the

View file

@ -138,7 +138,7 @@ class ConcurrentSocketActorSpec extends AkkaSpec {
}
}
protected def receive = {
def receive = {
case _
val payload = "%s".format(messageNumber)
messageNumber += 1

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.