Simplify backoff supervision api #26156

This commit is contained in:
Nicolas Vollmar 2018-12-19 08:08:48 +01:00
parent f2708c9dd3
commit d0a5ced319
5 changed files with 223 additions and 182 deletions

View file

@ -0,0 +1,9 @@
# Simplify backoff supervision API #19016
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.BackoffOnRestartSupervisor.replyWhileStopped")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.BackoffOnRestartSupervisor.finalStopMessage")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.BackoffOnRestartSupervisor.this")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.HandleBackoff.replyWhileStopped")
ProblemFilters.exclude[DirectMissingMethodProblem]("akka.pattern.HandleBackoff.finalStopMessage")
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.pattern.HandleBackoff.handleMessageToChild")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.pattern.Backoff.onFailure")
ProblemFilters.exclude[IncompatibleResultTypeProblem]("akka.pattern.Backoff.onStop")

View file

@ -4,11 +4,10 @@
package akka.pattern
import scala.concurrent.duration._
import akka.actor._
import akka.actor.OneForOneStrategy
import akka.actor.SupervisorStrategy._
import akka.actor.{ OneForOneStrategy, _ }
import scala.concurrent.duration._
/**
* Back-off supervisor that stops and starts a child actor when the child actor restarts.
@ -16,20 +15,19 @@ import akka.actor.SupervisorStrategy._
* with ``akka.pattern.Backoff.onFailure``.
*/
private class BackoffOnRestartSupervisor(
val childProps: Props,
val childName: String,
minBackoff: FiniteDuration,
maxBackoff: FiniteDuration,
val reset: BackoffReset,
randomFactor: Double,
strategy: OneForOneStrategy,
val replyWhileStopped: Option[Any],
val finalStopMessage: Option[Any Boolean])
val childProps: Props,
val childName: String,
minBackoff: FiniteDuration,
maxBackoff: FiniteDuration,
val reset: BackoffReset,
randomFactor: Double,
strategy: OneForOneStrategy,
replyWhileStopped: Option[Any])
extends Actor with HandleBackoff
with ActorLogging {
import context._
import BackoffSupervisor._
import context._
override val supervisorStrategy = OneForOneStrategy(strategy.maxNrOfRetries, strategy.withinTimeRange, strategy.loggingEnabled) {
case ex
@ -81,6 +79,15 @@ private class BackoffOnRestartSupervisor(
stop(self)
}
def receive = onTerminated orElse handleBackoff
def receive: Receive = onTerminated orElse handleBackoff
protected def handleMessageToChild(msg: Any): Unit = child match {
case Some(c)
c.forward(msg)
case None
replyWhileStopped match {
case None context.system.deadLetters.forward(msg)
case Some(r) sender() ! r
}
}
}

View file

@ -83,7 +83,7 @@ object Backoff {
minBackoff: FiniteDuration,
maxBackoff: FiniteDuration,
randomFactor: Double,
maxNrOfRetries: Int): BackoffOptions =
maxNrOfRetries: Int): BackoffOnFailureOptions =
BackoffOptionsImpl(RestartImpliesFailure, childProps, childName, minBackoff, maxBackoff, randomFactor).withMaxNrOfRetries(maxNrOfRetries)
/**
@ -137,7 +137,7 @@ object Backoff {
childName: String,
minBackoff: FiniteDuration,
maxBackoff: FiniteDuration,
randomFactor: Double): BackoffOptions =
randomFactor: Double): BackoffOnFailureOptions =
BackoffOptionsImpl(RestartImpliesFailure, childProps, childName, minBackoff, maxBackoff, randomFactor)
/**
@ -195,7 +195,7 @@ object Backoff {
minBackoff: java.time.Duration,
maxBackoff: java.time.Duration,
randomFactor: Double,
maxNrOfRetries: Int): BackoffOptions =
maxNrOfRetries: Int): BackoffOnFailureOptions =
onFailure(childProps, childName, minBackoff.asScala, maxBackoff.asScala, randomFactor, maxNrOfRetries)
/**
@ -250,7 +250,7 @@ object Backoff {
childName: String,
minBackoff: java.time.Duration,
maxBackoff: java.time.Duration,
randomFactor: Double): BackoffOptions =
randomFactor: Double): BackoffOnFailureOptions =
onFailure(childProps, childName, minBackoff.asScala, maxBackoff.asScala, randomFactor, -1)
/**
@ -315,7 +315,7 @@ object Backoff {
minBackoff: FiniteDuration,
maxBackoff: FiniteDuration,
randomFactor: Double,
maxNrOfRetries: Int): BackoffOptions =
maxNrOfRetries: Int): BackoffOnStopOptions =
BackoffOptionsImpl(StopImpliesFailure, childProps, childName, minBackoff, maxBackoff, randomFactor).withMaxNrOfRetries(maxNrOfRetries)
/**
@ -376,7 +376,7 @@ object Backoff {
childName: String,
minBackoff: FiniteDuration,
maxBackoff: FiniteDuration,
randomFactor: Double): BackoffOptions =
randomFactor: Double): BackoffOnStopOptions =
BackoffOptionsImpl(StopImpliesFailure, childProps, childName, minBackoff, maxBackoff, randomFactor)
/**
@ -441,7 +441,7 @@ object Backoff {
minBackoff: java.time.Duration,
maxBackoff: java.time.Duration,
randomFactor: Double,
maxNrOfRetries: Int): BackoffOptions =
maxNrOfRetries: Int): BackoffOnStopOptions =
onStop(childProps, childName, minBackoff.asScala, maxBackoff.asScala, randomFactor, maxNrOfRetries)
/**
@ -503,7 +503,7 @@ object Backoff {
childName: String,
minBackoff: java.time.Duration,
maxBackoff: java.time.Duration,
randomFactor: Double): BackoffOptions =
randomFactor: Double): BackoffOnStopOptions =
onStop(childProps, childName, minBackoff.asScala, maxBackoff.asScala, randomFactor, -1)
}
@ -548,23 +548,6 @@ trait BackoffOptions {
*/
def withDefaultStoppingStrategy: BackoffOptions
/**
* Returns a new BackoffOptions with a constant reply to messages that the supervisor receives while its
* child is stopped. By default, a message received while the child is stopped is forwarded to `deadLetters`.
* With this option, the supervisor will reply to the sender instead.
* @param replyWhileStopped The message that the supervisor will send in response to all messages while
* its child is stopped.
*/
def withReplyWhileStopped(replyWhileStopped: Any): BackoffOptions
/**
* Predicate evaluated for each message, if it returns true and the supervised actor is
* stopped then the supervisor will stop its self. If it returns true while
* the supervised actor is running then it will be forwarded to the supervised actor and
* when the supervised actor stops its self the supervisor will stop its self.
*/
def withFinalStopMessage(isFinalStopMessage: Any Boolean): BackoffOptions
/**
* Returns a new BackoffOptions with a maximum number of retries to restart the child actor.
* By default, the supervisor will retry infinitely.
@ -574,12 +557,42 @@ trait BackoffOptions {
*/
def withMaxNrOfRetries(maxNrOfRetries: Int): BackoffOptions
/**
* Returns a new BackoffOptions with a constant reply to messages that the supervisor receives while its
* child is stopped. By default, a message received while the child is stopped is forwarded to `deadLetters`.
* With this option, the supervisor will reply to the sender instead.
* @param replyWhileStopped The message that the supervisor will send in response to all messages while
* its child is stopped.
*/
def withReplyWhileStopped(replyWhileStopped: Any): BackoffOptions
// for backwards compability with 2.5.19
@deprecated("Use through BackoffOnStopOptions instead of BackoffOptions", since = "2.5.20")
def withFinalStopMessage(isFinalStopMessage: Any Boolean): BackoffOptions
/**
* Returns the props to create the back-off supervisor.
*/
private[akka] def props: Props
}
sealed trait BackoffOnStopOptions extends BackoffOptions {
/**
* Predicate evaluated for each message, if it returns true and the supervised actor is
* stopped then the supervisor will stop its self. If it returns true while
* the supervised actor is running then it will be forwarded to the supervised actor and
* when the supervised actor stops its self the supervisor will stop its self.
*/
def withFinalStopMessage(isFinalStopMessage: Any Boolean): BackoffOptions
}
sealed trait BackoffOnFailureOptions extends BackoffOptions {
// for backwards compability with 2.5.19
@deprecated("This has no effect for backoff on failure", since = "2.5.20")
def withFinalStopMessage(isFinalStopMessage: Any Boolean): BackoffOptions
}
private final case class BackoffOptionsImpl(
backoffType: BackoffType = RestartImpliesFailure,
childProps: Props,
@ -591,7 +604,7 @@ private final case class BackoffOptionsImpl(
supervisorStrategy: OneForOneStrategy = OneForOneStrategy()(SupervisorStrategy.defaultStrategy.decider),
replyWhileStopped: Option[Any] = None,
finalStopMessage: Option[Any Boolean] = None
) extends BackoffOptions {
) extends BackoffOptions with BackoffOnStopOptions with BackoffOnFailureOptions {
val backoffReset = reset.getOrElse(AutoReset(minBackoff))
@ -616,7 +629,7 @@ private final case class BackoffOptionsImpl(
backoffType match {
//onFailure method in companion object
case RestartImpliesFailure
Props(new BackoffOnRestartSupervisor(childProps, childName, minBackoff, maxBackoff, backoffReset, randomFactor, supervisorStrategy, replyWhileStopped, finalStopMessage))
Props(new BackoffOnRestartSupervisor(childProps, childName, minBackoff, maxBackoff, backoffReset, randomFactor, supervisorStrategy, replyWhileStopped))
//onStop method in companion object
case StopImpliesFailure
Props(new BackoffSupervisor(childProps, childName, minBackoff, maxBackoff, backoffReset, randomFactor, supervisorStrategy, replyWhileStopped, finalStopMessage))

View file

@ -4,11 +4,11 @@
package akka.pattern
import java.util.concurrent.ThreadLocalRandom
import java.util.Optional
import java.util.concurrent.ThreadLocalRandom
import akka.actor.{ Actor, ActorLogging, ActorRef, DeadLetterSuppression, OneForOneStrategy, Props, SupervisorStrategy, Terminated }
import akka.actor.SupervisorStrategy.{ Directive, Escalate }
import akka.actor.{ Actor, ActorLogging, ActorRef, DeadLetterSuppression, OneForOneStrategy, Props, SupervisorStrategy, Terminated }
import akka.util.JavaDurationConverters._
import scala.concurrent.duration.{ Duration, FiniteDuration }
@ -23,15 +23,15 @@ object BackoffSupervisor {
* most exceptions will immediately restart the child. You can define another
* supervision strategy by using [[#propsWithSupervisorStrategy]].
*
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param randomFactor after calculation of the exponential back-off an additional
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
*/
def props(
childProps: Props,
@ -49,18 +49,18 @@ object BackoffSupervisor {
* most exceptions will immediately restart the child. You can define another
* supervision strategy by using [[#propsWithSupervisorStrategy]].
*
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param randomFactor after calculation of the exponential back-off an additional
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param randomFactor after calculation of the exponential back-off an additional
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* @param maxNrOfRetries maximum number of attempts to restart the child actor.
* The supervisor will terminate itself after the maxNoOfRetries is reached.
* In order to restart infinitely pass in `-1`.
* The supervisor will terminate itself after the maxNoOfRetries is reached.
* In order to restart infinitely pass in `-1`.
*/
def props(
childProps: Props,
@ -83,15 +83,15 @@ object BackoffSupervisor {
* most exceptions will immediately restart the child. You can define another
* supervision strategy by using [[#propsWithSupervisorStrategy]].
*
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param randomFactor after calculation of the exponential back-off an additional
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
*/
def props(
childProps: Props,
@ -109,18 +109,18 @@ object BackoffSupervisor {
* most exceptions will immediately restart the child. You can define another
* supervision strategy by using [[#propsWithSupervisorStrategy]].
*
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param randomFactor after calculation of the exponential back-off an additional
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param randomFactor after calculation of the exponential back-off an additional
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* @param maxNrOfRetries maximum number of attempts to restart the child actor.
* The supervisor will terminate itself after the maxNoOfRetries is reached.
* In order to restart infinitely pass in `-1`.
* The supervisor will terminate itself after the maxNoOfRetries is reached.
* In order to restart infinitely pass in `-1`.
*/
def props(
childProps: Props,
@ -140,18 +140,18 @@ object BackoffSupervisor {
* `Restart` will perform a normal immediate restart of the child. A `Stop` will
* stop the child, but it will be started again after the back-off duration.
*
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param randomFactor after calculation of the exponential back-off an additional
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* @param strategy the supervision strategy to use for handling exceptions
* in the child. As the BackoffSupervisor creates a separate actor to handle the
* backoff process, only a [[OneForOneStrategy]] makes sense here.
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* @param strategy the supervision strategy to use for handling exceptions
* in the child. As the BackoffSupervisor creates a separate actor to handle the
* backoff process, only a [[OneForOneStrategy]] makes sense here.
*/
def propsWithSupervisorStrategy(
childProps: Props,
@ -174,18 +174,18 @@ object BackoffSupervisor {
* `Restart` will perform a normal immediate restart of the child. A `Stop` will
* stop the child, but it will be started again after the back-off duration.
*
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param childProps the [[akka.actor.Props]] of the child actor that
* will be started and supervised
* @param childName name of the child actor
* @param minBackoff minimum (initial) duration until the child actor will
* started again, if it is terminated
* @param maxBackoff the exponential back-off is capped to this duration
* @param randomFactor after calculation of the exponential back-off an additional
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* @param strategy the supervision strategy to use for handling exceptions
* in the child. As the BackoffSupervisor creates a separate actor to handle the
* backoff process, only a [[OneForOneStrategy]] makes sense here.
* random delay based on this factor is added, e.g. `0.2` adds up to `20%` delay.
* In order to skip this additional delay pass in `0`.
* @param strategy the supervision strategy to use for handling exceptions
* in the child. As the BackoffSupervisor creates a separate actor to handle the
* backoff process, only a [[OneForOneStrategy]] makes sense here.
*/
def propsWithSupervisorStrategy(
childProps: Props,
@ -284,15 +284,17 @@ object BackoffSupervisor {
* with `Backoff.onStop`.
*/
final class BackoffSupervisor(
val childProps: Props,
val childName: String,
minBackoff: FiniteDuration,
maxBackoff: FiniteDuration,
val reset: BackoffReset,
randomFactor: Double,
strategy: SupervisorStrategy,
val childProps: Props,
val childName: String,
minBackoff: FiniteDuration,
maxBackoff: FiniteDuration,
val reset: BackoffReset,
randomFactor: Double,
strategy: SupervisorStrategy,
@deprecated("Should be internal", since = "2.5.19") // since removed from HandleBackoff the val can be removed
val replyWhileStopped: Option[Any],
val finalStopMessage: Option[Any Boolean])
@deprecated("Should be internal", since = "2.5.19") // since removed from HandleBackoff the val can be removed
val finalStopMessage: Option[Any Boolean])
extends Actor with HandleBackoff
with ActorLogging {
@ -367,79 +369,25 @@ final class BackoffSupervisor(
}
def receive = onTerminated orElse handleBackoff
}
def receive: Receive = onTerminated orElse handleBackoff
private[akka] trait HandleBackoff { this: Actor
def childProps: Props
def childName: String
def reset: BackoffReset
def replyWhileStopped: Option[Any]
def finalStopMessage: Option[Any Boolean]
var child: Option[ActorRef] = None
var restartCount = 0
var finalStopMessageReceived = false
import BackoffSupervisor._
import context.dispatcher
override def preStart(): Unit = startChild()
def startChild(): Unit = {
if (child.isEmpty) {
child = Some(context.watch(context.actorOf(childProps, childName)))
}
}
def handleBackoff: Receive = {
case StartChild
startChild()
reset match {
case AutoReset(resetBackoff)
val _ = context.system.scheduler.scheduleOnce(resetBackoff, self, ResetRestartCount(restartCount))
case _ // ignore
protected def handleMessageToChild(msg: Any): Unit = child match {
case Some(c)
c.forward(msg)
if (!finalStopMessageReceived && finalStopMessage.isDefined) {
finalStopMessageReceived = finalStopMessage.get.apply(msg)
}
case Reset
reset match {
case ManualReset restartCount = 0
case msg unhandled(msg)
case None
replyWhileStopped match {
case None context.system.deadLetters.forward(msg)
case Some(r) sender() ! r
}
case ResetRestartCount(current)
if (current == restartCount) {
restartCount = 0
finalStopMessage match {
case None
case Some(fsm)
if (fsm(msg)) {
context.stop(self)
}
}
case GetRestartCount
sender() ! RestartCount(restartCount)
case GetCurrentChild
sender() ! CurrentChild(child)
case msg if child.contains(sender())
// use the BackoffSupervisor as sender
context.parent ! msg
case msg child match {
case Some(c)
c.forward(msg)
if (!finalStopMessageReceived && finalStopMessage.isDefined) {
finalStopMessageReceived = finalStopMessage.get.apply(msg)
}
case None
replyWhileStopped match {
case None context.system.deadLetters.forward(msg)
case Some(r) sender() ! r
}
finalStopMessage match {
case None
case Some(fsm)
if (fsm(msg)) {
context.stop(self)
}
}
}
}
}

View file

@ -0,0 +1,64 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.pattern
import akka.actor.{ Actor, ActorRef, Props }
private[akka] trait HandleBackoff {
this: Actor
def childProps: Props
def childName: String
def reset: BackoffReset
protected def handleMessageToChild(m: Any): Unit
var child: Option[ActorRef] = None
var restartCount = 0
var finalStopMessageReceived = false
import BackoffSupervisor._
import context.dispatcher
override def preStart(): Unit = startChild()
def startChild(): Unit = {
if (child.isEmpty) {
child = Some(context.watch(context.actorOf(childProps, childName)))
}
}
def handleBackoff: Receive = {
case StartChild
startChild()
reset match {
case AutoReset(resetBackoff)
val _ = context.system.scheduler.scheduleOnce(resetBackoff, self, ResetRestartCount(restartCount))
case _ // ignore
}
case Reset
reset match {
case ManualReset restartCount = 0
case msg unhandled(msg)
}
case ResetRestartCount(current)
if (current == restartCount) {
restartCount = 0
}
case GetRestartCount
sender() ! RestartCount(restartCount)
case GetCurrentChild
sender() ! CurrentChild(child)
case msg if child.contains(sender())
// use the BackoffSupervisor as sender
context.parent ! msg
case msg
handleMessageToChild(msg)
}
}