+tes #21198 Allow TestActor/TestKit/TestProbe to create child actors

This is useful in at least two scenarios:
- Unit testing actors that communicate to their parent directly
- Testing re-creating (typically persistent) actors with the same name
This commit is contained in:
Jan Ypma 2016-09-01 11:38:07 +02:00
parent 4821fb158e
commit 09f2d83e69
8 changed files with 248 additions and 33 deletions

View file

@ -4,14 +4,16 @@
package docs.testkit; package docs.testkit;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import akka.actor.*; import akka.actor.*;
import akka.japi.Creator; import akka.japi.Creator;
import akka.japi.Function; import akka.japi.Function;
import akka.testkit.AkkaJUnitActorSystemResource; import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.JavaTestKit;
import akka.testkit.TestActorRef; import akka.testkit.TestActorRef;
import akka.testkit.TestProbe; import akka.testkit.TestProbe;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
@ -187,6 +189,17 @@ public class ParentChildTest {
} }
//#test-fabricated-parent-creator //#test-fabricated-parent-creator
@Test
public void testProbeParentTest() throws Exception {
//#test-TestProbe-parent
JavaTestKit parent = new JavaTestKit(system);
ActorRef child = parent.childActorOf(Props.create(Child.class));
parent.send(child, "ping");
parent.expectMsgEquals("pong");
//#test-TestProbe-parent
}
@Test @Test
public void fabricatedParentTestsItsChildResponses() throws Exception { public void fabricatedParentTestsItsChildResponses() throws Exception {
// didn't put final on these in order to make the parent fit in one line in the html docs // didn't put final on these in order to make the parent fit in one line in the html docs

View file

@ -470,25 +470,51 @@ Testing parent-child relationships
The parent of an actor is always the actor that created it. At times this leads to The parent of an actor is always the actor that created it. At times this leads to
a coupling between the two that may not be straightforward to test. a coupling between the two that may not be straightforward to test.
Broadly, there are three approaches to improve testability of parent-child There are several approaches to improve testability of a child actor that
relationships: needs to refer to its parent:
1. when creating a child, pass an explicit reference to its parent 1. when creating a child, pass an explicit reference to its parent
2. when creating a parent, tell the parent how to create its child 2. create the child with a ``TestProbe`` as parent
3. create a fabricated parent when testing 3. create a fabricated parent when testing
Conversely, a parent's binding to its child can be lessened as follows:
4. when creating a parent, tell the parent how to create its child
For example, the structure of the code you want to test may follow this pattern: For example, the structure of the code you want to test may follow this pattern:
.. includecode:: code/docs/testkit/ParentChildTest.java#test-example .. includecode:: code/docs/testkit/ParentChildTest.java#test-example
Using dependency-injection Introduce child to its parent
^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The first option is to avoid use of the :meth:`context.parent` function and create The first option is to avoid use of the :meth:`context.parent` function and create
a child with a custom parent by passing an explicit reference to its parent instead. a child with a custom parent by passing an explicit reference to its parent instead.
.. includecode:: code/docs/testkit/ParentChildTest.java#test-dependentchild .. includecode:: code/docs/testkit/ParentChildTest.java#test-dependentchild
Create the child using JavaTestKit
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The ``JavaTestKit`` class can in fact create actors that will run with the test probe as parent.
This will cause any messages the the child actor sends to `context().getParent()` to
end up in the test probe.
.. includecode:: code/docs/testkit/ParentChildTest.java#test-TestProbe-parent
Using a fabricated parent
^^^^^^^^^^^^^^^^^^^^^^^^^
If you prefer to avoid modifying the child constructor you can
create a fabricated parent in your test. This, however, does not enable you to test
the parent actor in isolation.
.. includecode:: code/docs/testkit/ParentChildTest.java#test-fabricated-parent-creator
.. includecode:: code/docs/testkit/ParentChildTest.java#test-fabricated-parent
Externalize child making from the parent
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Alternatively, you can tell the parent how to create its child. There are two ways Alternatively, you can tell the parent how to create its child. There are two ways
to do this: by giving it a :class:`Props` object or by giving it a function which takes care of creating the child actor: to do this: by giving it a :class:`Props` object or by giving it a function which takes care of creating the child actor:
@ -503,19 +529,10 @@ And like this in your application code:
.. includecode:: code/docs/testkit/ParentChildTest.java#child-maker-prod .. includecode:: code/docs/testkit/ParentChildTest.java#child-maker-prod
Using a fabricated parent
^^^^^^^^^^^^^^^^^^^^^^^^^
If you prefer to avoid modifying the parent or child constructor you can
create a fabricated parent in your test. This, however, does not enable you to test
the parent actor in isolation.
.. includecode:: code/docs/testkit/ParentChildTest.java#test-fabricated-parent-creator
.. includecode:: code/docs/testkit/ParentChildTest.java#test-fabricated-parent
Which of these methods is the best depends on what is most important to test. The Which of these methods is the best depends on what is most important to test. The
most generic option is to create the parent actor by passing it a function that is most generic option is to create the parent actor by passing it a function that is
responsible for the Actor creation, but the fabricated parent is often sufficient. responsible for the Actor creation, but using TestProbe or having a fabricated parent is often sufficient.
.. _Java-CallingThreadDispatcher: .. _Java-CallingThreadDispatcher:

View file

@ -121,6 +121,17 @@ class ParentChildSpec extends WordSpec with Matchers with TestKitBase with Befor
} }
} }
//#test-TestProbe-parent
"A TestProbe serving as parent" should {
"test its child responses" in {
val parent = TestProbe()
val child = parent.childActorOf(Props[Child])
parent.send(child, "ping")
parent.expectMsg("pong")
}
}
//#test-TestProbe-parent
//#test-fabricated-parent //#test-fabricated-parent
"A fabricated parent" should { "A fabricated parent" should {
"test its child responses" in { "test its child responses" in {

View file

@ -548,26 +548,51 @@ Testing parent-child relationships
---------------------------------- ----------------------------------
The parent of an actor is always the actor that created it. At times this leads to The parent of an actor is always the actor that created it. At times this leads to
a coupling between the two that may not be straightforward to test. a coupling between the two that may not be straightforward to test.
Broadly, there are three approaches to improve testability of parent-child There are several approaches to improve testability of a child actor that
relationships: needs to refer to its parent:
1. when creating a child, pass an explicit reference to its parent 1. when creating a child, pass an explicit reference to its parent
2. when creating a parent, tell the parent how to create its child 2. create the child with a ``TestProbe`` as parent
3. create a fabricated parent when testing 3. create a fabricated parent when testing
Conversely, a parent's binding to its child can be lessened as follows:
4. when creating a parent, tell the parent how to create its child
For example, the structure of the code you want to test may follow this pattern: For example, the structure of the code you want to test may follow this pattern:
.. includecode:: code/docs/testkit/ParentChildSpec.scala#test-example .. includecode:: code/docs/testkit/ParentChildSpec.scala#test-example
Using dependency-injection Introduce child to its parent
^^^^^^^^^^^^^^^^^^^^^^^^^^ ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The first option is to avoid use of the :meth:`context.parent` function and create The first option is to avoid use of the :meth:`context.parent` function and create
a child with a custom parent by passing an explicit reference to its parent instead. a child with a custom parent by passing an explicit reference to its parent instead.
.. includecode:: code/docs/testkit/ParentChildSpec.scala#test-dependentchild .. includecode:: code/docs/testkit/ParentChildSpec.scala#test-dependentchild
Create the child using TestProbe
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The ``TestProbe`` class can in fact create actors that will run with the test probe as parent.
This will cause any messages the the child actor sends to `context.parent` to
end up in the test probe.
.. includecode:: code/docs/testkit/ParentChildSpec.scala##test-TestProbe-parent
Using a fabricated parent
^^^^^^^^^^^^^^^^^^^^^^^^^
If you prefer to avoid modifying the parent or child constructor you can
create a fabricated parent in your test. This, however, does not enable you to test
the parent actor in isolation.
.. includecode:: code/docs/testkit/ParentChildSpec.scala#test-fabricated-parent
Externalize child making from the parent
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
Alternatively, you can tell the parent how to create its child. There are two ways Alternatively, you can tell the parent how to create its child. There are two ways
to do this: by giving it a :class:`Props` object or by giving it a function which takes care of creating the child actor: to do this: by giving it a :class:`Props` object or by giving it a function which takes care of creating the child actor:
@ -581,14 +606,6 @@ And like this in your application code:
.. includecode:: code/docs/testkit/ParentChildSpec.scala#child-maker-prod .. includecode:: code/docs/testkit/ParentChildSpec.scala#child-maker-prod
Using a fabricated parent
^^^^^^^^^^^^^^^^^^^^^^^^^
If you prefer to avoid modifying the parent or child constructor you can
create a fabricated parent in your test. This, however, does not enable you to test
the parent actor in isolation.
.. includecode:: code/docs/testkit/ParentChildSpec.scala#test-fabricated-parent
Which of these methods is the best depends on what is most important to test. The Which of these methods is the best depends on what is most important to test. The
most generic option is to create the parent actor by passing it a function that is most generic option is to create the parent actor by passing it a function that is

View file

@ -4,9 +4,12 @@
package akka.testkit; package akka.testkit;
import akka.actor.Terminated; import akka.actor.Terminated;
import scala.Option;
import scala.runtime.AbstractFunction0; import scala.runtime.AbstractFunction0;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.actor.SupervisorStrategy;
import akka.event.Logging; import akka.event.Logging;
import akka.event.Logging.LogEvent; import akka.event.Logging.LogEvent;
import akka.japi.JavaPartialFunction; import akka.japi.JavaPartialFunction;
@ -684,4 +687,43 @@ public class JavaTestKit {
public void shutdown(ActorSystem actorSystem, Boolean verifySystemShutdown) { public void shutdown(ActorSystem actorSystem, Boolean verifySystemShutdown) {
shutdown(actorSystem, null, 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 Watch(ref: ActorRef) extends NoSerializationVerificationNeeded
final case class UnWatch(ref: ActorRef) extends NoSerializationVerificationNeeded final case class UnWatch(ref: ActorRef) extends NoSerializationVerificationNeeded
final case class SetAutoPilot(ap: AutoPilot) 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 { trait Message {
def msg: AnyRef def msg: AnyRef
@ -54,6 +60,31 @@ object TestActor {
val FALSE = (x: Any) false 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 // make creator serializable, for VerifySerializabilitySpec
def props(queue: BlockingDeque[Message]): Props = Props(classOf[TestActor], queue) def props(queue: BlockingDeque[Message]): Props = Props(classOf[TestActor], queue)
} }
@ -61,6 +92,8 @@ object TestActor {
class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor { class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
import TestActor._ import TestActor._
override val supervisorStrategy: DelegatingSupervisorStrategy = new DelegatingSupervisorStrategy
var ignore: Ignore = None var ignore: Ignore = None
var autopilot: AutoPilot = NoAutoPilot var autopilot: AutoPilot = NoAutoPilot
@ -70,6 +103,10 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
case Watch(ref) context.watch(ref) case Watch(ref) context.watch(ref)
case UnWatch(ref) context.unwatch(ref) case UnWatch(ref) context.unwatch(ref)
case SetAutoPilot(pilot) autopilot = pilot 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 case x: AnyRef
autopilot = autopilot.run(sender(), x) match { autopilot = autopilot.run(sender(), x) match {
case KeepRunning autopilot case KeepRunning autopilot
@ -102,7 +139,7 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
*/ */
trait TestKitBase { trait TestKitBase {
import TestActor.{ Message, RealMessage, NullMessage } import TestActor.{ Message, RealMessage, NullMessage, Spawn }
implicit val system: ActorSystem implicit val system: ActorSystem
val testKitSettings = TestKitExtension(system) val testKitSettings = TestKitExtension(system)
@ -688,6 +725,46 @@ trait TestKitBase {
TestKit.shutdownActorSystem(actorSystem, duration, verifySystemShutdown) 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) 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 scala.concurrent.duration._
import akka.pattern.ask import akka.pattern.ask
import scala.util.Try import scala.util.Try
import java.util.concurrent.atomic.AtomicInteger
class TestProbeSpec extends AkkaSpec with DefaultTimeout { class TestProbeSpec extends AkkaSpec with DefaultTimeout {
@ -39,6 +40,38 @@ class TestProbeSpec extends AkkaSpec with DefaultTimeout {
probe1.expectMsg(0 millis, "some hint here", "world") 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) { def assertFailureMessageContains(expectedHint: String)(block: Unit) {
Try { Try {
block block

View file

@ -951,7 +951,12 @@ object MiMa extends AutoPlugin {
// #21025 new orElse flow op // #21025 new orElse flow op
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.orElseGraph"), ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.orElseGraph"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.orElse"), ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOps.orElse"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOpsMat.orElseMat") ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.stream.scaladsl.FlowOpsMat.orElseMat"),
// #21201 adding childActorOf to TestActor / TestKit / TestProbe
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.testkit.TestKitBase.childActorOf$default$3"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.testkit.TestKitBase.childActorOf$default$2"),
ProblemFilters.exclude[ReversedMissingMethodProblem]("akka.testkit.TestKitBase.childActorOf")
) )
) )
} }