Merge branch 'master' into wip-sync-2.4.10-patriknw

This commit is contained in:
Patrik Nordwall 2016-09-09 14:12:16 +02:00
commit e8ce261faf
53 changed files with 1071 additions and 391 deletions

View file

@ -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);
}
}

View file

@ -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)
}

View file

@ -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