Merge branch 'master' into wip-sync-2.4.10-patriknw
This commit is contained in:
commit
e8ce261faf
53 changed files with 1071 additions and 391 deletions
|
|
@ -4,9 +4,12 @@
|
|||
package akka.testkit;
|
||||
|
||||
import akka.actor.Terminated;
|
||||
import scala.Option;
|
||||
import scala.runtime.AbstractFunction0;
|
||||
import akka.actor.ActorRef;
|
||||
import akka.actor.ActorSystem;
|
||||
import akka.actor.Props;
|
||||
import akka.actor.SupervisorStrategy;
|
||||
import akka.event.Logging;
|
||||
import akka.event.Logging.LogEvent;
|
||||
import akka.japi.JavaPartialFunction;
|
||||
|
|
@ -684,4 +687,43 @@ public class JavaTestKit {
|
|||
public void shutdown(ActorSystem actorSystem, Boolean verifySystemShutdown) {
|
||||
shutdown(actorSystem, null, verifySystemShutdown);
|
||||
}
|
||||
|
||||
/**
|
||||
* Spawns an actor as a child of this test actor, and returns the child's ActorRef.
|
||||
* @param props Props to create the child actor
|
||||
* @param name Actor name for the child actor
|
||||
* @param supervisorStrategy Strategy should decide what to do with failures in the actor.
|
||||
*/
|
||||
public ActorRef childActorOf(Props props, String name, SupervisorStrategy supervisorStrategy) {
|
||||
return p.childActorOf(props, name, supervisorStrategy);
|
||||
}
|
||||
|
||||
/**
|
||||
* Spawns an actor as a child of this test actor, and returns the child's ActorRef.
|
||||
* The actor will have an auto-generated name.
|
||||
* @param props Props to create the child actor
|
||||
* @param supervisorStrategy Strategy should decide what to do with failures in the actor.
|
||||
*/
|
||||
public ActorRef childActorOf(Props props, SupervisorStrategy supervisorStrategy) {
|
||||
return p.childActorOf(props, supervisorStrategy);
|
||||
}
|
||||
|
||||
/**
|
||||
* Spawns an actor as a child of this test actor, and returns the child's ActorRef.
|
||||
* The actor will be supervised using {@link SupervisorStrategy.stoppingStrategy}.
|
||||
* @param props Props to create the child actor
|
||||
* @param name Actor name for the child actor
|
||||
*/
|
||||
public ActorRef childActorOf(Props props, String name) {
|
||||
return p.childActorOf(props, name);
|
||||
}
|
||||
|
||||
/**
|
||||
* Spawns an actor as a child of this test actor, and returns the child's ActorRef.
|
||||
* The actor will have an auto-generated name and will be supervised using {@link SupervisorStrategy.stoppingStrategy}.
|
||||
* @param props Props to create the child actor
|
||||
*/
|
||||
public ActorRef childActorOf(Props props) {
|
||||
return p.childActorOf(props);
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -41,6 +41,12 @@ object TestActor {
|
|||
final case class Watch(ref: ActorRef) extends NoSerializationVerificationNeeded
|
||||
final case class UnWatch(ref: ActorRef) extends NoSerializationVerificationNeeded
|
||||
final case class SetAutoPilot(ap: AutoPilot) extends NoSerializationVerificationNeeded
|
||||
final case class Spawn(props: Props, name: Option[String] = None, strategy: Option[SupervisorStrategy] = None) extends NoSerializationVerificationNeeded {
|
||||
def apply(context: ActorRefFactory): ActorRef = name match {
|
||||
case Some(n) ⇒ context.actorOf(props, n)
|
||||
case None ⇒ context.actorOf(props)
|
||||
}
|
||||
}
|
||||
|
||||
trait Message {
|
||||
def msg: AnyRef
|
||||
|
|
@ -54,6 +60,31 @@ object TestActor {
|
|||
|
||||
val FALSE = (x: Any) ⇒ false
|
||||
|
||||
/** INTERNAL API */
|
||||
private[TestActor] class DelegatingSupervisorStrategy extends SupervisorStrategy {
|
||||
import SupervisorStrategy._
|
||||
|
||||
private var delegates = Map.empty[ActorRef, SupervisorStrategy]
|
||||
|
||||
private def delegate(child: ActorRef) = delegates.get(child).getOrElse(stoppingStrategy)
|
||||
|
||||
def update(child: ActorRef, supervisor: SupervisorStrategy): Unit = delegates += (child → supervisor)
|
||||
|
||||
override def decider = defaultDecider // not actually invoked
|
||||
|
||||
override def handleChildTerminated(context: ActorContext, child: ActorRef, children: Iterable[ActorRef]): Unit = {
|
||||
delegates -= child
|
||||
}
|
||||
|
||||
override def processFailure(context: ActorContext, restart: Boolean, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Unit = {
|
||||
delegates(child).processFailure(context, restart, child, cause, stats, children)
|
||||
}
|
||||
|
||||
override def handleFailure(context: ActorContext, child: ActorRef, cause: Throwable, stats: ChildRestartStats, children: Iterable[ChildRestartStats]): Boolean = {
|
||||
delegates(child).handleFailure(context, child, cause, stats, children)
|
||||
}
|
||||
}
|
||||
|
||||
// make creator serializable, for VerifySerializabilitySpec
|
||||
def props(queue: BlockingDeque[Message]): Props = Props(classOf[TestActor], queue)
|
||||
}
|
||||
|
|
@ -61,6 +92,8 @@ object TestActor {
|
|||
class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
|
||||
import TestActor._
|
||||
|
||||
override val supervisorStrategy: DelegatingSupervisorStrategy = new DelegatingSupervisorStrategy
|
||||
|
||||
var ignore: Ignore = None
|
||||
|
||||
var autopilot: AutoPilot = NoAutoPilot
|
||||
|
|
@ -70,6 +103,10 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
|
|||
case Watch(ref) ⇒ context.watch(ref)
|
||||
case UnWatch(ref) ⇒ context.unwatch(ref)
|
||||
case SetAutoPilot(pilot) ⇒ autopilot = pilot
|
||||
case spawn: Spawn ⇒
|
||||
val actor = spawn(context)
|
||||
for (s ← spawn.strategy) supervisorStrategy(actor) = s
|
||||
queue.offerLast(RealMessage(actor, self))
|
||||
case x: AnyRef ⇒
|
||||
autopilot = autopilot.run(sender(), x) match {
|
||||
case KeepRunning ⇒ autopilot
|
||||
|
|
@ -102,7 +139,7 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
|
|||
*/
|
||||
trait TestKitBase {
|
||||
|
||||
import TestActor.{ Message, RealMessage, NullMessage }
|
||||
import TestActor.{ Message, RealMessage, NullMessage, Spawn }
|
||||
|
||||
implicit val system: ActorSystem
|
||||
val testKitSettings = TestKitExtension(system)
|
||||
|
|
@ -688,6 +725,46 @@ trait TestKitBase {
|
|||
TestKit.shutdownActorSystem(actorSystem, duration, verifySystemShutdown)
|
||||
}
|
||||
|
||||
/**
|
||||
* Spawns an actor as a child of this test actor, and returns the child's ActorRef.
|
||||
* @param props Props to create the child actor
|
||||
* @param name Actor name for the child actor
|
||||
* @param supervisorStrategy Strategy should decide what to do with failures in the actor.
|
||||
*/
|
||||
def childActorOf(props: Props, name: String, supervisorStrategy: SupervisorStrategy): ActorRef = {
|
||||
testActor ! Spawn(props, Some(name), Some(supervisorStrategy))
|
||||
expectMsgType[ActorRef]
|
||||
}
|
||||
|
||||
/**
|
||||
* Spawns an actor as a child of this test actor with an auto-generated name, and returns the child's ActorRef.
|
||||
* @param props Props to create the child actor
|
||||
* @param supervisorStrategy Strategy should decide what to do with failures in the actor.
|
||||
*/
|
||||
def childActorOf(props: Props, supervisorStrategy: SupervisorStrategy): ActorRef = {
|
||||
testActor ! Spawn(props, None, Some(supervisorStrategy))
|
||||
expectMsgType[ActorRef]
|
||||
}
|
||||
|
||||
/**
|
||||
* Spawns an actor as a child of this test actor with a stopping supervisor strategy, and returns the child's ActorRef.
|
||||
* @param props Props to create the child actor
|
||||
* @param name Actor name for the child actor
|
||||
*/
|
||||
def childActorOf(props: Props, name: String): ActorRef = {
|
||||
testActor ! Spawn(props, Some(name), None)
|
||||
expectMsgType[ActorRef]
|
||||
}
|
||||
|
||||
/**
|
||||
* Spawns an actor as a child of this test actor with an auto-generated name and stopping supervisor strategy, returning the child's ActorRef.
|
||||
* @param props Props to create the child actor
|
||||
*/
|
||||
def childActorOf(props: Props): ActorRef = {
|
||||
testActor ! Spawn(props, None, None)
|
||||
expectMsgType[ActorRef]
|
||||
}
|
||||
|
||||
private def format(u: TimeUnit, d: Duration) = "%.3f %s".format(d.toUnit(u), u.toString.toLowerCase)
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -7,6 +7,7 @@ import scala.concurrent.{ Await }
|
|||
import scala.concurrent.duration._
|
||||
import akka.pattern.ask
|
||||
import scala.util.Try
|
||||
import java.util.concurrent.atomic.AtomicInteger
|
||||
|
||||
class TestProbeSpec extends AkkaSpec with DefaultTimeout {
|
||||
|
||||
|
|
@ -39,6 +40,38 @@ class TestProbeSpec extends AkkaSpec with DefaultTimeout {
|
|||
probe1.expectMsg(0 millis, "some hint here", "world")
|
||||
}
|
||||
|
||||
"create a child when invoking actorOf" in {
|
||||
val probe = TestProbe()
|
||||
val child = probe.childActorOf(TestActors.echoActorProps)
|
||||
child.path.parent should be(probe.ref.path)
|
||||
|
||||
val namedChild = probe.childActorOf(TestActors.echoActorProps, "actorName")
|
||||
namedChild.path.name should be("actorName")
|
||||
}
|
||||
|
||||
"restart a failing child if the given supervisor says so" in {
|
||||
val restarts = new AtomicInteger(0)
|
||||
|
||||
class FailingActor extends Actor {
|
||||
override def receive = msg ⇒ msg match {
|
||||
case _ ⇒
|
||||
throw new RuntimeException("simulated failure")
|
||||
}
|
||||
|
||||
override def postRestart(reason: Throwable): Unit = {
|
||||
restarts.incrementAndGet()
|
||||
}
|
||||
}
|
||||
|
||||
val probe = TestProbe()
|
||||
val child = probe.childActorOf(Props(new FailingActor), SupervisorStrategy.defaultStrategy)
|
||||
|
||||
awaitAssert {
|
||||
child ! "hello"
|
||||
restarts.get() should be > (1)
|
||||
}
|
||||
}
|
||||
|
||||
def assertFailureMessageContains(expectedHint: String)(block: ⇒ Unit) {
|
||||
Try {
|
||||
block
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue