diff --git a/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala b/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala
index ecc6dbfb4b..131cdeee8f 100644
--- a/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala
+++ b/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala
@@ -4,64 +4,48 @@
package akka.actor
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+import org.scalatest.BeforeAndAfterEach
+
+import akka.testing._
+import akka.testing.Testing.{testMillis, sleepFor}
+import akka.util.duration._
import akka.config.Supervision._
-import akka.{OneWay, Die, Ping}
+import akka.{Die, Ping}
import Actor._
-import org.scalatest.junit.JUnitSuite
-import org.junit.Test
import java.util.concurrent.atomic.AtomicInteger
-import java.util.concurrent. {CountDownLatch, TimeUnit, LinkedBlockingQueue}
+import java.util.concurrent.LinkedBlockingQueue
+
object SupervisorSpec {
+ val Timeout = 5 seconds
+ val TimeoutMillis = testMillis(Timeout).toInt
+
+ // =====================================================
+ // Message logs
+ // =====================================================
+
+ val PingMessage = "ping"
+ val PongMessage = "pong"
+ val ExceptionMessage = "Expected exception; to test fault-tolerance"
+
var messageLog = new LinkedBlockingQueue[String]
- var oneWayLog = new LinkedBlockingQueue[String]
- def clearMessageLogs {
- messageLog.clear
- oneWayLog.clear
- }
+ def messageLogPoll = messageLog.poll(Timeout.length, Timeout.unit)
- class PingPong1Actor extends Actor {
- import self._
+ // =====================================================
+ // Actors
+ // =====================================================
+
+ class PingPongActor extends Actor {
def receive = {
case Ping =>
- messageLog.put("ping")
- reply("pong")
-
- case OneWay =>
- oneWayLog.put("oneway")
-
+ messageLog.put(PingMessage)
+ self.reply_?(PongMessage)
case Die =>
- throw new RuntimeException("Expected exception; to test fault-tolerance")
- }
- override def postRestart(reason: Throwable) {
- messageLog.put(reason.getMessage)
- }
- }
-
- class PingPong2Actor extends Actor {
- import self._
- def receive = {
- case Ping =>
- messageLog.put("ping")
- reply("pong")
- case Die =>
- throw new RuntimeException("Expected exception; to test fault-tolerance")
- }
- override def postRestart(reason: Throwable) {
- messageLog.put(reason.getMessage)
- }
- }
-
- class PingPong3Actor extends Actor {
- import self._
- def receive = {
- case Ping =>
- messageLog.put("ping")
- reply("pong")
- case Die =>
- throw new RuntimeException("Expected exception; to test fault-tolerance")
+ throw new RuntimeException(ExceptionMessage)
}
override def postRestart(reason: Throwable) {
@@ -69,490 +53,74 @@ object SupervisorSpec {
}
}
- class TemporaryActor extends Actor {
- import self._
- lifeCycle = Temporary
- def receive = {
- case Ping =>
- messageLog.put("ping")
- reply("pong")
- case Die =>
- throw new RuntimeException("Expected exception; to test fault-tolerance")
- }
-
- override def postRestart(reason: Throwable) {
- messageLog.put(reason.getMessage)
- }
+ class TemporaryActor extends PingPongActor {
+ self.lifeCycle = Temporary
}
class Master extends Actor {
- self.faultHandler = OneForOneStrategy(List(classOf[Exception]), 5, 1000)
+ self.faultHandler = OneForOneStrategy(List(classOf[Exception]), 5, testMillis(1 second).toInt)
+
val temp = self.spawnLink[TemporaryActor]
+
override def receive = {
- case Die => temp !! (Die, 5000)
- }
- }
-}
-
-/**
- * @author Jonas Bonér
- */
-class SupervisorSpec extends JUnitSuite {
- import SupervisorSpec._
-
- var pingpong1: ActorRef = _
- var pingpong2: ActorRef = _
- var pingpong3: ActorRef = _
- var temporaryActor: ActorRef = _
-
- @Test def shoulNotRestartProgrammaticallyLinkedTemporaryActor = {
- clearMessageLogs
- val master = actorOf[Master].start
-
- intercept[RuntimeException] {
- master !! (Die, 5000)
- }
-
- Thread.sleep(1000)
- assert(messageLog.size === 0)
- }
-
- @Test def shoulNotRestartTemporaryActor = {
- clearMessageLogs
- val sup = getTemporaryActorAllForOneSupervisor
-
- intercept[RuntimeException] {
- temporaryActor !! (Die, 5000)
- }
-
- Thread.sleep(1000)
- assert(messageLog.size === 0)
- }
-
- @Test def shouldStartServerForNestedSupervisorHierarchy = {
- clearMessageLogs
- val sup = getNestedSupervisorsAllForOneConf
- sup.start
-
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ case Die => temp !! (Die, TimeoutMillis)
}
}
- @Test def shouldKillSingleActorOneForOne = {
- clearMessageLogs
- val sup = getSingleActorOneForOneSupervisor
+ // =====================================================
+ // Creating actors and supervisors
+ // =====================================================
- intercept[RuntimeException] {
- pingpong1 !! (Die, 5000)
- }
+ def temporaryActorAllForOne = {
+ val temporaryActor = actorOf[TemporaryActor].start
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldCallKillCallSingleActorOneForOne = {
- clearMessageLogs
- val sup = getSingleActorOneForOneSupervisor
-
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- intercept[RuntimeException] {
- pingpong1 !! (Die, 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldKillSingleActorAllForOne = {
- clearMessageLogs
- val sup = getSingleActorAllForOneSupervisor
-
- intercept[RuntimeException] {
- pingpong1 !! (Die, 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldCallKillCallSingleActorAllForOne = {
- clearMessageLogs
- val sup = getSingleActorAllForOneSupervisor
-
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- intercept[RuntimeException] {
- pingpong1 !! (Die, 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldKillMultipleActorsOneForOne1 = {
- clearMessageLogs
- val sup = getMultipleActorsOneForOneConf
-
- intercept[RuntimeException] {
- pingpong1 !! (Die, 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldKillMultipleActorsOneForOne2 = {
- clearMessageLogs
- val sup = getMultipleActorsOneForOneConf
-
- intercept[RuntimeException] {
- pingpong3 !! (Die, 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldKillCallMultipleActorsOneForOne = {
- clearMessageLogs
- val sup = getMultipleActorsOneForOneConf
-
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- intercept[RuntimeException] {
- pingpong2 !! (Die, 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldKillMultipleActorsAllForOne = {
- clearMessageLogs
- val sup = getMultipleActorsAllForOneConf
-
- intercept[RuntimeException] {
- pingpong2 !! (Die, 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldCallKillCallMultipleActorsAllForOne = {
- clearMessageLogs
- val sup = getMultipleActorsAllForOneConf
-
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- intercept[RuntimeException] {
- pingpong2 !! (Die, 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldOneWayKillSingleActorOneForOne = {
- clearMessageLogs
- val sup = getSingleActorOneForOneSupervisor
-
- pingpong1 ! Die
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldOneWayCallKillCallSingleActorOneForOne = {
- clearMessageLogs
- val sup = getSingleActorOneForOneSupervisor
-
- pingpong1 ! OneWay
-
- expect("oneway") {
- oneWayLog.poll(5, TimeUnit.SECONDS)
- }
- pingpong1 ! Die
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- pingpong1 ! OneWay
-
- expect("oneway") {
- oneWayLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldRestartKilledActorsForNestedSupervisorHierarchy = {
- clearMessageLogs
- val sup = getNestedSupervisorsAllForOneConf
-
-
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- intercept[RuntimeException] {
- pingpong2 !! (Die, 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5 , TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("pong") {
- (pingpong1 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldAttemptRestartWhenExceptionDuringRestart {
- val inits = new AtomicInteger(0)
- val dyingActor = actorOf(new Actor {
- self.lifeCycle = Permanent
- inits.incrementAndGet
-
- if (!(inits.get % 2 != 0))
- throw new IllegalStateException("Don't wanna!")
-
- def receive = {
- case Ping => self.reply_?("pong")
- case Die => throw new Exception("expected")
- }
- })
- val supervisor =
- Supervisor(
- SupervisorConfig(
- OneForOneStrategy(classOf[Exception] :: Nil,3,10000),
- Supervise(dyingActor,Permanent) :: Nil))
-
- intercept[Exception] {
- dyingActor !! (Die, 5000)
- }
-
- expect("pong") {
- (dyingActor !! (Ping, 5000)).getOrElse("nil")
- }
-
- expect(3) { inits.get }
- supervisor.shutdown
- }
-
- // =============================================
- // Create some supervisors with different configurations
-
- def getTemporaryActorAllForOneSupervisor: Supervisor = {
- temporaryActor = actorOf[TemporaryActor].start
-
- Supervisor(
+ val supervisor = Supervisor(
SupervisorConfig(
- AllForOneStrategy(List(classOf[Exception]), 3, 5000),
+ AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis),
Supervise(
temporaryActor,
Temporary)
:: Nil))
+
+ (temporaryActor, supervisor)
}
- def getSingleActorAllForOneSupervisor: Supervisor = {
- pingpong1 = actorOf[PingPong1Actor].start
+ def singleActorAllForOne = {
+ val pingpong = actorOf[PingPongActor].start
- Supervisor(
+ val supervisor = Supervisor(
SupervisorConfig(
- AllForOneStrategy(List(classOf[Exception]), 3, 5000),
+ AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis),
Supervise(
- pingpong1,
+ pingpong,
Permanent)
:: Nil))
+
+ (pingpong, supervisor)
}
- def getSingleActorOneForOneSupervisor: Supervisor = {
- pingpong1 = actorOf[PingPong1Actor].start
+ def singleActorOneForOne = {
+ val pingpong = actorOf[PingPongActor].start
- Supervisor(
+ val supervisor = Supervisor(
SupervisorConfig(
- OneForOneStrategy(List(classOf[Exception]), 3, 5000),
+ OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis),
Supervise(
- pingpong1,
+ pingpong,
Permanent)
:: Nil))
+
+ (pingpong, supervisor)
}
- def getMultipleActorsAllForOneConf: Supervisor = {
- pingpong1 = actorOf[PingPong1Actor].start
- pingpong2 = actorOf[PingPong2Actor].start
- pingpong3 = actorOf[PingPong3Actor].start
+ def multipleActorsAllForOne = {
+ val pingpong1 = actorOf[PingPongActor].start
+ val pingpong2 = actorOf[PingPongActor].start
+ val pingpong3 = actorOf[PingPongActor].start
- Supervisor(
+ val supervisor = Supervisor(
SupervisorConfig(
- AllForOneStrategy(List(classOf[Exception]), 3, 5000),
+ AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis),
Supervise(
pingpong1,
Permanent)
@@ -565,16 +133,18 @@ class SupervisorSpec extends JUnitSuite {
pingpong3,
Permanent)
:: Nil))
+
+ (pingpong1, pingpong2, pingpong3, supervisor)
}
- def getMultipleActorsOneForOneConf: Supervisor = {
- pingpong1 = actorOf[PingPong1Actor].start
- pingpong2 = actorOf[PingPong2Actor].start
- pingpong3 = actorOf[PingPong3Actor].start
+ def multipleActorsOneForOne = {
+ val pingpong1 = actorOf[PingPongActor].start
+ val pingpong2 = actorOf[PingPongActor].start
+ val pingpong3 = actorOf[PingPongActor].start
- Supervisor(
+ val supervisor = Supervisor(
SupervisorConfig(
- OneForOneStrategy(List(classOf[Exception]), 3, 5000),
+ OneForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis),
Supervise(
pingpong1,
Permanent)
@@ -587,22 +157,24 @@ class SupervisorSpec extends JUnitSuite {
pingpong3,
Permanent)
:: Nil))
+
+ (pingpong1, pingpong2, pingpong3, supervisor)
}
- def getNestedSupervisorsAllForOneConf: Supervisor = {
- pingpong1 = actorOf[PingPong1Actor].start
- pingpong2 = actorOf[PingPong2Actor].start
- pingpong3 = actorOf[PingPong3Actor].start
+ def nestedSupervisorsAllForOne = {
+ val pingpong1 = actorOf[PingPongActor]
+ val pingpong2 = actorOf[PingPongActor]
+ val pingpong3 = actorOf[PingPongActor]
- Supervisor(
+ val supervisor = Supervisor(
SupervisorConfig(
- AllForOneStrategy(List(classOf[Exception]), 3, 5000),
+ AllForOneStrategy(List(classOf[Exception]), 3, TimeoutMillis),
Supervise(
pingpong1,
Permanent)
::
SupervisorConfig(
- AllForOneStrategy(Nil, 3, 5000),
+ AllForOneStrategy(Nil, 3, TimeoutMillis),
Supervise(
pingpong2,
Permanent)
@@ -612,5 +184,204 @@ class SupervisorSpec extends JUnitSuite {
Permanent)
:: Nil)
:: Nil))
+
+ (pingpong1, pingpong2, pingpong3, supervisor)
}
}
+
+class SupervisorSpec extends WordSpec with MustMatchers with BeforeAndAfterEach {
+ import SupervisorSpec._
+
+ override def beforeEach() = {
+ messageLog.clear
+ }
+
+ def ping(pingPongActor: ActorRef) = {
+ (pingPongActor !! (Ping, TimeoutMillis)).getOrElse("nil") must be (PongMessage)
+ messageLogPoll must be (PingMessage)
+ }
+
+ def kill(pingPongActor: ActorRef) = {
+ intercept[RuntimeException] { pingPongActor !! (Die, TimeoutMillis) }
+ messageLogPoll must be (ExceptionMessage)
+ }
+
+ "A supervisor" must {
+
+ "not restart programmatically linked temporary actor" in {
+ val master = actorOf[Master].start
+
+ intercept[RuntimeException] {
+ master !! (Die, TimeoutMillis)
+ }
+
+ sleepFor(1 second)
+ messageLog.size must be (0)
+ }
+
+ "not restart temporary actor" in {
+ val (temporaryActor, supervisor) = temporaryActorAllForOne
+
+ intercept[RuntimeException] {
+ temporaryActor !! (Die, TimeoutMillis)
+ }
+
+ sleepFor(1 second)
+ messageLog.size must be (0)
+ }
+
+ "start server for nested supervisor hierarchy" in {
+ val (actor1, actor2, actor3, supervisor) = nestedSupervisorsAllForOne
+ ping(actor1)
+ }
+
+ "kill single actor OneForOne" in {
+ val (actor, supervisor) = singleActorOneForOne
+ kill(actor)
+ }
+
+ "call-kill-call single actor OneForOne" in {
+ val (actor, supervisor) = singleActorOneForOne
+ ping(actor)
+ kill(actor)
+ ping(actor)
+ }
+
+ "kill single actor AllForOne" in {
+ val (actor, supervisor) = singleActorAllForOne
+ kill(actor)
+ }
+
+ "call-kill-call single actor AllForOne" in {
+ val (actor, supervisor) = singleActorAllForOne
+ ping(actor)
+ kill(actor)
+ ping(actor)
+ }
+
+ "kill multiple actors OneForOne 1" in {
+ val (actor1, actor2, actor3, supervisor) = multipleActorsOneForOne
+ kill(actor1)
+ }
+
+ "kill multiple actors OneForOne 2" in {
+ val (actor1, actor2, actor3, supervisor) = multipleActorsOneForOne
+ kill(actor3)
+ }
+
+ "call-kill-call multiple actors OneForOne" in {
+ val (actor1, actor2, actor3, supervisor) = multipleActorsOneForOne
+
+ ping(actor1)
+ ping(actor2)
+ ping(actor3)
+
+ kill(actor2)
+
+ ping(actor1)
+ ping(actor2)
+ ping(actor3)
+ }
+
+ "kill multiple actors AllForOne" in {
+ val (actor1, actor2, actor3, supervisor) = multipleActorsAllForOne
+
+ kill(actor2)
+
+ // and two more exception messages
+ messageLogPoll must be (ExceptionMessage)
+ messageLogPoll must be (ExceptionMessage)
+ }
+
+ "call-kill-call multiple actors AllForOne" in {
+ val (actor1, actor2, actor3, supervisor) = multipleActorsAllForOne
+
+ ping(actor1)
+ ping(actor2)
+ ping(actor3)
+
+ kill(actor2)
+
+ // and two more exception messages
+ messageLogPoll must be (ExceptionMessage)
+ messageLogPoll must be (ExceptionMessage)
+
+ ping(actor1)
+ ping(actor2)
+ ping(actor3)
+ }
+
+ "one-way kill single actor OneForOne" in {
+ val (actor, supervisor) = singleActorOneForOne
+
+ actor ! Die
+ messageLogPoll must be (ExceptionMessage)
+ }
+
+ "one-way call-kill-call single actor OneForOne" in {
+ val (actor, supervisor) = singleActorOneForOne
+
+ actor ! Ping
+ messageLogPoll must be (PingMessage)
+
+ actor ! Die
+ messageLogPoll must be (ExceptionMessage)
+
+ actor ! Ping
+ messageLogPoll must be (PingMessage)
+ }
+
+ "restart killed actors in nested superviser hierarchy" in {
+ val (actor1, actor2, actor3, supervisor) = nestedSupervisorsAllForOne
+
+ ping(actor1)
+ ping(actor2)
+ ping(actor3)
+
+ kill(actor2)
+
+ // and two more exception messages
+ messageLogPoll must be (ExceptionMessage)
+ messageLogPoll must be (ExceptionMessage)
+
+ ping(actor1)
+ ping(actor2)
+ ping(actor3)
+ }
+
+ "must attempt restart when exception during restart" in {
+ val inits = new AtomicInteger(0)
+
+ val dyingActor = actorOf(new Actor {
+ self.lifeCycle = Permanent
+ inits.incrementAndGet
+
+ if (inits.get % 2 == 0) throw new IllegalStateException("Don't wanna!")
+
+ def receive = {
+ case Ping => self.reply_?(PongMessage)
+ case Die => throw new Exception("expected")
+ }
+ })
+
+ val supervisor =
+ Supervisor(
+ SupervisorConfig(
+ OneForOneStrategy(classOf[Exception] :: Nil, 3, 10000),
+ Supervise(dyingActor, Permanent) :: Nil))
+
+ intercept[Exception] {
+ dyingActor !! (Die, TimeoutMillis)
+ }
+
+ // give time for restart
+ sleepFor(3 seconds)
+
+ (dyingActor !! (Ping, TimeoutMillis)).getOrElse("nil") must be (PongMessage)
+
+ inits.get must be (3)
+
+ supervisor.shutdown
+ }
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/testing/Testing.scala b/akka-actor/src/test/scala/akka/testing/Testing.scala
index d957b26af7..98733c5434 100644
--- a/akka-actor/src/test/scala/akka/testing/Testing.scala
+++ b/akka-actor/src/test/scala/akka/testing/Testing.scala
@@ -25,5 +25,9 @@ object Testing {
def testTime(t: Float): Float = (timeFactor * t).toFloat
def testTime(t: Double): Double = timeFactor * t
+ def testSeconds(duration: Duration) = testTime(duration.toSeconds)
+ def testMillis(duration: Duration) = testTime(duration.toMillis)
+ def testNanos(duration: Duration) = testTime(duration.toNanos)
+
def sleepFor(duration: Duration) = Thread.sleep(testTime(duration.toMillis))
}
diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java
index 658ec71ff4..0fd24ac9a7 100644
--- a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java
+++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java
@@ -7,7 +7,7 @@ import akka.actor.UntypedActor;
import akka.stm.Ref;
public class UntypedCoordinatedCounter extends UntypedActor {
- private Ref count = new Ref(0);
+ private Ref count = new Ref(0);
private void increment() {
System.out.println("incrementing");
diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java
index 6f09a10173..3fc2ca502b 100644
--- a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java
+++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java
@@ -16,7 +16,7 @@ import java.util.concurrent.TimeUnit;
public class UntypedCoordinatedCounter extends UntypedActor {
private String name;
- private Ref count = new Ref(0);
+ private Ref count = new Ref(0);
private TransactionFactory txFactory = new TransactionFactoryBuilder()
.setTimeout(new FiniteDuration(3, TimeUnit.SECONDS))
.build();
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
index 941ec4c659..a6e17b5767 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
@@ -12,6 +12,10 @@ import akka.config.Supervision._
import java.util.concurrent.CountDownLatch
import akka.config.TypedActorConfigurator
+import akka.testing._
+import akka.util.duration._
+
+
/**
* @author Martin Krasser
*/
@@ -107,7 +111,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft
}
// allow some time for the actor to be stopped
- Thread.sleep(3000)
+ Testing.sleepFor(3 seconds)
val second = conf.getInstance(classOf[TypedActorFailer])
first should be (second)