From ceb6eec482b67e89360667faf882c579a82490a1 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 1 Aug 2012 14:17:09 +0200 Subject: [PATCH 01/16] randomize hierarchy setup and test Escalate, see #2336 - this uncovered a bug in clearing the isFailed state after an escalated Resume, which is fixed by keeping track of the perpetrator - also suppress ErrorLogs which are peripheral to the issue at hand --- .../akka/actor/SupervisorHierarchySpec.scala | 126 ++++++++++++------ .../main/scala/akka/actor/cell/Children.scala | 9 +- .../scala/akka/actor/cell/FaultHandling.scala | 19 +-- 3 files changed, 99 insertions(+), 55 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 7fde49c606..2c124546fe 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -53,12 +53,12 @@ object SupervisorHierarchySpec { } } + case class Children(refs: Vector[ActorRef]) case class Event(msg: Any) { val time: Long = System.nanoTime } case class ErrorLog(msg: String, log: Vector[Event]) - case class Failure(directive: Directive, log: Vector[Event]) extends RuntimeException with NoStackTrace { - override def toString = "Failure(" + directive + ")" + case class Failure(directive: Directive, log: Map[ActorRef, Vector[Event]], depth: Int) extends RuntimeException with NoStackTrace { + override def toString = "Failure(" + directive + ", " + depth + ")" } - val strategy = OneForOneStrategy() { case Failure(directive, _) ⇒ directive } val config = ConfigFactory.parseString(""" hierarchy { @@ -97,54 +97,86 @@ object SupervisorHierarchySpec { override def dispatcher(): MessageDispatcher = instance } - class Hierarchy(depth: Int, breadth: Int, listener: ActorRef) extends Actor { + class Hierarchy(size: Int, breadth: Int, listener: ActorRef) extends Actor { override def preStart { - if (depth > 1) - for (_ ← 1 to breadth) - context.watch(context.actorOf(Props(new Hierarchy(depth - 1, breadth, listener)).withDispatcher("hierarchy"))) - listener ! self + val s = size - 1 // subtract myself + if (s > 0) { + val kids = Random.nextInt(Math.min(breadth, s)) + 1 + val sizes = s / kids + var rest = s % kids + def kidSize = if (rest > 0) { rest -= 1; sizes + 1 } else sizes + for (_ ← 1 to kids) + context.watch(context.actorOf(Props(new Hierarchy(kidSize, breadth, listener)).withDispatcher("hierarchy"))) + } else context.parent ! Children(Vector(self)) } override def postRestart(cause: Throwable) { cause match { - case Failure(_, l) ⇒ log = l + case Failure(_, l, _) ⇒ log = l get self getOrElse Vector(Event("log lost")) } log :+= Event("restarted") } - override def supervisorStrategy = strategy + override val supervisorStrategy = OneForOneStrategy() { + case Failure(directive, _, 0) ⇒ + log :+= Event("applying " + directive + " to " + sender) + directive + case f @ Failure(directive, logs, x) ⇒ + import SupervisorStrategy._ + directive match { + case Restart ⇒ failed = true + case Resume ⇒ suspended = true + case _ ⇒ + } + log :+= Event("escalating " + f) + throw Failure(directive, logs + (self -> log), x - 1) + } override def preRestart(cause: Throwable, msg: Option[Any]): Unit = { // do not scrap children } override def postStop { - if (failed || suspended) { + if (failed || (suspended && !stopping)) { listener ! ErrorLog("not resumed (" + failed + ", " + suspended + ")", log) } } var failed = false var suspended = false + var stopping = false var log = Vector.empty[Event] - def check(msg: Any) = { + def check(msg: Any): Boolean = { suspended = false log :+= Event(msg) if (failed) { listener ! ErrorLog("processing message while failed", log) failed = false - context stop self - } + stop() + false + } else true } + def stop() { + stopping = true + context stop self + } + + var gotKidsFrom = Set.empty[ActorRef] + var kids = Vector.empty[ActorRef] + def receive = new Receive { val handler: Receive = { - case f @ Failure(Resume, _) ⇒ suspended = true; throw f.copy(log = log) - case f: Failure ⇒ failed = true; throw f.copy(log = log) - case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" - case Terminated(_) ⇒ listener ! ErrorLog("terminating", log); context stop self + case f @ Failure(Resume, _, _) ⇒ suspended = true; throw f.copy(log = Map(self -> log)) + case f: Failure ⇒ failed = true; throw f.copy(log = Map(self -> log)) + case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" + case Terminated(_) ⇒ listener ! ErrorLog("terminating", log); stop() + case Children(refs) ⇒ + kids ++= refs + gotKidsFrom += sender + if (context.children forall (gotKidsFrom contains)) context.parent ! Children(kids :+ self) } override def isDefinedAt(msg: Any) = handler.isDefinedAt(msg) - override def apply(msg: Any) = { check(msg); handler(msg) } + override def apply(msg: Any) = { if (check(msg)) handler(msg) } } } @@ -213,13 +245,6 @@ object SupervisorHierarchySpec { * all are terminated, transfer them to a WeakHashMap and verify that * they are indeed GCed * - * TODO RK: make hierarchy construction stochastic so that it includes - * different breadth (including the degenerate breadth-1 case). - * - * TODO RK: also test Escalate by adding an exception with a `var depth` - * which gets decremented within the supervisor and gets handled when zero - * is reached (Restart resolution) - * * TODO RK: also test exceptions during recreate * * TODO RK: also test recreate including terminating children @@ -230,7 +255,10 @@ object SupervisorHierarchySpec { class StressTest(testActor: ActorRef, depth: Int, breadth: Int) extends Actor with LoggingFSM[State, Null] { import context.system - override def supervisorStrategy = strategy + // don’t escalate from this one! + override val supervisorStrategy = OneForOneStrategy() { + case Failure(directive, _, _) ⇒ directive + } var children = Vector.empty[ActorRef] var idleChildren = Vector.empty[ActorRef] @@ -250,7 +278,6 @@ object SupervisorHierarchySpec { Fail(children(pick), if (x > 0.25) Restart else Resume) }) - val familySize = ((1 - BigInt(breadth).pow(depth)) / (1 - breadth)).toInt var hierarchy: ActorRef = _ override def preRestart(cause: Throwable, msg: Option[Any]) { @@ -269,34 +296,43 @@ object SupervisorHierarchySpec { when(Idle) { case Event(Init, _) ⇒ - hierarchy = context.watch(context.actorOf(Props(new Hierarchy(depth, breadth, self)).withDispatcher("hierarchy"))) + hierarchy = context.watch(context.actorOf(Props(new Hierarchy(size = 500, breadth = breadth, self)).withDispatcher("hierarchy"))) setTimer("phase", StateTimeout, 5 seconds, false) goto(Init) } when(Init) { - case Event(ref: ActorRef, _) ⇒ - if (idleChildren.nonEmpty || pingChildren.nonEmpty) - throw new IllegalStateException("received unexpected child " + children.size) - children :+= ref - if (children.size == familySize) { - idleChildren = children + case Event(Children(refs), _) ⇒ + children = refs + idleChildren = refs + if (children.toSet.size != children.size) { + testActor ! "children not unique" + stop() + } else { goto(Stress) - } else stay + } case Event(StateTimeout, _) ⇒ - testActor ! "only got %d out of %d refs".format(children.size, familySize) + testActor ! "did not get children list" stop() } onTransition { case Init -> Stress ⇒ - self ! Work(familySize * 1000) + self ! Work(500000) // set timeout for completion of the whole test (i.e. including Finishing and Stopping) setTimer("phase", StateTimeout, 30.seconds.dilated, false) } val workSchedule = 250.millis + private def randomDepth: Int = Random.nextFloat match { + case x if x > 0.1 ⇒ 0 + case x if x > 0.03 ⇒ 1 + case _ ⇒ 2 + } + + var ignoreNotResumedLogs = true + when(Stress) { case Event(w: Work, _) if idleChildren.isEmpty ⇒ context stop hierarchy @@ -304,7 +340,7 @@ object SupervisorHierarchySpec { case Event(Work(x), _) if x > 0 ⇒ nextJob.next match { case Ping(ref) ⇒ ref ! "ping" - case Fail(ref, dir) ⇒ ref ! Failure(dir, Vector.empty) + case Fail(ref, dir) ⇒ ref ! Failure(dir, Map.empty, randomDepth) } if (idleChildren.nonEmpty) self ! Work(x - 1) else context.system.scheduler.scheduleOnce(workSchedule, self, Work(x - 1)) @@ -338,7 +374,9 @@ object SupervisorHierarchySpec { } onTransition { - case _ -> Stopping ⇒ context stop hierarchy + case _ -> Stopping ⇒ + ignoreNotResumedLogs = false + context stop hierarchy } when(Stopping, stateTimeout = 5 seconds) { @@ -354,7 +392,8 @@ object SupervisorHierarchySpec { when(Failed, stateTimeout = 5 seconds) { case Event(e: ErrorLog, _) ⇒ - errors :+= sender -> e + if (!e.msg.startsWith("not resumed") || !ignoreNotResumedLogs) + errors :+= sender -> e stay case Event(Terminated(r), _) if r == hierarchy ⇒ printErrors() @@ -364,7 +403,8 @@ object SupervisorHierarchySpec { printErrors() testActor ! "timeout in Failed" stop - case Event("pong", _) ⇒ stay // don’t care? + case Event("pong", _) ⇒ stay // don’t care? + case Event(w: Work, _) ⇒ stay } def printErrors(): Unit = { @@ -494,7 +534,7 @@ class SupervisorHierarchySpec extends AkkaSpec(SupervisorHierarchySpec.config) w system.eventStream.publish(Mute(EventFilter[Failure]())) system.eventStream.publish(Mute(EventFilter.warning(start = "received dead letter"))) - val fsm = system.actorOf(Props(new StressTest(testActor, 6, 3)), "stressTest") + val fsm = system.actorOf(Props(new StressTest(testActor, depth = 6, breadth = 6)), "stressTest") fsm ! FSM.SubscribeTransitionCallBack(system.actorOf(Props(new Actor { def receive = { diff --git a/akka-actor/src/main/scala/akka/actor/cell/Children.scala b/akka-actor/src/main/scala/akka/actor/cell/Children.scala index eea7ed7508..6e77672f39 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/Children.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/Children.scala @@ -7,12 +7,12 @@ package akka.actor.cell import scala.annotation.tailrec import scala.collection.JavaConverters.asJavaIterableConverter import scala.util.control.NonFatal - import akka.actor.{ RepointableRef, Props, NoSerializationVerificationNeeded, InvalidActorNameException, InternalActorRef, ChildRestartStats, ActorRef } import akka.actor.ActorCell import akka.actor.ActorPath.ElementRegex import akka.serialization.SerializationExtension import akka.util.{ Unsafe, Helpers } +import akka.actor.ChildRestartStats private[akka] trait Children { this: ActorCell ⇒ @@ -119,8 +119,11 @@ private[akka] trait Children { this: ActorCell ⇒ case _ ⇒ } - protected def resumeChildren(): Unit = - childrenRefs.stats foreach (_.child.asInstanceOf[InternalActorRef].resume(inResponseToFailure = false)) + protected def resumeChildren(perp: ActorRef): Unit = + childrenRefs.stats foreach { + case ChildRestartStats(child: InternalActorRef, _, _) ⇒ + child.resume(inResponseToFailure = perp == child) + } def getChildByName(name: String): Option[ChildRestartStats] = childrenRefs.getByName(name) diff --git a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala index fce6a22e6f..bb6beebbce 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala @@ -36,10 +36,11 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ * a restart with dying children) * might well be replaced by ref to a Cancellable in the future (see #2299) */ - private var _failed = false - private def isFailed: Boolean = _failed - private def setFailed(): Unit = _failed = true - private def clearFailed(): Unit = _failed = false + private var _failed: ActorRef = null + private def isFailed: Boolean = _failed != null + private def setFailed(perpetrator: ActorRef): Unit = _failed = perpetrator + private def clearFailed(): Unit = _failed = null + private def perpetrator: ActorRef = _failed /** * Do re-create the actor in response to a failure. @@ -85,11 +86,12 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ * prompted this action. */ protected def faultResume(inResponseToFailure: Boolean): Unit = { + val perp = perpetrator // done always to keep that suspend counter balanced // must happen “atomically” try resumeNonRecursive() finally if (inResponseToFailure) clearFailed() - resumeChildren() + resumeChildren(perp) } protected def terminate() { @@ -106,7 +108,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ // do not process normal messages while waiting for all children to terminate suspendNonRecursive() // do not propagate failures during shutdown to the supervisor - setFailed() + setFailed(self) if (system.settings.DebugLifecycle) publish(Debug(self.path.toString, clazz(actor), "stopping")) } } else { @@ -120,11 +122,10 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ // prevent any further messages to be processed until the actor has been restarted if (!isFailed) try { suspendNonRecursive() - setFailed() // suspend children val skip: Set[ActorRef] = currentMessage match { - case Envelope(Failed(`t`), child) ⇒ Set(child) - case _ ⇒ Set.empty + case Envelope(Failed(_), child) ⇒ setFailed(child); Set(child) + case _ ⇒ setFailed(self); Set.empty } suspendChildren(skip) // tell supervisor From 5ffdd69bce367f431749239c26384c8c1dcfe26e Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 2 Aug 2012 10:23:28 +0200 Subject: [PATCH 02/16] verify preRestart is not called twice, see #2336 --- .../akka/actor/SupervisorHierarchySpec.scala | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 2c124546fe..fec850df33 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -131,8 +131,19 @@ object SupervisorHierarchySpec { log :+= Event("escalating " + f) throw Failure(directive, logs + (self -> log), x - 1) } + + def abort(msg: String) { + listener ! ErrorLog(msg, log) + stopping = true + context stop self + } + + var preRestartCalled = false override def preRestart(cause: Throwable, msg: Option[Any]): Unit = { // do not scrap children + if (preRestartCalled) abort("preRestart called twice") + log :+= Event("preRestart") + preRestartCalled = true } override def postStop { @@ -149,18 +160,12 @@ object SupervisorHierarchySpec { suspended = false log :+= Event(msg) if (failed) { - listener ! ErrorLog("processing message while failed", log) + abort("processing message while failed") failed = false - stop() false } else true } - def stop() { - stopping = true - context stop self - } - var gotKidsFrom = Set.empty[ActorRef] var kids = Vector.empty[ActorRef] @@ -169,7 +174,7 @@ object SupervisorHierarchySpec { case f @ Failure(Resume, _, _) ⇒ suspended = true; throw f.copy(log = Map(self -> log)) case f: Failure ⇒ failed = true; throw f.copy(log = Map(self -> log)) case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" - case Terminated(_) ⇒ listener ! ErrorLog("terminating", log); stop() + case Terminated(_) ⇒ abort("terminating") case Children(refs) ⇒ kids ++= refs gotKidsFrom += sender @@ -248,8 +253,6 @@ object SupervisorHierarchySpec { * TODO RK: also test exceptions during recreate * * TODO RK: also test recreate including terminating children - * - * TODO RK: also verify that preRestart is not called more than once per instance */ class StressTest(testActor: ActorRef, depth: Int, breadth: Int) extends Actor with LoggingFSM[State, Null] { From a19df590a70de9047a49ca7ecf64b1c70cf6e82e Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 2 Aug 2012 14:30:38 +0200 Subject: [PATCH 03/16] make actor-related exceptions be case classes for matching --- .../src/main/scala/akka/actor/Actor.scala | 63 ++++++++++++++----- .../src/main/scala/akka/actor/ActorCell.scala | 6 +- .../src/main/scala/akka/actor/Stash.scala | 2 +- .../src/main/scala/akka/routing/Routing.scala | 4 +- .../testkit/CallingThreadDispatcher.scala | 2 +- .../scala/akka/testkit/TestActorRef.scala | 2 +- 6 files changed, 54 insertions(+), 25 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index c1ae9c57bf..ea78bb705c 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -99,7 +99,7 @@ private[akka] case class SelectParent(next: Any) extends SelectionPath * IllegalActorStateException is thrown when a core invariant in the Actor implementation has been violated. * For instance, if you try to create an Actor that doesn't extend Actor. */ -class IllegalActorStateException private[akka] (message: String, cause: Throwable = null) +case class IllegalActorStateException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) { def this(msg: String) = this(msg, null) } @@ -107,7 +107,7 @@ class IllegalActorStateException private[akka] (message: String, cause: Throwabl /** * ActorKilledException is thrown when an Actor receives the akka.actor.Kill message */ -class ActorKilledException private[akka] (message: String, cause: Throwable) +case class ActorKilledException private[akka] (message: String, cause: Throwable) extends AkkaException(message, cause) with NoStackTrace { def this(msg: String) = this(msg, null) @@ -117,28 +117,43 @@ class ActorKilledException private[akka] (message: String, cause: Throwable) * An InvalidActorNameException is thrown when you try to convert something, usually a String, to an Actor name * which doesn't validate. */ -class InvalidActorNameException(message: String) extends AkkaException(message) +case class InvalidActorNameException(message: String) extends AkkaException(message) /** * An ActorInitializationException is thrown when the the initialization logic for an Actor fails. + * + * There is an extractor which works for ActorInitializationException and its subtypes: + * + * {{{ + * ex match { + * case ActorInitializationException(actor, message, cause) => ... + * } + * }}} */ -class ActorInitializationException private[akka] (val actor: ActorRef, message: String, cause: Throwable) +class ActorInitializationException protected (actor: ActorRef, message: String, cause: Throwable) extends AkkaException(message, cause) { - def this(msg: String) = this(null, msg, null) - def this(actor: ActorRef, msg: String) = this(actor, msg, null) + def getActor: ActorRef = actor +} +object ActorInitializationException { + private[akka] def apply(actor: ActorRef, message: String, cause: Throwable = null): ActorInitializationException = + new ActorInitializationException(actor, message, cause) + private[akka] def apply(message: String): ActorInitializationException = new ActorInitializationException(null, message, null) + def unapply(ex: ActorInitializationException): Option[(ActorRef, String, Throwable)] = Some((ex.getActor, ex.getMessage, ex.getCause)) } /** - * A PreRestartException is thrown when the preRestart() method failed. + * A PreRestartException is thrown when the preRestart() method failed; this + * exception is not propagated to the supervisor, as it originates from the + * already failed instance, hence it is only visible as log entry on the event + * stream. * * @param actor is the actor whose preRestart() hook failed * @param cause is the exception thrown by that actor within preRestart() * @param origCause is the exception which caused the restart in the first place * @param msg is the message which was optionally passed into preRestart() */ -class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, val origCause: Throwable, val msg: Option[Any]) - extends ActorInitializationException(actor, "exception in preRestart(" + origCause.getClass + ", " + msg.map(_.getClass) + ")", cause) { -} +case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, origCause: Throwable, msg: Option[Any]) + extends ActorInitializationException(actor, "exception in preRestart(" + origCause.getClass + ", " + msg.map(_.getClass) + ")", cause) /** * A PostRestartException is thrown when constructor or postRestart() method @@ -148,16 +163,30 @@ class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, val * @param cause is the exception thrown by that actor within preRestart() * @param origCause is the exception which caused the restart in the first place */ -class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, val origCause: Throwable) - extends ActorInitializationException(actor, "exception post restart (" + origCause.getClass + ")", cause) { +case class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, origCause: Throwable) + extends ActorInitializationException(actor, "exception post restart (" + origCause.getClass + ")", cause) + +/** + * This is an extractor for retrieving the original cause (i.e. the first + * failure) from a [[akka.actor.PostRestartException]]. In the face of multiple + * “nested” restarts it will walk the origCause-links until it arrives at a + * non-PostRestartException type. + */ +object OriginalRestartException { + def unapply(ex: PostRestartException): Option[Throwable] = { + def rec(ex: PostRestartException): Option[Throwable] = ex match { + case PostRestartException(_, _, e: PostRestartException) ⇒ rec(e) + case PostRestartException(_, _, e) ⇒ Some(e) + } + rec(ex) + } } /** - * InvalidMessageException is thrown when an invalid message is sent to an Actor. - * Technically it's only "null" which is an InvalidMessageException but who knows, - * there might be more of them in the future, or not. + * InvalidMessageException is thrown when an invalid message is sent to an Actor; + * Currently only `null` is an invalid message. */ -class InvalidMessageException private[akka] (message: String, cause: Throwable = null) +case class InvalidMessageException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) { def this(msg: String) = this(msg, null) } @@ -302,7 +331,7 @@ trait Actor { protected[akka] implicit val context: ActorContext = { val contextStack = ActorCell.contextStack.get if ((contextStack.isEmpty) || (contextStack.head eq null)) - throw new ActorInitializationException( + throw ActorInitializationException( "\n\tYou cannot create an instance of [" + getClass.getName + "] explicitly using the constructor (new)." + "\n\tYou have to use one of the factory methods to create a new actor. Either use:" + "\n\t\t'val actor = context.actorOf(Props[MyActor])' (to create a supervised child actor from within an actor), or" + diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 6650fffa94..020ea211fd 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -386,7 +386,7 @@ private[akka] class ActorCell( val instance = props.creator.apply() if (instance eq null) - throw new ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'") + throw ActorInitializationException(self, "Actor instance passed to actorOf can't be 'null'") // If no becomes were issued, the actors behavior is its receive method behaviorStack = if (behaviorStack.isEmpty) instance.receive :: behaviorStack else behaviorStack @@ -407,12 +407,12 @@ private[akka] class ActorCell( if (system.settings.DebugLifecycle) publish(Debug(self.path.toString, clazz(created), "started (" + created + ")")) } catch { case NonFatal(i: InstantiationException) ⇒ - throw new ActorInitializationException(self, + throw ActorInitializationException(self, """exception during creation, this problem is likely to occur because the class of the Actor you tried to create is either, a non-static inner class (in which case make it a static inner class or use Props(new ...) or Props( new UntypedActorFactory ... ) or is missing an appropriate, reachable no-args constructor. """, i.getCause) - case NonFatal(e) ⇒ throw new ActorInitializationException(self, "exception during creation", e) + case NonFatal(e) ⇒ throw ActorInitializationException(self, "exception during creation", e) } } diff --git a/akka-actor/src/main/scala/akka/actor/Stash.scala b/akka-actor/src/main/scala/akka/actor/Stash.scala index 2415b38618..7076a996df 100644 --- a/akka-actor/src/main/scala/akka/actor/Stash.scala +++ b/akka-actor/src/main/scala/akka/actor/Stash.scala @@ -69,7 +69,7 @@ trait Stash { private val mailbox: DequeBasedMessageQueue = { context.asInstanceOf[ActorCell].mailbox.messageQueue match { case queue: DequeBasedMessageQueue ⇒ queue - case other ⇒ throw new ActorInitializationException(self, "DequeBasedMailbox required, got: " + other.getClass() + """ + case other ⇒ throw ActorInitializationException(self, "DequeBasedMailbox required, got: " + other.getClass() + """ An (unbounded) deque-based mailbox can be configured as follows: my-custom-dispatcher { mailbox-type = "akka.dispatch.UnboundedDequeBasedMailbox" diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala index 09f1cf9bd8..7fd5470e4c 100644 --- a/akka-actor/src/main/scala/akka/routing/Routing.scala +++ b/akka-actor/src/main/scala/akka/routing/Routing.scala @@ -71,7 +71,7 @@ private[akka] class RoutedActorCell(_system: ActorSystemImpl, _ref: InternalActo } if (routerConfig.resizer.isEmpty && _routees.isEmpty) - throw new ActorInitializationException("router " + routerConfig + " did not register routees!") + throw ActorInitializationException("router " + routerConfig + " did not register routees!") start(sendSupervise = false) @@ -284,7 +284,7 @@ trait Router extends Actor { val ref = context match { case x: RoutedActorCell ⇒ x - case _ ⇒ throw new ActorInitializationException("Router actor can only be used in RoutedActorRef, not in " + context.getClass) + case _ ⇒ throw ActorInitializationException("Router actor can only be used in RoutedActorRef, not in " + context.getClass) } final def receive = ({ diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 26efaef84d..d25a156917 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -148,7 +148,7 @@ class CallingThreadDispatcher( val queue = mbox.queue queue.enter runQueue(mbox, queue) - case x ⇒ throw new ActorInitializationException("expected CallingThreadMailbox, got " + x.getClass) + case x ⇒ throw ActorInitializationException("expected CallingThreadMailbox, got " + x.getClass) } } diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index ba830b3855..d961a75fff 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -127,7 +127,7 @@ object TestActorRef { def apply[T <: Actor](name: String)(implicit t: ClassTag[T], system: ActorSystem): TestActorRef[T] = apply[T](Props({ system.asInstanceOf[ExtendedActorSystem].dynamicAccess.createInstanceFor[T](t.runtimeClass, Seq()) match { case Right(value) ⇒ value - case Left(exception) ⇒ throw new ActorInitializationException(null, + case Left(exception) ⇒ throw ActorInitializationException(null, "Could not instantiate Actor" + "\nMake sure Actor is NOT defined inside a class/trait," + "\nif so put it outside the class/trait, f.e. in a companion object," + From 2c9ddeb629a4ffcf29bb76b350fb1ff65c8a2b2b Mon Sep 17 00:00:00 2001 From: Roland Date: Thu, 2 Aug 2012 16:59:11 +0200 Subject: [PATCH 04/16] test exceptions during recreation, see #2336 fix two bugs: - resumeChildren should only check the perpetrator if inResponseToFailure is true - handleInvokeFailure must not suspend the survivors in case of an exception in postRestart --- .../akka/actor/SupervisorHierarchySpec.scala | 165 +++++++++++------- .../src/main/scala/akka/actor/ActorCell.scala | 4 +- .../main/scala/akka/actor/cell/Children.scala | 4 +- .../scala/akka/actor/cell/FaultHandling.scala | 48 ++--- .../main/scala/akka/dispatch/Mailbox.scala | 2 +- 5 files changed, 134 insertions(+), 89 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index fec850df33..b36bde739e 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -5,19 +5,15 @@ package akka.actor import language.postfixOps - import java.util.concurrent.{ TimeUnit, CountDownLatch } - import scala.concurrent.Await import scala.concurrent.util.Duration import scala.concurrent.util.duration.intToDurationInt import scala.math.BigInt.int2bigInt import scala.util.Random import scala.util.control.NoStackTrace - import com.typesafe.config.{ ConfigFactory, Config } - -import SupervisorStrategy.{ Resume, Restart, Directive } +import SupervisorStrategy.{ Resume, Restart, Stop, Directive } import akka.actor.SupervisorStrategy.seqThrowable2Decider import akka.dispatch.{ MessageDispatcher, DispatcherPrerequisites, DispatcherConfigurator, Dispatcher } import akka.pattern.ask @@ -56,9 +52,11 @@ object SupervisorHierarchySpec { case class Children(refs: Vector[ActorRef]) case class Event(msg: Any) { val time: Long = System.nanoTime } case class ErrorLog(msg: String, log: Vector[Event]) - case class Failure(directive: Directive, log: Map[ActorRef, Vector[Event]], depth: Int) extends RuntimeException with NoStackTrace { - override def toString = "Failure(" + directive + ", " + depth + ")" + case class Failure(directive: Directive, log: Map[ActorRef, Vector[Event]], depth: Int, var failPre: Int, var failPost: Int) + extends RuntimeException with NoStackTrace { + override def toString = "Failure(" + directive + ", " + depth + ", " + failPre + ", " + failPost + ")" } + case class Dump(level: Int) val config = ConfigFactory.parseString(""" hierarchy { @@ -82,14 +80,14 @@ object SupervisorHierarchySpec { override def suspend(cell: ActorCell): Unit = { val a = cell.actor.asInstanceOf[Hierarchy] - a.log :+= Event("suspended") + a.log :+= Event("suspended " + cell.mailbox.status / 4) super.suspend(cell) } override def resume(cell: ActorCell): Unit = { val a = cell.actor.asInstanceOf[Hierarchy] - a.log :+= Event("resumed") super.resume(cell) + a.log :+= Event("resumed " + cell.mailbox.status / 4) } } @@ -99,6 +97,24 @@ object SupervisorHierarchySpec { class Hierarchy(size: Int, breadth: Int, listener: ActorRef) extends Actor { + var failed = false + var suspended = false + var stopping = false + + def abort(msg: String) { + listener ! ErrorLog(msg, log) + stopping = true + context stop self + } + + def setFlags(directive: Directive): Unit = directive match { + case Restart ⇒ failed = true + case Resume ⇒ suspended = true + case _ ⇒ + } + + def suspendCount = context.asInstanceOf[ActorCell].mailbox.status / 4 + override def preStart { val s = size - 1 // subtract myself if (s > 0) { @@ -110,33 +126,6 @@ object SupervisorHierarchySpec { context.watch(context.actorOf(Props(new Hierarchy(kidSize, breadth, listener)).withDispatcher("hierarchy"))) } else context.parent ! Children(Vector(self)) } - override def postRestart(cause: Throwable) { - cause match { - case Failure(_, l, _) ⇒ log = l get self getOrElse Vector(Event("log lost")) - } - log :+= Event("restarted") - } - - override val supervisorStrategy = OneForOneStrategy() { - case Failure(directive, _, 0) ⇒ - log :+= Event("applying " + directive + " to " + sender) - directive - case f @ Failure(directive, logs, x) ⇒ - import SupervisorStrategy._ - directive match { - case Restart ⇒ failed = true - case Resume ⇒ suspended = true - case _ ⇒ - } - log :+= Event("escalating " + f) - throw Failure(directive, logs + (self -> log), x - 1) - } - - def abort(msg: String) { - listener ! ErrorLog(msg, log) - stopping = true - context stop self - } var preRestartCalled = false override def preRestart(cause: Throwable, msg: Option[Any]): Unit = { @@ -144,6 +133,37 @@ object SupervisorHierarchySpec { if (preRestartCalled) abort("preRestart called twice") log :+= Event("preRestart") preRestartCalled = true + cause match { + case f @ Failure(_, _, _, x, _) if x > 0 ⇒ f.failPre = x - 1; throw f + case _ ⇒ + } + } + + override val supervisorStrategy = OneForOneStrategy() { + case Failure(directive, _, 0, _, _) ⇒ + log :+= Event("applying " + directive + " to " + sender) + directive + case OriginalRestartException(f: Failure) ⇒ + log :+= Event("re-applying " + f.directive + " to " + sender) + f.directive + case f @ Failure(directive, logs, x, _, _) ⇒ + import SupervisorStrategy._ + setFlags(directive) + log :+= Event("escalating " + f) + throw f.copy(log = logs + (self -> log), depth = x - 1) + } + + override def postRestart(cause: Throwable) { + cause match { + case f: Failure ⇒ log = f.log get self getOrElse Vector(Event("log lost")) + case OriginalRestartException(f: Failure) ⇒ log = f.log get self getOrElse Vector(Event("log lost")) + case _ ⇒ + } + log :+= Event("restarted " + suspendCount) + cause match { + case f @ Failure(_, _, _, _, x) if x > 0 ⇒ f.failPost = x - 1; throw f + case _ ⇒ + } } override def postStop { @@ -152,9 +172,6 @@ object SupervisorHierarchySpec { } } - var failed = false - var suspended = false - var stopping = false var log = Vector.empty[Event] def check(msg: Any): Boolean = { suspended = false @@ -163,6 +180,9 @@ object SupervisorHierarchySpec { abort("processing message while failed") failed = false false + } else if (context.asInstanceOf[ActorCell].mailbox.isSuspended) { + abort("processing message while suspended") + false } else true } @@ -171,10 +191,11 @@ object SupervisorHierarchySpec { def receive = new Receive { val handler: Receive = { - case f @ Failure(Resume, _, _) ⇒ suspended = true; throw f.copy(log = Map(self -> log)) - case f: Failure ⇒ failed = true; throw f.copy(log = Map(self -> log)) - case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" - case Terminated(_) ⇒ abort("terminating") + case f: Failure ⇒ setFlags(f.directive); throw f.copy(log = Map(self -> log)) + case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" + case Dump(0) ⇒ context stop self + case Dump(level) ⇒ context.children foreach (_ ! Dump(level - 1)) + case Terminated(_) ⇒ abort("terminating") case Children(refs) ⇒ kids ++= refs gotKidsFrom += sender @@ -185,7 +206,7 @@ object SupervisorHierarchySpec { } } - case class Work(n: Int) + case object Work sealed trait Action case class Ping(ref: ActorRef) extends Action @@ -250,17 +271,16 @@ object SupervisorHierarchySpec { * all are terminated, transfer them to a WeakHashMap and verify that * they are indeed GCed * - * TODO RK: also test exceptions during recreate - * * TODO RK: also test recreate including terminating children */ - class StressTest(testActor: ActorRef, depth: Int, breadth: Int) extends Actor with LoggingFSM[State, Null] { + class StressTest(testActor: ActorRef, depth: Int, breadth: Int) extends Actor with LoggingFSM[State, Int] { import context.system // don’t escalate from this one! override val supervisorStrategy = OneForOneStrategy() { - case Failure(directive, _, _) ⇒ directive + case f: Failure ⇒ f.directive + case OriginalRestartException(f: Failure) ⇒ f.directive } var children = Vector.empty[ActorRef] @@ -295,7 +315,8 @@ object SupervisorHierarchySpec { testActor ! "stressTestStopped" } - startWith(Idle, null) + // number of Work packages to execute for the test + startWith(Idle, 500000) when(Idle) { case Event(Init, _) ⇒ @@ -321,14 +342,14 @@ object SupervisorHierarchySpec { onTransition { case Init -> Stress ⇒ - self ! Work(500000) + self ! Work // set timeout for completion of the whole test (i.e. including Finishing and Stopping) setTimer("phase", StateTimeout, 30.seconds.dilated, false) } val workSchedule = 250.millis - private def randomDepth: Int = Random.nextFloat match { + private def random012: Int = Random.nextFloat match { case x if x > 0.1 ⇒ 0 case x if x > 0.03 ⇒ 1 case _ ⇒ 2 @@ -337,22 +358,30 @@ object SupervisorHierarchySpec { var ignoreNotResumedLogs = true when(Stress) { - case Event(w: Work, _) if idleChildren.isEmpty ⇒ + case Event(Work, _) if idleChildren.isEmpty ⇒ + ignoreNotResumedLogs = false context stop hierarchy goto(Failed) - case Event(Work(x), _) if x > 0 ⇒ + case Event(Work, x) if x > 0 ⇒ nextJob.next match { case Ping(ref) ⇒ ref ! "ping" - case Fail(ref, dir) ⇒ ref ! Failure(dir, Map.empty, randomDepth) + case Fail(ref, dir) ⇒ ref ! Failure(dir, Map.empty, random012, random012, random012) } - if (idleChildren.nonEmpty) self ! Work(x - 1) - else context.system.scheduler.scheduleOnce(workSchedule, self, Work(x - 1)) - stay - case Event(Work(_), _) ⇒ if (pingChildren.isEmpty) goto(LastPing) else goto(Finishing) + if (idleChildren.nonEmpty) self ! Work + else context.system.scheduler.scheduleOnce(workSchedule, self, Work) + stay using (x - 1) + case Event(Work, _) ⇒ if (pingChildren.isEmpty) goto(LastPing) else goto(Finishing) case Event("pong", _) ⇒ pingChildren -= sender idleChildren :+= sender stay + case Event(StateTimeout, todo) ⇒ + log.info("dumping state due to StateTimeout") + log.info("children: " + children.size + " pinged: " + pingChildren.size + " idle: " + idleChildren.size + " work: " + todo) + println(system.asInstanceOf[ActorSystemImpl].printTree) + ignoreNotResumedLogs = false + hierarchy ! Dump(2) + goto(Failed) } when(Finishing) { @@ -403,11 +432,25 @@ object SupervisorHierarchySpec { testActor ! "stressTestFailed" stop case Event(StateTimeout, _) ⇒ + getErrors() printErrors() testActor ! "timeout in Failed" stop - case Event("pong", _) ⇒ stay // don’t care? - case Event(w: Work, _) ⇒ stay + case Event("pong", _) ⇒ stay // don’t care? + case Event(Work, _) ⇒ stay + } + + def getErrors() = { + def rec(target: ActorRef, depth: Int): Unit = { + target match { + case l: LocalActorRef ⇒ + errors :+= target -> ErrorLog("forced", l.underlying.actor.asInstanceOf[Hierarchy].log) + if (depth > 0) { + l.underlying.children foreach (rec(_, depth - 1)) + } + } + } + rec(hierarchy, 2) } def printErrors(): Unit = { @@ -534,7 +577,7 @@ class SupervisorHierarchySpec extends AkkaSpec(SupervisorHierarchySpec.config) w } "survive being stressed" in { - system.eventStream.publish(Mute(EventFilter[Failure]())) + system.eventStream.publish(Mute(EventFilter[Failure](), EventFilter[PreRestartException](), EventFilter[PostRestartException]())) system.eventStream.publish(Mute(EventFilter.warning(start = "received dead letter"))) val fsm = system.actorOf(Props(new StressTest(testActor, depth = 6, breadth = 6)), "stressTest") diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 020ea211fd..a5667f2634 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -314,7 +314,7 @@ private[akka] class ActorCell( case NoMessage ⇒ // only here to suppress warning } } catch { - case e @ (_: InterruptedException | NonFatal(_)) ⇒ handleInvokeFailure(e, "error while processing " + message) + case e @ (_: InterruptedException | NonFatal(_)) ⇒ handleInvokeFailure(Nil, e, "error while processing " + message) } //Memory consistency is handled by the Mailbox (reading mailbox status then processing messages, then writing mailbox status @@ -327,7 +327,7 @@ private[akka] class ActorCell( } currentMessage = null // reset current message after successful invocation } catch { - case e @ (_: InterruptedException | NonFatal(_)) ⇒ handleInvokeFailure(e, e.getMessage) + case e @ (_: InterruptedException | NonFatal(_)) ⇒ handleInvokeFailure(Nil, e, e.getMessage) } finally { checkReceiveTimeout // Reschedule receive timeout } diff --git a/akka-actor/src/main/scala/akka/actor/cell/Children.scala b/akka-actor/src/main/scala/akka/actor/cell/Children.scala index 6e77672f39..4716795c8b 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/Children.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/Children.scala @@ -119,10 +119,10 @@ private[akka] trait Children { this: ActorCell ⇒ case _ ⇒ } - protected def resumeChildren(perp: ActorRef): Unit = + protected def resumeChildren(inResponseToFailure: Boolean, perp: ActorRef): Unit = childrenRefs.stats foreach { case ChildRestartStats(child: InternalActorRef, _, _) ⇒ - child.resume(inResponseToFailure = perp == child) + child.resume(inResponseToFailure = inResponseToFailure && perp == child) } def getChildByName(name: String): Option[ChildRestartStats] = childrenRefs.getByName(name) diff --git a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala index bb6beebbce..9907f036a9 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala @@ -91,7 +91,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ // must happen “atomically” try resumeNonRecursive() finally if (inResponseToFailure) clearFailed() - resumeChildren(perp) + resumeChildren(inResponseToFailure, perp) } protected def terminate() { @@ -117,7 +117,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ } } - final def handleInvokeFailure(t: Throwable, message: String): Unit = { + final def handleInvokeFailure(childrenNotToSuspend: Iterable[ActorRef], t: Throwable, message: String): Unit = { publish(Error(t, self.path.toString, clazz(actor), message)) // prevent any further messages to be processed until the actor has been restarted if (!isFailed) try { @@ -127,7 +127,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ case Envelope(Failed(_), child) ⇒ setFailed(child); Set(child) case _ ⇒ setFailed(self); Set.empty } - suspendChildren(skip) + suspendChildren(skip ++ childrenNotToSuspend) // tell supervisor t match { // Wrap InterruptedExceptions and rethrow case _: InterruptedException ⇒ parent.tell(Failed(new ActorInterruptedException(t)), self); throw t @@ -156,30 +156,32 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ } } - private def finishRecreate(cause: Throwable, failedActor: Actor): Unit = try { - try resumeNonRecursive() - finally clearFailed() // must happen in any case, so that failure is propagated - + private def finishRecreate(cause: Throwable, failedActor: Actor): Unit = { // need to keep a snapshot of the surviving children before the new actor instance creates new ones val survivors = children - val freshActor = newActor() - 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. + try { + try resumeNonRecursive() + finally clearFailed() // must happen in any case, so that failure is propagated - freshActor.postRestart(cause) - if (system.settings.DebugLifecycle) publish(Debug(self.path.toString, clazz(freshActor), "restarted")) + val freshActor = newActor() + 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. - // only after parent is up and running again do restart the children which were not stopped - survivors foreach (child ⇒ - try child.asInstanceOf[InternalActorRef].restart(cause) - catch { - case NonFatal(e) ⇒ publish(Error(e, self.path.toString, clazz(freshActor), "restarting " + child)) - }) - } catch { - case NonFatal(e) ⇒ - clearActorFields(actor) // in order to prevent preRestart() from happening again - handleInvokeFailure(new PostRestartException(self, e, cause), e.getMessage) + freshActor.postRestart(cause) + if (system.settings.DebugLifecycle) publish(Debug(self.path.toString, clazz(freshActor), "restarted")) + + // only after parent is up and running again do restart the children which were not stopped + survivors foreach (child ⇒ + try child.asInstanceOf[InternalActorRef].restart(cause) + catch { + case NonFatal(e) ⇒ publish(Error(e, self.path.toString, clazz(freshActor), "restarting " + child)) + }) + } catch { + case NonFatal(e) ⇒ + clearActorFields(actor) // in order to prevent preRestart() from happening again + handleInvokeFailure(survivors, new PostRestartException(self, e, cause), e.getMessage) + } } final protected def handleFailure(child: ActorRef, cause: Throwable): Unit = getChildByRef(child) match { @@ -196,7 +198,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ */ try actor.supervisorStrategy.handleChildTerminated(this, child, children) catch { - case NonFatal(e) ⇒ handleInvokeFailure(e, "handleChildTerminated failed") + case NonFatal(e) ⇒ handleInvokeFailure(Nil, e, "handleChildTerminated failed") } /* * if the removal changed the state of the (terminating) children container, diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index f7062d5138..f1f887068b 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -271,7 +271,7 @@ private[akka] abstract class Mailbox(val messageQueue: MessageQueue) } } // if something happened while processing, fail this actor (most probable: exception in supervisorStrategy) - if (failure ne null) actor.handleInvokeFailure(failure, failure.getMessage) + if (failure ne null) actor.handleInvokeFailure(Nil, failure, failure.getMessage) } /** From 6aa6697fe5bcd498ced29f3013b82e8c8b20ab8c Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 2 Aug 2012 17:02:58 +0200 Subject: [PATCH 05/16] Initial IO work --- akka-actor/src/main/scala/akka/actor/IO.scala | 20 +++++++------------ 1 file changed, 7 insertions(+), 13 deletions(-) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 20bf78554c..de2b2a40d8 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -1037,14 +1037,10 @@ final class IOManagerActor extends Actor with ActorLogging { case IO.Write(handle, data) ⇒ if (channels contains handle) { - val queue = { - val existing = writes get handle - if (existing.isDefined) existing.get - else { - val q = new WriteBuffer(bufferSize) - writes update (handle, q) - q - } + val queue = writes get handle getOrElse { + val q = new WriteBuffer(bufferSize) + writes update (handle, q) + q } if (queue.isEmpty) addOps(handle, OP_WRITE) queue enqueue data @@ -1053,11 +1049,9 @@ final class IOManagerActor extends Actor with ActorLogging { run() case IO.Close(handle: IO.WriteHandle) ⇒ - if (writes get handle filterNot (_.isEmpty) isDefined) { - closing += handle - } else { - cleanup(handle, None) - } + //If we still have pending writes, add to set of closing handles + if (writes get handle exists (_.isEmpty == false)) closing += handle + else cleanup(handle, None) run() case IO.Close(handle) ⇒ From b8eb128fdaec8ec026abce467a1ee7ff279da89c Mon Sep 17 00:00:00 2001 From: Roland Date: Fri, 3 Aug 2012 14:19:42 +0200 Subject: [PATCH 06/16] =?UTF-8?q?clean=20up=20hierarchy=E2=80=99s=20log=20?= =?UTF-8?q?stashing=20in=20preparation=20for=20remaining=20TODOs?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../akka/actor/SupervisorHierarchySpec.scala | 85 ++++++++++++------- 1 file changed, 52 insertions(+), 33 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index b36bde739e..5e41955562 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -20,6 +20,7 @@ import akka.pattern.ask import akka.testkit.{ ImplicitSender, EventFilter, DefaultTimeout, AkkaSpec } import akka.testkit.{ filterException, duration2TestDuration, TestLatch } import akka.testkit.TestEvent.Mute +import java.util.concurrent.ConcurrentHashMap object SupervisorHierarchySpec { class FireWorkerException(msg: String) extends Exception(msg) @@ -52,9 +53,8 @@ object SupervisorHierarchySpec { case class Children(refs: Vector[ActorRef]) case class Event(msg: Any) { val time: Long = System.nanoTime } case class ErrorLog(msg: String, log: Vector[Event]) - case class Failure(directive: Directive, log: Map[ActorRef, Vector[Event]], depth: Int, var failPre: Int, var failPost: Int) - extends RuntimeException with NoStackTrace { - override def toString = "Failure(" + directive + ", " + depth + ", " + failPre + ", " + failPost + ")" + case class Failure(directive: Directive, depth: Int, var failPre: Int, var failPost: Int) extends RuntimeException with NoStackTrace { + override def toString = productPrefix + productIterator.mkString("(", ",", ")") } case class Dump(level: Int) @@ -95,15 +95,22 @@ object SupervisorHierarchySpec { override def dispatcher(): MessageDispatcher = instance } + /* + * This stores structural data of the hierarchy which would otherwise be lost + * upon Restart or would have to be managed by the highest supervisor (which + * is undesirable). + */ + case class HierarchyState(log: Vector[Event], kids: Map[String, Int]) + val stateCache = new ConcurrentHashMap[ActorRef, HierarchyState]() + class Hierarchy(size: Int, breadth: Int, listener: ActorRef) extends Actor { var failed = false var suspended = false - var stopping = false def abort(msg: String) { listener ! ErrorLog(msg, log) - stopping = true + log = Vector(Event("log sent")) context stop self } @@ -116,58 +123,68 @@ object SupervisorHierarchySpec { def suspendCount = context.asInstanceOf[ActorCell].mailbox.status / 4 override def preStart { + log :+= Event("started") val s = size - 1 // subtract myself - if (s > 0) { - val kids = Random.nextInt(Math.min(breadth, s)) + 1 - val sizes = s / kids - var rest = s % kids - def kidSize = if (rest > 0) { rest -= 1; sizes + 1 } else sizes - for (_ ← 1 to kids) - context.watch(context.actorOf(Props(new Hierarchy(kidSize, breadth, listener)).withDispatcher("hierarchy"))) - } else context.parent ! Children(Vector(self)) + val kidInfo: Map[String, Int] = + if (s > 0) { + val kids = Random.nextInt(Math.min(breadth, s)) + 1 + val sizes = s / kids + var rest = s % kids + val propsTemplate = Props.empty.withDispatcher("hierarchy") + (1 to kids).map { (id) ⇒ + val kidSize = if (rest > 0) { rest -= 1; sizes + 1 } else sizes + val props = propsTemplate.withCreator(new Hierarchy(kidSize, breadth, listener)) + val name = id.toString + context.watch(context.actorOf(props, name)) + (name, kidSize) + }(collection.breakOut) + } else { + context.parent ! Children(Vector(self)) + Map() + } + stateCache.put(self, HierarchyState(log, kidInfo)) } var preRestartCalled = false override def preRestart(cause: Throwable, msg: Option[Any]): Unit = { // do not scrap children if (preRestartCalled) abort("preRestart called twice") - log :+= Event("preRestart") - preRestartCalled = true - cause match { - case f @ Failure(_, _, _, x, _) if x > 0 ⇒ f.failPre = x - 1; throw f - case _ ⇒ + else { + log :+= Event("preRestart") + stateCache.put(self, stateCache.get(self).copy(log = log)) + preRestartCalled = true + cause match { + case f @ Failure(_, _, x, _) if x > 0 ⇒ f.failPre = x - 1; throw f + case _ ⇒ + } } } override val supervisorStrategy = OneForOneStrategy() { - case Failure(directive, _, 0, _, _) ⇒ + case Failure(directive, 0, _, _) ⇒ log :+= Event("applying " + directive + " to " + sender) directive case OriginalRestartException(f: Failure) ⇒ log :+= Event("re-applying " + f.directive + " to " + sender) f.directive - case f @ Failure(directive, logs, x, _, _) ⇒ + case f @ Failure(directive, x, _, _) ⇒ import SupervisorStrategy._ setFlags(directive) log :+= Event("escalating " + f) - throw f.copy(log = logs + (self -> log), depth = x - 1) + throw f.copy(depth = x - 1) } override def postRestart(cause: Throwable) { - cause match { - case f: Failure ⇒ log = f.log get self getOrElse Vector(Event("log lost")) - case OriginalRestartException(f: Failure) ⇒ log = f.log get self getOrElse Vector(Event("log lost")) - case _ ⇒ - } + log = stateCache.get(self).log log :+= Event("restarted " + suspendCount) cause match { - case f @ Failure(_, _, _, _, x) if x > 0 ⇒ f.failPost = x - 1; throw f - case _ ⇒ + case f @ Failure(_, _, _, x) if x > 0 ⇒ f.failPost = x - 1; throw f + case _ ⇒ } } override def postStop { - if (failed || (suspended && !stopping)) { + if (failed || suspended) { listener ! ErrorLog("not resumed (" + failed + ", " + suspended + ")", log) } } @@ -191,7 +208,7 @@ object SupervisorHierarchySpec { def receive = new Receive { val handler: Receive = { - case f: Failure ⇒ setFlags(f.directive); throw f.copy(log = Map(self -> log)) + case f: Failure ⇒ setFlags(f.directive); throw f case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" case Dump(0) ⇒ context stop self case Dump(level) ⇒ context.children foreach (_ ! Dump(level - 1)) @@ -272,6 +289,8 @@ object SupervisorHierarchySpec { * they are indeed GCed * * TODO RK: also test recreate including terminating children + * + * TODO RK: test exceptions in constructor */ class StressTest(testActor: ActorRef, depth: Int, breadth: Int) extends Actor with LoggingFSM[State, Int] { @@ -320,7 +339,7 @@ object SupervisorHierarchySpec { when(Idle) { case Event(Init, _) ⇒ - hierarchy = context.watch(context.actorOf(Props(new Hierarchy(size = 500, breadth = breadth, self)).withDispatcher("hierarchy"))) + hierarchy = context.watch(context.actorOf(Props(new Hierarchy(size = 500, breadth = breadth, self)).withDispatcher("hierarchy"), "head")) setTimer("phase", StateTimeout, 5 seconds, false) goto(Init) } @@ -344,7 +363,7 @@ object SupervisorHierarchySpec { case Init -> Stress ⇒ self ! Work // set timeout for completion of the whole test (i.e. including Finishing and Stopping) - setTimer("phase", StateTimeout, 30.seconds.dilated, false) + setTimer("phase", StateTimeout, 50.seconds.dilated, false) } val workSchedule = 250.millis @@ -365,7 +384,7 @@ object SupervisorHierarchySpec { case Event(Work, x) if x > 0 ⇒ nextJob.next match { case Ping(ref) ⇒ ref ! "ping" - case Fail(ref, dir) ⇒ ref ! Failure(dir, Map.empty, random012, random012, random012) + case Fail(ref, dir) ⇒ ref ! Failure(dir, random012, random012, random012) } if (idleChildren.nonEmpty) self ! Work else context.system.scheduler.scheduleOnce(workSchedule, self, Work) From 1d9c237d7c5da116f3077c27527cc96d80baf0f5 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 2 Aug 2012 22:43:07 +0200 Subject: [PATCH 07/16] Making read buffer size, select interval and default backlog size configurable in .conf and making it possible to override backlog on a per-Listen basis. Also Adding causes to a could of cleanup calls. --- akka-actor/src/main/resources/reference.conf | 12 +++ akka-actor/src/main/scala/akka/actor/IO.scala | 87 +++++++++++++------ 2 files changed, 71 insertions(+), 28 deletions(-) diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 7bedfa3f02..8fbf16b427 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -319,4 +319,16 @@ akka { tick-duration = 100ms ticks-per-wheel = 512 } + + io { + # In bytes, the size of the shared read buffer. In the span 0..Int.MaxValue + # + read-buffer-size = 8KiB + + # + select-interval = 100 + + # + default-backlog = 1000 + } } diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index de2b2a40d8..9b50aa2c99 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -11,7 +11,6 @@ import scala.concurrent.util.Duration import scala.util.control.NonFatal import akka.util.ByteString import java.net.{ SocketAddress, InetSocketAddress } -import java.io.IOException import java.nio.ByteBuffer import java.nio.channels.{ SelectableChannel, @@ -27,6 +26,8 @@ import scala.collection.mutable import scala.annotation.tailrec import scala.collection.generic.CanBuildFrom import java.util.UUID +import java.io.{EOFException, IOException} + /** * IO messages and iteratees. * @@ -177,6 +178,13 @@ object IO { */ case class ReuseAddress(on: Boolean) extends SocketOption with ServerSocketOption + /** + * [[akka.actor.IO.ServerSocketOption]] to set the maximum backlog of connections. 0 or negative means that the platform default will be used. + * For more information see [[http://docs.oracle.com/javase/7/docs/api/java/nio/channels/ServerSocketChannel.html#bind(java.net.SocketAddress, int)]] + * @param numberOfConnections + */ + case class Backlog(numberOfConnections: Int) extends ServerSocketOption + /** * [[akka.actor.IO.SocketOption]] to set the SO_SNDBUF option for this * [[akka.actor.IO.SocketHandle]]. @@ -806,13 +814,30 @@ object IO { * An IOManager does not need to be manually stopped when not in use as it will * automatically enter an idle state when it has no channels to manage. */ -final class IOManager private (system: ActorSystem) extends Extension { //FIXME how about taking an ActorContext +final class IOManager private (system: ExtendedActorSystem) extends Extension { //FIXME how about taking an ActorContext + /** * A reference to the [[akka.actor.IOManagerActor]] that performs the actual * IO. It communicates with other actors using subclasses of * [[akka.actor.IO.IOMessage]]. */ - val actor = system.actorOf(Props[IOManagerActor], "io-manager") + val actor = { + val c = system.settings.config.getConfig("akka.io") + val readBufferSize = { + val sz = c.getBytes("read-buffer-size") + require(sz <= Int.MaxValue && sz > 0) + sz.toInt + } + + val selectInterval = { + val i = c.getInt("select-interval") + require(i > 0) + i + } + val defaultBacklog = c.getInt("default-backlog") + + system.actorOf(Props(new IOManagerActor(readBufferSize, selectInterval, defaultBacklog)), "io-manager") + } /** * Create a ServerSocketChannel listening on an address. Messages will be @@ -898,11 +923,13 @@ object IOManager extends ExtensionId[IOManager] with ExtensionIdProvider { * * Use [[akka.actor.IOManager]] to retrieve an instance of this Actor. */ -final class IOManagerActor extends Actor with ActorLogging { +final class IOManagerActor( + val readBufferSize: Int, + val selectInterval: Int, + /** force a select when lastSelect reaches this amount */ + val defaultBacklog: Int) extends Actor with ActorLogging { import SelectionKey.{ OP_READ, OP_WRITE, OP_ACCEPT, OP_CONNECT } - private val bufferSize = 8192 // FIXME TODO: make configurable - private type ReadChannel = ReadableByteChannel with SelectableChannel private type WriteChannel = WritableByteChannel with SelectableChannel @@ -918,14 +945,11 @@ final class IOManagerActor extends Actor with ActorLogging { private val closing = mutable.Set.empty[IO.Handle] /** Buffer used for all reads */ - private val buffer = ByteBuffer.allocate(bufferSize) + private val buffer = ByteBuffer.allocate(readBufferSize) /** a counter that is incremented each time a message is retrieved */ private var lastSelect = 0 - /** force a select when lastSelect reaches this amount */ - private val selectAt = 100 // FIXME TODO: make configurable - /** true while the selector is open and channels.nonEmpty */ private var running = false @@ -952,10 +976,14 @@ final class IOManagerActor extends Actor with ActorLogging { } } lastSelect += 1 - if (lastSelect >= selectAt) select() + if (lastSelect >= selectInterval) + running = select() } - private def select() { + /** + * @return true if we should be running and false if not + */ + private def select(): Boolean = try { if (selector.isOpen) { // TODO: Make select behaviour configurable. // Blocking 1ms reduces allocations during idle times, non blocking gives better performance. @@ -965,12 +993,13 @@ final class IOManagerActor extends Actor with ActorLogging { while (keys.hasNext) { val key = keys.next() keys.remove() - if (key.isValid) { process(key) } + if (key.isValid) process(key) } - if (channels.isEmpty) running = false + if (channels.isEmpty) false else running } else { - running = false + false } + } finally { lastSelect = 0 } @@ -994,7 +1023,7 @@ final class IOManagerActor extends Actor with ActorLogging { def receive = { case Select ⇒ - select() + running = select() if (running) self ! Select selectSent = running @@ -1002,18 +1031,20 @@ final class IOManagerActor extends Actor with ActorLogging { val channel = ServerSocketChannel open () channel configureBlocking false + var backlog = defaultBacklog val sock = channel.socket options foreach { case IO.ReceiveBufferSize(size) ⇒ forwardFailure(sock.setReceiveBufferSize(size)) case IO.ReuseAddress(on) ⇒ forwardFailure(sock.setReuseAddress(on)) case IO.PerformancePreferences(connTime, latency, bandwidth) ⇒ forwardFailure(sock.setPerformancePreferences(connTime, latency, bandwidth)) + case IO.Backlog(number) ⇒ backlog = number } - channel.socket bind (address, 1000) // FIXME TODO: make backlog configurable + channel.socket bind (address, backlog) channels update (server, channel) channel register (selector, OP_ACCEPT, server) - server.owner ! IO.Listening(server, channel.socket.getLocalSocketAddress()) + server.owner ! IO.Listening(server, sock.getLocalSocketAddress()) run() case IO.Connect(socket, address, options) ⇒ @@ -1038,13 +1069,13 @@ final class IOManagerActor extends Actor with ActorLogging { case IO.Write(handle, data) ⇒ if (channels contains handle) { val queue = writes get handle getOrElse { - val q = new WriteBuffer(bufferSize) + val q = new WriteBuffer(readBufferSize) writes update (handle, q) q } if (queue.isEmpty) addOps(handle, OP_WRITE) queue enqueue data - if (queue.length >= bufferSize) write(handle, channels(handle).asInstanceOf[WriteChannel]) + if (queue.length >= readBufferSize) write(handle, channels(handle).asInstanceOf[WriteChannel]) } run() @@ -1083,12 +1114,12 @@ final class IOManagerActor extends Actor with ActorLogging { case server: IO.ServerHandle ⇒ accepted -= server case writable: IO.WriteHandle ⇒ writes -= writable } - channels.get(handle) match { - case Some(channel) ⇒ - channel.close - channels -= handle - if (!handle.owner.isTerminated) handle.owner ! IO.Closed(handle, cause) - case None ⇒ + channels.get(handle) foreach { + channel ⇒ + try channel.close finally { + channels -= handle + if (!handle.owner.isTerminated) handle.owner ! IO.Closed(handle, cause) + } } } @@ -1109,7 +1140,7 @@ final class IOManagerActor extends Actor with ActorLogging { removeOps(socket, OP_CONNECT) socket.owner ! IO.Connected(socket, channel.socket.getRemoteSocketAddress()) } else { - cleanup(socket, None) // TODO: Add a cause + cleanup(socket, Some(new IllegalStateException("Channel for socket handle [%s] didn't finish connect" format socket))) } } @@ -1138,7 +1169,7 @@ final class IOManagerActor extends Actor with ActorLogging { buffer.clear val readLen = channel read buffer if (readLen == -1) { - cleanup(handle, None) // TODO: Add a cause + cleanup(handle, Some(new EOFException("Elvis has left the building"))) } else if (readLen > 0) { buffer.flip handle.owner ! IO.Read(handle, ByteString(buffer)) From d1b57b4db3659fb5e432180417ebf894b85b70b1 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Fri, 3 Aug 2012 17:30:59 +0200 Subject: [PATCH 08/16] Code formatting --- akka-actor/src/main/scala/akka/actor/IO.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/IO.scala b/akka-actor/src/main/scala/akka/actor/IO.scala index 9b50aa2c99..22dad7b069 100644 --- a/akka-actor/src/main/scala/akka/actor/IO.scala +++ b/akka-actor/src/main/scala/akka/actor/IO.scala @@ -26,7 +26,7 @@ import scala.collection.mutable import scala.annotation.tailrec import scala.collection.generic.CanBuildFrom import java.util.UUID -import java.io.{EOFException, IOException} +import java.io.{ EOFException, IOException } /** * IO messages and iteratees. From 7849354093a36d0c059e8084ec4ca8fd9f30cbcb Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 6 Aug 2012 12:01:33 +0200 Subject: [PATCH 09/16] stress test also the Stop directive, see #2336 This required some more rework of the idleChildren management. --- .../akka/actor/SupervisorHierarchySpec.scala | 202 ++++++++++++------ 1 file changed, 140 insertions(+), 62 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 5e41955562..2a0827c713 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -21,6 +21,7 @@ import akka.testkit.{ ImplicitSender, EventFilter, DefaultTimeout, AkkaSpec } import akka.testkit.{ filterException, duration2TestDuration, TestLatch } import akka.testkit.TestEvent.Mute import java.util.concurrent.ConcurrentHashMap +import java.lang.ref.WeakReference object SupervisorHierarchySpec { class FireWorkerException(msg: String) extends Exception(msg) @@ -50,10 +51,14 @@ object SupervisorHierarchySpec { } } - case class Children(refs: Vector[ActorRef]) + case class Ready(ref: ActorRef) + case class Died(ref: ActorRef) + case class Cleared(ref: ActorRef) + case object Abort case class Event(msg: Any) { val time: Long = System.nanoTime } case class ErrorLog(msg: String, log: Vector[Event]) - case class Failure(directive: Directive, depth: Int, var failPre: Int, var failPost: Int) extends RuntimeException with NoStackTrace { + case class Failure(directive: Directive, stop: Boolean, depth: Int, var failPre: Int, var failPost: Int) + extends RuntimeException with NoStackTrace { override def toString = productPrefix + productIterator.mkString("(", ",", ")") } case class Dump(level: Int) @@ -100,10 +105,10 @@ object SupervisorHierarchySpec { * upon Restart or would have to be managed by the highest supervisor (which * is undesirable). */ - case class HierarchyState(log: Vector[Event], kids: Map[String, Int]) + case class HierarchyState(log: Vector[Event], kids: Map[ActorRef, Int]) val stateCache = new ConcurrentHashMap[ActorRef, HierarchyState]() - class Hierarchy(size: Int, breadth: Int, listener: ActorRef) extends Actor { + class Hierarchy(size: Int, breadth: Int, listener: ActorRef, myLevel: Int) extends Actor { var failed = false var suspended = false @@ -111,6 +116,7 @@ object SupervisorHierarchySpec { def abort(msg: String) { listener ! ErrorLog(msg, log) log = Vector(Event("log sent")) + context.parent ! Abort context stop self } @@ -124,8 +130,9 @@ object SupervisorHierarchySpec { override def preStart { log :+= Event("started") + listener ! Ready(self) val s = size - 1 // subtract myself - val kidInfo: Map[String, Int] = + val kidInfo: Map[ActorRef, Int] = if (s > 0) { val kids = Random.nextInt(Math.min(breadth, s)) + 1 val sizes = s / kids @@ -133,15 +140,10 @@ object SupervisorHierarchySpec { val propsTemplate = Props.empty.withDispatcher("hierarchy") (1 to kids).map { (id) ⇒ val kidSize = if (rest > 0) { rest -= 1; sizes + 1 } else sizes - val props = propsTemplate.withCreator(new Hierarchy(kidSize, breadth, listener)) - val name = id.toString - context.watch(context.actorOf(props, name)) - (name, kidSize) + val props = propsTemplate.withCreator(new Hierarchy(kidSize, breadth, listener, myLevel + 1)) + (context.watch(context.actorOf(props, id.toString)), kidSize) }(collection.breakOut) - } else { - context.parent ! Children(Vector(self)) - Map() - } + } else Map() stateCache.put(self, HierarchyState(log, kidInfo)) } @@ -154,20 +156,20 @@ object SupervisorHierarchySpec { stateCache.put(self, stateCache.get(self).copy(log = log)) preRestartCalled = true cause match { - case f @ Failure(_, _, x, _) if x > 0 ⇒ f.failPre = x - 1; throw f - case _ ⇒ + case f: Failure if f.failPre > 0 ⇒ f.failPre -= 1; throw f + case _ ⇒ } } } override val supervisorStrategy = OneForOneStrategy() { - case Failure(directive, 0, _, _) ⇒ - log :+= Event("applying " + directive + " to " + sender) - directive - case OriginalRestartException(f: Failure) ⇒ - log :+= Event("re-applying " + f.directive + " to " + sender) - f.directive - case f @ Failure(directive, x, _, _) ⇒ + case Failure(directive, stop, 0, _, failPost) ⇒ + log :+= Event("applying (" + directive + ", " + stop + ", " + failPost + ") to " + sender) + if (myLevel > 3 && failPost == 0 && stop) Stop else directive + case PostRestartException(_, Failure(directive, stop, 0, _, failPost), _) ⇒ + log :+= Event("re-applying (" + directive + ", " + stop + ", " + failPost + ") to " + sender) + if (myLevel > 3 && failPost == 0 && stop) Stop else directive + case f @ Failure(directive, _, x, _, _) ⇒ import SupervisorStrategy._ setFlags(directive) log :+= Event("escalating " + f) @@ -178,8 +180,9 @@ object SupervisorHierarchySpec { log = stateCache.get(self).log log :+= Event("restarted " + suspendCount) cause match { - case f @ Failure(_, _, _, x) if x > 0 ⇒ f.failPost = x - 1; throw f - case _ ⇒ + case f: Failure if f.failPost > 0 ⇒ f.failPost -= 1; throw f + case PostRestartException(`self`, f: Failure, _) if f.failPost > 0 ⇒ f.failPost -= 1; throw f + case _ ⇒ } } @@ -203,20 +206,18 @@ object SupervisorHierarchySpec { } else true } - var gotKidsFrom = Set.empty[ActorRef] - var kids = Vector.empty[ActorRef] - def receive = new Receive { val handler: Receive = { - case f: Failure ⇒ setFlags(f.directive); throw f - case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" - case Dump(0) ⇒ context stop self - case Dump(level) ⇒ context.children foreach (_ ! Dump(level - 1)) - case Terminated(_) ⇒ abort("terminating") - case Children(refs) ⇒ - kids ++= refs - gotKidsFrom += sender - if (context.children forall (gotKidsFrom contains)) context.parent ! Children(kids :+ self) + case f: Failure ⇒ setFlags(f.directive); throw f + case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" + case Dump(0) ⇒ abort("dump") + case Dump(level) ⇒ context.children foreach (_ ! Dump(level - 1)) + case Terminated(ref) ⇒ listener ! Died(ref) + case Cleared(ref) ⇒ + val kids = stateCache.get(self).kids(ref) + val props = Props(new Hierarchy(kids, breadth, listener, myLevel + 1)) + context.watch(context.actorOf(props, ref.path.name)) + case Abort ⇒ abort("terminating") } override def isDefinedAt(msg: Any) = handler.isDefinedAt(msg) override def apply(msg: Any) = { if (check(msg)) handler(msg) } @@ -224,6 +225,7 @@ object SupervisorHierarchySpec { } case object Work + case class GCcheck(kids: Vector[WeakReference[ActorRef]]) sealed trait Action case class Ping(ref: ActorRef) extends Action @@ -236,6 +238,7 @@ object SupervisorHierarchySpec { case object Finishing extends State case object LastPing extends State case object Stopping extends State + case object GC extends State case object Failed extends State /* @@ -282,18 +285,22 @@ object SupervisorHierarchySpec { * - accumulate ErrorLog messages * - upon termination of the hierarchy send back failed result and print * the logs, merged and in chronological order. - * - * TODO RK: also test Stop directive, and keep a complete list of all - * actors ever created, then verify after stop()ping the hierarchy that - * all are terminated, transfer them to a WeakHashMap and verify that - * they are indeed GCed + * + * Remark about test failures which lead to stopping: + * The FSM needs to know not the send more things to the dead guy, but it + * also must not watch all targets, because the dead guy’s supervisor also + * watches him and creates a new guy in response to the Terminated; given + * that there is no ordering relationship guaranteed by DeathWatch it could + * happen that the new guy sends his Ready before the FSM has gotten the + * Terminated, which would screw things up big time. Solution is to let the + * supervisor do all, including notifying the FSM of the death of the guy. * * TODO RK: also test recreate including terminating children * * TODO RK: test exceptions in constructor */ - class StressTest(testActor: ActorRef, depth: Int, breadth: Int) extends Actor with LoggingFSM[State, Int] { + class StressTest(testActor: ActorRef, size: Int, breadth: Int) extends Actor with LoggingFSM[State, Int] { import context.system // don’t escalate from this one! @@ -335,24 +342,24 @@ object SupervisorHierarchySpec { } // number of Work packages to execute for the test - startWith(Idle, 500000) + startWith(Idle, size * 1000) when(Idle) { case Event(Init, _) ⇒ - hierarchy = context.watch(context.actorOf(Props(new Hierarchy(size = 500, breadth = breadth, self)).withDispatcher("hierarchy"), "head")) + hierarchy = context.watch(context.actorOf(Props(new Hierarchy(size, breadth, self, 0)).withDispatcher("hierarchy"), "head")) setTimer("phase", StateTimeout, 5 seconds, false) goto(Init) } when(Init) { - case Event(Children(refs), _) ⇒ - children = refs - idleChildren = refs - if (children.toSet.size != children.size) { + case Event(Ready(ref), _) ⇒ + if (children contains ref) { testActor ! "children not unique" stop() } else { - goto(Stress) + children :+= ref + if (children.size == size) goto(Stress) + else stay } case Event(StateTimeout, _) ⇒ testActor ! "did not get children list" @@ -362,6 +369,7 @@ object SupervisorHierarchySpec { onTransition { case Init -> Stress ⇒ self ! Work + idleChildren = children // set timeout for completion of the whole test (i.e. including Finishing and Stopping) setTimer("phase", StateTimeout, 50.seconds.dilated, false) } @@ -373,23 +381,32 @@ object SupervisorHierarchySpec { case x if x > 0.03 ⇒ 1 case _ ⇒ 2 } + private def bury(ref: ActorRef): Unit = { + val deadGuy = ref.path.elements + val deadGuySize = deadGuy.size + val isChild = (other: ActorRef) ⇒ other.path.elements.take(deadGuySize) == deadGuy + idleChildren = idleChildren filterNot isChild + pingChildren = pingChildren filterNot isChild + } var ignoreNotResumedLogs = true when(Stress) { case Event(Work, _) if idleChildren.isEmpty ⇒ - ignoreNotResumedLogs = false - context stop hierarchy - goto(Failed) + context.system.scheduler.scheduleOnce(workSchedule, self, Work) + stay case Event(Work, x) if x > 0 ⇒ nextJob.next match { case Ping(ref) ⇒ ref ! "ping" - case Fail(ref, dir) ⇒ ref ! Failure(dir, random012, random012, random012) + case Fail(ref, dir) ⇒ ref ! Failure(dir, stop = random012 > 0, depth = random012, failPre = random012, failPost = random012) } if (idleChildren.nonEmpty) self ! Work else context.system.scheduler.scheduleOnce(workSchedule, self, Work) stay using (x - 1) case Event(Work, _) ⇒ if (pingChildren.isEmpty) goto(LastPing) else goto(Finishing) + case Event(Died(ref), _) ⇒ + bury(ref) + stay replying (Cleared(ref)) case Event("pong", _) ⇒ pingChildren -= sender idleChildren :+= sender @@ -408,6 +425,10 @@ object SupervisorHierarchySpec { pingChildren -= sender idleChildren :+= sender if (pingChildren.isEmpty) goto(LastPing) else stay + case Event(Died(ref), _) ⇒ + bury(ref) + sender ! Cleared(ref) + if (pingChildren.isEmpty) goto(LastPing) else stay } onTransition { @@ -422,6 +443,10 @@ object SupervisorHierarchySpec { pingChildren -= sender idleChildren :+= sender if (pingChildren.isEmpty) goto(Stopping) else stay + case Event(Died(ref), _) ⇒ + bury(ref) + sender ! Cleared(ref) + if (pingChildren.isEmpty) goto(Stopping) else stay } onTransition { @@ -432,11 +457,53 @@ object SupervisorHierarchySpec { when(Stopping, stateTimeout = 5 seconds) { case Event(Terminated(r), _) if r == hierarchy ⇒ - testActor ! "stressTestSuccessful" - stop + val undead = children filterNot (_.isTerminated) + if (undead.nonEmpty) { + log.info("undead:\n" + undead.mkString("\n")) + testActor ! "stressTestFailed (" + undead.size + " undead)" + stop + } else if (false) { + /* + * This part of the test is normally disabled, because it does not + * work reliably: even though I found only these weak references + * using YourKit just now, GC wouldn’t collect them and the test + * failed. I’m leaving this code in so that manual inspection remains + * an option (by setting the above condition to “true”). + */ + val weak = children map (new WeakReference(_)) + children = Vector.empty + pingChildren = Set.empty + idleChildren = Vector.empty + context.system.scheduler.scheduleOnce(workSchedule, self, GCcheck(weak)) + System.gc() + goto(GC) + } else { + testActor ! "stressTestSuccessful" + stop + } case Event(StateTimeout, _) ⇒ testActor ! "timeout in Stopping" stop + case Event(e: ErrorLog, _) ⇒ + errors :+= sender -> e + goto(Failed) + } + + when(GC, stateTimeout = 10 seconds) { + case Event(GCcheck(weak), _) ⇒ + val next = weak filter (_.get ne null) + if (next.nonEmpty) { + println(next.size + " left") + context.system.scheduler.scheduleOnce(workSchedule, self, GCcheck(next)) + System.gc() + stay + } else { + testActor ! "stressTestSuccessful" + stop + } + case Event(StateTimeout, _) ⇒ + testActor ! "timeout in GC" + stop } var errors = Vector.empty[(ActorRef, ErrorLog)] @@ -473,7 +540,7 @@ object SupervisorHierarchySpec { } def printErrors(): Unit = { - val merged = errors flatMap { + val merged = errors.sortBy(_._1.toString) flatMap { case (ref, ErrorLog(msg, log)) ⇒ println(ref + " " + msg) log map (l ⇒ (l.time, ref, l.msg.toString)) @@ -482,14 +549,25 @@ object SupervisorHierarchySpec { } whenUnhandled { + case Event(Ready(ref), _) ⇒ + children :+= ref + idleChildren :+= ref + stay case Event(e: ErrorLog, _) ⇒ - errors :+= sender -> e - // don’t stop the hierarchy, that is going to happen all by itself and in the right order - goto(Failed) + if (e.msg.startsWith("not resumed")) stay + else { + errors :+= sender -> e + // don’t stop the hierarchy, that is going to happen all by itself and in the right order + goto(Failed) + } case Event(StateTimeout, _) ⇒ - println("pingChildren:\n" + pingChildren.mkString("\n")) + println("pingChildren:\n" + pingChildren.view.map(_.path.toString).toSeq.sorted.mkString("\n")) + ignoreNotResumedLogs = false context stop hierarchy goto(Failed) + case Event(Abort, _) ⇒ + log.info("received Abort") + goto(Failed) case Event(msg, _) ⇒ testActor ! ("received unexpected msg: " + msg) stop @@ -597,9 +675,9 @@ class SupervisorHierarchySpec extends AkkaSpec(SupervisorHierarchySpec.config) w "survive being stressed" in { system.eventStream.publish(Mute(EventFilter[Failure](), EventFilter[PreRestartException](), EventFilter[PostRestartException]())) - system.eventStream.publish(Mute(EventFilter.warning(start = "received dead letter"))) + system.eventStream.publish(Mute(EventFilter.warning(start = "received dead "))) - val fsm = system.actorOf(Props(new StressTest(testActor, depth = 6, breadth = 6)), "stressTest") + val fsm = system.actorOf(Props(new StressTest(testActor, size = 500, breadth = 6)), "stressTest") fsm ! FSM.SubscribeTransitionCallBack(system.actorOf(Props(new Actor { def receive = { From b45e0ab2378628985a27cd89c488e38a6412a1be Mon Sep 17 00:00:00 2001 From: Roland Date: Mon, 6 Aug 2012 22:48:51 +0200 Subject: [PATCH 10/16] add terminating children during restart to stress test, see #2336 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This uncovered a bug with executing system messages while waiting for a restart to complete: restart needs to be “atomic”, hence recreate/suspend/resume must be kept back until finished. --- .../akka/actor/SupervisorHierarchySpec.scala | 84 +++++++++++++------ .../src/main/scala/akka/actor/ActorCell.scala | 53 ++++++++---- .../main/scala/akka/actor/cell/Children.scala | 5 ++ .../akka/actor/cell/ChildrenContainer.scala | 3 +- .../scala/akka/actor/cell/FaultHandling.scala | 10 +-- 5 files changed, 108 insertions(+), 47 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 2a0827c713..ce35fd8c82 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -53,11 +53,10 @@ object SupervisorHierarchySpec { case class Ready(ref: ActorRef) case class Died(ref: ActorRef) - case class Cleared(ref: ActorRef) case object Abort case class Event(msg: Any) { val time: Long = System.nanoTime } case class ErrorLog(msg: String, log: Vector[Event]) - case class Failure(directive: Directive, stop: Boolean, depth: Int, var failPre: Int, var failPost: Int) + case class Failure(directive: Directive, stop: Boolean, depth: Int, var failPre: Int, var failPost: Int, stopKids: Int) extends RuntimeException with NoStackTrace { override def toString = productPrefix + productIterator.mkString("(", ",", ")") } @@ -153,32 +152,52 @@ object SupervisorHierarchySpec { if (preRestartCalled) abort("preRestart called twice") else { log :+= Event("preRestart") - stateCache.put(self, stateCache.get(self).copy(log = log)) preRestartCalled = true cause match { - case f: Failure if f.failPre > 0 ⇒ f.failPre -= 1; throw f - case _ ⇒ + case f: Failure ⇒ + context.children.take(f.stopKids) foreach { child ⇒ + log :+= Event("killing " + child) + context.unwatch(child) + context.stop(child) + } + stateCache.put(self, stateCache.get(self).copy(log = log)) + if (f.failPre > 0) { + f.failPre -= 1 + throw f + } + case _ ⇒ stateCache.put(self, stateCache.get(self).copy(log = log)) } } } override val supervisorStrategy = OneForOneStrategy() { - case Failure(directive, stop, 0, _, failPost) ⇒ + case Failure(directive, stop, 0, _, failPost, _) ⇒ log :+= Event("applying (" + directive + ", " + stop + ", " + failPost + ") to " + sender) if (myLevel > 3 && failPost == 0 && stop) Stop else directive - case PostRestartException(_, Failure(directive, stop, 0, _, failPost), _) ⇒ + case PostRestartException(_, Failure(directive, stop, 0, _, failPost, _), _) ⇒ log :+= Event("re-applying (" + directive + ", " + stop + ", " + failPost + ") to " + sender) if (myLevel > 3 && failPost == 0 && stop) Stop else directive - case f @ Failure(directive, _, x, _, _) ⇒ + case f: Failure ⇒ import SupervisorStrategy._ - setFlags(directive) + setFlags(f.directive) log :+= Event("escalating " + f) - throw f.copy(depth = x - 1) + throw f.copy(depth = f.depth - 1) + case _ ⇒ sender ! Dump(0); Resume } override def postRestart(cause: Throwable) { - log = stateCache.get(self).log + val state = stateCache.get(self) + log = state.log log :+= Event("restarted " + suspendCount) + state.kids foreach { + case (child, kidSize) ⇒ + val name = child.path.name + if (context.actorFor(name).isTerminated) { + listener ! Died(child) + val props = Props(new Hierarchy(kidSize, breadth, listener, myLevel + 1)) + context.watch(context.actorOf(props, name)) + } + } cause match { case f: Failure if f.failPost > 0 ⇒ f.failPost -= 1; throw f case PostRestartException(`self`, f: Failure, _) if f.failPost > 0 ⇒ f.failPost -= 1; throw f @@ -208,15 +227,22 @@ object SupervisorHierarchySpec { def receive = new Receive { val handler: Receive = { - case f: Failure ⇒ setFlags(f.directive); throw f - case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" - case Dump(0) ⇒ abort("dump") - case Dump(level) ⇒ context.children foreach (_ ! Dump(level - 1)) - case Terminated(ref) ⇒ listener ! Died(ref) - case Cleared(ref) ⇒ - val kids = stateCache.get(self).kids(ref) - val props = Props(new Hierarchy(kids, breadth, listener, myLevel + 1)) - context.watch(context.actorOf(props, ref.path.name)) + case f: Failure ⇒ setFlags(f.directive); throw f + case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" + case Dump(0) ⇒ abort("dump") + case Dump(level) ⇒ context.children foreach (_ ! Dump(level - 1)) + case Terminated(ref) ⇒ + /* + * It might be that we acted upon this death already in postRestart + * (if the unwatch() came too late), so just ignore in this case. + */ + val name = ref.path.name + if (context.actorFor(name).isTerminated) { + listener ! Died(ref) + val kids = stateCache.get(self).kids(ref) + val props = Props(new Hierarchy(kids, breadth, listener, myLevel + 1)) + context.watch(context.actorOf(props, name)) + } case Abort ⇒ abort("terminating") } override def isDefinedAt(msg: Any) = handler.isDefinedAt(msg) @@ -295,8 +321,6 @@ object SupervisorHierarchySpec { * Terminated, which would screw things up big time. Solution is to let the * supervisor do all, including notifying the FSM of the death of the guy. * - * TODO RK: also test recreate including terminating children - * * TODO RK: test exceptions in constructor */ @@ -307,6 +331,7 @@ object SupervisorHierarchySpec { override val supervisorStrategy = OneForOneStrategy() { case f: Failure ⇒ f.directive case OriginalRestartException(f: Failure) ⇒ f.directive + case _ ⇒ Stop } var children = Vector.empty[ActorRef] @@ -397,8 +422,15 @@ object SupervisorHierarchySpec { stay case Event(Work, x) if x > 0 ⇒ nextJob.next match { - case Ping(ref) ⇒ ref ! "ping" - case Fail(ref, dir) ⇒ ref ! Failure(dir, stop = random012 > 0, depth = random012, failPre = random012, failPost = random012) + case Ping(ref) ⇒ ref ! "ping" + case Fail(ref, dir) ⇒ + val f = Failure(dir, stop = random012 > 0, depth = random012, failPre = random012, failPost = random012, + stopKids = random012 match { + case 0 ⇒ 0 + case 1 ⇒ Random.nextInt(breadth / 2) + case 2 ⇒ 1000 + }) + ref ! f } if (idleChildren.nonEmpty) self ! Work else context.system.scheduler.scheduleOnce(workSchedule, self, Work) @@ -406,7 +438,7 @@ object SupervisorHierarchySpec { case Event(Work, _) ⇒ if (pingChildren.isEmpty) goto(LastPing) else goto(Finishing) case Event(Died(ref), _) ⇒ bury(ref) - stay replying (Cleared(ref)) + stay case Event("pong", _) ⇒ pingChildren -= sender idleChildren :+= sender @@ -427,7 +459,6 @@ object SupervisorHierarchySpec { if (pingChildren.isEmpty) goto(LastPing) else stay case Event(Died(ref), _) ⇒ bury(ref) - sender ! Cleared(ref) if (pingChildren.isEmpty) goto(LastPing) else stay } @@ -445,7 +476,6 @@ object SupervisorHierarchySpec { if (pingChildren.isEmpty) goto(Stopping) else stay case Event(Died(ref), _) ⇒ bury(ref) - sender ! Cleared(ref) if (pingChildren.isEmpty) goto(Stopping) else stay } diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index a5667f2634..e7808b2cc9 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -300,21 +300,46 @@ private[akka] class ActorCell( * MESSAGE PROCESSING */ //Memory consistency is handled by the Mailbox (reading mailbox status then processing messages, then writing mailbox status - final def systemInvoke(message: SystemMessage): Unit = try { - message match { - case Create() ⇒ create() - case Recreate(cause) ⇒ faultRecreate(cause) - case Watch(watchee, watcher) ⇒ addWatcher(watchee, watcher) - case Unwatch(watchee, watcher) ⇒ remWatcher(watchee, watcher) - case Suspend() ⇒ faultSuspend() - case Resume(inRespToFailure) ⇒ faultResume(inRespToFailure) - case Terminate() ⇒ terminate() - case Supervise(child) ⇒ supervise(child) - case ChildTerminated(child) ⇒ handleChildTerminated(child) - case NoMessage ⇒ // only here to suppress warning + @tailrec final def systemInvoke(message: SystemMessage): Unit = { + /* + * When recreate/suspend/resume are received while restarting (i.e. between + * preRestart and postRestart, waiting for children to terminate), these + * must not be executed immediately, but instead queued and released after + * finishRecreate returns. This can only ever be triggered by + * ChildTerminated, and ChildTerminated is not one of the queued message + * types (hence the overwrite further down). Mailbox sets message.next=null + * before systemInvoke, so this will only be non-null during such a replay. + */ + var todo = message.next + try { + message match { + case Create() ⇒ create() + case Watch(watchee, watcher) ⇒ addWatcher(watchee, watcher) + case Unwatch(watchee, watcher) ⇒ remWatcher(watchee, watcher) + case Recreate(cause) ⇒ + recreationOrNull match { + case null ⇒ faultRecreate(cause) + case r ⇒ message.next = r.todo; r.todo = message + } + case Suspend() ⇒ + recreationOrNull match { + case null ⇒ faultSuspend() + case r ⇒ message.next = r.todo; r.todo = message + } + case Resume(inRespToFailure) ⇒ + recreationOrNull match { + case null ⇒ faultResume(inRespToFailure) + case r ⇒ message.next = r.todo; r.todo = message + } + case Terminate() ⇒ terminate() + case Supervise(child) ⇒ supervise(child) + case ChildTerminated(child) ⇒ todo = handleChildTerminated(child) + case NoMessage ⇒ // only here to suppress warning + } + } catch { + case e @ (_: InterruptedException | NonFatal(_)) ⇒ handleInvokeFailure(Nil, e, "error while processing " + message) } - } catch { - case e @ (_: InterruptedException | NonFatal(_)) ⇒ handleInvokeFailure(Nil, e, "error while processing " + message) + if (todo != null) systemInvoke(todo) } //Memory consistency is handled by the Mailbox (reading mailbox status then processing messages, then writing mailbox status diff --git a/akka-actor/src/main/scala/akka/actor/cell/Children.scala b/akka-actor/src/main/scala/akka/actor/cell/Children.scala index 4716795c8b..202c5e4016 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/Children.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/Children.scala @@ -113,6 +113,11 @@ private[akka] trait Children { this: ActorCell ⇒ protected def isTerminating = childrenRefs.isTerminating + protected def recreationOrNull = childrenRefs match { + case TerminatingChildrenContainer(_, _, r: Recreation) ⇒ r + case _ ⇒ null + } + protected def suspendChildren(skip: Set[ActorRef] = Set.empty): Unit = childrenRefs.stats foreach { case ChildRestartStats(child, _, _) if !(skip contains child) ⇒ child.asInstanceOf[InternalActorRef].suspend() diff --git a/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala b/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala index 98679862ba..09303bd24d 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala @@ -7,6 +7,7 @@ package akka.actor.cell import scala.collection.immutable.TreeMap import akka.actor.{ InvalidActorNameException, ChildStats, ChildRestartStats, ChildNameReserved, ActorRef } +import akka.dispatch.SystemMessage /** * INTERNAL API @@ -43,7 +44,7 @@ private[akka] object ChildrenContainer { sealed trait SuspendReason case object UserRequest extends SuspendReason - case class Recreation(cause: Throwable) extends SuspendReason + case class Recreation(cause: Throwable, var todo: SystemMessage = null) extends SuspendReason case object Termination extends SuspendReason trait EmptyChildrenContainer extends ChildrenContainer { diff --git a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala index 9907f036a9..9cdfdb2d37 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala @@ -5,11 +5,11 @@ package akka.actor.cell import scala.annotation.tailrec - import akka.actor.{ PreRestartException, PostRestartException, InternalActorRef, Failed, ActorRef, ActorInterruptedException, ActorCell, Actor } import akka.dispatch.{ Envelope, ChildTerminated } import akka.event.Logging.{ Warning, Error, Debug } import scala.util.control.NonFatal +import akka.dispatch.SystemMessage private[akka] trait FaultHandling { this: ActorCell ⇒ @@ -189,7 +189,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ case None ⇒ publish(Warning(self.path.toString, clazz(actor), "dropping Failed(" + cause + ") from unknown child " + child)) } - final protected def handleChildTerminated(child: ActorRef): Unit = { + final protected def handleChildTerminated(child: ActorRef): SystemMessage = { val status = removeChildAndGetStateChange(child) /* * if this fails, we do nothing in case of terminating/restarting state, @@ -205,9 +205,9 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ * then we are continuing the previously suspended recreate/terminate action */ status match { - case Some(ChildrenContainer.Recreation(cause)) ⇒ finishRecreate(cause, actor) - case Some(ChildrenContainer.Termination) ⇒ finishTerminate() - case _ ⇒ + case Some(ChildrenContainer.Recreation(cause, todo)) ⇒ finishRecreate(cause, actor); SystemMessage.reverse(todo) + case Some(ChildrenContainer.Termination) ⇒ finishTerminate(); null + case _ ⇒ null } } } From 810d65068ec57fe5554ea9146baca26f0a41e578 Mon Sep 17 00:00:00 2001 From: Roland Date: Tue, 7 Aug 2012 22:11:40 +0200 Subject: [PATCH 11/16] test exceptions from actor constructor in hierarchy, see #2336 This has brought to light some interesting effects (aka bugs) both in the general implementation as well as in previous fixes. SupervisorHierarchySpec is without TODOs now and GREEN. --- .../test/scala/akka/actor/FSMTimingSpec.scala | 2 +- .../akka/actor/SupervisorHierarchySpec.scala | 172 +++++++++++++----- .../akka/actor/dispatch/ActorModelSpec.scala | 2 +- .../dispatch/PriorityDispatcherSpec.scala | 2 +- .../src/main/scala/akka/actor/Actor.scala | 6 +- .../src/main/scala/akka/actor/ActorCell.scala | 2 +- .../src/main/scala/akka/actor/ActorRef.scala | 6 +- .../main/scala/akka/actor/FaultHandling.scala | 4 +- .../akka/actor/RepointableActorRef.scala | 4 +- .../main/scala/akka/actor/cell/Children.scala | 6 +- .../main/scala/akka/actor/cell/Dispatch.scala | 2 +- .../scala/akka/actor/cell/FaultHandling.scala | 26 ++- .../akka/dispatch/AbstractDispatcher.scala | 2 +- .../main/scala/akka/dispatch/Mailbox.scala | 3 + .../src/main/scala/akka/agent/Agent.scala | 2 +- .../akka/remote/RemoteActorRefProvider.scala | 2 +- .../testkit/CallingThreadDispatcher.scala | 2 +- 17 files changed, 167 insertions(+), 78 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index 1fa9a1ff03..049a5891e2 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -147,7 +147,7 @@ object FSMTimingSpec { } def resume(actorRef: ActorRef): Unit = actorRef match { - case l: ActorRefWithCell ⇒ l.resume(inResponseToFailure = false) + case l: ActorRefWithCell ⇒ l.resume(inResponseToFailure = null) case _ ⇒ } diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index ce35fd8c82..d455104378 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -22,6 +22,7 @@ import akka.testkit.{ filterException, duration2TestDuration, TestLatch } import akka.testkit.TestEvent.Mute import java.util.concurrent.ConcurrentHashMap import java.lang.ref.WeakReference +import akka.event.Logging object SupervisorHierarchySpec { class FireWorkerException(msg: String) extends Exception(msg) @@ -54,9 +55,11 @@ object SupervisorHierarchySpec { case class Ready(ref: ActorRef) case class Died(ref: ActorRef) case object Abort + case object PingOfDeath + case object PongOfDeath case class Event(msg: Any) { val time: Long = System.nanoTime } case class ErrorLog(msg: String, log: Vector[Event]) - case class Failure(directive: Directive, stop: Boolean, depth: Int, var failPre: Int, var failPost: Int, stopKids: Int) + case class Failure(directive: Directive, stop: Boolean, depth: Int, var failPre: Int, var failPost: Int, val failConstr: Int, stopKids: Int) extends RuntimeException with NoStackTrace { override def toString = productPrefix + productIterator.mkString("(", ",", ")") } @@ -83,15 +86,19 @@ object SupervisorHierarchySpec { Duration(config.getMilliseconds("shutdown-timeout"), TimeUnit.MILLISECONDS)) { override def suspend(cell: ActorCell): Unit = { - val a = cell.actor.asInstanceOf[Hierarchy] - a.log :+= Event("suspended " + cell.mailbox.status / 4) + cell.actor match { + case h: Hierarchy ⇒ h.log :+= Event("suspended " + cell.mailbox.suspendCount) + case _ ⇒ + } super.suspend(cell) } override def resume(cell: ActorCell): Unit = { - val a = cell.actor.asInstanceOf[Hierarchy] super.resume(cell) - a.log :+= Event("resumed " + cell.mailbox.status / 4) + cell.actor match { + case h: Hierarchy ⇒ h.log :+= Event("resumed " + cell.mailbox.suspendCount) + case _ ⇒ + } } } @@ -104,11 +111,20 @@ object SupervisorHierarchySpec { * upon Restart or would have to be managed by the highest supervisor (which * is undesirable). */ - case class HierarchyState(log: Vector[Event], kids: Map[ActorRef, Int]) + case class HierarchyState(log: Vector[Event], kids: Map[ActorRef, Int], failConstr: Failure) val stateCache = new ConcurrentHashMap[ActorRef, HierarchyState]() class Hierarchy(size: Int, breadth: Int, listener: ActorRef, myLevel: Int) extends Actor { + var log = Vector.empty[Event] + + stateCache.get(self) match { + case hs @ HierarchyState(_, _, f: Failure) if f.failConstr > 0 ⇒ + stateCache.put(self, hs.copy(failConstr = f.copy(failConstr = f.failConstr - 1))) + throw f + case _ ⇒ + } + var failed = false var suspended = false @@ -125,7 +141,7 @@ object SupervisorHierarchySpec { case _ ⇒ } - def suspendCount = context.asInstanceOf[ActorCell].mailbox.status / 4 + def suspendCount = context.asInstanceOf[ActorCell].mailbox.suspendCount override def preStart { log :+= Event("started") @@ -143,7 +159,7 @@ object SupervisorHierarchySpec { (context.watch(context.actorOf(props, id.toString)), kidSize) }(collection.breakOut) } else Map() - stateCache.put(self, HierarchyState(log, kidInfo)) + stateCache.put(self, HierarchyState(log, kidInfo, null)) } var preRestartCalled = false @@ -170,20 +186,31 @@ object SupervisorHierarchySpec { } } - override val supervisorStrategy = OneForOneStrategy() { - case Failure(directive, stop, 0, _, failPost, _) ⇒ - log :+= Event("applying (" + directive + ", " + stop + ", " + failPost + ") to " + sender) - if (myLevel > 3 && failPost == 0 && stop) Stop else directive - case PostRestartException(_, Failure(directive, stop, 0, _, failPost, _), _) ⇒ - log :+= Event("re-applying (" + directive + ", " + stop + ", " + failPost + ") to " + sender) - if (myLevel > 3 && failPost == 0 && stop) Stop else directive - case f: Failure ⇒ - import SupervisorStrategy._ - setFlags(f.directive) - log :+= Event("escalating " + f) - throw f.copy(depth = f.depth - 1) - case _ ⇒ sender ! Dump(0); Resume + val unwrap: PartialFunction[Throwable, (Throwable, Throwable)] = { + case x @ PostRestartException(_, f: Failure, _) ⇒ (f, x) + case x @ ActorInitializationException(_, _, f: Failure) ⇒ (f, x) + case x ⇒ (x, x) } + override val supervisorStrategy = OneForOneStrategy()(unwrap andThen { + case _ if pongsToGo > 0 ⇒ Resume + case (f: Failure, orig) ⇒ + if (f.depth > 0) { + setFlags(f.directive) + log :+= Event("escalating " + f) + throw f.copy(depth = f.depth - 1) + } + val prefix = orig match { + case f: Failure ⇒ "applying " + case _ ⇒ "re-applying " + } + log :+= Event(prefix + f + " to " + sender) + if (myLevel > 3 && f.failPost == 0 && f.stop) Stop else f.directive + case (_, x) ⇒ + log :+= Event("unhandled exception" + Logging.stackTraceFor(x)) + sender ! Dump(0) + context.system.scheduler.scheduleOnce(1 second, self, Dump(0)) + Resume + }) override def postRestart(cause: Throwable) { val state = stateCache.get(self) @@ -194,7 +221,7 @@ object SupervisorHierarchySpec { val name = child.path.name if (context.actorFor(name).isTerminated) { listener ! Died(child) - val props = Props(new Hierarchy(kidSize, breadth, listener, myLevel + 1)) + val props = Props(new Hierarchy(kidSize, breadth, listener, myLevel + 1)).withDispatcher("hierarchy") context.watch(context.actorOf(props, name)) } } @@ -208,10 +235,11 @@ object SupervisorHierarchySpec { override def postStop { if (failed || suspended) { listener ! ErrorLog("not resumed (" + failed + ", " + suspended + ")", log) + } else { + stateCache.put(self, HierarchyState(log, Map(), null)) } } - var log = Vector.empty[Event] def check(msg: Any): Boolean = { suspended = false log :+= Event(msg) @@ -222,12 +250,21 @@ object SupervisorHierarchySpec { } else if (context.asInstanceOf[ActorCell].mailbox.isSuspended) { abort("processing message while suspended") false + } else if (!Thread.currentThread.getName.startsWith("SupervisorHierarchySpec-hierarchy")) { + abort("running on wrong thread " + Thread.currentThread + " dispatcher=" + context.props.dispatcher + "=>" + + context.asInstanceOf[ActorCell].dispatcher.id) + false } else true } + var pongsToGo = 0 + def receive = new Receive { val handler: Receive = { - case f: Failure ⇒ setFlags(f.directive); throw f + case f: Failure ⇒ + setFlags(f.directive) + stateCache.put(self, stateCache.get(self).copy(failConstr = f.copy())) + throw f case "ping" ⇒ Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" case Dump(0) ⇒ abort("dump") case Dump(level) ⇒ context.children foreach (_ ! Dump(level - 1)) @@ -237,13 +274,27 @@ object SupervisorHierarchySpec { * (if the unwatch() came too late), so just ignore in this case. */ val name = ref.path.name - if (context.actorFor(name).isTerminated) { + if (pongsToGo == 0 && context.actorFor(name).isTerminated) { listener ! Died(ref) val kids = stateCache.get(self).kids(ref) - val props = Props(new Hierarchy(kids, breadth, listener, myLevel + 1)) + val props = Props(new Hierarchy(kids, breadth, listener, myLevel + 1)).withDispatcher("hierarchy") context.watch(context.actorOf(props, name)) } case Abort ⇒ abort("terminating") + case PingOfDeath ⇒ + if (size > 1) { + pongsToGo = context.children.size + context.children foreach (_ ! PingOfDeath) + } else { + context stop self + context.parent ! PongOfDeath + } + case PongOfDeath ⇒ + pongsToGo -= 1 + if (pongsToGo == 0) { + context stop self + context.parent ! PongOfDeath + } } override def isDefinedAt(msg: Any) = handler.isDefinedAt(msg) override def apply(msg: Any) = { if (check(msg)) handler(msg) } @@ -289,6 +340,9 @@ object SupervisorHierarchySpec { * 100 millis * - when receiving a Work() while all actors are "pinged", stop the * hierarchy and go to the Stopping state + * - make sure to remove all actors which die in the course of the test + * from the pinged and idle sets (others will be spawned from within the + * hierarchy) * * Finishing: * - after dealing out the last action, wait for the outstanding "pong" @@ -320,8 +374,6 @@ object SupervisorHierarchySpec { * happen that the new guy sends his Ready before the FSM has gotten the * Terminated, which would screw things up big time. Solution is to let the * supervisor do all, including notifying the FSM of the death of the guy. - * - * TODO RK: test exceptions in constructor */ class StressTest(testActor: ActorRef, size: Int, breadth: Int) extends Actor with LoggingFSM[State, Int] { @@ -329,9 +381,10 @@ object SupervisorHierarchySpec { // don’t escalate from this one! override val supervisorStrategy = OneForOneStrategy() { - case f: Failure ⇒ f.directive - case OriginalRestartException(f: Failure) ⇒ f.directive - case _ ⇒ Stop + case f: Failure ⇒ f.directive + case OriginalRestartException(f: Failure) ⇒ f.directive + case ActorInitializationException(f: Failure) ⇒ f.directive + case _ ⇒ Stop } var children = Vector.empty[ActorRef] @@ -424,7 +477,7 @@ object SupervisorHierarchySpec { nextJob.next match { case Ping(ref) ⇒ ref ! "ping" case Fail(ref, dir) ⇒ - val f = Failure(dir, stop = random012 > 0, depth = random012, failPre = random012, failPost = random012, + val f = Failure(dir, stop = random012 > 0, depth = random012, failPre = random012, failPost = random012, failConstr = random012, stopKids = random012 match { case 0 ⇒ 0 case 1 ⇒ Random.nextInt(breadth / 2) @@ -482,10 +535,11 @@ object SupervisorHierarchySpec { onTransition { case _ -> Stopping ⇒ ignoreNotResumedLogs = false - context stop hierarchy + hierarchy ! PingOfDeath } when(Stopping, stateTimeout = 5 seconds) { + case Event(PongOfDeath, _) ⇒ stay case Event(Terminated(r), _) if r == hierarchy ⇒ val undead = children filterNot (_.isTerminated) if (undead.nonEmpty) { @@ -548,34 +602,50 @@ object SupervisorHierarchySpec { testActor ! "stressTestFailed" stop case Event(StateTimeout, _) ⇒ - getErrors() + getErrors(hierarchy, 2) printErrors() testActor ! "timeout in Failed" stop - case Event("pong", _) ⇒ stay // don’t care? - case Event(Work, _) ⇒ stay + case Event("pong", _) ⇒ stay // don’t care? + case Event(Work, _) ⇒ stay + case Event(Died(_), _) ⇒ stay } - def getErrors() = { - def rec(target: ActorRef, depth: Int): Unit = { - target match { - case l: LocalActorRef ⇒ - errors :+= target -> ErrorLog("forced", l.underlying.actor.asInstanceOf[Hierarchy].log) - if (depth > 0) { - l.underlying.children foreach (rec(_, depth - 1)) - } - } + def getErrors(target: ActorRef, depth: Int): Unit = { + target match { + case l: LocalActorRef ⇒ + l.underlying.actor match { + case h: Hierarchy ⇒ errors :+= target -> ErrorLog("forced", h.log) + case _ ⇒ errors :+= target -> ErrorLog("fetched", stateCache.get(target).log) + } + if (depth > 0) { + l.underlying.children foreach (getErrors(_, depth - 1)) + } + } + } + + def getErrorsUp(target: ActorRef): Unit = { + target match { + case l: LocalActorRef ⇒ + l.underlying.actor match { + case h: Hierarchy ⇒ errors :+= target -> ErrorLog("forced", h.log) + case _ ⇒ errors :+= target -> ErrorLog("fetched", stateCache.get(target).log) + } + if (target != hierarchy) getErrorsUp(l.getParent) } - rec(hierarchy, 2) } def printErrors(): Unit = { + errors collect { + case (origin, ErrorLog("dump", _)) ⇒ getErrors(origin, 1) + case (origin, ErrorLog(msg, _)) if msg startsWith "not resumed" ⇒ getErrorsUp(origin) + } val merged = errors.sortBy(_._1.toString) flatMap { case (ref, ErrorLog(msg, log)) ⇒ println(ref + " " + msg) log map (l ⇒ (l.time, ref, l.msg.toString)) } - merged.sorted foreach println + merged.sorted.distinct foreach println } whenUnhandled { @@ -704,8 +774,12 @@ class SupervisorHierarchySpec extends AkkaSpec(SupervisorHierarchySpec.config) w } "survive being stressed" in { - system.eventStream.publish(Mute(EventFilter[Failure](), EventFilter[PreRestartException](), EventFilter[PostRestartException]())) - system.eventStream.publish(Mute(EventFilter.warning(start = "received dead "))) + system.eventStream.publish(Mute( + EventFilter[Failure](), + EventFilter[ActorInitializationException](), + EventFilter[NoSuchElementException]("head of empty list"), + EventFilter.error(start = "changing Resume into Restart"), + EventFilter.warning(start = "received dead "))) val fsm = system.actorOf(Props(new StressTest(testActor, size = 500, breadth = 6)), "stressTest") diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index b5d284d7af..52b3686aa1 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -348,7 +348,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa assertNoCountDown(done, 1000, "Should not process messages while suspended") assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, suspensions = 1) - a.resume(inResponseToFailure = false) + a.resume(inResponseToFailure = null) assertCountDown(done, 3.seconds.dilated.toMillis, "Should resume processing of messages when resumed") assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, msgsProcessed = 1, suspensions = 1, resumes = 1) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala index 1d0f3ec416..41ce7db15f 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -65,7 +65,7 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit val msgs = (1 to 100).toList for (m ← msgs) actor ! m - actor.resume(inResponseToFailure = false) //Signal the actor to start treating it's message backlog + actor.resume(inResponseToFailure = null) //Signal the actor to start treating it's message backlog Await.result(actor.?('Result).mapTo[List[Int]], timeout.duration) must be === msgs.reverse } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index ea78bb705c..86f910ac08 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -153,7 +153,8 @@ object ActorInitializationException { * @param msg is the message which was optionally passed into preRestart() */ case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, origCause: Throwable, msg: Option[Any]) - extends ActorInitializationException(actor, "exception in preRestart(" + origCause.getClass + ", " + msg.map(_.getClass) + ")", cause) + extends ActorInitializationException(actor, + "exception in preRestart(" + (if (origCause == null) "null" else origCause.getClass) + ", " + msg.map(_.getClass) + ")", cause) /** * A PostRestartException is thrown when constructor or postRestart() method @@ -164,7 +165,8 @@ case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, * @param origCause is the exception which caused the restart in the first place */ case class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, origCause: Throwable) - extends ActorInitializationException(actor, "exception post restart (" + origCause.getClass + ")", cause) + extends ActorInitializationException(actor, + "exception post restart (" + (if (origCause == null) "null" else origCause.getClass) + ")", cause) /** * This is an extractor for retrieving the original cause (i.e. the first diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index e7808b2cc9..09523fbab3 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -188,7 +188,7 @@ private[akka] trait Cell { /** * Recursively resume this actor and all its children. */ - def resume(inResponseToFailure: Boolean): Unit + def resume(inResponseToFailure: Throwable): Unit /** * Restart this actor (will recursively restart or stop all children). */ diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index 00a84f956a..e8cdf2955e 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -191,7 +191,7 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe /* * Actor life-cycle management, invoked only internally (in response to user requests via ActorContext). */ - def resume(inResponseToFailure: Boolean): Unit + def resume(inResponseToFailure: Throwable): Unit def suspend(): Unit def restart(cause: Throwable): Unit def stop(): Unit @@ -288,7 +288,7 @@ private[akka] class LocalActorRef private[akka] ( /** * Resumes a suspended actor. */ - override def resume(inResponseToFailure: Boolean): Unit = actorCell.resume(inResponseToFailure) + override def resume(inResponseToFailure: Throwable): Unit = actorCell.resume(inResponseToFailure) /** * Shuts down the actor and its message queue @@ -388,7 +388,7 @@ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef { override def getChild(names: Iterator[String]): InternalActorRef = if (names.forall(_.isEmpty)) this else Nobody override def suspend(): Unit = () - override def resume(inResponseToFailure: Boolean): Unit = () + override def resume(inResponseToFailure: Throwable): Unit = () override def stop(): Unit = () override def isTerminated = false diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 7b1da320b6..64a4c2c7b1 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -253,7 +253,7 @@ abstract class SupervisorStrategy { def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Boolean = { val directive = if (decider.isDefinedAt(cause)) decider(cause) else Escalate //FIXME applyOrElse in Scala 2.10 directive match { - case Resume ⇒ resumeChild(child); true + case Resume ⇒ resumeChild(child, cause); true case Restart ⇒ processFailure(context, true, child, cause, stats, children); true case Stop ⇒ processFailure(context, false, child, cause, stats, children); true case Escalate ⇒ false @@ -265,7 +265,7 @@ abstract class SupervisorStrategy { * is not the currently failing child. Suspend/resume needs to be done in * matching pairs, otherwise actors will wake up too soon or never at all. */ - final def resumeChild(child: ActorRef): Unit = child.asInstanceOf[InternalActorRef].resume(inResponseToFailure = true) + final def resumeChild(child: ActorRef, cause: Throwable): Unit = child.asInstanceOf[InternalActorRef].resume(inResponseToFailure = cause) /** * Restart the given child, possibly suspending it first. diff --git a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala index caad67503a..7977be5e5b 100644 --- a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala @@ -81,7 +81,7 @@ private[akka] class RepointableActorRef( def suspend(): Unit = underlying.suspend() - def resume(inResponseToFailure: Boolean): Unit = underlying.resume(inResponseToFailure) + def resume(inResponseToFailure: Throwable): Unit = underlying.resume(inResponseToFailure) def stop(): Unit = underlying.stop() @@ -171,7 +171,7 @@ private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, val self: Rep def system: ActorSystem = systemImpl def suspend(): Unit = { lock.lock(); try suspendCount += 1 finally lock.unlock() } - def resume(inResponseToFailure: Boolean): Unit = { lock.lock(); try suspendCount -= 1 finally lock.unlock() } + def resume(inResponseToFailure: Throwable): Unit = { lock.lock(); try suspendCount -= 1 finally lock.unlock() } def restart(cause: Throwable): Unit = { lock.lock(); try suspendCount -= 1 finally lock.unlock() } def stop(): Unit = sendSystemMessage(Terminate()) def isTerminated: Boolean = false diff --git a/akka-actor/src/main/scala/akka/actor/cell/Children.scala b/akka-actor/src/main/scala/akka/actor/cell/Children.scala index 202c5e4016..900bc60a0c 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/Children.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/Children.scala @@ -124,10 +124,10 @@ private[akka] trait Children { this: ActorCell ⇒ case _ ⇒ } - protected def resumeChildren(inResponseToFailure: Boolean, perp: ActorRef): Unit = + protected def resumeChildren(inResponseToFailure: Throwable, perp: ActorRef): Unit = childrenRefs.stats foreach { case ChildRestartStats(child: InternalActorRef, _, _) ⇒ - child.resume(inResponseToFailure = inResponseToFailure && perp == child) + child.resume(if (perp == child) inResponseToFailure else null) } def getChildByName(name: String): Option[ChildRestartStats] = childrenRefs.getByName(name) @@ -188,6 +188,8 @@ private[akka] trait Children { this: ActorCell ⇒ unreserveChild(name) throw e } + // mailbox==null during RoutedActorCell constructor, where suspends are queued otherwise + if (mailbox ne null) for (_ ← 1 to mailbox.suspendCount) actor.suspend() addChild(actor) actor } diff --git a/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala b/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala index 8c849366d8..a473e4df66 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala @@ -62,7 +62,7 @@ private[akka] trait Dispatch { this: ActorCell ⇒ final def suspend(): Unit = dispatcher.systemDispatch(this, Suspend()) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ - final def resume(inResponseToFailure: Boolean): Unit = dispatcher.systemDispatch(this, Resume(inResponseToFailure)) + final def resume(inResponseToFailure: Throwable): Unit = dispatcher.systemDispatch(this, Resume(inResponseToFailure)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ final def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause)) diff --git a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala index 9cdfdb2d37..feabe3104d 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala @@ -10,6 +10,7 @@ import akka.dispatch.{ Envelope, ChildTerminated } import akka.event.Logging.{ Warning, Error, Debug } import scala.util.control.NonFatal import akka.dispatch.SystemMessage +import akka.event.Logging private[akka] trait FaultHandling { this: ActorCell ⇒ @@ -66,7 +67,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ if (!setChildrenTerminationReason(ChildrenContainer.Recreation(cause))) finishRecreate(cause, failedActor) } else { // need to keep that suspend counter balanced - faultResume(inResponseToFailure = false) + faultResume(inResponseToFailure = null) } /** @@ -85,13 +86,19 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ * @param inResponseToFailure signifies if it was our own failure which * prompted this action. */ - protected def faultResume(inResponseToFailure: Boolean): Unit = { - val perp = perpetrator - // done always to keep that suspend counter balanced - // must happen “atomically” - try resumeNonRecursive() - finally if (inResponseToFailure) clearFailed() - resumeChildren(inResponseToFailure, perp) + protected def faultResume(inResponseToFailure: Throwable): Unit = { + if ((actor == null || actor.context == null) && inResponseToFailure != null) { + system.eventStream.publish(Error(self.path.toString, clazz(actor), + "changing Resume into Restart after " + inResponseToFailure)) + faultRecreate(inResponseToFailure) + } else { + val perp = perpetrator + // done always to keep that suspend counter balanced + // must happen “atomically” + try resumeNonRecursive() + finally if (inResponseToFailure != null) clearFailed() + resumeChildren(inResponseToFailure, perp) + } } protected def terminate() { @@ -135,7 +142,8 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ } } catch { case NonFatal(e) ⇒ - publish(Error(e, self.path.toString, clazz(actor), "emergency stop: exception in failure handling")) + publish(Error(e, self.path.toString, clazz(actor), + "emergency stop: exception in failure handling for " + t.getClass + Logging.stackTraceFor(t))) try children foreach stop finally finishTerminate() } diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index fff56a3776..34047677aa 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -90,7 +90,7 @@ private[akka] case class Suspend() extends SystemMessage // sent to self from Ac /** * INTERNAL API */ -private[akka] case class Resume(inResponseToFailure: Boolean) extends SystemMessage // sent to self from ActorCell.resume +private[akka] case class Resume(inResponseToFailure: Throwable) extends SystemMessage // sent to self from ActorCell.resume /** * INTERNAL API */ diff --git a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala index f1f887068b..fd7057a963 100644 --- a/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala +++ b/akka-actor/src/main/scala/akka/dispatch/Mailbox.scala @@ -107,6 +107,9 @@ private[akka] abstract class Mailbox(val messageQueue: MessageQueue) @inline final def shouldProcessMessage: Boolean = (status & shouldNotProcessMask) == 0 + @inline + final def suspendCount: Int = status / suspendUnit + @inline final def isSuspended: Boolean = (status & suspendMask) != 0 diff --git a/akka-agent/src/main/scala/akka/agent/Agent.scala b/akka-agent/src/main/scala/akka/agent/Agent.scala index bd77c1d9ff..03ba363c89 100644 --- a/akka-agent/src/main/scala/akka/agent/Agent.scala +++ b/akka-agent/src/main/scala/akka/agent/Agent.scala @@ -214,7 +214,7 @@ class Agent[T](initialValue: T, system: ActorSystem) { /** * Resumes processing of `send` actions for the agent. */ - def resume(): Unit = updater.resume(inResponseToFailure = false) + def resume(): Unit = updater.resume(inResponseToFailure = null) /** * Closes the agents and makes it eligible for garbage collection. diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index cff9dda274..910c57502a 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -235,7 +235,7 @@ private[akka] class RemoteActorRef private[akka] ( def suspend(): Unit = sendSystemMessage(Suspend()) - def resume(inResponseToFailure: Boolean): Unit = sendSystemMessage(Resume(inResponseToFailure)) + def resume(inResponseToFailure: Throwable): Unit = sendSystemMessage(Resume(inResponseToFailure)) def stop(): Unit = sendSystemMessage(Terminate()) diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index d25a156917..5028ab7dc9 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -171,7 +171,7 @@ class CallingThreadDispatcher( if (switched && !wasActive) { runQueue(mbox, queue) } - case m ⇒ m.systemEnqueue(actor.self, Resume(false)) + case m ⇒ m.systemEnqueue(actor.self, Resume(inResponseToFailure = null)) } } From 6145d4313bdec98ec847f7fb8eda91c4cd51bc2a Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 8 Aug 2012 14:13:52 +0200 Subject: [PATCH 12/16] =?UTF-8?q?some=20cleanup=20after=20Viktor=E2=80=99s?= =?UTF-8?q?=20comments?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit - remove useless `cause` argument from some akka exceptions - improve names of some method arguments - eliminate some closure allocation --- .../test/scala/akka/actor/FSMTimingSpec.scala | 2 +- .../akka/actor/dispatch/ActorModelSpec.scala | 2 +- .../dispatch/PriorityDispatcherSpec.scala | 2 +- .../src/main/scala/akka/actor/Actor.scala | 30 ++++++++----------- .../src/main/scala/akka/actor/ActorCell.scala | 2 +- .../src/main/scala/akka/actor/ActorRef.scala | 6 ++-- .../main/scala/akka/actor/FaultHandling.scala | 2 +- .../akka/actor/RepointableActorRef.scala | 4 +-- .../main/scala/akka/actor/cell/Children.scala | 8 ++--- .../akka/actor/cell/ChildrenContainer.scala | 1 + .../main/scala/akka/actor/cell/Dispatch.scala | 2 +- .../scala/akka/actor/cell/FaultHandling.scala | 18 +++++------ .../akka/dispatch/AbstractDispatcher.scala | 2 +- .../src/main/scala/akka/agent/Agent.scala | 2 +- .../akka/remote/RemoteActorRefProvider.scala | 2 +- .../testkit/CallingThreadDispatcher.scala | 2 +- 16 files changed, 41 insertions(+), 46 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala index 049a5891e2..39b4299af1 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/FSMTimingSpec.scala @@ -147,7 +147,7 @@ object FSMTimingSpec { } def resume(actorRef: ActorRef): Unit = actorRef match { - case l: ActorRefWithCell ⇒ l.resume(inResponseToFailure = null) + case l: ActorRefWithCell ⇒ l.resume(causedByFailure = null) case _ ⇒ } diff --git a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala index 52b3686aa1..64d283d72a 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/dispatch/ActorModelSpec.scala @@ -348,7 +348,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa assertNoCountDown(done, 1000, "Should not process messages while suspended") assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, suspensions = 1) - a.resume(inResponseToFailure = null) + a.resume(causedByFailure = null) assertCountDown(done, 3.seconds.dilated.toMillis, "Should resume processing of messages when resumed") assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, msgsProcessed = 1, suspensions = 1, resumes = 1) diff --git a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala index 41ce7db15f..a1de1f84bd 100644 --- a/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/dispatch/PriorityDispatcherSpec.scala @@ -65,7 +65,7 @@ class PriorityDispatcherSpec extends AkkaSpec(PriorityDispatcherSpec.config) wit val msgs = (1 to 100).toList for (m ← msgs) actor ! m - actor.resume(inResponseToFailure = null) //Signal the actor to start treating it's message backlog + actor.resume(causedByFailure = null) //Signal the actor to start treating it's message backlog Await.result(actor.?('Result).mapTo[List[Int]], timeout.duration) must be === msgs.reverse } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 86f910ac08..04a9499ebf 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -8,6 +8,7 @@ import akka.AkkaException import scala.reflect.BeanProperty import scala.util.control.NoStackTrace import java.util.regex.Pattern +import scala.annotation.tailrec /** * Marker trait to show which Messages are automatically handled by Akka @@ -99,19 +100,12 @@ private[akka] case class SelectParent(next: Any) extends SelectionPath * IllegalActorStateException is thrown when a core invariant in the Actor implementation has been violated. * For instance, if you try to create an Actor that doesn't extend Actor. */ -case class IllegalActorStateException private[akka] (message: String, cause: Throwable = null) - extends AkkaException(message, cause) { - def this(msg: String) = this(msg, null) -} +case class IllegalActorStateException private[akka] (message: String) extends AkkaException(message) /** * ActorKilledException is thrown when an Actor receives the akka.actor.Kill message */ -case class ActorKilledException private[akka] (message: String, cause: Throwable) - extends AkkaException(message, cause) - with NoStackTrace { - def this(msg: String) = this(msg, null) -} +case class ActorKilledException private[akka] (message: String) extends AkkaException(message) with NoStackTrace /** * An InvalidActorNameException is thrown when you try to convert something, usually a String, to an Actor name @@ -152,9 +146,12 @@ object ActorInitializationException { * @param origCause is the exception which caused the restart in the first place * @param msg is the message which was optionally passed into preRestart() */ -case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, origCause: Throwable, msg: Option[Any]) +case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, originalCause: Throwable, messageOption: Option[Any]) extends ActorInitializationException(actor, - "exception in preRestart(" + (if (origCause == null) "null" else origCause.getClass) + ", " + msg.map(_.getClass) + ")", cause) + "exception in preRestart(" + + (if (originalCause == null) "null" else originalCause.getClass) + ", " + + (messageOption match { case Some(m: AnyRef) ⇒ m.getClass; case _ ⇒ "None" }) + + ")", cause) /** * A PostRestartException is thrown when constructor or postRestart() method @@ -164,9 +161,9 @@ case class PreRestartException private[akka] (actor: ActorRef, cause: Throwable, * @param cause is the exception thrown by that actor within preRestart() * @param origCause is the exception which caused the restart in the first place */ -case class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, origCause: Throwable) +case class PostRestartException private[akka] (actor: ActorRef, cause: Throwable, originalCause: Throwable) extends ActorInitializationException(actor, - "exception post restart (" + (if (origCause == null) "null" else origCause.getClass) + ")", cause) + "exception post restart (" + (if (originalCause == null) "null" else originalCause.getClass) + ")", cause) /** * This is an extractor for retrieving the original cause (i.e. the first @@ -176,7 +173,7 @@ case class PostRestartException private[akka] (actor: ActorRef, cause: Throwable */ object OriginalRestartException { def unapply(ex: PostRestartException): Option[Throwable] = { - def rec(ex: PostRestartException): Option[Throwable] = ex match { + @tailrec def rec(ex: PostRestartException): Option[Throwable] = ex match { case PostRestartException(_, _, e: PostRestartException) ⇒ rec(e) case PostRestartException(_, _, e) ⇒ Some(e) } @@ -188,10 +185,7 @@ object OriginalRestartException { * InvalidMessageException is thrown when an invalid message is sent to an Actor; * Currently only `null` is an invalid message. */ -case class InvalidMessageException private[akka] (message: String, cause: Throwable = null) - extends AkkaException(message, cause) { - def this(msg: String) = this(msg, null) -} +case class InvalidMessageException private[akka] (message: String) extends AkkaException(message) /** * A DeathPactException is thrown by an Actor that receives a Terminated(someActor) message diff --git a/akka-actor/src/main/scala/akka/actor/ActorCell.scala b/akka-actor/src/main/scala/akka/actor/ActorCell.scala index 09523fbab3..13c449d99c 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorCell.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorCell.scala @@ -188,7 +188,7 @@ private[akka] trait Cell { /** * Recursively resume this actor and all its children. */ - def resume(inResponseToFailure: Throwable): Unit + def resume(causedByFailure: Throwable): Unit /** * Restart this actor (will recursively restart or stop all children). */ diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index e8cdf2955e..d74fc5293d 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -191,7 +191,7 @@ private[akka] abstract class InternalActorRef extends ActorRef with ScalaActorRe /* * Actor life-cycle management, invoked only internally (in response to user requests via ActorContext). */ - def resume(inResponseToFailure: Throwable): Unit + def resume(causedByFailure: Throwable): Unit def suspend(): Unit def restart(cause: Throwable): Unit def stop(): Unit @@ -288,7 +288,7 @@ private[akka] class LocalActorRef private[akka] ( /** * Resumes a suspended actor. */ - override def resume(inResponseToFailure: Throwable): Unit = actorCell.resume(inResponseToFailure) + override def resume(causedByFailure: Throwable): Unit = actorCell.resume(causedByFailure) /** * Shuts down the actor and its message queue @@ -388,7 +388,7 @@ private[akka] trait MinimalActorRef extends InternalActorRef with LocalRef { override def getChild(names: Iterator[String]): InternalActorRef = if (names.forall(_.isEmpty)) this else Nobody override def suspend(): Unit = () - override def resume(inResponseToFailure: Throwable): Unit = () + override def resume(causedByFailure: Throwable): Unit = () override def stop(): Unit = () override def isTerminated = false diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 64a4c2c7b1..9ef60f2316 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -265,7 +265,7 @@ abstract class SupervisorStrategy { * is not the currently failing child. Suspend/resume needs to be done in * matching pairs, otherwise actors will wake up too soon or never at all. */ - final def resumeChild(child: ActorRef, cause: Throwable): Unit = child.asInstanceOf[InternalActorRef].resume(inResponseToFailure = cause) + final def resumeChild(child: ActorRef, cause: Throwable): Unit = child.asInstanceOf[InternalActorRef].resume(causedByFailure = cause) /** * Restart the given child, possibly suspending it first. diff --git a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala index 7977be5e5b..57a375dbad 100644 --- a/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/RepointableActorRef.scala @@ -81,7 +81,7 @@ private[akka] class RepointableActorRef( def suspend(): Unit = underlying.suspend() - def resume(inResponseToFailure: Throwable): Unit = underlying.resume(inResponseToFailure) + def resume(causedByFailure: Throwable): Unit = underlying.resume(causedByFailure) def stop(): Unit = underlying.stop() @@ -171,7 +171,7 @@ private[akka] class UnstartedCell(val systemImpl: ActorSystemImpl, val self: Rep def system: ActorSystem = systemImpl def suspend(): Unit = { lock.lock(); try suspendCount += 1 finally lock.unlock() } - def resume(inResponseToFailure: Throwable): Unit = { lock.lock(); try suspendCount -= 1 finally lock.unlock() } + def resume(causedByFailure: Throwable): Unit = { lock.lock(); try suspendCount -= 1 finally lock.unlock() } def restart(cause: Throwable): Unit = { lock.lock(); try suspendCount -= 1 finally lock.unlock() } def stop(): Unit = sendSystemMessage(Terminate()) def isTerminated: Boolean = false diff --git a/akka-actor/src/main/scala/akka/actor/cell/Children.scala b/akka-actor/src/main/scala/akka/actor/cell/Children.scala index 900bc60a0c..2b0fa76db4 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/Children.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/Children.scala @@ -118,16 +118,16 @@ private[akka] trait Children { this: ActorCell ⇒ case _ ⇒ null } - protected def suspendChildren(skip: Set[ActorRef] = Set.empty): Unit = + protected def suspendChildren(exceptFor: Set[ActorRef] = Set.empty): Unit = childrenRefs.stats foreach { - case ChildRestartStats(child, _, _) if !(skip contains child) ⇒ child.asInstanceOf[InternalActorRef].suspend() + case ChildRestartStats(child, _, _) if !(exceptFor contains child) ⇒ child.asInstanceOf[InternalActorRef].suspend() case _ ⇒ } - protected def resumeChildren(inResponseToFailure: Throwable, perp: ActorRef): Unit = + protected def resumeChildren(causedByFailure: Throwable, perp: ActorRef): Unit = childrenRefs.stats foreach { case ChildRestartStats(child: InternalActorRef, _, _) ⇒ - child.resume(if (perp == child) inResponseToFailure else null) + child.resume(if (perp == child) causedByFailure else null) } def getChildByName(name: String): Option[ChildRestartStats] = childrenRefs.getByName(name) diff --git a/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala b/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala index 09303bd24d..1fc21c08eb 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/ChildrenContainer.scala @@ -44,6 +44,7 @@ private[akka] object ChildrenContainer { sealed trait SuspendReason case object UserRequest extends SuspendReason + // careful with those system messages, all handling to be taking place in ActorCell.scala! case class Recreation(cause: Throwable, var todo: SystemMessage = null) extends SuspendReason case object Termination extends SuspendReason diff --git a/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala b/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala index a473e4df66..3e11071f82 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/Dispatch.scala @@ -62,7 +62,7 @@ private[akka] trait Dispatch { this: ActorCell ⇒ final def suspend(): Unit = dispatcher.systemDispatch(this, Suspend()) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ - final def resume(inResponseToFailure: Throwable): Unit = dispatcher.systemDispatch(this, Resume(inResponseToFailure)) + final def resume(causedByFailure: Throwable): Unit = dispatcher.systemDispatch(this, Resume(causedByFailure)) // ➡➡➡ NEVER SEND THE SAME SYSTEM MESSAGE OBJECT TO TWO ACTORS ⬅⬅⬅ final def restart(cause: Throwable): Unit = dispatcher.systemDispatch(this, Recreate(cause)) diff --git a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala index feabe3104d..b9155887e5 100644 --- a/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/cell/FaultHandling.scala @@ -67,7 +67,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ if (!setChildrenTerminationReason(ChildrenContainer.Recreation(cause))) finishRecreate(cause, failedActor) } else { // need to keep that suspend counter balanced - faultResume(inResponseToFailure = null) + faultResume(causedByFailure = null) } /** @@ -83,21 +83,21 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ /** * Do resume the actor in response to a failure. * - * @param inResponseToFailure signifies if it was our own failure which + * @param causedByFailure signifies if it was our own failure which * prompted this action. */ - protected def faultResume(inResponseToFailure: Throwable): Unit = { - if ((actor == null || actor.context == null) && inResponseToFailure != null) { + protected def faultResume(causedByFailure: Throwable): Unit = { + if ((actor == null || actor.context == null) && causedByFailure != null) { system.eventStream.publish(Error(self.path.toString, clazz(actor), - "changing Resume into Restart after " + inResponseToFailure)) - faultRecreate(inResponseToFailure) + "changing Resume into Restart after " + causedByFailure)) + faultRecreate(causedByFailure) } else { val perp = perpetrator // done always to keep that suspend counter balanced // must happen “atomically” try resumeNonRecursive() - finally if (inResponseToFailure != null) clearFailed() - resumeChildren(inResponseToFailure, perp) + finally if (causedByFailure != null) clearFailed() + resumeChildren(causedByFailure, perp) } } @@ -134,7 +134,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒ case Envelope(Failed(_), child) ⇒ setFailed(child); Set(child) case _ ⇒ setFailed(self); Set.empty } - suspendChildren(skip ++ childrenNotToSuspend) + suspendChildren(exceptFor = skip ++ childrenNotToSuspend) // tell supervisor t match { // Wrap InterruptedExceptions and rethrow case _: InterruptedException ⇒ parent.tell(Failed(new ActorInterruptedException(t)), self); throw t diff --git a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala index 34047677aa..31308ab4f7 100644 --- a/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala +++ b/akka-actor/src/main/scala/akka/dispatch/AbstractDispatcher.scala @@ -90,7 +90,7 @@ private[akka] case class Suspend() extends SystemMessage // sent to self from Ac /** * INTERNAL API */ -private[akka] case class Resume(inResponseToFailure: Throwable) extends SystemMessage // sent to self from ActorCell.resume +private[akka] case class Resume(causedByFailure: Throwable) extends SystemMessage // sent to self from ActorCell.resume /** * INTERNAL API */ diff --git a/akka-agent/src/main/scala/akka/agent/Agent.scala b/akka-agent/src/main/scala/akka/agent/Agent.scala index 03ba363c89..09bf6f2ad1 100644 --- a/akka-agent/src/main/scala/akka/agent/Agent.scala +++ b/akka-agent/src/main/scala/akka/agent/Agent.scala @@ -214,7 +214,7 @@ class Agent[T](initialValue: T, system: ActorSystem) { /** * Resumes processing of `send` actions for the agent. */ - def resume(): Unit = updater.resume(inResponseToFailure = null) + def resume(): Unit = updater.resume(causedByFailure = null) /** * Closes the agents and makes it eligible for garbage collection. diff --git a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala index 910c57502a..131b5d76be 100644 --- a/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala +++ b/akka-remote/src/main/scala/akka/remote/RemoteActorRefProvider.scala @@ -235,7 +235,7 @@ private[akka] class RemoteActorRef private[akka] ( def suspend(): Unit = sendSystemMessage(Suspend()) - def resume(inResponseToFailure: Throwable): Unit = sendSystemMessage(Resume(inResponseToFailure)) + def resume(causedByFailure: Throwable): Unit = sendSystemMessage(Resume(causedByFailure)) def stop(): Unit = sendSystemMessage(Terminate()) diff --git a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala index 5028ab7dc9..831033da03 100644 --- a/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala +++ b/akka-testkit/src/main/scala/akka/testkit/CallingThreadDispatcher.scala @@ -171,7 +171,7 @@ class CallingThreadDispatcher( if (switched && !wasActive) { runQueue(mbox, queue) } - case m ⇒ m.systemEnqueue(actor.self, Resume(inResponseToFailure = null)) + case m ⇒ m.systemEnqueue(actor.self, Resume(causedByFailure = null)) } } From d4ce9e67509a4a10cc04ca1d27ee426116b3d245 Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 8 Aug 2012 14:44:15 +0200 Subject: [PATCH 13/16] clarify ScalaDoc of SupervisorStrategy --- akka-actor/src/main/scala/akka/actor/FaultHandling.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala index 9ef60f2316..5de2653b04 100644 --- a/akka-actor/src/main/scala/akka/actor/FaultHandling.scala +++ b/akka-actor/src/main/scala/akka/actor/FaultHandling.scala @@ -273,7 +273,7 @@ abstract class SupervisorStrategy { * IMPORTANT: * * If the child is the currently failing one, it will already have been - * suspended, hence `suspendFirst` is false. If the child is not the + * suspended, hence `suspendFirst` must be false. If the child is not the * currently failing one, then it did not request this treatment and is * therefore not prepared to be resumed without prior suspend. */ From 5a42a46ea540d3214fa8134585ea092c0edee99f Mon Sep 17 00:00:00 2001 From: Roland Date: Wed, 8 Aug 2012 17:15:39 +0200 Subject: [PATCH 14/16] ease debugging of SupervisorHierarchySpec with more precise logging --- .../test/scala/akka/actor/SupervisorHierarchySpec.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala index 6139dc86ae..dadddf7b2d 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/SupervisorHierarchySpec.scala @@ -167,7 +167,7 @@ object SupervisorHierarchySpec { // do not scrap children if (preRestartCalled) abort("preRestart called twice") else { - log :+= Event("preRestart") + log :+= Event("preRestart " + cause) preRestartCalled = true cause match { case f: Failure ⇒ @@ -196,7 +196,7 @@ object SupervisorHierarchySpec { case (f: Failure, orig) ⇒ if (f.depth > 0) { setFlags(f.directive) - log :+= Event("escalating " + f) + log :+= Event("escalating " + f + " from " + sender) throw f.copy(depth = f.depth - 1) } val prefix = orig match { @@ -206,7 +206,7 @@ object SupervisorHierarchySpec { log :+= Event(prefix + f + " to " + sender) if (myLevel > 3 && f.failPost == 0 && f.stop) Stop else f.directive case (_, x) ⇒ - log :+= Event("unhandled exception" + Logging.stackTraceFor(x)) + log :+= Event("unhandled exception from " + sender + Logging.stackTraceFor(x)) sender ! Dump(0) context.system.scheduler.scheduleOnce(1 second, self, Dump(0)) Resume @@ -215,7 +215,7 @@ object SupervisorHierarchySpec { override def postRestart(cause: Throwable) { val state = stateCache.get(self) log = state.log - log :+= Event("restarted " + suspendCount) + log :+= Event("restarted " + suspendCount + " " + cause) state.kids foreach { case (child, kidSize) ⇒ val name = child.path.name From 8df9f30221ac9049e9a64afbe942479b5489466f Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Wed, 8 Aug 2012 22:26:03 +0200 Subject: [PATCH 15/16] =?UTF-8?q?Upgrade=20to=20Netty=203.5.3.Final=20as?= =?UTF-8?q?=20suggested=20by=20Bj=C3=B6rn=20Harrtell?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- project/AkkaBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/AkkaBuild.scala b/project/AkkaBuild.scala index a29f0ed290..36748fe553 100644 --- a/project/AkkaBuild.scala +++ b/project/AkkaBuild.scala @@ -532,7 +532,7 @@ object Dependency { // Compile val camelCore = "org.apache.camel" % "camel-core" % "2.8.0" exclude("org.slf4j", "slf4j-api") // ApacheV2 val config = "com.typesafe" % "config" % "0.5.0" // ApacheV2 - val netty = "io.netty" % "netty" % "3.5.1.Final" // ApacheV2 + val netty = "io.netty" % "netty" % "3.5.3.Final" // ApacheV2 val protobuf = "com.google.protobuf" % "protobuf-java" % "2.4.1" // New BSD val scalaStm = "org.scala-tools" %% "scala-stm" % "0.6" // Modified BSD (Scala) val slf4jApi = "org.slf4j" % "slf4j-api" % "1.6.4" // MIT From c1b2a32a3553b9d8e3a8f30db9aa485b17f25498 Mon Sep 17 00:00:00 2001 From: Viktor Klang Date: Thu, 9 Aug 2012 16:38:33 +0200 Subject: [PATCH 16/16] Review fixes, adding descriptions of the io-configuration parameters --- akka-actor/src/main/resources/reference.conf | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/akka-actor/src/main/resources/reference.conf b/akka-actor/src/main/resources/reference.conf index 8fbf16b427..723347ce98 100644 --- a/akka-actor/src/main/resources/reference.conf +++ b/akka-actor/src/main/resources/reference.conf @@ -321,14 +321,15 @@ akka { } io { - # In bytes, the size of the shared read buffer. In the span 0..Int.MaxValue + # In bytes, the size of the shared read buffer. In the span 0b..2GiB. # read-buffer-size = 8KiB - # + # Specifies how many ops are done between every descriptor selection select-interval = 100 - # + # Number of connections that are allowed in the backlog. + # 0 or negative means that the platform default will be used. default-backlog = 1000 } }