Merge pull request #247 from jboner/wip-1711-supervisorStrategy-patriknw

FaultHandler as method in Actor instead of in Props. See #1711
This commit is contained in:
patriknw 2012-01-23 08:40:59 -08:00
commit d21d03207f
34 changed files with 471 additions and 345 deletions

View file

@ -80,7 +80,7 @@ class ActorFireForgetRequestReplySpec extends AkkaSpec with BeforeAndAfterEach w
"should shutdown crashed temporary actor" in { "should shutdown crashed temporary actor" in {
filterEvents(EventFilter[Exception]("Expected exception")) { filterEvents(EventFilter[Exception]("Expected exception")) {
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(0)))))
val actor = Await.result((supervisor ? Props[CrashingActor]).mapTo[ActorRef], timeout.duration) val actor = Await.result((supervisor ? Props[CrashingActor]).mapTo[ActorRef], timeout.duration)
actor.isTerminated must be(false) actor.isTerminated must be(false)
actor ! "Die" actor ! "Die"

View file

@ -35,7 +35,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS
"invoke preRestart, preStart, postRestart when using OneForOneStrategy" in { "invoke preRestart, preStart, postRestart when using OneForOneStrategy" in {
filterException[ActorKilledException] { filterException[ActorKilledException] {
val id = newUuid().toString val id = newUuid().toString
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(3)))))
val gen = new AtomicInteger(0) val gen = new AtomicInteger(0)
val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen) { val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen) {
override def preRestart(reason: Throwable, message: Option[Any]) { report("preRestart") } override def preRestart(reason: Throwable, message: Option[Any]) { report("preRestart") }
@ -69,7 +69,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS
"default for preRestart and postRestart is to call postStop and preStart respectively" in { "default for preRestart and postRestart is to call postStop and preStart respectively" in {
filterException[ActorKilledException] { filterException[ActorKilledException] {
val id = newUuid().toString val id = newUuid().toString
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(3)))))
val gen = new AtomicInteger(0) val gen = new AtomicInteger(0)
val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen)) val restarterProps = Props(new LifeCycleTestActor(testActor, id, gen))
val restarter = Await.result((supervisor ? restarterProps).mapTo[ActorRef], timeout.duration) val restarter = Await.result((supervisor ? restarterProps).mapTo[ActorRef], timeout.duration)
@ -99,7 +99,7 @@ class ActorLifeCycleSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitS
"not invoke preRestart and postRestart when never restarted using OneForOneStrategy" in { "not invoke preRestart and postRestart when never restarted using OneForOneStrategy" in {
val id = newUuid().toString val id = newUuid().toString
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(3)))) val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(3)))))
val gen = new AtomicInteger(0) val gen = new AtomicInteger(0)
val props = Props(new LifeCycleTestActor(testActor, id, gen)) val props = Props(new LifeCycleTestActor(testActor, id, gen))
val a = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) val a = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration)

View file

@ -374,6 +374,8 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
val boss = system.actorOf(Props(new Actor { val boss = system.actorOf(Props(new Actor {
override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 2, 1000)
val ref = context.actorOf( val ref = context.actorOf(
Props(new Actor { Props(new Actor {
def receive = { case _ } def receive = { case _ }
@ -382,7 +384,7 @@ class ActorRefSpec extends AkkaSpec with DefaultTimeout {
})) }))
protected def receive = { case "sendKill" ref ! Kill } protected def receive = { case "sendKill" ref ! Kill }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 1000))) }))
boss ! "sendKill" boss ! "sendKill"
Await.ready(latch, 5 seconds) Await.ready(latch, 5 seconds)

View file

@ -23,7 +23,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout
import DeathWatchSpec._ import DeathWatchSpec._
lazy val supervisor = system.actorOf(Props[Supervisor], "watchers") lazy val supervisor = system.actorOf(Props(new Supervisor(SupervisorStrategy.defaultStrategy)), "watchers")
def startWatching(target: ActorRef) = Await.result((supervisor ? props(target, testActor)).mapTo[ActorRef], 3 seconds) def startWatching(target: ActorRef) = Await.result((supervisor ? props(target, testActor)).mapTo[ActorRef], 3 seconds)
@ -94,7 +94,7 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout
"notify with a Terminated message once when an Actor is stopped but not when restarted" in { "notify with a Terminated message once when an Actor is stopped but not when restarted" in {
filterException[ActorKilledException] { filterException[ActorKilledException] {
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(2)))) val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), Some(2)))))
val terminalProps = Props(context { case x context.sender ! x }) val terminalProps = Props(context { case x context.sender ! x })
val terminal = Await.result((supervisor ? terminalProps).mapTo[ActorRef], timeout.duration) val terminal = Await.result((supervisor ? terminalProps).mapTo[ActorRef], timeout.duration)
@ -115,13 +115,13 @@ trait DeathWatchSpec { this: AkkaSpec with ImplicitSender with DefaultTimeout
"fail a monitor which does not handle Terminated()" in { "fail a monitor which does not handle Terminated()" in {
filterEvents(EventFilter[ActorKilledException](), EventFilter[DeathPactException]()) { filterEvents(EventFilter[ActorKilledException](), EventFilter[DeathPactException]()) {
case class FF(fail: Failed) case class FF(fail: Failed)
val supervisor = system.actorOf(Props[Supervisor] val strategy = new OneForOneStrategy(SupervisorStrategy.makeDecider(List(classOf[Exception])), Some(0)) {
.withFaultHandler(new OneForOneStrategy(FaultHandlingStrategy.makeDecider(List(classOf[Exception])), Some(0)) { override def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]) = {
override def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]) = { testActor.tell(FF(Failed(cause)), child)
testActor.tell(FF(Failed(cause)), child) super.handleFailure(context, child, cause, stats, children)
super.handleFailure(context, child, cause, stats, children) }
} }
})) val supervisor = system.actorOf(Props(new Supervisor(strategy)))
val failed = Await.result((supervisor ? Props.empty).mapTo[ActorRef], timeout.duration) val failed = Await.result((supervisor ? Props.empty).mapTo[ActorRef], timeout.duration)
val brother = Await.result((supervisor ? Props(new Actor { val brother = Await.result((supervisor ? Props(new Actor {

View file

@ -72,8 +72,9 @@ class FSMTransitionSpec extends AkkaSpec with ImplicitSender {
val fsm = system.actorOf(Props(new MyFSM(testActor))) val fsm = system.actorOf(Props(new MyFSM(testActor)))
val sup = system.actorOf(Props(new Actor { val sup = system.actorOf(Props(new Actor {
context.watch(fsm) context.watch(fsm)
override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), None, None)
def receive = { case _ } def receive = { case _ }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) }))
within(300 millis) { within(300 millis) {
fsm ! SubscribeTransitionCallBack(forward) fsm ! SubscribeTransitionCallBack(forward)

View file

@ -28,7 +28,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
"A RestartStrategy" must { "A RestartStrategy" must {
"ensure that slave stays dead after max restarts within time range" in { "ensure that slave stays dead after max restarts within time range" in {
val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 1000))) val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), 2, 1000))))
val restartLatch = new TestLatch val restartLatch = new TestLatch
val secondRestartLatch = new TestLatch val secondRestartLatch = new TestLatch
@ -74,7 +74,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
} }
"ensure that slave is immortal without max restarts and time range" in { "ensure that slave is immortal without max restarts and time range" in {
val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, None))) val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), None, None))))
val countDownLatch = new TestLatch(100) val countDownLatch = new TestLatch(100)
@ -96,7 +96,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
} }
"ensure that slave restarts after number of crashes not within time range" in { "ensure that slave restarts after number of crashes not within time range" in {
val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 2, 500))) val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), 2, 500))))
val restartLatch = new TestLatch val restartLatch = new TestLatch
val secondRestartLatch = new TestLatch val secondRestartLatch = new TestLatch
@ -153,7 +153,7 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
} }
"ensure that slave is not restarted after max retries" in { "ensure that slave is not restarted after max retries" in {
val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), Some(2), None))) val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Throwable]), Some(2), None))))
val restartLatch = new TestLatch val restartLatch = new TestLatch
val secondRestartLatch = new TestLatch val secondRestartLatch = new TestLatch
@ -208,11 +208,12 @@ class RestartStrategySpec extends AkkaSpec with DefaultTimeout {
val countDownLatch = new TestLatch(2) val countDownLatch = new TestLatch(2)
val boss = system.actorOf(Props(new Actor { val boss = system.actorOf(Props(new Actor {
override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), None, Some(1000))
def receive = { def receive = {
case p: Props sender ! context.watch(context.actorOf(p)) case p: Props sender ! context.watch(context.actorOf(p))
case t: Terminated maxNoOfRestartsLatch.open() case t: Terminated maxNoOfRestartsLatch.open()
} }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), None, Some(1000)))) }))
val slaveProps = Props(new Actor { val slaveProps = Props(new Actor {

View file

@ -133,7 +133,7 @@ class SchedulerSpec extends AkkaSpec with BeforeAndAfterEach with DefaultTimeout
val restartLatch = new TestLatch val restartLatch = new TestLatch
val pingLatch = new TestLatch(6) val pingLatch = new TestLatch(6)
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, 1000))) val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, 1000))))
val props = Props(new Actor { val props = Props(new Actor {
def receive = { def receive = {
case Ping pingLatch.countDown() case Ping pingLatch.countDown()

View file

@ -3,7 +3,12 @@
*/ */
package akka.actor package akka.actor
class Supervisor extends Actor { /**
* For testing Supervisor behavior, normally you don't supply the strategy
* from the outside like this.
*/
class Supervisor(override val supervisorStrategy: SupervisorStrategy) extends Actor {
def receive = { def receive = {
case x: Props sender ! context.actorOf(x) case x: Props sender ! context.actorOf(x)
} }

View file

@ -12,7 +12,12 @@ import akka.dispatch.Await
object SupervisorHierarchySpec { object SupervisorHierarchySpec {
class FireWorkerException(msg: String) extends Exception(msg) class FireWorkerException(msg: String) extends Exception(msg)
class CountDownActor(countDown: CountDownLatch) extends Actor { /**
* For testing Supervisor behavior, normally you don't supply the strategy
* from the outside like this.
*/
class CountDownActor(countDown: CountDownLatch, override val supervisorStrategy: SupervisorStrategy) extends Actor {
protected def receive = { protected def receive = {
case p: Props sender ! context.actorOf(p) case p: Props sender ! context.actorOf(p)
} }
@ -33,12 +38,12 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout {
"restart manager and workers in AllForOne" in { "restart manager and workers in AllForOne" in {
val countDown = new CountDownLatch(4) val countDown = new CountDownLatch(4)
val boss = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), None, None))) val boss = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), None, None))))
val managerProps = Props(new CountDownActor(countDown)).withFaultHandler(AllForOneStrategy(List(), None, None)) val managerProps = Props(new CountDownActor(countDown, AllForOneStrategy(List(), None, None)))
val manager = Await.result((boss ? managerProps).mapTo[ActorRef], timeout.duration) val manager = Await.result((boss ? managerProps).mapTo[ActorRef], timeout.duration)
val workerProps = Props(new CountDownActor(countDown)) val workerProps = Props(new CountDownActor(countDown, SupervisorStrategy.defaultStrategy))
val workerOne, workerTwo, workerThree = Await.result((manager ? workerProps).mapTo[ActorRef], timeout.duration) val workerOne, workerTwo, workerThree = Await.result((manager ? workerProps).mapTo[ActorRef], timeout.duration)
filterException[ActorKilledException] { filterException[ActorKilledException] {
@ -55,13 +60,15 @@ class SupervisorHierarchySpec extends AkkaSpec with DefaultTimeout {
val countDownMessages = new CountDownLatch(1) val countDownMessages = new CountDownLatch(1)
val countDownMax = new CountDownLatch(1) val countDownMax = new CountDownLatch(1)
val boss = system.actorOf(Props(new Actor { val boss = system.actorOf(Props(new Actor {
val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages)))) override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 1, 5000)
val crasher = context.watch(context.actorOf(Props(new CountDownActor(countDownMessages, SupervisorStrategy.defaultStrategy))))
protected def receive = { protected def receive = {
case "killCrasher" crasher ! Kill case "killCrasher" crasher ! Kill
case Terminated(_) countDownMax.countDown() case Terminated(_) countDownMax.countDown()
} }
}).withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 1, 5000))) }))
filterException[ActorKilledException] { filterException[ActorKilledException] {
boss ! "killCrasher" boss ! "killCrasher"

View file

@ -28,7 +28,7 @@ class SupervisorMiscSpec extends AkkaSpec(SupervisorMiscSpec.config) with Defaul
filterEvents(EventFilter[Exception]("Kill")) { filterEvents(EventFilter[Exception]("Kill")) {
val countDownLatch = new CountDownLatch(4) val countDownLatch = new CountDownLatch(4)
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 5000))) val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), 3, 5000))))
val workerProps = Props(new Actor { val workerProps = Props(new Actor {
override def postRestart(cause: Throwable) { countDownLatch.countDown() } override def postRestart(cause: Throwable) { countDownLatch.countDown() }

View file

@ -53,6 +53,8 @@ object SupervisorSpec {
var s: ActorRef = _ var s: ActorRef = _
override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), Some(0))
def receive = { def receive = {
case Die temp forward Die case Die temp forward Die
case Terminated(`temp`) sendTo ! "terminated" case Terminated(`temp`) sendTo ! "terminated"
@ -75,45 +77,45 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
private def child(supervisor: ActorRef, props: Props): ActorRef = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration) private def child(supervisor: ActorRef, props: Props): ActorRef = Await.result((supervisor ? props).mapTo[ActorRef], timeout.duration)
def temporaryActorAllForOne = { def temporaryActorAllForOne = {
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), Some(0)))) val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), Some(0)))))
val temporaryActor = child(supervisor, Props(new PingPongActor(testActor))) val temporaryActor = child(supervisor, Props(new PingPongActor(testActor)))
(temporaryActor, supervisor) (temporaryActor, supervisor)
} }
def singleActorAllForOne = { def singleActorAllForOne = {
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))))
val pingpong = child(supervisor, Props(new PingPongActor(testActor))) val pingpong = child(supervisor, Props(new PingPongActor(testActor)))
(pingpong, supervisor) (pingpong, supervisor)
} }
def singleActorOneForOne = { def singleActorOneForOne = {
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))))
val pingpong = child(supervisor, Props(new PingPongActor(testActor))) val pingpong = child(supervisor, Props(new PingPongActor(testActor)))
(pingpong, supervisor) (pingpong, supervisor)
} }
def multipleActorsAllForOne = { def multipleActorsAllForOne = {
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))))
val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor))) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor)))
(pingpong1, pingpong2, pingpong3, supervisor) (pingpong1, pingpong2, pingpong3, supervisor)
} }
def multipleActorsOneForOne = { def multipleActorsOneForOne = {
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))))
val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor))) val pingpong1, pingpong2, pingpong3 = child(supervisor, Props(new PingPongActor(testActor)))
(pingpong1, pingpong2, pingpong3, supervisor) (pingpong1, pingpong2, pingpong3, supervisor)
} }
def nestedSupervisorsAllForOne = { def nestedSupervisorsAllForOne = {
val topSupervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))) val topSupervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis))))
val pingpong1 = child(topSupervisor, Props(new PingPongActor(testActor))) val pingpong1 = child(topSupervisor, Props(new PingPongActor(testActor)))
val middleSupervisor = child(topSupervisor, Props[Supervisor].withFaultHandler(AllForOneStrategy(Nil, 3, TimeoutMillis))) val middleSupervisor = child(topSupervisor, Props(new Supervisor(AllForOneStrategy(Nil, 3, TimeoutMillis))))
val pingpong2, pingpong3 = child(middleSupervisor, Props(new PingPongActor(testActor))) val pingpong2, pingpong3 = child(middleSupervisor, Props(new PingPongActor(testActor)))
(pingpong1, pingpong2, pingpong3, topSupervisor) (pingpong1, pingpong2, pingpong3, topSupervisor)
@ -141,7 +143,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
"A supervisor" must { "A supervisor" must {
"not restart child more times than permitted" in { "not restart child more times than permitted" in {
val master = system.actorOf(Props(new Master(testActor)).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), Some(0)))) val master = system.actorOf(Props(new Master(testActor)))
master ! Die master ! Die
expectMsg(3 seconds, "terminated") expectMsg(3 seconds, "terminated")
@ -277,7 +279,7 @@ class SupervisorSpec extends AkkaSpec with BeforeAndAfterEach with ImplicitSende
"must attempt restart when exception during restart" in { "must attempt restart when exception during restart" in {
val inits = new AtomicInteger(0) val inits = new AtomicInteger(0)
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(OneForOneStrategy(classOf[Exception] :: Nil, 3, 10000))) val supervisor = system.actorOf(Props(new Supervisor(OneForOneStrategy(classOf[Exception] :: Nil, 3, 10000))))
val dyingProps = Props(new Actor { val dyingProps = Props(new Actor {
inits.incrementAndGet inits.incrementAndGet

View file

@ -22,11 +22,12 @@ class SupervisorTreeSpec extends AkkaSpec with ImplicitSender with DefaultTimeou
EventFilter[ActorKilledException](occurrences = 1) intercept { EventFilter[ActorKilledException](occurrences = 1) intercept {
within(5 seconds) { within(5 seconds) {
val p = Props(new Actor { val p = Props(new Actor {
override val supervisorStrategy = OneForOneStrategy(List(classOf[Exception]), 3, 1000)
def receive = { def receive = {
case p: Props sender ! context.actorOf(p) case p: Props sender ! context.actorOf(p)
} }
override def preRestart(cause: Throwable, msg: Option[Any]) { testActor ! self.path } override def preRestart(cause: Throwable, msg: Option[Any]) { testActor ! self.path }
}).withFaultHandler(OneForOneStrategy(List(classOf[Exception]), 3, 1000)) })
val headActor = system.actorOf(p) val headActor = system.actorOf(p)
val middleActor = Await.result((headActor ? p).mapTo[ActorRef], timeout.duration) val middleActor = Await.result((headActor ? p).mapTo[ActorRef], timeout.duration)
val lastActor = Await.result((middleActor ? p).mapTo[ActorRef], timeout.duration) val lastActor = Await.result((middleActor ? p).mapTo[ActorRef], timeout.duration)

View file

@ -24,7 +24,7 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender
"A supervised actor with lifecycle PERMANENT" should { "A supervised actor with lifecycle PERMANENT" should {
"be able to reply on failure during preRestart" in { "be able to reply on failure during preRestart" in {
filterEvents(EventFilter[Exception]("test", occurrences = 1)) { filterEvents(EventFilter[Exception]("test", occurrences = 1)) {
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), 5, 10000))) val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), 5, 10000))))
val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration) val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration)
supervised.!("test")(testActor) supervised.!("test")(testActor)
@ -35,7 +35,7 @@ class Ticket669Spec extends AkkaSpec with BeforeAndAfterAll with ImplicitSender
"be able to reply on failure during postStop" in { "be able to reply on failure during postStop" in {
filterEvents(EventFilter[Exception]("test", occurrences = 1)) { filterEvents(EventFilter[Exception]("test", occurrences = 1)) {
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(AllForOneStrategy(List(classOf[Exception]), Some(0), None))) val supervisor = system.actorOf(Props(new Supervisor(AllForOneStrategy(List(classOf[Exception]), Some(0), None))))
val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration) val supervised = Await.result((supervisor ? Props[Supervised]).mapTo[ActorRef], timeout.duration)
supervised.!("test")(testActor) supervised.!("test")(testActor)

View file

@ -298,10 +298,13 @@ class TypedActorSpec extends AkkaSpec(TypedActorSpec.config)
"be able to handle exceptions when calling methods" in { "be able to handle exceptions when calling methods" in {
filterEvents(EventFilter[IllegalStateException]("expected")) { filterEvents(EventFilter[IllegalStateException]("expected")) {
val boss = system.actorOf(Props(context { val boss = system.actorOf(Props(new Actor {
case p: TypedProps[_] context.sender ! TypedActor(context).typedActorOf(p) override val supervisorStrategy = OneForOneStrategy {
}).withFaultHandler(OneForOneStrategy { case e: IllegalStateException if e.getMessage == "expected" SupervisorStrategy.Resume
case e: IllegalStateException if e.getMessage == "expected" FaultHandlingStrategy.Resume }
def receive = {
case p: TypedProps[_] context.sender ! TypedActor(context).typedActorOf(p)
}
})) }))
val t = Await.result((boss ? TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(2 seconds)).mapTo[Foo], timeout.duration) val t = Await.result((boss ? TypedProps[Bar](classOf[Foo], classOf[Bar]).withTimeout(2 seconds)).mapTo[Foo], timeout.duration)

View file

@ -15,6 +15,7 @@ import akka.actor._
object LoggingReceiveSpec { object LoggingReceiveSpec {
class TestLogActor extends Actor { class TestLogActor extends Actor {
override val supervisorStrategy = OneForOneStrategy(List(classOf[Throwable]), 5, 5000)
def receive = { case _ } def receive = { case _ }
} }
} }
@ -149,7 +150,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
within(3 seconds) { within(3 seconds) {
val lifecycleGuardian = appLifecycle.asInstanceOf[ActorSystemImpl].guardian val lifecycleGuardian = appLifecycle.asInstanceOf[ActorSystemImpl].guardian
val lname = lifecycleGuardian.path.toString val lname = lifecycleGuardian.path.toString
val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000))) val supervisor = TestActorRef[TestLogActor](Props[TestLogActor])
val sname = supervisor.path.toString val sname = supervisor.path.toString
val sclass = classOf[TestLogActor] val sclass = classOf[TestLogActor]

View file

@ -141,6 +141,14 @@ object Actor {
* *
* {{{ * {{{
* class ExampleActor extends Actor { * class ExampleActor extends Actor {
*
* override val supervisorStrategy = OneForOneStrategy({
* case _: ArithmeticException Resume
* case _: NullPointerException Restart
* case _: IllegalArgumentException Stop
* case _: Exception Escalate
* }: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000))
*
* def receive = { * def receive = {
* // directly calculated reply * // directly calculated reply
* case Request(r) => sender ! calculate(r) * case Request(r) => sender ! calculate(r)
@ -224,6 +232,12 @@ trait Actor {
*/ */
protected def receive: Receive protected def receive: Receive
/**
* User overridable definition the strategy to use for supervising
* child actors.
*/
def supervisorStrategy(): SupervisorStrategy = SupervisorStrategy.defaultStrategy
/** /**
* User overridable callback. * User overridable callback.
* <p/> * <p/>

View file

@ -395,7 +395,7 @@ private[akka] class ActorCell(
dispatcher.resume(this) //FIXME should this be moved down? dispatcher.resume(this) //FIXME should this be moved down?
props.faultHandler.handleSupervisorRestarted(cause, self, children) actor.supervisorStrategy.handleSupervisorRestarted(cause, self, children)
} catch { } catch {
// TODO catching all and continue isn't good for OOME, ticket #1418 // TODO catching all and continue isn't good for OOME, ticket #1418
case e try { case e try {
@ -491,11 +491,11 @@ private[akka] class ActorCell(
// make sure that InterruptedException does not leave this thread // make sure that InterruptedException does not leave this thread
if (e.isInstanceOf[InterruptedException]) { if (e.isInstanceOf[InterruptedException]) {
val ex = ActorInterruptedException(e) val ex = ActorInterruptedException(e)
props.faultHandler.handleSupervisorFailing(self, children) actor.supervisorStrategy.handleSupervisorFailing(self, children)
parent.tell(Failed(ex), self) parent.tell(Failed(ex), self)
throw e //Re-throw InterruptedExceptions as expected throw e //Re-throw InterruptedExceptions as expected
} else { } else {
props.faultHandler.handleSupervisorFailing(self, children) actor.supervisorStrategy.handleSupervisorFailing(self, children)
parent.tell(Failed(e), self) parent.tell(Failed(e), self)
} }
} finally { } finally {
@ -569,7 +569,7 @@ private[akka] class ActorCell(
} }
final def handleFailure(child: ActorRef, cause: Throwable): Unit = childrenRefs.get(child.path.name) match { final def handleFailure(child: ActorRef, cause: Throwable): Unit = childrenRefs.get(child.path.name) match {
case Some(stats) if stats.child == child if (!props.faultHandler.handleFailure(this, child, cause, stats, childrenRefs.values)) throw cause case Some(stats) if stats.child == child if (!actor.supervisorStrategy.handleFailure(this, child, cause, stats, childrenRefs.values)) throw cause
case Some(stats) system.eventStream.publish(Warning(self.path.toString, clazz(actor), "dropping Failed(" + cause + ") from unknown child " + child + " matching names but not the same, was: " + stats.child)) case Some(stats) system.eventStream.publish(Warning(self.path.toString, clazz(actor), "dropping Failed(" + cause + ") from unknown child " + child + " matching names but not the same, was: " + stats.child))
case None system.eventStream.publish(Warning(self.path.toString, clazz(actor), "dropping Failed(" + cause + ") from unknown child " + child)) case None system.eventStream.publish(Warning(self.path.toString, clazz(actor), "dropping Failed(" + cause + ") from unknown child " + child))
} }
@ -577,7 +577,7 @@ private[akka] class ActorCell(
final def handleChildTerminated(child: ActorRef): Unit = { final def handleChildTerminated(child: ActorRef): Unit = {
if (childrenRefs contains child.path.name) { if (childrenRefs contains child.path.name) {
childrenRefs -= child.path.name childrenRefs -= child.path.name
props.faultHandler.handleChildTerminated(this, child, children) actor.supervisorStrategy.handleChildTerminated(this, child, children)
if (stopping && childrenRefs.isEmpty) doTerminate() if (stopping && childrenRefs.isEmpty) doTerminate()
} else system.locker ! ChildTerminated(child) } else system.locker ! ChildTerminated(child)
} }

View file

@ -334,6 +334,16 @@ class LocalActorRefProvider(
* exceptions which might have occurred. * exceptions which might have occurred.
*/ */
private class Guardian extends Actor { private class Guardian extends Actor {
override val supervisorStrategy = {
import akka.actor.SupervisorStrategy._
OneForOneStrategy {
case _: ActorKilledException Stop
case _: ActorInitializationException Stop
case _: Exception Restart
}
}
def receive = { def receive = {
case Terminated(_) context.stop(self) case Terminated(_) context.stop(self)
case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case e: Exception e }) case CreateChild(child, name) sender ! (try context.actorOf(child, name) catch { case e: Exception e })
@ -366,16 +376,6 @@ class LocalActorRefProvider(
override def preRestart(cause: Throwable, msg: Option[Any]) {} override def preRestart(cause: Throwable, msg: Option[Any]) {}
} }
private val guardianFaultHandlingStrategy = {
import akka.actor.FaultHandlingStrategy._
OneForOneStrategy {
case _: ActorKilledException Stop
case _: ActorInitializationException Stop
case _: Exception Restart
}
}
private val guardianProps = Props(new Guardian).withFaultHandler(guardianFaultHandlingStrategy)
/* /*
* The problem is that ActorRefs need a reference to the ActorSystem to * The problem is that ActorRefs need a reference to the ActorSystem to
* provide their service. Hence they cannot be created while the * provide their service. Hence they cannot be created while the
@ -401,6 +401,8 @@ class LocalActorRefProvider(
*/ */
def registerExtraNames(_extras: Map[String, InternalActorRef]): Unit = extraNames ++= _extras def registerExtraNames(_extras: Map[String, InternalActorRef]): Unit = extraNames ++= _extras
private val guardianProps = Props(new Guardian)
lazy val rootGuardian: InternalActorRef = lazy val rootGuardian: InternalActorRef =
new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) { new LocalActorRef(system, guardianProps, theOneWhoWalksTheBubblesOfSpaceTime, rootPath, true) {
object Extra { object Extra {

View file

@ -44,7 +44,7 @@ case class ChildRestartStats(val child: ActorRef, var maxNrOfRetriesCount: Int =
} }
} }
object FaultHandlingStrategy { object SupervisorStrategy {
sealed trait Action sealed trait Action
/** /**
@ -95,6 +95,16 @@ object FaultHandlingStrategy {
*/ */
def escalate = Escalate def escalate = Escalate
final val defaultStrategy: SupervisorStrategy = {
def defaultDecider: Decider = {
case _: ActorInitializationException Stop
case _: ActorKilledException Stop
case _: Exception Restart
case _ Escalate
}
OneForOneStrategy(defaultDecider, None, None)
}
type Decider = PartialFunction[Throwable, Action] type Decider = PartialFunction[Throwable, Action]
type JDecider = akka.japi.Function[Throwable, Action] type JDecider = akka.japi.Function[Throwable, Action]
type CauseAction = (Class[_ <: Throwable], Action) type CauseAction = (Class[_ <: Throwable], Action)
@ -148,9 +158,9 @@ object FaultHandlingStrategy {
} }
} }
abstract class FaultHandlingStrategy { abstract class SupervisorStrategy {
import FaultHandlingStrategy._ import SupervisorStrategy._
def decider: Decider def decider: Decider
@ -190,12 +200,12 @@ abstract class FaultHandlingStrategy {
object AllForOneStrategy { object AllForOneStrategy {
def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): AllForOneStrategy = def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): AllForOneStrategy =
new AllForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit),
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): AllForOneStrategy = def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): AllForOneStrategy =
new AllForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange)
def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int]): AllForOneStrategy = def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int]): AllForOneStrategy =
new AllForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), maxNrOfRetries, None) new AllForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, None)
} }
/** /**
@ -204,22 +214,22 @@ object AllForOneStrategy {
* maxNrOfRetries = the number of times an actor is allowed to be restarted * maxNrOfRetries = the number of times an actor is allowed to be restarted
* withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window * withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window
*/ */
case class AllForOneStrategy(decider: FaultHandlingStrategy.Decider, case class AllForOneStrategy(decider: SupervisorStrategy.Decider,
maxNrOfRetries: Option[Int] = None, maxNrOfRetries: Option[Int] = None,
withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy { withinTimeRange: Option[Int] = None) extends SupervisorStrategy {
def this(decider: FaultHandlingStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) = def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) =
this(FaultHandlingStrategy.makeDecider(decider), this(SupervisorStrategy.makeDecider(decider),
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
if (withinTimeRange < 0) None else Some(withinTimeRange)) if (withinTimeRange < 0) None else Some(withinTimeRange))
def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
this(FaultHandlingStrategy.makeDecider(trapExit), this(SupervisorStrategy.makeDecider(trapExit),
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
if (withinTimeRange < 0) None else Some(withinTimeRange)) if (withinTimeRange < 0) None else Some(withinTimeRange))
def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
this(FaultHandlingStrategy.makeDecider(trapExit), this(SupervisorStrategy.makeDecider(trapExit),
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
if (withinTimeRange < 0) None else Some(withinTimeRange)) if (withinTimeRange < 0) None else Some(withinTimeRange))
@ -247,12 +257,12 @@ case class AllForOneStrategy(decider: FaultHandlingStrategy.Decider,
object OneForOneStrategy { object OneForOneStrategy {
def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): OneForOneStrategy = def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): OneForOneStrategy =
new OneForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit),
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange)) if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): OneForOneStrategy = def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): OneForOneStrategy =
new OneForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange) new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, withinTimeRange)
def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int]): OneForOneStrategy = def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Option[Int]): OneForOneStrategy =
new OneForOneStrategy(FaultHandlingStrategy.makeDecider(trapExit), maxNrOfRetries, None) new OneForOneStrategy(SupervisorStrategy.makeDecider(trapExit), maxNrOfRetries, None)
} }
/** /**
@ -261,22 +271,22 @@ object OneForOneStrategy {
* maxNrOfRetries = the number of times an actor is allowed to be restarted * maxNrOfRetries = the number of times an actor is allowed to be restarted
* withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window * withinTimeRange = millisecond time window for maxNrOfRetries, negative means no window
*/ */
case class OneForOneStrategy(decider: FaultHandlingStrategy.Decider, case class OneForOneStrategy(decider: SupervisorStrategy.Decider,
maxNrOfRetries: Option[Int] = None, maxNrOfRetries: Option[Int] = None,
withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy { withinTimeRange: Option[Int] = None) extends SupervisorStrategy {
def this(decider: FaultHandlingStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) = def this(decider: SupervisorStrategy.JDecider, maxNrOfRetries: Int, withinTimeRange: Int) =
this(FaultHandlingStrategy.makeDecider(decider), this(SupervisorStrategy.makeDecider(decider),
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
if (withinTimeRange < 0) None else Some(withinTimeRange)) if (withinTimeRange < 0) None else Some(withinTimeRange))
def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = def this(trapExit: JIterable[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
this(FaultHandlingStrategy.makeDecider(trapExit), this(SupervisorStrategy.makeDecider(trapExit),
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
if (withinTimeRange < 0) None else Some(withinTimeRange)) if (withinTimeRange < 0) None else Some(withinTimeRange))
def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) = def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
this(FaultHandlingStrategy.makeDecider(trapExit), this(SupervisorStrategy.makeDecider(trapExit),
if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
if (withinTimeRange < 0) None else Some(withinTimeRange)) if (withinTimeRange < 0) None else Some(withinTimeRange))

View file

@ -18,19 +18,11 @@ import akka.routing._
* Used when creating new actors through; <code>ActorSystem.actorOf</code> and <code>ActorContext.actorOf</code>. * Used when creating new actors through; <code>ActorSystem.actorOf</code> and <code>ActorContext.actorOf</code>.
*/ */
object Props { object Props {
import FaultHandlingStrategy._
final val defaultCreator: () Actor = () throw new UnsupportedOperationException("No actor creator specified!") final val defaultCreator: () Actor = () throw new UnsupportedOperationException("No actor creator specified!")
final val defaultDecider: Decider = {
case _: ActorInitializationException Stop
case _: ActorKilledException Stop
case _: Exception Restart
case _ Escalate
}
final val defaultRoutedProps: RouterConfig = NoRouter final val defaultRoutedProps: RouterConfig = NoRouter
final val defaultFaultHandler: FaultHandlingStrategy = OneForOneStrategy(defaultDecider, None, None)
final val noHotSwap: Stack[Actor.Receive] = Stack.empty final val noHotSwap: Stack[Actor.Receive] = Stack.empty
final val empty = new Props(() new Actor { def receive = Actor.emptyBehavior }) final val empty = new Props(() new Actor { def receive = Actor.emptyBehavior })
@ -79,8 +71,6 @@ object Props {
def apply(behavior: ActorContext Actor.Receive): Props = def apply(behavior: ActorContext Actor.Receive): Props =
apply(new Actor { def receive = behavior(context) }) apply(new Actor { def receive = behavior(context) })
def apply(faultHandler: FaultHandlingStrategy): Props =
apply(new Actor { def receive = { case _ } }).withFaultHandler(faultHandler)
} }
/** /**
@ -94,14 +84,10 @@ object Props {
* val props = Props( * val props = Props(
* creator = .., * creator = ..,
* dispatcher = .., * dispatcher = ..,
* faultHandler = ..,
* routerConfig = .. * routerConfig = ..
* ) * )
* val props = Props().withCreator(new MyActor) * val props = Props().withCreator(new MyActor)
* val props = Props[MyActor].withRouter(RoundRobinRouter(..)) * val props = Props[MyActor].withRouter(RoundRobinRouter(..))
* val props = Props[MyActor].withFaultHandler(OneForOneStrategy {
* case e: IllegalStateException Resume
* })
* }}} * }}}
* *
* Examples on Java API: * Examples on Java API:
@ -114,14 +100,12 @@ object Props {
* } * }
* }); * });
* Props props = new Props().withCreator(new UntypedActorFactory() { ... }); * Props props = new Props().withCreator(new UntypedActorFactory() { ... });
* Props props = new Props(MyActor.class).withFaultHandler(new OneForOneStrategy(...));
* Props props = new Props(MyActor.class).withRouter(new RoundRobinRouter(..)); * Props props = new Props(MyActor.class).withRouter(new RoundRobinRouter(..));
* }}} * }}}
*/ */
case class Props( case class Props(
creator: () Actor = Props.defaultCreator, creator: () Actor = Props.defaultCreator,
dispatcher: String = Dispatchers.DefaultDispatcherId, dispatcher: String = Dispatchers.DefaultDispatcherId,
faultHandler: FaultHandlingStrategy = Props.defaultFaultHandler,
routerConfig: RouterConfig = Props.defaultRoutedProps) { routerConfig: RouterConfig = Props.defaultRoutedProps) {
/** /**
@ -129,16 +113,14 @@ case class Props(
*/ */
def this() = this( def this() = this(
creator = Props.defaultCreator, creator = Props.defaultCreator,
dispatcher = Dispatchers.DefaultDispatcherId, dispatcher = Dispatchers.DefaultDispatcherId)
faultHandler = Props.defaultFaultHandler)
/** /**
* Java API. * Java API.
*/ */
def this(factory: UntypedActorFactory) = this( def this(factory: UntypedActorFactory) = this(
creator = () factory.create(), creator = () factory.create(),
dispatcher = Dispatchers.DefaultDispatcherId, dispatcher = Dispatchers.DefaultDispatcherId)
faultHandler = Props.defaultFaultHandler)
/** /**
* Java API. * Java API.
@ -146,7 +128,6 @@ case class Props(
def this(actorClass: Class[_ <: Actor]) = this( def this(actorClass: Class[_ <: Actor]) = this(
creator = () actorClass.newInstance, creator = () actorClass.newInstance,
dispatcher = Dispatchers.DefaultDispatcherId, dispatcher = Dispatchers.DefaultDispatcherId,
faultHandler = Props.defaultFaultHandler,
routerConfig = Props.defaultRoutedProps) routerConfig = Props.defaultRoutedProps)
/** /**
@ -175,11 +156,6 @@ case class Props(
*/ */
def withDispatcher(d: String) = copy(dispatcher = d) def withDispatcher(d: String) = copy(dispatcher = d)
/**
* Returns a new Props with the specified faulthandler set.
*/
def withFaultHandler(f: FaultHandlingStrategy) = copy(faultHandler = f)
/** /**
* Returns a new Props with the specified router config set. * Returns a new Props with the specified router config set.
*/ */

View file

@ -218,6 +218,11 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
TypedActor.currentContext set null TypedActor.currentContext set null
} }
override def supervisorStrategy(): SupervisorStrategy = me match {
case l: Supervisor l.supervisorStrategy
case _ super.supervisorStrategy
}
override def preStart(): Unit = me match { override def preStart(): Unit = me match {
case l: PreStart l.preStart() case l: PreStart l.preStart()
case _ super.preStart() case _ super.preStart()
@ -275,6 +280,17 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
} }
} }
/**
* Mix this into your TypedActor to be able to define supervisor strategy
*/
trait Supervisor {
/**
* User overridable definition the strategy to use for supervising
* child actors.
*/
def supervisorStrategy(): SupervisorStrategy = SupervisorStrategy.defaultStrategy
}
/** /**
* Mix this into your TypedActor to be able to hook into its lifecycle * Mix this into your TypedActor to be able to hook into its lifecycle
*/ */
@ -355,7 +371,6 @@ object TypedActor extends ExtensionId[TypedActorExtension] with ExtensionIdProvi
object TypedProps { object TypedProps {
val defaultDispatcherId: String = Dispatchers.DefaultDispatcherId val defaultDispatcherId: String = Dispatchers.DefaultDispatcherId
val defaultFaultHandler: FaultHandlingStrategy = akka.actor.Props.defaultFaultHandler
val defaultTimeout: Option[Timeout] = None val defaultTimeout: Option[Timeout] = None
val defaultLoader: Option[ClassLoader] = None val defaultLoader: Option[ClassLoader] = None
@ -415,7 +430,6 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] (
interfaces: Seq[Class[_]], interfaces: Seq[Class[_]],
creator: () T, creator: () T,
dispatcher: String = TypedProps.defaultDispatcherId, dispatcher: String = TypedProps.defaultDispatcherId,
faultHandler: FaultHandlingStrategy = TypedProps.defaultFaultHandler,
timeout: Option[Timeout] = TypedProps.defaultTimeout, timeout: Option[Timeout] = TypedProps.defaultTimeout,
loader: Option[ClassLoader] = TypedProps.defaultLoader) { loader: Option[ClassLoader] = TypedProps.defaultLoader) {
@ -458,11 +472,6 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] (
*/ */
def withDispatcher(d: String) = copy(dispatcher = d) def withDispatcher(d: String) = copy(dispatcher = d)
/**
* Returns a new Props with the specified faulthandler set.
*/
def withFaultHandler(f: FaultHandlingStrategy) = copy(faultHandler = f)
/** /**
* @returns a new Props that will use the specified ClassLoader to create its proxy class in * @returns a new Props that will use the specified ClassLoader to create its proxy class in
* If loader is null, it will use the bootstrap classloader. * If loader is null, it will use the bootstrap classloader.
@ -512,8 +521,8 @@ case class TypedProps[T <: AnyRef] protected[TypedProps] (
import akka.actor.{ Props ActorProps } import akka.actor.{ Props ActorProps }
def actorProps(): ActorProps = def actorProps(): ActorProps =
if (dispatcher == ActorProps().dispatcher && faultHandler == ActorProps().faultHandler) ActorProps() if (dispatcher == ActorProps().dispatcher) ActorProps()
else ActorProps(dispatcher = dispatcher, faultHandler = faultHandler) else ActorProps(dispatcher = dispatcher)
} }
case class ContextualTypedActorFactory(typedActor: TypedActorExtension, actorFactory: ActorContext) extends TypedActorFactory { case class ContextualTypedActorFactory(typedActor: TypedActorExtension, actorFactory: ActorContext) extends TypedActorFactory {

View file

@ -37,6 +37,26 @@ import akka.dispatch.{ MessageDispatcher, Promise }
* } * }
* } * }
* *
* private static SupervisorStrategy strategy = new OneForOneStrategy(new Function<Throwable, Action>() {
* @Override
* public Action apply(Throwable t) {
* if (t instanceof ArithmeticException) {
* return resume();
* } else if (t instanceof NullPointerException) {
* return restart();
* } else if (t instanceof IllegalArgumentException) {
* return stop();
* } else {
* return escalate();
* }
* }
* }, 10, 60000);
*
* @Override
* public SupervisorStrategy supervisorStrategy() {
* return strategy;
* }
*
* public void onReceive(Object message) throws Exception { * public void onReceive(Object message) throws Exception {
* if (message instanceof String) { * if (message instanceof String) {
* String msg = (String)message; * String msg = (String)message;
@ -92,6 +112,12 @@ abstract class UntypedActor extends Actor {
*/ */
def getSender(): ActorRef = sender def getSender(): ActorRef = sender
/**
* User overridable definition the strategy to use for supervising
* child actors.
*/
override def supervisorStrategy(): SupervisorStrategy = super.supervisorStrategy()
/** /**
* User overridable callback. * User overridable callback.
* <p/> * <p/>

View file

@ -3,90 +3,90 @@
Actor Systems Actor Systems
============= =============
Actors are objects which encapsulate state and behavior, they communicate Actors are objects which encapsulate state and behavior, they communicate
exclusively by exchanging messages which are placed into the recipients exclusively by exchanging messages which are placed into the recipients
mailbox. In a sense, actors are the most strigent form of object-oriented mailbox. In a sense, actors are the most strigent form of object-oriented
programming, but it serves better to view them as persons: while modeling a programming, but it serves better to view them as persons: while modeling a
solution with actors, envision a group of people and assign sub-tasks to them, solution with actors, envision a group of people and assign sub-tasks to them,
arrange their functions into an organizational structure and think about how to arrange their functions into an organizational structure and think about how to
escalate failure (all with the benefit of not actually dealing with people, escalate failure (all with the benefit of not actually dealing with people,
which means that we need not concern ourselves with their emotional state or which means that we need not concern ourselves with their emotional state or
moral issues). The result can then serve as a mental scaffolding for building moral issues). The result can then serve as a mental scaffolding for building
the software implementation. the software implementation.
Hierarchical Structure Hierarchical Structure
---------------------- ----------------------
Like in an economic organization, actors naturally form hierarchies. One actor, Like in an economic organization, actors naturally form hierarchies. One actor,
which is to oversee a certain function in the program might want to split up which is to oversee a certain function in the program might want to split up
its task into smaller, more manageable pieces. For this purpose it starts child its task into smaller, more manageable pieces. For this purpose it starts child
actors which it supervises. While the details of supervision are explained actors which it supervises. While the details of supervision are explained
:ref:`here <supervision>`, we shall concentrate on the underlying concepts in :ref:`here <supervision>`, we shall concentrate on the underlying concepts in
this section. The only prerequisite is to know that each actor has exactly one this section. The only prerequisite is to know that each actor has exactly one
supervisor, which is the actor that created it. supervisor, which is the actor that created it.
The quintessential feature of actor systems is that tasks are split up and The quintessential feature of actor systems is that tasks are split up and
delegated until they become small enough to be handled in one piece. In doing delegated until they become small enough to be handled in one piece. In doing
so, not only is the task itself clearly structured, but the resulting actors so, not only is the task itself clearly structured, but the resulting actors
can be reasoned about in terms of which messages they should process, how they can be reasoned about in terms of which messages they should process, how they
should react nominally and how failure should be handled. If one actor does not should react nominally and how failure should be handled. If one actor does not
have the means for dealing with a certain situation, it sends a corresponding have the means for dealing with a certain situation, it sends a corresponding
failure message to its supervisor, asking for help. The recursive structure failure message to its supervisor, asking for help. The recursive structure
then allows to handle failure at the right level. then allows to handle failure at the right level.
Compare this to layered software design which easily devolves into defensive Compare this to layered software design which easily devolves into defensive
programming with the aim of not leaking any failure out: if the problem is programming with the aim of not leaking any failure out: if the problem is
communicated to the right person, a better solution can be found than if communicated to the right person, a better solution can be found than if
trying to keep everything “under the carpet”. trying to keep everything “under the carpet”.
Now, the difficulty in designing such a system is how to decide who should Now, the difficulty in designing such a system is how to decide who should
supervise what. There is of course no single best solution, but there are a few supervise what. There is of course no single best solution, but there are a few
guide lines which might be helpful: guide lines which might be helpful:
- If one actor manages the work another actor is doing, e.g. by passing on - If one actor manages the work another actor is doing, e.g. by passing on
sub-tasks, then the manager should supervise the child. The reason is that sub-tasks, then the manager should supervise the child. The reason is that
the manager knows which kind of failures are expected and how to handle the manager knows which kind of failures are expected and how to handle
them. them.
- If one actor carries very important data (i.e. its state shall not be lost - If one actor carries very important data (i.e. its state shall not be lost
if avoidable), this actor should source out any possibly dangerous sub-tasks if avoidable), this actor should source out any possibly dangerous sub-tasks
to children it supervises and handle failures of these children as to children it supervises and handle failures of these children as
appropriate. Depending on the nature of the requests, it may be best to appropriate. Depending on the nature of the requests, it may be best to
create a new child for each request, which simplifies state management for create a new child for each request, which simplifies state management for
collecting the replies. This is known as the “Error Kernel Pattern” from collecting the replies. This is known as the “Error Kernel Pattern” from
Erlang. Erlang.
- If one actor depends on another actor for carrying out its duty, it should - If one actor depends on another actor for carrying out its duty, it should
watch that other actors liveness and act upon receiving a termination watch that other actors liveness and act upon receiving a termination
notice. This is different from supervision, as the watching party has no notice. This is different from supervision, as the watching party has no
influence on the supervision strategy, and it should be noted that a influence on the supervisor strategy, and it should be noted that a
functional dependency alone is not a criterion for deciding where to place a functional dependency alone is not a criterion for deciding where to place a
certain child actor in the hierarchy. certain child actor in the hierarchy.
There are of course always exceptions to these rules, but no matter whether you There are of course always exceptions to these rules, but no matter whether you
follow the rules or break them, you should always have a reason. follow the rules or break them, you should always have a reason.
Configuration Container Configuration Container
----------------------- -----------------------
The actor system as a collaborating ensemble of actors is the natural unit for The actor system as a collaborating ensemble of actors is the natural unit for
managing shared facilities like scheduling services, configuration, logging, managing shared facilities like scheduling services, configuration, logging,
etc. Several actor systems with different configuration may co-exist within the etc. Several actor systems with different configuration may co-exist within the
same JVM without problems, there is no global shared state within Akka itself. same JVM without problems, there is no global shared state within Akka itself.
Couple this with the transparent communication between actor systems—within one Couple this with the transparent communication between actor systems—within one
node or across a network connection—to see that actor systems themselves can be node or across a network connection—to see that actor systems themselves can be
used as building blocks in a functional hierarchy. used as building blocks in a functional hierarchy.
Actor Best Practices Actor Best Practices
-------------------- --------------------
#. Actors should be like nice co-workers: do their job efficiently without #. Actors should be like nice co-workers: do their job efficiently without
bothering everyone else needlessly and avoid hogging resources. Translated bothering everyone else needlessly and avoid hogging resources. Translated
to programming this means to process events and generate responses (or more to programming this means to process events and generate responses (or more
requests) in an event-driven manner. Actors should not block (i.e. passively requests) in an event-driven manner. Actors should not block (i.e. passively
wait while occupying a Thread) on some external entity, which might be a wait while occupying a Thread) on some external entity, which might be a
lock, a network socket, etc. The blocking operations should be done in some lock, a network socket, etc. The blocking operations should be done in some
special-cased thread which sends messages to the actors which shall act on special-cased thread which sends messages to the actors which shall act on
them. them.
#. Do not pass mutable objects between actors. In order to ensure that, prefer #. Do not pass mutable objects between actors. In order to ensure that, prefer
@ -94,21 +94,21 @@ Actor Best Practices
their mutable state to the outside, you are back in normal Java concurrency their mutable state to the outside, you are back in normal Java concurrency
land with all the drawbacks. land with all the drawbacks.
#. Actors are made to be containers for behavior and state, embracing this #. Actors are made to be containers for behavior and state, embracing this
means to not routinely send behavior within messages (which may be tempting means to not routinely send behavior within messages (which may be tempting
using Scala closures). One of the risks is to accidentally share mutable using Scala closures). One of the risks is to accidentally share mutable
state between actors, and this violation of the actor model unfortunately state between actors, and this violation of the actor model unfortunately
breaks all the properties which make programming in actors such a nice breaks all the properties which make programming in actors such a nice
experience. experience.
What you should not concern yourself with What you should not concern yourself with
----------------------------------------- -----------------------------------------
An actor system manages the resources it is configured to use in order to run An actor system manages the resources it is configured to use in order to run
the actors which it contains. There may be millions of actors within one such the actors which it contains. There may be millions of actors within one such
system, after all the mantra is to view them as abundant and they weigh in at system, after all the mantra is to view them as abundant and they weigh in at
an overhead of only roughly 300 bytes per instance. Naturally, the exact order an overhead of only roughly 300 bytes per instance. Naturally, the exact order
in which messages are processed in large systems is not controllable by the in which messages are processed in large systems is not controllable by the
application author, but this is also not intended. Take a step back and relax application author, but this is also not intended. Take a step back and relax
while Akka does the heavy lifting under the hood. while Akka does the heavy lifting under the hood.

View file

@ -3,123 +3,121 @@
What is an Actor? What is an Actor?
================= =================
The previous section about :ref:`actor-systems` explained how actors form The previous section about :ref:`actor-systems` explained how actors form
hierarchies and are the smallest unit when building an application. This hierarchies and are the smallest unit when building an application. This
section looks at one such actor in isolation, explaining the concepts you section looks at one such actor in isolation, explaining the concepts you
encounter while implementing it. For more an in depth reference with all the encounter while implementing it. For more an in depth reference with all the
details please refer to :ref:`actors-scala` and :ref:`untyped-actors-java`. details please refer to :ref:`actors-scala` and :ref:`untyped-actors-java`.
An actor is a container for `State`_, `Behavior`_, a `Mailbox`_, `Children`_ An actor is a container for `State`_, `Behavior`_, a `Mailbox`_, `Children`_
and a `Fault Handling Strategy`_. All of this is encapsulated behind an `Actor and a `Supervisor Strategy`_. All of this is encapsulated behind an `Actor
Reference`_. Finally, this happens `When an Actor Terminates`_. Reference`_. Finally, this happens `When an Actor Terminates`_.
Actor Reference Actor Reference
--------------- ---------------
As detailed below, an actor object needs to be shielded from the outside in As detailed below, an actor object needs to be shielded from the outside in
order to benefit from the actor model. Therefore, actors are represented to the order to benefit from the actor model. Therefore, actors are represented to the
outside using actor references, which are objects that can be passed around outside using actor references, which are objects that can be passed around
freely and without restriction. This split into inner and outer object enables freely and without restriction. This split into inner and outer object enables
transparency for all the desired operations: restarting an actor without transparency for all the desired operations: restarting an actor without
needing to update references elsewhere, placing the actual actor object on needing to update references elsewhere, placing the actual actor object on
remote hosts, sending messages to actors in completely different applications. remote hosts, sending messages to actors in completely different applications.
But the most important aspect is that it is not possible to look inside an But the most important aspect is that it is not possible to look inside an
actor and get hold of its state from the outside, unless the actor unwisely actor and get hold of its state from the outside, unless the actor unwisely
publishes this information itself. publishes this information itself.
State State
----- -----
Actor objects will typically contain some variables which reflect possible Actor objects will typically contain some variables which reflect possible
states the actor may be in. This can be an explicit state machine (e.g. using states the actor may be in. This can be an explicit state machine (e.g. using
the :ref:`fsm` module), or it could be a counter, set of listeners, pending the :ref:`fsm` module), or it could be a counter, set of listeners, pending
requests, etc. These data are what make an actor valuable, and they must be requests, etc. These data are what make an actor valuable, and they must be
protected from corruption by other actors. The good news is that Akka actors protected from corruption by other actors. The good news is that Akka actors
conceptually each have their own light-weight thread, which is completely conceptually each have their own light-weight thread, which is completely
shielded from the rest of the system. This means that instead of having to shielded from the rest of the system. This means that instead of having to
synchronize access using locks you can just write your actor code without synchronize access using locks you can just write your actor code without
worrying about concurrency at all. worrying about concurrency at all.
Behind the scenes Akka will run sets of actors on sets of real threads, where Behind the scenes Akka will run sets of actors on sets of real threads, where
typically many actors share one thread, and subsequent invocations of one actor typically many actors share one thread, and subsequent invocations of one actor
may end up being processed on different threads. Akka ensures that this may end up being processed on different threads. Akka ensures that this
implementation detail does not affect the single-threadedness of handling the implementation detail does not affect the single-threadedness of handling the
actors state. actors state.
Because the internal state is vital to an actors operations, having Because the internal state is vital to an actors operations, having
inconsistent state is fatal. Thus, when the actor fails and is restarted by its inconsistent state is fatal. Thus, when the actor fails and is restarted by its
supervisor, the state will be created from scratch, like upon first creating supervisor, the state will be created from scratch, like upon first creating
the actor. This is to enable the ability of self-healing of the system. the actor. This is to enable the ability of self-healing of the system.
Behavior Behavior
-------- --------
Every time a message is processed, it is matched against the current behavior Every time a message is processed, it is matched against the current behavior
of the actor. Behavior means a function which defines the actions to be taken of the actor. Behavior means a function which defines the actions to be taken
in reaction to the message at that point in time, say forward a request if the in reaction to the message at that point in time, say forward a request if the
client is authorized, deny it otherwise. This behavior may change over time, client is authorized, deny it otherwise. This behavior may change over time,
e.g. because different clients obtain authorization over time, or because the e.g. because different clients obtain authorization over time, or because the
actor may go into an “out-of-service” mode and later come back. These changes actor may go into an “out-of-service” mode and later come back. These changes
are achieved by either encoding them in state variables which are read from the are achieved by either encoding them in state variables which are read from the
behavior logic, or the function itself may be swapped out at runtime, see the behavior logic, or the function itself may be swapped out at runtime, see the
``become`` and ``unbecome`` operations. However, the initial behavior defined ``become`` and ``unbecome`` operations. However, the initial behavior defined
during construction of the actor object is special in the sense that a restart during construction of the actor object is special in the sense that a restart
of the actor will reset its behavior to this initial one. of the actor will reset its behavior to this initial one.
Mailbox Mailbox
------- -------
An actors purpose is the processing of messages, and these messages were sent An actors purpose is the processing of messages, and these messages were sent
to the actor from other actors (or from outside the actor system). The piece to the actor from other actors (or from outside the actor system). The piece
which connects sender and receiver is the actors mailbox: each actor has which connects sender and receiver is the actors mailbox: each actor has
exactly one mailbox to which all senders enqueue their messages. Enqueuing exactly one mailbox to which all senders enqueue their messages. Enqueuing
happens in the time-order of send operations, which means that messages sent happens in the time-order of send operations, which means that messages sent
from different actors may not have a defined order at runtime due to the from different actors may not have a defined order at runtime due to the
apparent randomness of distributing actors across threads. Sending multiple apparent randomness of distributing actors across threads. Sending multiple
messages to the same target from the same actor, on the other hand, will messages to the same target from the same actor, on the other hand, will
enqueue them in the same order. enqueue them in the same order.
There are different mailbox implementations to choose from, the default being a There are different mailbox implementations to choose from, the default being a
FIFO: the order of the messages processed by the actor matches the order in FIFO: the order of the messages processed by the actor matches the order in
which they were enqueued. This is usually a good default, but applications may which they were enqueued. This is usually a good default, but applications may
need to prioritize some messages over others. In this case, a priority mailbox need to prioritize some messages over others. In this case, a priority mailbox
will enqueue not always at the end but at a position as given by the message will enqueue not always at the end but at a position as given by the message
priority, which might even be at the front. While using such a queue, the order priority, which might even be at the front. While using such a queue, the order
of messages processed will naturally be defined by the queues algorithm and in of messages processed will naturally be defined by the queues algorithm and in
general not be FIFO. general not be FIFO.
An important feature in which Akka differs from some other actor model An important feature in which Akka differs from some other actor model
implementations is that the current behavior must always handle the next implementations is that the current behavior must always handle the next
dequeued message, there is no scanning the mailbox for the next matching one. dequeued message, there is no scanning the mailbox for the next matching one.
Failure to handle a message will typically be treated as a failure, unless this Failure to handle a message will typically be treated as a failure, unless this
behavior is overridden. behavior is overridden.
Children Children
-------- --------
Each actor is potentially a supervisor: if it creates children for delegating Each actor is potentially a supervisor: if it creates children for delegating
sub-tasks, it will automatically supervise them. The list of children is sub-tasks, it will automatically supervise them. The list of children is
maintained within the actors context and the actor has access to it. maintained within the actors context and the actor has access to it.
Modifications to the list are done by creating (``context.actorOf(...)``) or Modifications to the list are done by creating (``context.actorOf(...)``) or
stopping (``context.stop(child)``) children and these actions are reflected stopping (``context.stop(child)``) children and these actions are reflected
immediately. The actual creation and termination actions happen behind the immediately. The actual creation and termination actions happen behind the
scenes in an asynchronous way, so they do not “block” their supervisor. scenes in an asynchronous way, so they do not “block” their supervisor.
Fault Handling Strategy Supervisor Strategy
----------------------- -------------------
The final piece of an actor is its strategy for handling faults of its The final piece of an actor is its strategy for handling faults of its
children. To keep it simple and robust, this is declared outside of the actors children. Fault handling is then done transparently by Akka, applying one
code and has no access to the actors state. Fault handling is then done of the strategies described in :ref:`supervision` for each incoming failure.
transparently by Akka, applying one of the strategies described in As this strategy is fundamental to how an actor system is structured, it
:ref:`supervision` for each incoming failure. As this strategy is fundamental cannot be changed once an actor has been created.
to how an actor system is structured, it cannot be changed once an actor has
been created.
Considering that there is only one such strategy for each actor, this means Considering that there is only one such strategy for each actor, this means
that if different strategies apply to the various children of an actor, the that if different strategies apply to the various children of an actor, the
children should be grouped beneath intermediate supervisors with matching children should be grouped beneath intermediate supervisors with matching
strategies, preferring once more the structuring of actor systems according to strategies, preferring once more the structuring of actor systems according to
the splitting of tasks into sub-tasks. the splitting of tasks into sub-tasks.
When an Actor Terminates When an Actor Terminates

View file

@ -6,8 +6,8 @@ package akka.docs.actor;
//#testkit //#testkit
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.actor.FaultHandlingStrategy; import akka.actor.SupervisorStrategy;
import static akka.actor.FaultHandlingStrategy.*; import static akka.actor.SupervisorStrategy.*;
import akka.actor.OneForOneStrategy; import akka.actor.OneForOneStrategy;
import akka.actor.Props; import akka.actor.Props;
import akka.actor.Terminated; import akka.actor.Terminated;
@ -36,6 +36,30 @@ public class FaultHandlingTestBase {
//#testkit //#testkit
//#supervisor //#supervisor
static public class Supervisor extends UntypedActor { static public class Supervisor extends UntypedActor {
//#strategy
private static SupervisorStrategy strategy = new OneForOneStrategy(new Function<Throwable, Action>() {
@Override
public Action apply(Throwable t) {
if (t instanceof ArithmeticException) {
return resume();
} else if (t instanceof NullPointerException) {
return restart();
} else if (t instanceof IllegalArgumentException) {
return stop();
} else {
return escalate();
}
}
}, 10, 60000);
@Override
public SupervisorStrategy supervisorStrategy() {
return strategy;
}
//#strategy
public void onReceive(Object o) { public void onReceive(Object o) {
if (o instanceof Props) { if (o instanceof Props) {
getSender().tell(getContext().actorOf((Props) o)); getSender().tell(getContext().actorOf((Props) o));
@ -44,10 +68,35 @@ public class FaultHandlingTestBase {
} }
} }
} }
//#supervisor //#supervisor
//#supervisor2 //#supervisor2
static public class Supervisor2 extends UntypedActor { static public class Supervisor2 extends UntypedActor {
//#strategy2
private static SupervisorStrategy strategy = new OneForOneStrategy(new Function<Throwable, Action>() {
@Override
public Action apply(Throwable t) {
if (t instanceof ArithmeticException) {
return resume();
} else if (t instanceof NullPointerException) {
return restart();
} else if (t instanceof IllegalArgumentException) {
return stop();
} else {
return escalate();
}
}
}, 10, 60000);
@Override
public SupervisorStrategy supervisorStrategy() {
return strategy;
}
//#strategy2
public void onReceive(Object o) { public void onReceive(Object o) {
if (o instanceof Props) { if (o instanceof Props) {
getSender().tell(getContext().actorOf((Props) o)); getSender().tell(getContext().actorOf((Props) o));
@ -55,18 +104,19 @@ public class FaultHandlingTestBase {
unhandled(o); unhandled(o);
} }
} }
@Override @Override
public void preRestart(Throwable cause, Option<Object> msg) { public void preRestart(Throwable cause, Option<Object> msg) {
// do not kill all children, which is the default here // do not kill all children, which is the default here
} }
} }
//#supervisor2 //#supervisor2
//#child //#child
static public class Child extends UntypedActor { static public class Child extends UntypedActor {
int state = 0; int state = 0;
public void onReceive(Object o) throws Exception { public void onReceive(Object o) throws Exception {
if (o instanceof Exception) { if (o instanceof Exception) {
throw (Exception) o; throw (Exception) o;
@ -79,41 +129,25 @@ public class FaultHandlingTestBase {
} }
} }
} }
//#child //#child
//#strategy
static FaultHandlingStrategy strategy = new OneForOneStrategy(new Function<Throwable, Action>() {
@Override
public Action apply(Throwable t) {
if (t instanceof ArithmeticException) {
return resume();
} else if (t instanceof NullPointerException) {
return restart();
} else if (t instanceof IllegalArgumentException) {
return stop();
} else {
return escalate();
}
}
}, 10, 60000);
//#strategy
//#testkit //#testkit
static ActorSystem system; static ActorSystem system;
Duration timeout = Duration.create(5, SECONDS); Duration timeout = Duration.create(5, SECONDS);
@BeforeClass @BeforeClass
public static void start() { public static void start() {
system = ActorSystem.create("test", AkkaSpec.testConf()); system = ActorSystem.create("test", AkkaSpec.testConf());
} }
@AfterClass @AfterClass
public static void cleanup() { public static void cleanup() {
system.shutdown(); system.shutdown();
} }
@Test @Test
public void mustEmployFaultHandler() { public void mustEmploySupervisorStrategy() {
// code here // code here
//#testkit //#testkit
EventFilter ex1 = (EventFilter) new ErrorFilter(ArithmeticException.class); EventFilter ex1 = (EventFilter) new ErrorFilter(ArithmeticException.class);
@ -122,32 +156,32 @@ public class FaultHandlingTestBase {
EventFilter ex4 = (EventFilter) new ErrorFilter(Exception.class); EventFilter ex4 = (EventFilter) new ErrorFilter(Exception.class);
Seq<EventFilter> ignoreExceptions = seq(ex1, ex2, ex3, ex4); Seq<EventFilter> ignoreExceptions = seq(ex1, ex2, ex3, ex4);
system.eventStream().publish(new TestEvent.Mute(ignoreExceptions)); system.eventStream().publish(new TestEvent.Mute(ignoreExceptions));
//#create //#create
Props superprops = new Props(Supervisor.class).withFaultHandler(strategy); Props superprops = new Props(Supervisor.class);
ActorRef supervisor = system.actorOf(superprops, "supervisor"); ActorRef supervisor = system.actorOf(superprops, "supervisor");
ActorRef child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout); ActorRef child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout);
//#create //#create
//#resume //#resume
child.tell(42); child.tell(42);
assert Await.result(child.ask("get", 5000), timeout).equals(42); assert Await.result(child.ask("get", 5000), timeout).equals(42);
child.tell(new ArithmeticException()); child.tell(new ArithmeticException());
assert Await.result(child.ask("get", 5000), timeout).equals(42); assert Await.result(child.ask("get", 5000), timeout).equals(42);
//#resume //#resume
//#restart //#restart
child.tell(new NullPointerException()); child.tell(new NullPointerException());
assert Await.result(child.ask("get", 5000), timeout).equals(0); assert Await.result(child.ask("get", 5000), timeout).equals(0);
//#restart //#restart
//#stop //#stop
final TestProbe probe = new TestProbe(system); final TestProbe probe = new TestProbe(system);
probe.watch(child); probe.watch(child);
child.tell(new IllegalArgumentException()); child.tell(new IllegalArgumentException());
probe.expectMsg(new Terminated(child)); probe.expectMsg(new Terminated(child));
//#stop //#stop
//#escalate-kill //#escalate-kill
child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout); child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout);
probe.watch(child); probe.watch(child);
@ -155,9 +189,9 @@ public class FaultHandlingTestBase {
child.tell(new Exception()); child.tell(new Exception());
probe.expectMsg(new Terminated(child)); probe.expectMsg(new Terminated(child));
//#escalate-kill //#escalate-kill
//#escalate-restart //#escalate-restart
superprops = new Props(Supervisor2.class).withFaultHandler(strategy); superprops = new Props(Supervisor2.class);
supervisor = system.actorOf(superprops, "supervisor2"); supervisor = system.actorOf(superprops, "supervisor2");
child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout); child = (ActorRef) Await.result(supervisor.ask(new Props(Child.class), 5000), timeout);
child.tell(23); child.tell(23);
@ -167,10 +201,11 @@ public class FaultHandlingTestBase {
//#escalate-restart //#escalate-restart
//#testkit //#testkit
} }
//#testkit
//#testkit
public <A> Seq<A> seq(A... args) { public <A> Seq<A> seq(A... args) {
return JavaConverters.collectionAsScalaIterableConverter(java.util.Arrays.asList(args)).asScala().toSeq(); return JavaConverters.collectionAsScalaIterableConverter(java.util.Arrays.asList(args)).asScala().toSeq();
} }
//#testkit //#testkit
} }
//#testkit //#testkit

View file

@ -1,19 +1,19 @@
.. _fault-tolerance-java: .. _fault-tolerance-java:
Fault Handling Strategies (Java) Fault Tolerance (Java)
================================= ======================
.. sidebar:: Contents .. sidebar:: Contents
.. contents:: :local: .. contents:: :local:
As explained in :ref:`actor-systems` each actor is the supervisor of its As explained in :ref:`actor-systems` each actor is the supervisor of its
children, and as such each actor is given a fault handling strategy when it is children, and as such each actor defines fault handling supervisor strategy.
created. This strategy cannot be changed afterwards as it is an integral part This strategy cannot be changed afterwards as it is an integral part of the
of the actor systems structure. actor systems structure.
Creating a Fault Handling Strategy Creating a Supervisor Strategy
---------------------------------- ------------------------------
For the sake of demonstration let us consider the following strategy: For the sake of demonstration let us consider the following strategy:
@ -26,7 +26,7 @@ First off, it is a one-for-one strategy, meaning that each child is treated
separately (an all-for-one strategy works very similarly, the only difference separately (an all-for-one strategy works very similarly, the only difference
is that any decision is applied to all children of the supervisor, not only the is that any decision is applied to all children of the supervisor, not only the
failing one). There are limits set on the restart frequency, namely maximum 10 failing one). There are limits set on the restart frequency, namely maximum 10
restarts per minute; each of these settings defaults to could be left out, which means restarts per minute; each of these settings could be left out, which means
that the respective limit does not apply, leaving the possibility to specify an that the respective limit does not apply, leaving the possibility to specify an
absolute upper limit on the restarts or to make the restarts work infinitely. absolute upper limit on the restarts or to make the restarts work infinitely.
@ -50,7 +50,7 @@ where ``TestProbe`` provides an actor ref useful for receiving and inspecting re
.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java .. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java
:include: testkit :include: testkit
Using the strategy shown above let us create actors: Let us create actors:
.. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java .. includecode:: code/akka/docs/actor/FaultHandlingTestBase.java
:include: create :include: create

View file

@ -30,7 +30,7 @@ it's located in ``akka.actor.TypedActor``.
:include: typed-actor-extension-tools :include: typed-actor-extension-tools
.. warning:: .. warning::
Same as not exposing ``this`` of an Akka Actor, it's important not to expose ``this`` of a Typed Actor, Same as not exposing ``this`` of an Akka Actor, it's important not to expose ``this`` of a Typed Actor,
instead you should pass the external proxy reference, which is obtained from within your Typed Actor as instead you should pass the external proxy reference, which is obtained from within your Typed Actor as
``TypedActor.self()``, this is your external identity, as the ``ActorRef`` is the external identity of ``TypedActor.self()``, this is your external identity, as the ``ActorRef`` is the external identity of
@ -127,7 +127,7 @@ Request-reply-with-future message send
.. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java .. includecode:: code/akka/docs/actor/TypedActorDocTestBase.java
:include: typed-actor-call-future :include: typed-actor-call-future
This call is asynchronous, and the Future returned can be used for asynchronous composition. This call is asynchronous, and the Future returned can be used for asynchronous composition.
Stopping Typed Actors Stopping Typed Actors
--------------------- ---------------------
@ -153,6 +153,13 @@ you can create child Typed Actors by invoking ``typedActorOf(..)`` on that.
This also works for creating child Typed Actors in regular Akka Actors. This also works for creating child Typed Actors in regular Akka Actors.
Supervisor Strategy
-------------------
By having your Typed Actor implementation class implement ``TypedActor.Supervisor``
you can define the strategy to use for supervising child actors, as described in
:ref:`supervision` and :ref:`fault-tolerance-java`.
Lifecycle callbacks Lifecycle callbacks
------------------- -------------------

View file

@ -129,6 +129,7 @@ In addition, it offers:
* :obj:`getSelf()` reference to the :class:`ActorRef` of the actor * :obj:`getSelf()` reference to the :class:`ActorRef` of the actor
* :obj:`getSender()` reference sender Actor of the last received message, typically used as described in :ref:`UntypedActor.Reply` * :obj:`getSender()` reference sender Actor of the last received message, typically used as described in :ref:`UntypedActor.Reply`
* :obj:`supervisorStrategy()` user overridable definition the strategy to use for supervising child actors
* :obj:`getContext()` exposes contextual information for the actor and the current message, such as: * :obj:`getContext()` exposes contextual information for the actor and the current message, such as:
* factory methods to create child actors (:meth:`actorOf`) * factory methods to create child actors (:meth:`actorOf`)

View file

@ -403,7 +403,7 @@ v2.0::
context.parent context.parent
*Fault handling strategy* *Supervisor Strategy*
v1.3:: v1.3::
@ -420,14 +420,18 @@ v1.3::
v2.0:: v2.0::
val strategy = OneForOneStrategy({ class MyActor extends Actor {
case _: ArithmeticException ⇒ Resume override val supervisorStrategy = OneForOneStrategy({
case _: NullPointerException ⇒ Restart case _: ArithmeticException ⇒ Resume
case _: IllegalArgumentException ⇒ Stop case _: NullPointerException ⇒ Restart
case _: Exception ⇒ Escalate case _: IllegalArgumentException ⇒ Stop
}: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000)) case _: Exception ⇒ Escalate
}: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000))
val supervisor = system.actorOf(Props[Supervisor].withFaultHandler(strategy), "supervisor") def receive = {
case x =>
}
}
Documentation: Documentation:

View file

@ -154,6 +154,7 @@ In addition, it offers:
* :obj:`self` reference to the :class:`ActorRef` of the actor * :obj:`self` reference to the :class:`ActorRef` of the actor
* :obj:`sender` reference sender Actor of the last received message, typically used as described in :ref:`Actor.Reply` * :obj:`sender` reference sender Actor of the last received message, typically used as described in :ref:`Actor.Reply`
* :obj:`supervisorStrategy` user overridable definition the strategy to use for supervising child actors
* :obj:`context` exposes contextual information for the actor and the current message, such as: * :obj:`context` exposes contextual information for the actor and the current message, such as:
* factory methods to create child actors (:meth:`actorOf`) * factory methods to create child actors (:meth:`actorOf`)

View file

@ -17,6 +17,18 @@ object FaultHandlingDocSpec {
//#supervisor //#supervisor
//#supervisor //#supervisor
class Supervisor extends Actor { class Supervisor extends Actor {
//#strategy
import akka.actor.OneForOneStrategy
import akka.actor.SupervisorStrategy._
override val supervisorStrategy = OneForOneStrategy({
case _: ArithmeticException Resume
case _: NullPointerException Restart
case _: IllegalArgumentException Stop
case _: Exception Escalate
}: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000))
//#strategy
def receive = { def receive = {
case p: Props sender ! context.actorOf(p) case p: Props sender ! context.actorOf(p)
} }
@ -25,6 +37,18 @@ object FaultHandlingDocSpec {
//#supervisor2 //#supervisor2
class Supervisor2 extends Actor { class Supervisor2 extends Actor {
//#strategy2
import akka.actor.OneForOneStrategy
import akka.actor.SupervisorStrategy._
override val supervisorStrategy = OneForOneStrategy({
case _: ArithmeticException Resume
case _: NullPointerException Restart
case _: IllegalArgumentException Stop
case _: Exception Escalate
}: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000))
//#strategy2
def receive = { def receive = {
case p: Props sender ! context.actorOf(p) case p: Props sender ! context.actorOf(p)
} }
@ -56,21 +80,9 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender {
"apply the chosen strategy for its child" in { "apply the chosen strategy for its child" in {
//#testkit //#testkit
//#strategy
import akka.actor.OneForOneStrategy
import akka.actor.FaultHandlingStrategy._
val strategy = OneForOneStrategy({
case _: ArithmeticException Resume
case _: NullPointerException Restart
case _: IllegalArgumentException Stop
case _: Exception Escalate
}: Decider, maxNrOfRetries = Some(10), withinTimeRange = Some(60000))
//#strategy
//#create //#create
val superprops = Props[Supervisor].withFaultHandler(strategy) val supervisor = system.actorOf(Props[Supervisor], "supervisor")
val supervisor = system.actorOf(superprops, "supervisor")
supervisor ! Props[Child] supervisor ! Props[Child]
val child = expectMsgType[ActorRef] // retrieve answer from TestKits testActor val child = expectMsgType[ActorRef] // retrieve answer from TestKits testActor
@ -114,8 +126,7 @@ class FaultHandlingDocSpec extends AkkaSpec with ImplicitSender {
expectMsg(Terminated(child2)) expectMsg(Terminated(child2))
//#escalate-kill //#escalate-kill
//#escalate-restart //#escalate-restart
val superprops2 = Props[Supervisor2].withFaultHandler(strategy) val supervisor2 = system.actorOf(Props[Supervisor2], "supervisor2")
val supervisor2 = system.actorOf(superprops2, "supervisor2")
supervisor2 ! Props[Child] supervisor2 ! Props[Child]
val child3 = expectMsgType[ActorRef] val child3 = expectMsgType[ActorRef]

View file

@ -1,19 +1,19 @@
.. _fault-tolerance-scala: .. _fault-tolerance-scala:
Fault Handling Strategies (Scala) Fault Tolerance (Scala)
================================= =======================
.. sidebar:: Contents .. sidebar:: Contents
.. contents:: :local: .. contents:: :local:
As explained in :ref:`actor-systems` each actor is the supervisor of its As explained in :ref:`actor-systems` each actor is the supervisor of its
children, and as such each actor is given a fault handling strategy when it is children, and as such each actor defines fault handling supervisor strategy.
created. This strategy cannot be changed afterwards as it is an integral part This strategy cannot be changed afterwards as it is an integral part of the
of the actor systems structure. actor systems structure.
Creating a Fault Handling Strategy Creating a Supervisor Strategy
---------------------------------- ------------------------------
For the sake of demonstration let us consider the following strategy: For the sake of demonstration let us consider the following strategy:
@ -56,7 +56,7 @@ MustMatchers``
.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala .. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala
:include: testkit :include: testkit
Using the strategy shown above let us create actors: Let us create actors:
.. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala .. includecode:: code/akka/docs/actor/FaultHandlingDocSpec.scala
:include: create :include: create

View file

@ -30,7 +30,7 @@ it's located in ``akka.actor.TypedActor``.
:include: typed-actor-extension-tools :include: typed-actor-extension-tools
.. warning:: .. warning::
Same as not exposing ``this`` of an Akka Actor, it's important not to expose ``this`` of a Typed Actor, Same as not exposing ``this`` of an Akka Actor, it's important not to expose ``this`` of a Typed Actor,
instead you should pass the external proxy reference, which is obtained from within your Typed Actor as instead you should pass the external proxy reference, which is obtained from within your Typed Actor as
``TypedActor.self``, this is your external identity, as the ``ActorRef`` is the external identity of ``TypedActor.self``, this is your external identity, as the ``ActorRef`` is the external identity of
@ -127,7 +127,7 @@ Request-reply-with-future message send
.. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala .. includecode:: code/akka/docs/actor/TypedActorDocSpec.scala
:include: typed-actor-call-future :include: typed-actor-call-future
This call is asynchronous, and the Future returned can be used for asynchronous composition. This call is asynchronous, and the Future returned can be used for asynchronous composition.
Stopping Typed Actors Stopping Typed Actors
--------------------- ---------------------
@ -153,6 +153,13 @@ you can create child Typed Actors by invoking ``typedActorOf(..)`` on that.
This also works for creating child Typed Actors in regular Akka Actors. This also works for creating child Typed Actors in regular Akka Actors.
Supervisor Strategy
-------------------
By having your Typed Actor implementation class implement ``TypedActor.Supervisor``
you can define the strategy to use for supervising child actors, as described in
:ref:`supervision` and :ref:`fault-tolerance-scala`.
Lifecycle callbacks Lifecycle callbacks
------------------- -------------------

View file

@ -181,8 +181,10 @@ class TestActorRefSpec extends AkkaSpec("disp1.type=Dispatcher") with BeforeAndA
override def postRestart(reason: Throwable) { counter -= 1 } override def postRestart(reason: Throwable) { counter -= 1 }
}), self, "child") }), self, "child")
override def supervisorStrategy = OneForOneStrategy(List(classOf[ActorKilledException]), 5, 1000)
def receiveT = { case "sendKill" ref ! Kill } def receiveT = { case "sendKill" ref ! Kill }
}).withFaultHandler(OneForOneStrategy(List(classOf[ActorKilledException]), 5, 1000))) }))
boss ! "sendKill" boss ! "sendKill"