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:
parent
369811981e
commit
f4d59383d7
17 changed files with 185 additions and 80 deletions
|
|
@ -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 {
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
|
|
||||||
|
|
@ -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")
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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 there’s 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))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
-------------------------------
|
-------------------------------
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
----------------------------------
|
----------------------------------
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
----------------------------------
|
----------------------------------
|
||||||
|
|
|
||||||
|
|
@ -93,7 +93,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender {
|
||||||
supervisor ! Props[Child]
|
supervisor ! Props[Child]
|
||||||
val child = expectMsgType[ActorRef] // retrieve answer from TestKit’s testActor
|
val child = expectMsgType[ActorRef] // retrieve answer from TestKit’s 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]
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
-------------------------------
|
-------------------------------
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -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]
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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)
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue