Move hotswap stack into Actor trait. See #1717

This commit is contained in:
Patrik Nordwall 2012-01-30 13:27:48 +01:00
parent c1178c9b1a
commit d12d56a1ae
6 changed files with 78 additions and 23 deletions

View file

@ -0,0 +1,49 @@
import akka.actor._
import com.typesafe.config.ConfigFactory
import akka.event.LoggingReceive
import scala.annotation.tailrec
object Errtest extends App {
val config = ConfigFactory.parseString("""
akka.loglevel = DEBUG
akka.actor.debug {
receive = on
lifecycle = on
}
""")
val sys = ActorSystem("ErrSys", config)
val top = sys.actorOf(Props[Top], name = "top")
for (n 1 to 100) {
top ! "run " + n
Thread.sleep(1000)
}
}
class Top extends Actor {
var c: ActorRef = _
def receive = LoggingReceive {
case x
c = context.actorOf(Props[Child]);
c ! "ok"
}
}
class Child extends Actor {
//throw new Error("Simulated ERR")
blowUp(0)
//not @tailrec
private final def blowUp(n: Long): Long = {
blowUp(n + 1) + 1
}
def receive = LoggingReceive {
case x
//context.system.shutdown();
}
}

View file

@ -7,6 +7,7 @@ package akka.actor
import akka.AkkaException import akka.AkkaException
import scala.reflect.BeanProperty import scala.reflect.BeanProperty
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import scala.collection.immutable.Stack
import java.util.regex.Pattern import java.util.regex.Pattern
/** /**
@ -112,6 +113,8 @@ object Actor {
def isDefinedAt(x: Any) = false def isDefinedAt(x: Any) = false
def apply(x: Any) = throw new UnsupportedOperationException("Empty behavior apply()") def apply(x: Any) = throw new UnsupportedOperationException("Empty behavior apply()")
} }
private final val emptyBehaviourStack: Stack[Actor.Receive] = Stack.empty
} }
/** /**
@ -172,7 +175,7 @@ trait Actor {
type Receive = Actor.Receive type Receive = Actor.Receive
/** /**
* Stores the context for this actor, including self, sender, and hotswap. * Stores the context for this actor, including self, and sender.
* It is implicit to support operations such as `forward`. * It is implicit to support operations such as `forward`.
* *
* [[akka.actor.ActorContext]] is the Scala API. `getContext` returns a * [[akka.actor.ActorContext]] is the Scala API. `getContext` returns a
@ -282,7 +285,6 @@ trait Actor {
// ========================================= // =========================================
private[akka] final def apply(msg: Any) = { private[akka] final def apply(msg: Any) = {
val behaviorStack = context.asInstanceOf[ActorCell].hotswap
msg match { msg match {
case msg if behaviorStack.nonEmpty && behaviorStack.head.isDefinedAt(msg) behaviorStack.head.apply(msg) case msg if behaviorStack.nonEmpty && behaviorStack.head.isDefinedAt(msg) behaviorStack.head.apply(msg)
case msg if behaviorStack.isEmpty && processingBehavior.isDefinedAt(msg) processingBehavior.apply(msg) case msg if behaviorStack.isEmpty && processingBehavior.isDefinedAt(msg) processingBehavior.apply(msg)
@ -291,5 +293,18 @@ trait Actor {
} }
private[this] val processingBehavior = receive //ProcessingBehavior is the original behavior private[this] val processingBehavior = receive //ProcessingBehavior is the original behavior
private[akka] def pushBehavior(behavior: Receive): Unit = {
behaviorStack = behaviorStack.push(behavior)
}
private[akka] def popBehavior(): Unit = {
val stack = behaviorStack
if (stack.nonEmpty) behaviorStack = stack.pop
}
private[akka] def clearBehaviorStack(): Unit = { behaviorStack = emptyBehaviourStack }
private var behaviorStack: Stack[PartialFunction[Any, Unit]] = emptyBehaviourStack
} }

View file

@ -174,8 +174,7 @@ private[akka] class ActorCell(
val self: InternalActorRef, val self: InternalActorRef,
val props: Props, val props: Props,
@volatile var parent: InternalActorRef, @volatile var parent: InternalActorRef,
/*no member*/ _receiveTimeout: Option[Duration], /*no member*/ _receiveTimeout: Option[Duration]) extends UntypedActorContext {
var hotswap: Stack[PartialFunction[Any, Unit]]) extends UntypedActorContext {
import ActorCell._ import ActorCell._
@ -389,7 +388,6 @@ private[akka] class ActorCell(
} }
} }
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
hotswap = Props.noHotSwap // Reset the behavior
freshActor.postRestart(cause) freshActor.postRestart(cause)
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(freshActor), "restarted")) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(freshActor), "restarted"))
@ -509,9 +507,9 @@ private[akka] class ActorCell(
} }
} }
def become(behavior: Actor.Receive, discardOld: Boolean = true) { def become(behavior: Actor.Receive, discardOld: Boolean = true): Unit = {
if (discardOld) unbecome() if (discardOld) unbecome()
hotswap = hotswap.push(behavior) actor.pushBehavior(behavior)
} }
/** /**
@ -527,10 +525,7 @@ private[akka] class ActorCell(
become(newReceive, discardOld) become(newReceive, discardOld)
} }
def unbecome() { def unbecome(): Unit = actor.popBehavior()
val h = hotswap
if (h.nonEmpty) hotswap = h.pop
}
def autoReceiveMessage(msg: Envelope) { def autoReceiveMessage(msg: Envelope) {
if (system.settings.DebugAutoReceive) if (system.settings.DebugAutoReceive)
@ -547,9 +542,9 @@ private[akka] class ActorCell(
} }
private def doTerminate() { private def doTerminate() {
val a = actor
try { try {
try { try {
val a = actor
if (a ne null) a.postStop() if (a ne null) a.postStop()
} finally { } finally {
dispatcher.detach(this) dispatcher.detach(this)
@ -563,7 +558,7 @@ private[akka] class ActorCell(
} finally { } finally {
currentMessage = null currentMessage = null
clearActorFields() clearActorFields()
hotswap = Props.noHotSwap if (a ne null) a.clearBehaviorStack()
} }
} }
} }

View file

@ -224,8 +224,7 @@ private[akka] class LocalActorRef private[akka] (
_supervisor: InternalActorRef, _supervisor: InternalActorRef,
val path: ActorPath, val path: ActorPath,
val systemService: Boolean = false, val systemService: Boolean = false,
_receiveTimeout: Option[Duration] = None, _receiveTimeout: Option[Duration] = None)
_hotswap: Stack[PartialFunction[Any, Unit]] = Props.noHotSwap)
extends InternalActorRef with LocalRef { extends InternalActorRef with LocalRef {
/* /*
@ -238,7 +237,7 @@ private[akka] class LocalActorRef private[akka] (
* us to use purely factory methods for creating LocalActorRefs. * us to use purely factory methods for creating LocalActorRefs.
*/ */
@volatile @volatile
private var actorCell = newActorCell(_system, this, _props, _supervisor, _receiveTimeout, _hotswap) private var actorCell = newActorCell(_system, this, _props, _supervisor, _receiveTimeout)
actorCell.start() actorCell.start()
protected def newActorCell( protected def newActorCell(
@ -246,9 +245,8 @@ private[akka] class LocalActorRef private[akka] (
ref: InternalActorRef, ref: InternalActorRef,
props: Props, props: Props,
supervisor: InternalActorRef, supervisor: InternalActorRef,
receiveTimeout: Option[Duration], receiveTimeout: Option[Duration]): ActorCell =
hotswap: Stack[PartialFunction[Any, Unit]]): ActorCell = new ActorCell(system, ref, props, supervisor, receiveTimeout)
new ActorCell(system, ref, props, supervisor, receiveTimeout, hotswap)
protected def actorContext: ActorContext = actorCell protected def actorContext: ActorContext = actorCell

View file

@ -22,7 +22,6 @@ object Props {
final val defaultRoutedProps: RouterConfig = NoRouter final val defaultRoutedProps: RouterConfig = NoRouter
final val noHotSwap: Stack[Actor.Receive] = Stack.empty
final val empty = new Props(() new Actor { def receive = Actor.emptyBehavior }) final val empty = new Props(() new Actor { def receive = Actor.emptyBehavior })
/** /**

View file

@ -41,9 +41,8 @@ class TestActorRef[T <: Actor](
ref: InternalActorRef, ref: InternalActorRef,
props: Props, props: Props,
supervisor: InternalActorRef, supervisor: InternalActorRef,
receiveTimeout: Option[Duration], receiveTimeout: Option[Duration]): ActorCell =
hotswap: Stack[PartialFunction[Any, Unit]]): ActorCell = new ActorCell(system, ref, props, supervisor, receiveTimeout) {
new ActorCell(system, ref, props, supervisor, receiveTimeout, hotswap) {
override def autoReceiveMessage(msg: Envelope) { override def autoReceiveMessage(msg: Envelope) {
msg.message match { msg.message match {
case InternalGetActor sender ! actor case InternalGetActor sender ! actor