Log actor failures in supervisor, see #2824

* To make it possible to override with application
  specific logging, or mute logging for certain failures
* Changed log level of Resume failures to WARNING, which
  caused all the changes to the tests
This commit is contained in:
Patrik Nordwall 2013-03-07 12:10:30 +01:00
parent 369811981e
commit f4d59383d7
17 changed files with 185 additions and 80 deletions

View file

@ -67,6 +67,7 @@ class ActorDSLSpec extends AkkaSpec {
"have a maximum queue size" in { "have a maximum queue size" in {
val i = inbox() val i = inbox()
system.eventStream.subscribe(testActor, classOf[Warning]) system.eventStream.subscribe(testActor, classOf[Warning])
try {
for (_ 1 to 1000) i.receiver ! 0 for (_ 1 to 1000) i.receiver ! 0
expectNoMsg(1 second) expectNoMsg(1 second)
EventFilter.warning(start = "dropping message", occurrences = 1) intercept { EventFilter.warning(start = "dropping message", occurrences = 1) intercept {
@ -80,6 +81,9 @@ class ActorDSLSpec extends AkkaSpec {
intercept[TimeoutException] { intercept[TimeoutException] {
i.receive(1 second) i.receive(1 second)
} }
} finally {
system.eventStream.unsubscribe(testActor, classOf[Warning])
}
} }
"have a default and custom timeouts" in { "have a default and custom timeouts" in {
@ -188,8 +192,8 @@ class ActorDSLSpec extends AkkaSpec {
} }
}) })
a ! testActor a ! testActor
EventFilter[Exception](occurrences = 1) intercept { EventFilter.warning("hi", occurrences = 1) intercept {
a ! new Exception a ! new Exception("hi")
} }
expectNoMsg(1 second) expectNoMsg(1 second)
EventFilter[Exception]("hello", occurrences = 1) intercept { EventFilter[Exception]("hello", occurrences = 1) intercept {

View file

@ -233,8 +233,8 @@ object SupervisorHierarchySpec {
abort("invariant violated: " + state.kids.size + " != " + context.children.size) abort("invariant violated: " + state.kids.size + " != " + context.children.size)
} }
cause match { cause match {
case f: Failure if f.failPost > 0 f.failPost -= 1; throw f 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 PostRestartException(`self`, f: Failure, _) if f.failPost > 0 { f.failPost -= 1; throw f }
case _ case _
} }
} }
@ -272,7 +272,7 @@ object SupervisorHierarchySpec {
setFlags(f.directive) setFlags(f.directive)
stateCache.put(self, stateCache.get(self).copy(failConstr = f.copy())) stateCache.put(self, stateCache.get(self).copy(failConstr = f.copy()))
throw f throw f
case "ping" Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" case "ping" { Thread.sleep((Random.nextFloat * 1.03).toLong); sender ! "pong" }
case Dump(0) abort("dump") case Dump(0) abort("dump")
case Dump(level) context.children foreach (_ ! Dump(level - 1)) case Dump(level) context.children foreach (_ ! Dump(level - 1))
case Terminated(ref) case Terminated(ref)
@ -756,7 +756,7 @@ class SupervisorHierarchySpec extends AkkaSpec(SupervisorHierarchySpec.config) w
val worker = expectMsgType[ActorRef] val worker = expectMsgType[ActorRef]
worker ! "ping" worker ! "ping"
expectMsg("pong") expectMsg("pong")
EventFilter[Exception]("expected", occurrences = 1) intercept { EventFilter.warning("expected", occurrences = 1) intercept {
middle ! "fail" middle ! "fail"
} }
middle ! "ping" middle ! "ping"
@ -781,13 +781,13 @@ class SupervisorHierarchySpec extends AkkaSpec(SupervisorHierarchySpec.config) w
val worker = expectMsgType[ActorRef] val worker = expectMsgType[ActorRef]
worker ! "ping" worker ! "ping"
expectMsg("pong") expectMsg("pong")
EventFilter[Exception]("expected", occurrences = 1) intercept { EventFilter.warning("expected", occurrences = 1) intercept {
boss ! "fail" boss ! "fail"
}
awaitCond(worker.asInstanceOf[LocalActorRef].underlying.mailbox.isSuspended) awaitCond(worker.asInstanceOf[LocalActorRef].underlying.mailbox.isSuspended)
worker ! "ping" worker ! "ping"
expectNoMsg(2 seconds) expectNoMsg(2 seconds)
latch.countDown() latch.countDown()
}
expectMsg("pong") expectMsg("pong")
} }

View file

@ -171,7 +171,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
override def preStart() { preStarts += 1; testActor ! ("preStart" + preStarts) } override def preStart() { preStarts += 1; testActor ! ("preStart" + preStarts) }
override def postStop() { postStops += 1; testActor ! ("postStop" + postStops) } override def postStop() { postStops += 1; testActor ! ("postStop" + postStops) }
def receive = { def receive = {
case "crash" testActor ! "crashed"; throw new RuntimeException("Expected") case "crash" { testActor ! "crashed"; throw new RuntimeException("Expected") }
case "ping" sender ! "pong" case "ping" sender ! "pong"
} }
} }
@ -385,7 +385,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
val child = context.watch(context.actorOf(Props(new Actor { val child = context.watch(context.actorOf(Props(new Actor {
override def postRestart(reason: Throwable): Unit = testActor ! "child restarted" override def postRestart(reason: Throwable): Unit = testActor ! "child restarted"
def receive = { def receive = {
case l: TestLatch Await.ready(l, 5 seconds); throw new IllegalStateException("OHNOES") case l: TestLatch { Await.ready(l, 5 seconds); throw new IllegalStateException("OHNOES") }
case "test" sender ! "child green" case "test" sender ! "child green"
} }
}), "child")) }), "child"))
@ -403,7 +403,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
val latch = TestLatch() val latch = TestLatch()
parent ! latch parent ! latch
parent ! "testchild" parent ! "testchild"
EventFilter[IllegalStateException]("OHNOES", occurrences = 2) intercept { EventFilter[IllegalStateException]("OHNOES", occurrences = 1) intercept {
latch.countDown() latch.countDown()
} }
expectMsg("parent restarted") expectMsg("parent restarted")

View file

@ -192,7 +192,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
EventFilter[ActorKilledException](occurrences = 1) intercept { EventFilter[ActorKilledException](occurrences = 1) intercept {
actor ! Kill actor ! Kill
val set = receiveWhile(messages = 3) { val set = receiveWhile(messages = 3) {
case Logging.Error(_: ActorKilledException, `aname`, `aclass`, "Kill") 1 case Logging.Error(_: ActorKilledException, `aname`, _, "Kill") 1
case Logging.Debug(`aname`, `aclass`, "restarting") 2 case Logging.Debug(`aname`, `aclass`, "restarting") 2
case Logging.Debug(`aname`, `aclass`, "restarted") 3 case Logging.Debug(`aname`, `aclass`, "restarted") 3
}.toSet }.toSet

View file

@ -169,14 +169,14 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
RoundRobinRouter(1, supervisorStrategy = escalator))) RoundRobinRouter(1, supervisorStrategy = escalator)))
//#supervision //#supervision
router ! CurrentRoutees router ! CurrentRoutees
EventFilter[ActorKilledException](occurrences = 2) intercept { EventFilter[ActorKilledException](occurrences = 1) intercept {
expectMsgType[RouterRoutees].routees.head ! Kill expectMsgType[RouterRoutees].routees.head ! Kill
} }
expectMsgType[ActorKilledException] expectMsgType[ActorKilledException]
val router2 = system.actorOf(Props.empty.withRouter(RoundRobinRouter(1).withSupervisorStrategy(escalator))) val router2 = system.actorOf(Props.empty.withRouter(RoundRobinRouter(1).withSupervisorStrategy(escalator)))
router2 ! CurrentRoutees router2 ! CurrentRoutees
EventFilter[ActorKilledException](occurrences = 2) intercept { EventFilter[ActorKilledException](occurrences = 1) intercept {
expectMsgType[RouterRoutees].routees.head ! Kill expectMsgType[RouterRoutees].routees.head ! Kill
} }
expectMsgType[ActorKilledException] expectMsgType[ActorKilledException]
@ -194,7 +194,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
override def postRestart(reason: Throwable): Unit = testActor ! "restarted" override def postRestart(reason: Throwable): Unit = testActor ! "restarted"
}).withRouter(RoundRobinRouter(3)) }).withRouter(RoundRobinRouter(3))
val router = expectMsgType[ActorRef] val router = expectMsgType[ActorRef]
EventFilter[Exception]("die", occurrences = 2) intercept { EventFilter[Exception]("die", occurrences = 1) intercept {
router ! "die" router ! "die"
} }
expectMsgType[Exception].getMessage must be("die") expectMsgType[Exception].getMessage must be("die")

View file

@ -383,7 +383,7 @@ private[akka] class ActorCell(
case NoMessage // only here to suppress warning case NoMessage // only here to suppress warning
} }
} catch handleNonFatalOrInterruptedException { e } catch handleNonFatalOrInterruptedException { e
handleInvokeFailure(Nil, e, "error while processing " + message) handleInvokeFailure(Nil, e)
} }
if (todo != null) systemInvoke(todo) if (todo != null) systemInvoke(todo)
} }
@ -398,7 +398,7 @@ private[akka] class ActorCell(
} }
currentMessage = null // reset current message after successful invocation currentMessage = null // reset current message after successful invocation
} catch handleNonFatalOrInterruptedException { e } catch handleNonFatalOrInterruptedException { e
handleInvokeFailure(Nil, e, e.getMessage) handleInvokeFailure(Nil, e)
} finally { } finally {
checkReceiveTimeout // Reschedule receive timeout checkReceiveTimeout // Reschedule receive timeout
} }

View file

@ -4,13 +4,16 @@
package akka.actor package akka.actor
import language.implicitConversions import language.implicitConversions
import java.lang.{ Iterable JIterable } import java.lang.{ Iterable JIterable }
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import akka.japi.Util.immutableSeq import akka.japi.Util.immutableSeq
import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.ArrayBuffer
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import akka.event.Logging.LogEvent
import akka.event.Logging.Error
import akka.event.Logging.Warning
import scala.util.control.NonFatal
/** /**
* INTERNAL API * INTERNAL API
@ -35,7 +38,7 @@ case class ChildRestartStats(child: ActorRef, var maxNrOfRetriesCount: Int = 0,
def requestRestartPermission(retriesWindow: (Option[Int], Option[Int])): Boolean = def requestRestartPermission(retriesWindow: (Option[Int], Option[Int])): Boolean =
retriesWindow match { retriesWindow match {
case (Some(retries), _) if retries < 1 false case (Some(retries), _) if retries < 1 false
case (Some(retries), None) maxNrOfRetriesCount += 1; maxNrOfRetriesCount <= retries case (Some(retries), None) { maxNrOfRetriesCount += 1; maxNrOfRetriesCount <= retries }
case (x, Some(window)) retriesInWindowOkay(if (x.isDefined) x.get else 1, window) case (x, Some(window)) retriesInWindowOkay(if (x.isDefined) x.get else 1, window)
case (None, _) true case (None, _) true
} }
@ -273,18 +276,64 @@ abstract class SupervisorStrategy {
* failure, which will lead to this actor re-throwing the exception which * failure, which will lead to this actor re-throwing the exception which
* caused the failure. The exception will not be wrapped. * caused the failure. The exception will not be wrapped.
* *
* This method calls [[akka.actor.SupervisorStrategy#logFailure]], which will
* log the failure unless it is escalated. You can customize the logging by
* setting [[akka.actor.SupervisorStrategy#loggingEnabled]] to `false` and
* do the logging inside the `decider` or override the `logFailure` method.
*
* @param children is a lazy collection (a view) * @param children is a lazy collection (a view)
*/ */
def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Boolean = { def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Boolean = {
val directive = decider.applyOrElse(cause, escalateDefault) val directive = decider.applyOrElse(cause, escalateDefault)
directive match { directive match {
case Resume resumeChild(child, cause); true case d @ Resume
case Restart processFailure(context, true, child, cause, stats, children); true logFailure(context, child, cause, d)
case Stop processFailure(context, false, child, cause, stats, children); true resumeChild(child, cause)
case Escalate false true
case d @ Restart
logFailure(context, child, cause, d)
processFailure(context, true, child, cause, stats, children)
true
case d @ Stop
logFailure(context, child, cause, d)
processFailure(context, false, child, cause, stats, children)
true
case d @ Escalate
logFailure(context, child, cause, d)
false
} }
} }
/**
* Logging of actor failures is done when this is `true`.
*/
protected def loggingEnabled: Boolean = true
/**
* Default logging of actor failures when
* [[akka.actor.SupervisorStrategy#loggingEnabled]] is `true`.
* `Escalate` failures are not logged here, since they are supposed
* to be handled at a level higher up in the hierarchy.
* `Resume` failures are logged at `Warning` level.
* `Stop` and `Restart` failures are logged at `Error` level.
*/
protected def logFailure(context: ActorContext, child: ActorRef, cause: Throwable, decision: Directive): Unit =
if (loggingEnabled) {
val logMessage = cause match {
case e: ActorInitializationException e.getCause.getMessage
case e e.getMessage
}
decision match {
case Resume publish(context, Warning(child.path.toString, getClass, logMessage))
case Escalate // don't log here
case _ publish(context, Error(cause, child.path.toString, getClass, logMessage))
}
}
// logging is not the main purpose, and if it fails theres nothing we can do
private def publish(context: ActorContext, logEvent: LogEvent): Unit =
try context.system.eventStream.publish(logEvent) catch { case NonFatal(_) }
/** /**
* Resume the previously failed child: <b>do never apply this to a child which * Resume the previously failed child: <b>do never apply this to a child which
* is not the currently failing child</b>. Suspend/resume needs to be done in * is not the currently failing child</b>. Suspend/resume needs to be done in
@ -319,12 +368,19 @@ abstract class SupervisorStrategy {
* @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window * @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window
* @param decider mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a * @param decider mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a
* `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates. * `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates.
* @param loggingEnabled the strategy logs the failure if this is enabled (true), by default it is enabled
*/ */
case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider) case class AllForOneStrategy(
maxNrOfRetries: Int = -1,
withinTimeRange: Duration = Duration.Inf,
override val loggingEnabled: Boolean = true)(val decider: SupervisorStrategy.Decider)
extends SupervisorStrategy { extends SupervisorStrategy {
import SupervisorStrategy._ import SupervisorStrategy._
def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider, loggingEnabled: Boolean) =
this(maxNrOfRetries, withinTimeRange, loggingEnabled)(SupervisorStrategy.makeDecider(decider))
def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider) = def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider) =
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(decider)) this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(decider))
@ -358,10 +414,17 @@ case class AllForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration
* @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window * @param withinTimeRange duration of the time window for maxNrOfRetries, Duration.Inf means no window
* @param decider mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a * @param decider mapping from Throwable to [[akka.actor.SupervisorStrategy.Directive]], you can also use a
* `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates. * `Seq` of Throwables which maps the given Throwables to restarts, otherwise escalates.
* @param loggingEnabled the strategy logs the failure if this is enabled (true), by default it is enabled
*/ */
case class OneForOneStrategy(maxNrOfRetries: Int = -1, withinTimeRange: Duration = Duration.Inf)(val decider: SupervisorStrategy.Decider) case class OneForOneStrategy(
maxNrOfRetries: Int = -1,
withinTimeRange: Duration = Duration.Inf,
override val loggingEnabled: Boolean = true)(val decider: SupervisorStrategy.Decider)
extends SupervisorStrategy { extends SupervisorStrategy {
def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider, loggingEnabled: Boolean) =
this(maxNrOfRetries, withinTimeRange, loggingEnabled)(SupervisorStrategy.makeDecider(decider))
def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider) = def this(maxNrOfRetries: Int, withinTimeRange: Duration, decider: SupervisorStrategy.JDecider) =
this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(decider)) this(maxNrOfRetries, withinTimeRange)(SupervisorStrategy.makeDecider(decider))

View file

@ -163,15 +163,14 @@ private[akka] trait FaultHandling { this: ActorCell ⇒
} }
} }
final def handleInvokeFailure(childrenNotToSuspend: immutable.Iterable[ActorRef], t: Throwable, message: String): Unit = { final def handleInvokeFailure(childrenNotToSuspend: immutable.Iterable[ActorRef], t: Throwable): Unit = {
publish(Error(t, self.path.toString, clazz(actor), message))
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
if (!isFailed) try { if (!isFailed) try {
suspendNonRecursive() suspendNonRecursive()
// suspend children // suspend children
val skip: Set[ActorRef] = currentMessage match { val skip: Set[ActorRef] = currentMessage match {
case Envelope(Failed(_, _), child) setFailed(child); Set(child) case Envelope(Failed(_, _), child) { setFailed(child); Set(child) }
case _ setFailed(self); Set.empty case _ { setFailed(self); Set.empty }
} }
suspendChildren(exceptFor = skip ++ childrenNotToSuspend) suspendChildren(exceptFor = skip ++ childrenNotToSuspend)
t match { t match {
@ -233,7 +232,7 @@ private[akka] trait FaultHandling { this: ActorCell ⇒
}) })
} catch handleNonFatalOrInterruptedException { e } catch handleNonFatalOrInterruptedException { e
clearActorFields(actor) // in order to prevent preRestart() from happening again clearActorFields(actor) // in order to prevent preRestart() from happening again
handleInvokeFailure(survivors, new PostRestartException(self, e, cause), e.getMessage) handleInvokeFailure(survivors, new PostRestartException(self, e, cause))
} }
} }
@ -263,7 +262,8 @@ private[akka] trait FaultHandling { this: ActorCell ⇒
if (actor != null) { if (actor != null) {
try actor.supervisorStrategy.handleChildTerminated(this, child, children) try actor.supervisorStrategy.handleChildTerminated(this, child, children)
catch handleNonFatalOrInterruptedException { e catch handleNonFatalOrInterruptedException { e
handleInvokeFailure(Nil, e, "handleChildTerminated failed") publish(Error(e, self.path.toString, clazz(actor), "handleChildTerminated failed"))
handleInvokeFailure(Nil, e)
} }
} }
/* /*
@ -271,9 +271,9 @@ private[akka] trait FaultHandling { this: ActorCell ⇒
* then we are continuing the previously suspended recreate/create/terminate action * then we are continuing the previously suspended recreate/create/terminate action
*/ */
status match { status match {
case Some(c @ ChildrenContainer.Recreation(cause)) finishRecreate(cause, actor); c.dequeueAll() case Some(c @ ChildrenContainer.Recreation(cause)) { finishRecreate(cause, actor); c.dequeueAll() }
case Some(c @ ChildrenContainer.Creation()) finishCreate(); c.dequeueAll() case Some(c @ ChildrenContainer.Creation()) { finishCreate(); c.dequeueAll() }
case Some(ChildrenContainer.Termination) finishTerminate(); null case Some(ChildrenContainer.Termination) { finishTerminate(); null }
case _ null case _ null
} }
} }

View file

@ -29,7 +29,7 @@ class ConsumerIntegrationTest extends WordSpec with MustMatchers with NonSharedC
implicit def ec: ExecutionContext = system.dispatcher implicit def ec: ExecutionContext = system.dispatcher
"Consumer must throw FailedToCreateRouteException, while awaiting activation, if endpoint is invalid" in { "Consumer must throw FailedToCreateRouteException, while awaiting activation, if endpoint is invalid" in {
filterEvents(EventFilter[ActorActivationException](occurrences = 1)) { filterEvents(EventFilter.warning(pattern = "failed to activate.*", occurrences = 1)) {
val actorRef = system.actorOf(Props(new TestActor(uri = "some invalid uri")), "invalidActor") val actorRef = system.actorOf(Props(new TestActor(uri = "some invalid uri")), "invalidActor")
intercept[FailedToCreateRouteException] { intercept[FailedToCreateRouteException] {
Await.result(camel.activationFutureFor(actorRef), defaultTimeoutDuration) Await.result(camel.activationFutureFor(actorRef), defaultTimeoutDuration)

View file

@ -52,6 +52,13 @@ restarts per minute. ``-1`` and ``Duration.Inf()`` means that the respective lim
does not apply, leaving the possibility to specify an absolute upper limit on the does not apply, leaving the possibility to specify an absolute upper limit on the
restarts or to make the restarts work infinitely. restarts or to make the restarts work infinitely.
.. note::
If the strategy is declared inside the supervising actor (as opposed to
a separate class) its decider has access to all internal state of
the actor in a thread-safe fashion, including obtaining a reference to the
currently failed child (available as the ``getSender`` of the failure message).
Default Supervisor Strategy Default Supervisor Strategy
^^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^
@ -78,6 +85,22 @@ loss of the child. This strategy is also provided pre-packaged as
:class:`StoppingSupervisorStrategy` configurator to be used when you want the :class:`StoppingSupervisorStrategy` configurator to be used when you want the
``"/user"`` guardian to apply it. ``"/user"`` guardian to apply it.
Logging of Actor Failures
^^^^^^^^^^^^^^^^^^^^^^^^^
By default the ``SupervisorStrategy`` logs failures unless they are escalated.
Escalated failures are supposed to be handled, and potentially logged, at a level
higher in the hierarchy.
You can mute the default logging of a ``SupervisorStrategy`` by setting
``loggingEnabled`` to ``false`` when instantiating it. Customized logging
can be done inside the ``Decider``. Note that the reference to the currently
failed child is available as the ``getSender`` when the ``SupervisorStrategy`` is
declared inside the supervising actor.
You may also customize the logging in your own ``SupervisorStrategy`` implementation
by overriding the ``logFailure`` method.
Supervision of Top-Level Actors Supervision of Top-Level Actors
------------------------------- -------------------------------

View file

@ -739,7 +739,7 @@ Please note, that the child actors are *still restarted*, but no new ``ActorRef`
the same principles for the children, ensuring that their ``preStart()`` method is called only at the creation of their the same principles for the children, ensuring that their ``preStart()`` method is called only at the creation of their
refs. refs.
For more information see :ref:`what-restarting-means-scala`. For more information see :ref:`supervision-restart`.
Initialization via message passing Initialization via message passing
---------------------------------- ----------------------------------

View file

@ -882,7 +882,7 @@ Please note, that the child actors are *still restarted*, but no new ``ActorRef`
the same principles for the children, ensuring that their ``preStart()`` method is called only at the creation of their the same principles for the children, ensuring that their ``preStart()`` method is called only at the creation of their
refs. refs.
For more information see :ref:`what-restarting-means-scala`. For more information see :ref:`supervision-restart`.
Initialization via message passing Initialization via message passing
---------------------------------- ----------------------------------

View file

@ -93,7 +93,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender {
supervisor ! Props[Child] supervisor ! Props[Child]
val child = expectMsgType[ActorRef] // retrieve answer from TestKits testActor val child = expectMsgType[ActorRef] // retrieve answer from TestKits testActor
//#create //#create
EventFilter[ArithmeticException](occurrences = 1) intercept { EventFilter.warning(occurrences = 1) intercept {
//#resume //#resume
child ! 42 // set state to 42 child ! 42 // set state to 42
child ! "get" child ! "get"
@ -121,7 +121,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender {
child.isTerminated must be(true) child.isTerminated must be(true)
//#stop //#stop
} }
EventFilter[Exception]("CRASH", occurrences = 4) intercept { EventFilter[Exception]("CRASH", occurrences = 2) intercept {
//#escalate-kill //#escalate-kill
supervisor ! Props[Child] // create new child supervisor ! Props[Child] // create new child
val child2 = expectMsgType[ActorRef] val child2 = expectMsgType[ActorRef]

View file

@ -82,6 +82,22 @@ loss of the child. This strategy is also provided pre-packaged as
:class:`StoppingSupervisorStrategy` configurator to be used when you want the :class:`StoppingSupervisorStrategy` configurator to be used when you want the
``"/user"`` guardian to apply it. ``"/user"`` guardian to apply it.
Logging of Actor Failures
^^^^^^^^^^^^^^^^^^^^^^^^^
By default the ``SupervisorStrategy`` logs failures unless they are escalated.
Escalated failures are supposed to be handled, and potentially logged, at a level
higher in the hierarchy.
You can mute the default logging of a ``SupervisorStrategy`` by setting
``loggingEnabled`` to ``false`` when instantiating it. Customized logging
can be done inside the ``Decider``. Note that the reference to the currently
failed child is available as the ``sender`` when the ``SupervisorStrategy`` is
declared inside the supervising actor.
You may also customize the logging in your own ``SupervisorStrategy`` implementation
by overriding the ``logFailure`` method.
Supervision of Top-Level Actors Supervision of Top-Level Actors
------------------------------- -------------------------------

View file

@ -240,9 +240,10 @@ class BarrierSpec extends AkkaSpec(BarrierSpec.config) with ImplicitSender {
expectMsg(ToClient(Done)) expectMsg(ToClient(Done))
b ! Remove(B) b ! Remove(B)
b ! Remove(A) b ! Remove(A)
EventFilter[BarrierEmpty](occurrences = 1) intercept { EventFilter.warning(start = "cannot remove", occurrences = 1) intercept {
b ! Remove(A) b ! Remove(A)
} }
Thread.sleep(5000)
} }
} }

View file

@ -210,9 +210,7 @@ akka.actor.deployment {
router ! CurrentRoutees router ! CurrentRoutees
EventFilter[ActorKilledException](occurrences = 1) intercept { EventFilter[ActorKilledException](occurrences = 1) intercept {
EventFilter[ActorKilledException](occurrences = 1).intercept {
expectMsgType[RouterRoutees].routees.head ! Kill expectMsgType[RouterRoutees].routees.head ! Kill
}(otherSystem)
} }
expectMsgType[ActorKilledException] expectMsgType[ActorKilledException]
} }

View file

@ -197,7 +197,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
expectMsg(42) expectMsg(42)
EventFilter[Exception]("crash", occurrences = 1).intercept { EventFilter[Exception]("crash", occurrences = 1).intercept {
r ! new Exception("crash") r ! new Exception("crash")
}(other) }
expectMsg("preRestart") expectMsg("preRestart")
r ! 42 r ! 42
expectMsg(42) expectMsg(42)
@ -242,7 +242,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
expectMsg(42) expectMsg(42)
EventFilter[Exception]("crash", occurrences = 1).intercept { EventFilter[Exception]("crash", occurrences = 1).intercept {
r ! new Exception("crash") r ! new Exception("crash")
}(other) }
expectMsg("preRestart") expectMsg("preRestart")
r ! 42 r ! 42
expectMsg(42) expectMsg(42)
@ -258,7 +258,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
expectMsg(10.seconds, 42) expectMsg(10.seconds, 42)
EventFilter[Exception]("crash", occurrences = 1).intercept { EventFilter[Exception]("crash", occurrences = 1).intercept {
r ! new Exception("crash") r ! new Exception("crash")
}(other) }
expectMsg("preRestart") expectMsg("preRestart")
r ! 42 r ! 42
expectMsg(42) expectMsg(42)
@ -274,7 +274,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
expectMsg(10.seconds, 42) expectMsg(10.seconds, 42)
EventFilter[Exception]("crash", occurrences = 1).intercept { EventFilter[Exception]("crash", occurrences = 1).intercept {
r ! new Exception("crash") r ! new Exception("crash")
}(other) }
expectMsg("preRestart") expectMsg("preRestart")
r ! 42 r ! 42
expectMsg(42) expectMsg(42)