Merge branch 'wip-1952-testkit-java-∂π'

This commit is contained in:
Roland 2012-07-04 17:38:20 +02:00
commit cde7b29a33
18 changed files with 1586 additions and 82 deletions

1
.gitignore vendored
View file

@ -64,3 +64,4 @@ mongoDB/
redis/
beanstalk/
.scalastyle
bin/

View file

@ -0,0 +1,11 @@
package akka.japi;
import scala.collection.Seq;
public class JAPI {
public static <T> Seq<T> seq(T... ts) {
return Util.arrayToSeq(ts);
}
}

View file

@ -586,6 +586,7 @@ object Logging {
/** Null Object used for errors without cause Throwable */
object NoCause extends NoStackTrace
}
def noCause = Error.NoCause
/**
* For WARNING Logging

View file

@ -5,6 +5,7 @@
package akka.japi
import scala.Some
import scala.util.control.NoStackTrace
/**
* A Function interface. Used to create first-class-functions is Java.
@ -44,6 +45,76 @@ trait Creator[T] {
def create(): T
}
object PurePartialFunction {
sealed abstract class NoMatchException extends RuntimeException with NoStackTrace
case object NoMatch extends NoMatchException
final def noMatch(): RuntimeException = NoMatch
}
/**
* Helper for implementing a *pure* partial function: it will possibly be
* invoked multiple times for a single application, because its only abstract
* method is used for both isDefinedAt() and apply(); the former is mapped to
* `isCheck == true` and the latter to `isCheck == false` for those cases where
* this is important to know.
*
* Failure to match is signaled by throwing `noMatch()`, i.e. not returning
* normally (the exception used in this case is pre-allocated, hence not
* <i>that</i> expensive).
*
* {{{
* new PurePartialFunction<Object, String>() {
* public String apply(Object in, boolean isCheck) {
* if (in instanceof TheThing) {
* if (isCheck) return null; // to spare the expensive or side-effecting code
* return doSomethingWithTheThing((TheThing) in);
* } else {
* throw noMatch();
* }
* }
* }
* }}}
*
* The typical use of partial functions from Akka looks like the following:
*
* {{{
* if (pf.isDefinedAt(x)) {
* pf.apply(x);
* }
* }}}
*
* i.e. it will first call `PurePartialFunction.apply(x, true)` and if that
* does not throw `noMatch()` it will continue with calling
* `PurePartialFunction.apply(x, false)`.
*/
abstract class PurePartialFunction[A, B] extends scala.runtime.AbstractFunction1[A, B] with PartialFunction[A, B] {
import PurePartialFunction._
def apply(x: A, isCheck: Boolean): B
final def isDefinedAt(x: A): Boolean = try { apply(x, true); true } catch { case NoMatch false }
final def apply(x: A): B = try apply(x, false) catch { case NoMatch throw new MatchError }
}
/**
* This is a specialized variant of PartialFunction which is <b><i>only
* applicable if you know that `isDefinedAt(x)` is always called before
* `apply(x)`with the same `x` of course.</i></b>
*
* `match(x)` will be called for `isDefinedAt(x)` only, and its semantics
* are the same as for [[akka.japi.PurePartialFunction]] (apart from the
* missing because unneeded boolean argument).
*/
abstract class CachingPartialFunction[A, B <: AnyRef] extends scala.runtime.AbstractFunction1[A, B] with PartialFunction[A, B] {
import PurePartialFunction._
def `match`(x: A): B
var cache: B = _
final def isDefinedAt(x: A): Boolean = try { cache = `match`(x); true } catch { case NoMatch cache = null.asInstanceOf[B]; false }
final def apply(x: A): B = cache
}
/**
* This class represents optional values. Instances of <code>Option</code>
* are either instances of case class <code>Some</code> or it is case
@ -117,4 +188,8 @@ object Util {
* Given a Class returns a Scala Manifest of that Class
*/
def manifest[T](clazz: Class[T]): Manifest[T] = Manifest.classType(clazz)
def arrayToSeq[T](arr: Array[T]): Seq[T] = arr.toSeq
def arrayToSeq(classes: Array[Class[_]]): Seq[Class[_]] = classes.toSeq
}

View file

@ -0,0 +1,399 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.testkit;
import static org.junit.Assert.*;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import com.typesafe.config.ConfigFactory;
import com.typesafe.config.Config;
import akka.actor.ActorKilledException;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Kill;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
import akka.dispatch.Await;
import akka.dispatch.Future;
import akka.testkit.CallingThreadDispatcher;
import akka.testkit.TestActor;
import akka.testkit.TestActor.AutoPilot;
import akka.testkit.TestActorRef;
import akka.testkit.JavaTestKit;
import akka.util.Duration;
public class TestKitDocTest {
//#test-actor-ref
static class MyActor extends UntypedActor {
public void onReceive(Object o) throws Exception {
if (o.equals("say42")) {
getSender().tell(42, getSelf());
} else if (o instanceof Exception) {
throw (Exception) o;
}
}
public boolean testMe() { return true; }
}
//#test-actor-ref
private static ActorSystem system;
@BeforeClass
public static void setup() {
final Config config = ConfigFactory.parseString(
"akka.event-handlers = [akka.testkit.TestEventListener]");
system = ActorSystem.create("demoSystem", config);
}
@AfterClass
public static void cleanup() {
system.shutdown();
}
//#test-actor-ref
@Test
public void demonstrateTestActorRef() {
final Props props = new Props(MyActor.class);
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "testA");
final MyActor actor = ref.underlyingActor();
assertTrue(actor.testMe());
}
//#test-actor-ref
@Test
public void demonstrateAsk() throws Exception {
//#test-behavior
final Props props = new Props(MyActor.class);
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "testB");
final Future<Object> future = akka.pattern.Patterns.ask(ref, "say42", 3000);
assertTrue(future.isCompleted());
assertEquals(42, Await.result(future, Duration.Zero()));
//#test-behavior
}
@Test
public void demonstrateExceptions() {
//#test-expecting-exceptions
final Props props = new Props(MyActor.class);
final TestActorRef<MyActor> ref = TestActorRef.create(system, props, "myActor");
try {
ref.receive(new Exception("expected"));
fail("expected an exception to be thrown");
} catch (Exception e) {
assertEquals("expected", e.getMessage());
}
//#test-expecting-exceptions
}
@Test
public void demonstrateWithin() {
//#test-within
new JavaTestKit(system) {{
getRef().tell(42);
new Within(Duration.Zero(), Duration.parse("1 second")) {
// do not put code outside this method, will run afterwards
public void run() {
assertEquals((Integer) 42, expectMsgClass(Integer.class));
}
};
}};
//#test-within
}
@Test
public void demonstrateExpectMsg() {
//#test-expectmsg
new JavaTestKit(system) {{
getRef().tell(42);
final String out = new ExpectMsg<String>("match hint") {
// do not put code outside this method, will run afterwards
protected String match(Object in) {
if (in instanceof Integer) {
return "match";
} else {
throw noMatch();
}
}
}.get(); // this extracts the received message
assertEquals("match", out);
}};
//#test-expectmsg
}
@Test
public void demonstrateReceiveWhile() {
//#test-receivewhile
new JavaTestKit(system) {{
getRef().tell(42);
getRef().tell(43);
getRef().tell("hello");
final String[] out =
new ReceiveWhile<String>(String.class, duration("1 second")) {
// do not put code outside this method, will run afterwards
protected String match(Object in) {
if (in instanceof Integer) {
return in.toString();
} else {
throw noMatch();
}
}
}.get(); // this extracts the received messages
assertArrayEquals(new String[] {"42", "43"}, out);
expectMsgEquals("hello");
}};
//#test-receivewhile
new JavaTestKit(system) {{
//#test-receivewhile-full
new ReceiveWhile<String>( // type of array to be created must match ...
String.class, // ... this class which is needed to that end
duration("100 millis"), // maximum collect time
duration("50 millis"), // maximum time between messages
12 // maximum number of messages to collect
) {
//#match-elided
protected String match(Object in) {
throw noMatch();
}
//#match-elided
};
//#test-receivewhile-full
}};
}
@Test
public void demonstrateAwaitCond() {
//#test-awaitCond
new JavaTestKit(system) {{
getRef().tell(42);
new AwaitCond(
duration("1 second"), // maximum wait time
duration("100 millis") // interval at which to check the condition
) {
// do not put code outside this method, will run afterwards
protected boolean cond() {
// typically used to wait for something to start up
return msgAvailable();
}
};
}};
//#test-awaitCond
}
@Test
@SuppressWarnings("unchecked") // due to generic varargs
public void demonstrateExpect() {
new JavaTestKit(system) {{
getRef().tell("hello");
getRef().tell("hello");
getRef().tell("hello");
getRef().tell("world");
getRef().tell(42);
getRef().tell(42);
//#test-expect
final String hello = expectMsgEquals("hello");
final Object any = expectMsgAnyOf("hello", "world");
final Object[] all = expectMsgAllOf("hello", "world");
final int i = expectMsgClass(Integer.class);
final Number j = expectMsgAnyClassOf(Integer.class, Long.class);
expectNoMsg();
//#test-expect
assertEquals("hello", hello);
assertEquals("hello", any);
assertEquals(42, i);
assertEquals(42, j);
assertArrayEquals(new String[] {"hello", "world"}, all);
}};
}
@Test
public void demonstrateIgnoreMsg() {
//#test-ignoreMsg
new JavaTestKit(system) {{
// ignore all Strings
new IgnoreMsg() {
protected boolean ignore(Object msg) {
return msg instanceof String;
}
};
getRef().tell("hello");
getRef().tell(42);
expectMsgEquals(42);
// remove message filter
ignoreNoMsg();
getRef().tell("hello");
expectMsgEquals("hello");
}};
//#test-ignoreMsg
}
@Test
public void demonstrateDilated() {
//#duration-dilation
new JavaTestKit(system) {{
final Duration original = duration("1 second");
final Duration stretched = dilated(original);
assertTrue("dilated", stretched.gteq(original));
}};
//#duration-dilation
}
@Test
public void demonstrateProbe() {
//#test-probe
// simple actor which just forwards messages
class Forwarder extends UntypedActor {
final ActorRef target;
public Forwarder(ActorRef target) {
this.target = target;
}
public void onReceive(Object msg) {
target.forward(msg, getContext());
}
}
new JavaTestKit(system) {{
// create a test probe
final JavaTestKit probe = new JavaTestKit(system);
// create a forwarder, injecting the probes testActor
final Props props = new Props(new UntypedActorFactory() {
private static final long serialVersionUID = 8927158735963950216L;
public UntypedActor create() {
return new Forwarder(probe.getRef());
}
});
final ActorRef forwarder = system.actorOf(props, "forwarder");
// verify correct forwarding
forwarder.tell(42, getRef());
probe.expectMsgEquals(42);
assertEquals(getRef(), probe.getLastSender());
}};
//#test-probe
}
@Test
public void demonstrateSpecialProbe() {
//#test-special-probe
new JavaTestKit(system) {{
class MyProbe extends JavaTestKit {
public MyProbe() {
super(system);
}
public void assertHello() {
expectMsgEquals("hello");
}
}
final MyProbe probe = new MyProbe();
probe.getRef().tell("hello");
probe.assertHello();
}};
//#test-special-probe
}
@Test
public void demonstrateReply() {
//#test-probe-reply
new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system);
probe.getRef().tell("hello", getRef());
probe.expectMsgEquals("hello");
probe.reply("world");
expectMsgEquals("world");
assertEquals(probe.getRef(), getLastSender());
}};
//#test-probe-reply
}
@Test
public void demonstrateForward() {
//#test-probe-forward
new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system);
probe.getRef().tell("hello", getRef());
probe.expectMsgEquals("hello");
probe.forward(getRef());
expectMsgEquals("hello");
assertEquals(getRef(), getLastSender());
}};
//#test-probe-forward
}
@Test
public void demonstrateWithinProbe() {
try {
//#test-within-probe
new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system);
new Within(duration("1 second")) {
public void run() {
probe.expectMsgEquals("hello");
}
};
}};
//#test-within-probe
} catch (AssertionError e) {
// expected to fail
}
}
@Test
public void demonstrateAutoPilot() {
//#test-auto-pilot
new JavaTestKit(system) {{
final JavaTestKit probe = new JavaTestKit(system);
// install auto-pilot
probe.setAutoPilot(new TestActor.AutoPilot() {
public AutoPilot run(ActorRef sender, Object msg) {
sender.tell(msg);
return noAutoPilot();
}
});
// first one is replied to directly ...
probe.getRef().tell("hello", getRef());
expectMsgEquals("hello");
// ... but then the auto-pilot switched itself off
probe.getRef().tell("world", getRef());
expectNoMsg();
}};
//#test-auto-pilot
}
// only compilation
public void demonstrateCTD() {
//#calling-thread-dispatcher
system.actorOf(
new Props(MyActor.class)
.withDispatcher(CallingThreadDispatcher.Id()));
//#calling-thread-dispatcher
}
@Test
public void demonstrateEventFilter() {
//#test-event-filter
new JavaTestKit(system) {{
assertEquals("demoSystem", system.name());
final ActorRef victim = system.actorOf(Props.empty(), "victim");
final int result = new EventFilter<Integer>(ActorKilledException.class) {
protected Integer run() {
victim.tell(Kill.getInstance());
return 42;
}
}.from("akka://demoSystem/user/victim").occurrences(1).exec();
assertEquals(42, result);
}};
//#test-event-filter
}
}

View file

@ -0,0 +1,95 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.testkit;
//#fullsample
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.actor.Props;
import akka.actor.UntypedActor;
import akka.testkit.JavaTestKit;
import akka.util.Duration;
public class TestKitSampleTest {
public static class SomeActor extends UntypedActor {
ActorRef target = null;
public void onReceive(Object msg) {
if (msg.equals("hello")) {
getSender().tell("world");
if (target != null) target.forward(msg, getContext());
} else if (msg instanceof ActorRef) {
target = (ActorRef) msg;
getSender().tell("done");
}
}
}
static ActorSystem system;
@BeforeClass
public static void setup() {
system = ActorSystem.create();
}
@AfterClass
public static void teardown() {
system.shutdown();
}
@Test
public void testIt() {
/*
* Wrap the whole test procedure within a testkit constructor
* if you want to receive actor replies or use Within(), etc.
*/
new JavaTestKit(system) {{
final Props props = new Props(SomeActor.class);
final ActorRef subject = system.actorOf(props);
// can also use JavaTestKit from the outside
final JavaTestKit probe = new JavaTestKit(system);
// inject the probe by passing it to the test subject
// like a real resource would be passed in production
subject.tell(probe.getRef(), getRef());
// await the correct response
expectMsgEquals(duration("1 second"), "done");
// the run() method needs to finish within 3 seconds
new Within(duration("3 seconds")) {
protected void run() {
subject.tell("hello", getRef());
// This is a demo: would normally use expectMsgEquals().
// Wait time is bounded by 3-second deadline above.
new AwaitCond() {
protected boolean cond() {
return probe.msgAvailable();
}
};
// response must have been enqueued to us before probe
expectMsgEquals(Duration.Zero(), "world");
// check that the probe we injected earlier got the msg
probe.expectMsgEquals(Duration.Zero(), "hello");
Assert.assertEquals(getRef(), probe.getLastSender());
// Will wait for the rest of the 3 seconds
expectNoMsg();
}
};
}};
}
}
//#fullsample

View file

@ -92,7 +92,7 @@ There are 4 different types of message dispatchers:
* CallingThreadDispatcher
- This dispatcher runs invocations on the current thread only. This dispatcher does not create any new threads,
but it can be used from different threads concurrently for the same actor. See :ref:`TestCallingThreadDispatcherRef`
but it can be used from different threads concurrently for the same actor. See :ref:`Java-CallingThreadDispatcher`
for details and restrictions.
- Sharability: Unlimited

View file

@ -4,11 +4,574 @@
Testing Actor Systems (Java)
##############################
Due to the conciseness of test DSLs available for Scala, it may be a good idea
to write the test suite in that language even if the main project is written in
Java. If that is not desirable, you can also use :class:`TestKit` and friends
from Java, albeit with more verbose syntax Munish Gupta has `published a nice
post <http://www.akkaessentials.in/2012/05/using-testkit-with-java.html>`_
showing several patterns you may find useful, and for reference documentation
please refer to :ref:`akka-testkit` until that section has been ported over to
cover Java in full.
As with any piece of software, automated tests are a very important part of the
development cycle. The actor model presents a different view on how units of
code are delimited and how they interact, which has an influence on how to
perform tests.
.. note::
Due to the conciseness of test DSLs available for Scala (`ScalaTest`_,
`Specs2`_, `ScalaCheck`_), it may be a good idea to write the test suite in
that language even if the main project is written in Java. If that is not
desirable, you can also use :class:`TestKit` and friends from Java, albeit
with more verbose syntax which is covered below. Munish Gupta has `published
a nice post
<http://www.akkaessentials.in/2012/05/using-testkit-with-java.html>`_ showing
several patterns you may find useful.
.. _ScalaTest: http://scalatest.org/
.. _Specs2: http://specs2.org/
.. _ScalaCheck: http://code.google.com/p/scalacheck/
Akka comes with a dedicated module :mod:`akka-testkit` for supporting tests at
different levels, which fall into two clearly distinct categories:
- Testing isolated pieces of code without involving the actor model, meaning
without multiple threads; this implies completely deterministic behavior
concerning the ordering of events and no concurrency concerns and will be
called **Unit Testing** in the following.
- Testing (multiple) encapsulated actors including multi-threaded scheduling;
this implies non-deterministic order of events but shielding from
concurrency concerns by the actor model and will be called **Integration
Testing** in the following.
There are of course variations on the granularity of tests in both categories,
where unit testing reaches down to white-box tests and integration testing can
encompass functional tests of complete actor networks. The important
distinction lies in whether concurrency concerns are part of the test or not.
The tools offered are described in detail in the following sections.
.. note::
Be sure to add the module :mod:`akka-testkit` to your dependencies.
Unit Testing with :class:`TestActorRef`
=======================================
Testing the business logic inside :class:`Actor` classes can be divided into
two parts: first, each atomic operation must work in isolation, then sequences
of incoming events must be processed correctly, even in the presence of some
possible variability in the ordering of events. The former is the primary use
case for single-threaded unit testing, while the latter can only be verified in
integration tests.
Normally, the :class:`ActorRef` shields the underlying :class:`Actor` instance
from the outside, the only communications channel is the actor's mailbox. This
restriction is an impediment to unit testing, which led to the inception of the
:class:`TestActorRef`. This special type of reference is designed specifically
for test purposes and allows access to the actor in two ways: either by
obtaining a reference to the underlying actor instance, or by invoking or
querying the actor's behaviour (:meth:`receive`). Each one warrants its own
section below.
Obtaining a Reference to an :class:`Actor`
------------------------------------------
Having access to the actual :class:`Actor` object allows application of all
traditional unit testing techniques on the contained methods. Obtaining a
reference is done like this:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-actor-ref
Since :class:`TestActorRef` is generic in the actor type it returns the
underlying actor with its proper static type. From this point on you may bring
any unit testing tool to bear on your actor as usual.
Testing the Actor's Behavior
----------------------------
When the dispatcher invokes the processing behavior of an actor on a message,
it actually calls :meth:`apply` on the current behavior registered for the
actor. This starts out with the return value of the declared :meth:`receive`
method, but it may also be changed using :meth:`become` and :meth:`unbecome` in
response to external messages. All of this contributes to the overall actor
behavior and it does not lend itself to easy testing on the :class:`Actor`
itself. Therefore the :class:`TestActorRef` offers a different mode of
operation to complement the :class:`Actor` testing: it supports all operations
also valid on normal :class:`ActorRef`. Messages sent to the actor are
processed synchronously on the current thread and answers may be sent back as
usual. This trick is made possible by the :class:`CallingThreadDispatcher`
described below (see `CallingThreadDispatcher`_); this dispatcher is set
implicitly for any actor instantiated into a :class:`TestActorRef`.
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-behavior
As the :class:`TestActorRef` is a subclass of :class:`LocalActorRef` with a few
special extras, also aspects like supervision and restarting work properly, but
beware that execution is only strictly synchronous as long as all actors
involved use the :class:`CallingThreadDispatcher`. As soon as you add elements
which include more sophisticated scheduling you leave the realm of unit testing
as you then need to think about asynchronicity again (in most cases the problem
will be to wait until the desired effect had a chance to happen).
One more special aspect which is overridden for single-threaded tests is the
:meth:`receiveTimeout`, as including that would entail asynchronous queuing of
:obj:`ReceiveTimeout` messages, violating the synchronous contract.
.. warning::
To summarize: :class:`TestActorRef` overwrites two fields: it sets the
dispatcher to :obj:`CallingThreadDispatcher.global` and it sets the
:obj:`receiveTimeout` to None.
The Way In-Between: Expecting Exceptions
----------------------------------------
If you want to test the actor behavior, including hotswapping, but without
involving a dispatcher and without having the :class:`TestActorRef` swallow
any thrown exceptions, then there is another mode available for you: just use
the :meth:`receive` method :class:`TestActorRef`, which will be forwarded to the
underlying actor:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-expecting-exceptions
Use Cases
---------
You may of course mix and match both modi operandi of :class:`TestActorRef` as
suits your test needs:
- one common use case is setting up the actor into a specific internal state
before sending the test message
- another is to verify correct internal state transitions after having sent
the test message
Feel free to experiment with the possibilities, and if you find useful
patterns, don't hesitate to let the Akka forums know about them! Who knows,
common operations might even be worked into nice DSLs.
Integration Testing with :class:`JavaTestKit`
=============================================
When you are reasonably sure that your actor's business logic is correct, the
next step is verifying that it works correctly within its intended environment.
The definition of the environment depends of course very much on the problem at
hand and the level at which you intend to test, ranging for
functional/integration tests to full system tests. The minimal setup consists
of the test procedure, which provides the desired stimuli, the actor under
test, and an actor receiving replies. Bigger systems replace the actor under
test with a network of actors, apply stimuli at varying injection points and
arrange results to be sent from different emission points, but the basic
principle stays the same in that a single procedure drives the test.
The :class:`JavaTestKit` class contains a collection of tools which makes this
common task easy.
.. includecode:: code/docs/testkit/TestKitSampleTest.java#fullsample
The :class:`JavaTestKit` contains an actor named :obj:`testActor` which is the
entry point for messages to be examined with the various ``expectMsg...``
assertions detailed below. The test actors reference is obtained using the
:meth:`getRef()` method as demonstrated above. The :obj:`testActor` may also
be passed to other actors as usual, usually subscribing it as notification
listener. There is a whole set of examination methods, e.g. receiving all
consecutive messages matching certain criteria, receiving a whole sequence of
fixed messages or classes, receiving nothing for some time, etc.
The ActorSystem passed in to the constructor of JavaTestKit is accessible via the
:meth:`getSystem()` method.
.. note::
Remember to shut down the actor system after the test is finished (also in
case of failure) so that all actors—including the test actor—are stopped.
Built-In Assertions
-------------------
The above mentioned :meth:`expectMsgEquals` is not the only method for
formulating assertions concerning received messages, the full set is this:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-expect
In these examples, the maximum durations you will find mentioned below are left
out, in which case they use the default value from configuration item
``akka.test.single-expect-default`` which itself defaults to 3 seconds (or they
obey the innermost enclosing :class:`Within` as detailed :ref:`below
<JavaTestKit.within>`). The full signatures are:
* :meth:`public <T> T expectMsgEquals(Duration max, T msg)`
The given message object must be received within the specified time; the
object will be returned.
* :meth:`public Object expectMsgAnyOf(Duration max, Object... msg)`
An object must be received within the given time, and it must be equal
(compared with ``equals()``) to at least one of the passed reference
objects; the received object will be returned.
* :meth:`public Object[] expectMsgAllOf(Duration max, Object... msg)`
A number of objects matching the size of the supplied object array must be
received within the given time, and for each of the given objects there
must exist at least one among the received ones which equals it (compared
with ``equals()``). The full sequence of received objects is returned in
the order received.
* :meth:`public <T> T expectMsgClass(Duration max, Class<T> c)`
An object which is an instance of the given :class:`Class` must be received
within the allotted time frame; the object will be returned. Note that this
does a conformance check, if you need the class to be equal you need to
verify that afterwards.
* :meth:`public <T> T expectMsgAnyClassOf(Duration max, Class<? extends T>... c)`
An object must be received within the given time, and it must be an
instance of at least one of the supplied :class:`Class` objects; the
received object will be returned. Note that this does a conformance check,
if you need the class to be equal you need to verify that afterwards.
.. note::
Because of a limitation in Javas type system it may be necessary to add
``@SuppressWarnings("unchecked")`` when using this method.
* :meth:`public void expectNoMsg(Duration max)`
No message must be received within the given time. This also fails if a
message has been received before calling this method which has not been
removed from the queue using one of the other methods.
For cases which require more refined conditions there are constructs which take
code blocks:
* **ExpectMsg<T>**
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-expectmsg
The :meth:`match(Object in)` method will be evaluated once a message has
been received within the allotted time (which may be given as constructor
argument). If it throws ``noMatch()`` (where it is sufficient to call that
method; the ``throw`` keyword is only needed in cases where the compiler
would otherwise complain about wrong return types—Java is lacking Scalas
notion of a type which signifies “will not ever return normally”), then the
expectation fails with an :class:`AssertionError`, otherwise the matched
and possibly transformed object is stored for retrieval using the
:meth:`get()` method.
* **ReceiveWhile<T>**
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-receivewhile
This construct works like ExpectMsg, but it continually collects messages
as long as they match the criteria, and it does not fail when a
non-matching one is encountered. Collecting messages also ends when the
time is up, when too much time passes between messages or when enough
messages have been received.
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-receivewhile-full
:exclude: match-elided
The need to specify the ``String`` result type twice results from the need
to create a correctly typed array and Javas inability to infer the classs
type argument.
* **AwaitCond**
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-awaitCond
This general construct is not connected with the test kits message
reception, the embedded condition can compute the boolean result from
anything in scope.
There are also cases where not all messages sent to the test kit are actually
relevant to the test, but removing them would mean altering the actors under
test. For this purpose it is possible to ignore certain messages:
* **IgnoreMsg**
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-ignoreMsg
Expecting Log Messages
----------------------
Since an integration test does not allow to the internal processing of the
participating actors, verifying expected exceptions cannot be done directly.
Instead, use the logging system for this purpose: replacing the normal event
handler with the :class:`TestEventListener` and using an :class:`EventFilter`
allows assertions on log messages, including those which are generated by
exceptions:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-event-filter
If a number of occurrences is specific—as demonstrated above—then ``exec()``
will block until that number of matching messages have been received or the
timeout configured in ``akka.test.filter-leeway`` is used up (time starts
counting after the ``run()`` method returns). In case of a timeout the test
fails.
.. note::
Be sure to exchange the default event handler with the
:class:`TestEventListener` in your ``application.conf`` to enable this
function::
akka.event-handlers = [akka.testkit.TestEventListener]
.. _JavaTestKit.within:
Timing Assertions
-----------------
Another important part of functional testing concerns timing: certain events
must not happen immediately (like a timer), others need to happen before a
deadline. Therefore, all examination methods accept an upper time limit within
the positive or negative result must be obtained. Lower time limits need to be
checked external to the examination, which is facilitated by a new construct
for managing time constraints:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-within
The block in :meth:`Within.run()` must complete after a :ref:`Duration` which
is between :obj:`min` and :obj:`max`, where the former defaults to zero. The
deadline calculated by adding the :obj:`max` parameter to the block's start
time is implicitly available within the block to all examination methods, if
you do not specify it, it is inherited from the innermost enclosing
:meth:`within` block.
It should be noted that if the last message-receiving assertion of the block is
:meth:`expectNoMsg` or :meth:`receiveWhile`, the final check of the
:meth:`within` is skipped in order to avoid false positives due to wake-up
latencies. This means that while individual contained assertions still use the
maximum time bound, the overall block may take arbitrarily longer in this case.
.. note::
All times are measured using ``System.nanoTime``, meaning that they describe
wall time, not CPU time or system time.
Accounting for Slow Test Systems
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The tight timeouts you use during testing on your lightning-fast notebook will
invariably lead to spurious test failures on the heavily loaded Jenkins server
(or similar). To account for this situation, all maximum durations are
internally scaled by a factor taken from the :ref:`configuration`,
``akka.test.timefactor``, which defaults to 1.
You can scale other durations with the same factor by using the implicit conversion
in ``akka.testkit`` package object to add dilated function to :class:`Duration`.
.. includecode:: code/docs/testkit/TestKitDocTest.java#duration-dilation
Using Multiple Probe Actors
---------------------------
When the actors under test are supposed to send various messages to different
destinations, it may be difficult distinguishing the message streams arriving
at the :obj:`testActor` when using the :class:`JavaTestKit` as shown until now.
Another approach is to use it for creation of simple probe actors to be
inserted in the message flows. The functionality is best explained using a
small example:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-probe
This simple test verifies an equally simple Forwarder actor by injecting a
probe as the forwarders target. Another example would be two actors A and B
which collaborate by A sending messages to B. In order to verify this message
flow, a :class:`TestProbe` could be inserted as target of A, using the
forwarding capabilities or auto-pilot described below to include a real B in
the test setup.
Probes may also be equipped with custom assertions to make your test code even
more concise and clear:
.. includecode:: code/docs/testkit/TestKitDocTest.java
:include: test-special-probe
You have complete flexibility here in mixing and matching the
:class:`JavaTestKit` facilities with your own checks and choosing an intuitive
name for it. In real life your code will probably be a bit more complicated
than the example given above; just use the power!
Replying to Messages Received by Probes
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The probe stores the sender of the last dequeued message (i.e. after its
``expectMsg*`` reception), which may be retrieved using the
:meth:`getLastSender()` method. This information can also implicitly be used
for having the probe reply to the last received message:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-probe-reply
Forwarding Messages Received by Probes
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The probe can also forward a received message (i.e. after its ``expectMsg*``
reception), retaining the original sender:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-probe-forward
Auto-Pilot
^^^^^^^^^^
Receiving messages in a queue for later inspection is nice, but in order to
keep a test running and verify traces later you can also install an
:class:`AutoPilot` in the participating test probes (actually in any
:class:`TestKit`) which is invoked before enqueueing to the inspection queue.
This code can be used to forward messages, e.g. in a chain ``A --> Probe -->
B``, as long as a certain protocol is obeyed.
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-auto-pilot
The :meth:`run` method must return the auto-pilot for the next message, wrapped
in an :class:`Option`; setting it to :obj:`None` terminates the auto-pilot.
Caution about Timing Assertions
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
The behavior of :meth:`within` blocks when using test probes might be perceived
as counter-intuitive: you need to remember that the nicely scoped deadline as
described :ref:`above <JavaTestKit.within>` is local to each probe. Hence, probes
do not react to each other's deadlines or to the deadline set in an enclosing
:class:`JavaTestKit` instance:
.. includecode:: code/docs/testkit/TestKitDocTest.java#test-within-probe
Here, the ``expectMsgEquals`` call will use the default timeout.
.. _Java-CallingThreadDispatcher:
CallingThreadDispatcher
=======================
The :class:`CallingThreadDispatcher` serves good purposes in unit testing, as
described above, but originally it was conceived in order to allow contiguous
stack traces to be generated in case of an error. As this special dispatcher
runs everything which would normally be queued directly on the current thread,
the full history of a message's processing chain is recorded on the call stack,
so long as all intervening actors run on this dispatcher.
How to use it
-------------
Just set the dispatcher as you normally would:
.. includecode:: code/docs/testkit/TestKitDocTest.java#calling-thread-dispatcher
How it works
------------
When receiving an invocation, the :class:`CallingThreadDispatcher` checks
whether the receiving actor is already active on the current thread. The
simplest example for this situation is an actor which sends a message to
itself. In this case, processing cannot continue immediately as that would
violate the actor model, so the invocation is queued and will be processed when
the active invocation on that actor finishes its processing; thus, it will be
processed on the calling thread, but simply after the actor finishes its
previous work. In the other case, the invocation is simply processed
immediately on the current thread. Futures scheduled via this dispatcher are
also executed immediately.
This scheme makes the :class:`CallingThreadDispatcher` work like a general
purpose dispatcher for any actors which never block on external events.
In the presence of multiple threads it may happen that two invocations of an
actor running on this dispatcher happen on two different threads at the same
time. In this case, both will be processed directly on their respective
threads, where both compete for the actor's lock and the loser has to wait.
Thus, the actor model is left intact, but the price is loss of concurrency due
to limited scheduling. In a sense this is equivalent to traditional mutex style
concurrency.
The other remaining difficulty is correct handling of suspend and resume: when
an actor is suspended, subsequent invocations will be queued in thread-local
queues (the same ones used for queuing in the normal case). The call to
:meth:`resume`, however, is done by one specific thread, and all other threads
in the system will probably not be executing this specific actor, which leads
to the problem that the thread-local queues cannot be emptied by their native
threads. Hence, the thread calling :meth:`resume` will collect all currently
queued invocations from all threads into its own queue and process them.
Limitations
-----------
If an actor's behavior blocks on a something which would normally be affected
by the calling actor after having sent the message, this will obviously
dead-lock when using this dispatcher. This is a common scenario in actor tests
based on :class:`CountDownLatch` for synchronization:
.. code-block:: scala
val latch = new CountDownLatch(1)
actor ! startWorkAfter(latch) // actor will call latch.await() before proceeding
doSomeSetupStuff()
latch.countDown()
The example would hang indefinitely within the message processing initiated on
the second line and never reach the fourth line, which would unblock it on a
normal dispatcher.
Thus, keep in mind that the :class:`CallingThreadDispatcher` is not a
general-purpose replacement for the normal dispatchers. On the other hand it
may be quite useful to run your actor network on it for testing, because if it
runs without dead-locking chances are very high that it will not dead-lock in
production.
.. warning::
The above sentence is unfortunately not a strong guarantee, because your
code might directly or indirectly change its behavior when running on a
different dispatcher. If you are looking for a tool to help you debug
dead-locks, the :class:`CallingThreadDispatcher` may help with certain error
scenarios, but keep in mind that it has may give false negatives as well as
false positives.
Benefits
--------
To summarize, these are the features with the :class:`CallingThreadDispatcher`
has to offer:
- Deterministic execution of single-threaded tests while retaining nearly full
actor semantics
- Full message processing history leading up to the point of failure in
exception stack traces
- Exclusion of certain classes of dead-lock scenarios
.. _actor.logging-java:
Tracing Actor Invocations
=========================
The testing facilities described up to this point were aiming at formulating
assertions about a systems behavior. If a test fails, it is usually your job
to find the cause, fix it and verify the test again. This process is supported
by debuggers as well as logging, where the Akka toolkit offers the following
options:
* *Logging of exceptions thrown within Actor instances*
This is always on; in contrast to the other logging mechanisms, this logs at
``ERROR`` level.
* *Logging of special messages*
Actors handle certain special messages automatically, e.g. :obj:`Kill`,
:obj:`PoisonPill`, etc. Tracing of these message invocations is enabled by
the setting ``akka.actor.debug.autoreceive``, which enables this on all
actors.
* *Logging of the actor lifecycle*
Actor creation, start, restart, monitor start, monitor stop and stop may be traced by
enabling the setting ``akka.actor.debug.lifecycle``; this, too, is enabled
uniformly on all actors.
All these messages are logged at ``DEBUG`` level. To summarize, you can enable
full logging of actor activities using this configuration fragment::
akka {
loglevel = DEBUG
actor {
debug {
autoreceive = on
lifecycle = on
}
}
}

View file

@ -125,7 +125,10 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
val actorRef = TestActorRef(new MyActor)
// hypothetical message stimulating a '42' answer
val result = Await.result((actorRef ? Say42), 5 seconds).asInstanceOf[Int]
val future = actorRef ? Say42
val result = future.value.get match {
case Right(x: Int) x
}
result must be(42)
//#test-behavior
}
@ -146,7 +149,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
val actorRef = TestActorRef(new Actor {
def receive = {
case boom throw new IllegalArgumentException("boom")
case "hello" throw new IllegalArgumentException("boom")
}
})
intercept[IllegalArgumentException] { actorRef.receive("hello") }
@ -272,4 +275,15 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
//#test-kit-base
}
"demonstrate within() nesting" in {
intercept[AssertionError] {
//#test-within-probe
val probe = TestProbe()
within(1 second) {
probe.expectMsg("hello")
}
//#test-within-probe
}
}
}

View file

@ -93,7 +93,7 @@ There are 4 different types of message dispatchers:
* CallingThreadDispatcher
- This dispatcher runs invocations on the current thread only. This dispatcher does not create any new threads,
but it can be used from different threads concurrently for the same actor. See :ref:`TestCallingThreadDispatcherRef`
but it can be used from different threads concurrently for the same actor. See :ref:`Scala-CallingThreadDispatcher`
for details and restrictions.
- Sharability: Unlimited

View file

@ -424,7 +424,7 @@ This FSM will log at DEBUG level:
* all state transitions
Life cycle changes and special messages can be logged as described for
:ref:`Actors <actor.logging>`.
:ref:`Actors <actor.logging-scala>`.
Rolling Event Log
-----------------

View file

@ -67,15 +67,6 @@ Since :class:`TestActorRef` is generic in the actor type it returns the
underlying actor with its proper static type. From this point on you may bring
any unit testing tool to bear on your actor as usual.
Expecting Exceptions
--------------------
Testing that an expected exception is thrown while processing a message sent to
the actor under test can be done by using a :class:`TestActorRef` :meth:`receive` based
invocation:
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-expecting-exceptions
.. _TestFSMRef:
Testing Finite State Machines
@ -111,8 +102,8 @@ operation to complement the :class:`Actor` testing: it supports all operations
also valid on normal :class:`ActorRef`. Messages sent to the actor are
processed synchronously on the current thread and answers may be sent back as
usual. This trick is made possible by the :class:`CallingThreadDispatcher`
described below; this dispatcher is set implicitly for any actor instantiated
into a :class:`TestActorRef`.
described below (see `CallingThreadDispatcher`_); this dispatcher is set
implicitly for any actor instantiated into a :class:`TestActorRef`.
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-behavior
@ -134,8 +125,8 @@ One more special aspect which is overridden for single-threaded tests is the
dispatcher to :obj:`CallingThreadDispatcher.global` and it sets the
:obj:`receiveTimeout` to None.
The Way In-Between
------------------
The Way In-Between: Expecting Exceptions
----------------------------------------
If you want to test the actor behavior, including hotswapping, but without
involving a dispatcher and without having the :class:`TestActorRef` swallow
@ -143,10 +134,7 @@ any thrown exceptions, then there is another mode available for you: just use
the :meth:`receive` method :class:`TestActorRef`, which will be forwarded to the
underlying actor:
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-unhandled
The above sample assumes the default behavior for unhandled messages, i.e.
that the actor doesn't swallow all messages and doesn't override :meth:`unhandled`.
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-expecting-exceptions
Use Cases
---------
@ -205,12 +193,12 @@ Built-In Assertions
The above mentioned :meth:`expectMsg` is not the only method for formulating
assertions concerning received messages. Here is the full list:
* :meth:`expectMsg[T](d: Duration, msg: T): T`
* :meth:`expectMsg[T](d: Duration, msg: T): T`
The given message object must be received within the specified time; the
object will be returned.
* :meth:`expectMsgPF[T](d: Duration)(pf: PartialFunction[Any, T]): T`
* :meth:`expectMsgPF[T](d: Duration)(pf: PartialFunction[Any, T]): T`
Within the given time period, a message must be received and the given
partial function must be defined for that message; the result from applying
@ -219,40 +207,40 @@ assertions concerning received messages. Here is the full list:
the deadline from the innermost enclosing :ref:`within <TestKit.within>`
block instead.
* :meth:`expectMsgClass[T](d: Duration, c: Class[T]): T`
* :meth:`expectMsgClass[T](d: Duration, c: Class[T]): T`
An object which is an instance of the given :class:`Class` must be received
within the allotted time frame; the object will be returned. Note that this
does a conformance check; if you need the class to be equal, have a look at
:meth:`expectMsgAllClassOf` with a single given class argument.
* :meth:`expectMsgType[T: Manifest](d: Duration)`
* :meth:`expectMsgType[T: Manifest](d: Duration)`
An object which is an instance of the given type (after erasure) must be
received within the allotted time frame; the object will be returned. This
method is approximately equivalent to
``expectMsgClass(manifest[T].erasure)``.
* :meth:`expectMsgAnyOf[T](d: Duration, obj: T*): T`
* :meth:`expectMsgAnyOf[T](d: Duration, obj: T*): T`
An object must be received within the given time, and it must be equal (
compared with ``==``) to at least one of the passed reference objects; the
received object will be returned.
* :meth:`expectMsgAnyClassOf[T](d: Duration, obj: Class[_ <: T]*): T`
* :meth:`expectMsgAnyClassOf[T](d: Duration, obj: Class[_ <: T]*): T`
An object must be received within the given time, and it must be an
instance of at least one of the supplied :class:`Class` objects; the
received object will be returned.
* :meth:`expectMsgAllOf[T](d: Duration, obj: T*): Seq[T]`
* :meth:`expectMsgAllOf[T](d: Duration, obj: T*): Seq[T]`
A number of objects matching the size of the supplied object array must be
received within the given time, and for each of the given objects there
must exist at least one among the received ones which equals (compared with
``==``) it. The full sequence of received objects is returned.
* :meth:`expectMsgAllClassOf[T](d: Duration, c: Class[_ <: T]*): Seq[T]`
* :meth:`expectMsgAllClassOf[T](d: Duration, c: Class[_ <: T]*): Seq[T]`
A number of objects matching the size of the supplied :class:`Class` array
must be received within the given time, and for each of the given classes
@ -260,25 +248,25 @@ assertions concerning received messages. Here is the full list:
(compared with ``==``) it (this is *not* a conformance check). The full
sequence of received objects is returned.
* :meth:`expectMsgAllConformingOf[T](d: Duration, c: Class[_ <: T]*): Seq[T]`
* :meth:`expectMsgAllConformingOf[T](d: Duration, c: Class[_ <: T]*): Seq[T]`
A number of objects matching the size of the supplied :class:`Class` array
must be received within the given time, and for each of the given classes
there must exist at least one among the received objects which is an
instance of this class. The full sequence of received objects is returned.
* :meth:`expectNoMsg(d: Duration)`
* :meth:`expectNoMsg(d: Duration)`
No message must be received within the given time. This also fails if a
message has been received before calling this method which has not been
removed from the queue using one of the other methods.
* :meth:`receiveN(n: Int, d: Duration): Seq[AnyRef]`
* :meth:`receiveN(n: Int, d: Duration): Seq[AnyRef]`
``n`` messages must be received within the given time; the received
messages are returned.
* :meth:`fishForMessage(max: Duration, hint: String)(pf: PartialFunction[Any, Boolean]): Any`
* :meth:`fishForMessage(max: Duration, hint: String)(pf: PartialFunction[Any, Boolean]): Any`
Keep receiving messages as long as the time is not used up and the partial
function matches and returns ``false``. Returns the message received for
@ -288,13 +276,13 @@ assertions concerning received messages. Here is the full list:
In addition to message reception assertions there are also methods which help
with message flows:
* :meth:`receiveOne(d: Duration): AnyRef`
* :meth:`receiveOne(d: Duration): AnyRef`
Tries to receive one message for at most the given time interval and
returns ``null`` in case of failure. If the given Duration is zero, the
call is non-blocking (polling mode).
* :meth:`receiveWhile[T](max: Duration, idle: Duration, messages: Int)(pf: PartialFunction[Any, T]): Seq[T]`
* :meth:`receiveWhile[T](max: Duration, idle: Duration, messages: Int)(pf: PartialFunction[Any, T]): Seq[T]`
Collect messages as long as
@ -309,14 +297,14 @@ with message flows:
idle timeout feature). The number of expected messages defaults to
``Int.MaxValue``, which effectively disables this limit.
* :meth:`awaitCond(p: => Boolean, max: Duration, interval: Duration)`
* :meth:`awaitCond(p: => Boolean, max: Duration, interval: Duration)`
Poll the given condition every :obj:`interval` until it returns ``true`` or
the :obj:`max` duration is used up. The interval defaults to 100 ms and the
maximum defaults to the time remaining in the innermost enclosing
:ref:`within <TestKit.within>` block.
* :meth:`ignoreMsg(pf: PartialFunction[AnyRef, Boolean])`
* :meth:`ignoreMsg(pf: PartialFunction[AnyRef, Boolean])`
:meth:`ignoreNoMsg`
@ -329,8 +317,8 @@ with message flows:
This feature is useful e.g. when testing a logging system, where you want
to ignore regular messages and are only interested in your specific ones.
Expecting Exceptions
--------------------
Expecting Log Messages
----------------------
Since an integration test does not allow to the internal processing of the
participating actors, verifying expected exceptions cannot be done directly.
@ -341,6 +329,20 @@ exceptions:
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#event-filter
If a number of occurrences is specific—as demonstrated above—then ``intercept``
will block until that number of matching messages have been received or the
timeout configured in ``akka.test.filter-leeway`` is used up (time starts
counting after the passed-in block of code returns). In case of a timeout the
test fails.
.. note::
Be sure to exchange the default event handler with the
:class:`TestEventListener` in your ``application.conf`` to enable this
function::
akka.event-handlers = [akka.testkit.TestEventListener]
.. _TestKit.within:
Timing Assertions
@ -363,7 +365,7 @@ The block given to :meth:`within` must complete after a :ref:`Duration` which
is between :obj:`min` and :obj:`max`, where the former defaults to zero. The
deadline calculated by adding the :obj:`max` parameter to the block's start
time is implicitly available within the block to all examination methods, if
you do not specify it, is is inherited from the innermost enclosing
you do not specify it, it is inherited from the innermost enclosing
:meth:`within` block.
It should be noted that if the last message-receiving assertion of the block is
@ -473,8 +475,9 @@ B``, as long as a certain protocol is obeyed.
.. includecode:: ../../akka-testkit/src/test/scala/akka/testkit/TestProbeSpec.scala#autopilot
The :meth:`run` method must return the auto-pilot for the next message, wrapped
in an :class:`Option`; setting it to :obj:`None` terminates the auto-pilot.
The :meth:`run` method must return the auto-pilot for the next message, which
may be :class:`KeepRunning` to retain the current one or :class:`NoAutoPilot`
to switch it off.
Caution about Timing Assertions
^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^^
@ -483,23 +486,13 @@ The behavior of :meth:`within` blocks when using test probes might be perceived
as counter-intuitive: you need to remember that the nicely scoped deadline as
described :ref:`above <TestKit.within>` is local to each probe. Hence, probes
do not react to each other's deadlines or to the deadline set in an enclosing
:class:`TestKit` instance::
:class:`TestKit` instance:
class SomeTest extends TestKit(_system: ActorSystem) with ImplicitSender {
.. includecode:: code/docs/testkit/TestkitDocSpec.scala#test-within-probe
val probe = TestProbe()
Here, the ``expectMsg`` call will use the default timeout.
within(100 millis) {
probe.expectMsg("hallo") // Will hang forever!
}
}
This test will hang indefinitely, because the :meth:`expectMsg` call does not
see any deadline. Currently, the only option is to use ``probe.within`` in the
above code to make it work; later versions may include lexically scoped
deadlines using implicit arguments.
.. _TestCallingThreadDispatcherRef:
.. _Scala-CallingThreadDispatcher:
CallingThreadDispatcher
=======================
@ -598,7 +591,7 @@ has to offer:
exception stack traces
- Exclusion of certain classes of dead-lock scenarios
.. _actor.logging:
.. _actor.logging-scala:
Tracing Actor Invocations
=========================

View file

@ -0,0 +1,329 @@
/**
* Copyright (C) 2009-2012 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.testkit;
import scala.runtime.AbstractFunction0;
import akka.actor.ActorRef;
import akka.actor.ActorSystem;
import akka.event.Logging;
import akka.event.Logging.LogEvent;
import akka.japi.PurePartialFunction;
import akka.japi.CachingPartialFunction;
import akka.japi.Util;
import akka.util.Duration;
/**
* Java API for the TestProbe. Proper JavaDocs to come once JavaDoccing is implemented.
*/
public class JavaTestKit {
private final TestProbe p;
public JavaTestKit(ActorSystem system) {
p = new TestProbe(system);
}
public ActorRef getRef() {
return p.ref();
}
public ActorSystem getSystem() {
return p.system();
}
static public Duration duration(String s) {
return Duration.parse(s);
}
public Duration dilated(Duration d) {
return d.mul(TestKitExtension.get(p.system()).TestTimeFactor());
}
public boolean msgAvailable() {
return p.msgAvailable();
}
public ActorRef getLastSender() {
return p.lastMessage().sender();
}
public void send(ActorRef actor, Object msg) {
actor.tell(msg, p.ref());
}
public void forward(ActorRef actor) {
actor.tell(p.lastMessage().msg(), p.lastMessage().sender());
}
public void reply(Object msg) {
p.lastMessage().sender().tell(msg, p.ref());
}
public Duration getRemainingTime() {
return p.remaining();
}
public Duration getRemainingTimeOr(Duration def) {
return p.remainingOr(def);
}
public ActorRef watch(ActorRef ref) {
return p.watch(ref);
}
public ActorRef unwatch(ActorRef ref) {
return p.unwatch(ref);
}
public abstract class IgnoreMsg {
abstract protected boolean ignore(Object msg);
public IgnoreMsg() {
p.ignoreMsg(new PurePartialFunction<Object, Object>() {
public Boolean apply(Object in, boolean isCheck) {
return ignore(in);
}
});
}
}
public void ignoreNoMsg() {
p.ignoreNoMsg();
}
public void setAutoPilot(TestActor.AutoPilot pilot) {
p.setAutoPilot(pilot);
}
public abstract class Within {
protected abstract void run();
public Within(Duration max) {
p.within(max, new AbstractFunction0<Object>() {
public Object apply() {
run();
return null;
}
});
}
public Within(Duration min, Duration max) {
p.within(min, max, new AbstractFunction0<Object>() {
public Object apply() {
run();
return null;
}
});
}
}
public abstract class AwaitCond {
protected abstract boolean cond();
public AwaitCond() {
this(Duration.Undefined(), p.awaitCond$default$3());
}
public AwaitCond(Duration max) {
this(max, p.awaitCond$default$3());
}
public AwaitCond(Duration max, Duration interval) {
p.awaitCond(new AbstractFunction0<Object>() {
public Object apply() {
return cond();
}
}, max, interval);
}
}
public abstract class ExpectMsg<T> {
private final T result;
public ExpectMsg(String hint) {
this(Duration.Undefined(), hint);
}
public ExpectMsg(Duration max, String hint) {
final Object received = p.receiveOne(max);
try {
result = match(received);
} catch (PurePartialFunction.NoMatchException ex) {
throw new AssertionError("while expecting '" + hint
+ "' received unexpected: " + received);
}
}
abstract protected T match(Object msg);
protected RuntimeException noMatch() {
throw PurePartialFunction.noMatch();
}
public T get() {
return result;
}
}
public <T> T expectMsgEquals(T msg) {
return p.expectMsg(msg);
}
public <T> T expectMsgEquals(Duration max, T msg) {
return p.expectMsg(max, msg);
}
public <T> T expectMsgClass(Class<T> clazz) {
return p.expectMsgClass(clazz);
}
public <T> T expectMsgClass(Duration max, Class<T> clazz) {
return p.expectMsgClass(max, clazz);
}
public Object expectMsgAnyOf(Object... msgs) {
return p.expectMsgAnyOf(Util.arrayToSeq(msgs));
}
public Object expectMsgAnyOf(Duration max, Object... msgs) {
return p.expectMsgAnyOf(max, Util.arrayToSeq(msgs));
}
public Object[] expectMsgAllOf(Object... msgs) {
return (Object[]) p.expectMsgAllOf(Util.arrayToSeq(msgs)).toArray(
Util.manifest(Object.class));
}
public Object[] expectMsgAllOf(Duration max, Object... msgs) {
return (Object[]) p.expectMsgAllOf(max, Util.arrayToSeq(msgs)).toArray(
Util.manifest(Object.class));
}
@SuppressWarnings("unchecked")
public <T> T expectMsgAnyClassOf(Class<? extends T>... classes) {
final Object result = p.expectMsgAnyClassOf(Util.arrayToSeq(classes));
return (T) result;
}
public Object expectMsgAnyClassOf(Duration max, Class<?>... classes) {
return p.expectMsgAnyClassOf(max, Util.arrayToSeq(classes));
}
public void expectNoMsg() {
p.expectNoMsg();
}
public void expectNoMsg(Duration max) {
p.expectNoMsg(max);
}
public abstract class ReceiveWhile<T> {
abstract protected T match(Object msg);
private Object results;
public ReceiveWhile(Class<T> clazz) {
this(clazz, Duration.Undefined());
}
public ReceiveWhile(Class<T> clazz, Duration max) {
this(clazz, max, Duration.Inf(), Integer.MAX_VALUE);
}
public ReceiveWhile(Class<T> clazz, Duration max, int messages) {
this(clazz, max, Duration.Inf(), messages);
}
@SuppressWarnings("unchecked")
public ReceiveWhile(Class<T> clazz, Duration max, Duration idle, int messages) {
results = p.receiveWhile(max, idle, messages,
new CachingPartialFunction<Object, T>() {
public T match(Object msg) {
return ReceiveWhile.this.match(msg);
}
}).toArray(Util.manifest(clazz));
}
protected RuntimeException noMatch() {
throw PurePartialFunction.noMatch();
}
@SuppressWarnings("unchecked")
public T[] get() {
return (T[]) results;
}
}
public abstract class EventFilter<T> {
abstract protected T run();
private final Class<? extends Logging.LogEvent> clazz;
private String source = null;
private String message = null;
private boolean pattern = false;
private boolean complete = false;
private int occurrences = Integer.MAX_VALUE;
private Class<? extends Throwable> exceptionType = null;
@SuppressWarnings("unchecked")
public EventFilter(Class<?> clazz) {
if (Throwable.class.isAssignableFrom(clazz)) {
this.clazz = Logging.Error.class;
exceptionType = (Class<? extends Throwable>) clazz;
} else if (Logging.LogEvent.class.isAssignableFrom(clazz)) {
this.clazz = (Class<? extends LogEvent>) clazz;
} else throw new IllegalArgumentException("supplied class must either be LogEvent or Throwable");
}
public T exec() {
akka.testkit.EventFilter filter;
if (clazz == Logging.Error.class) {
if (exceptionType == null) exceptionType = Logging.noCause().getClass();
filter = new ErrorFilter(exceptionType, source, message, pattern, complete, occurrences);
} else if (clazz == Logging.Warning.class) {
filter = new WarningFilter(source, message, pattern, complete, occurrences);
} else if (clazz == Logging.Info.class) {
filter = new InfoFilter(source, message, pattern, complete, occurrences);
} else if (clazz == Logging.Debug.class) {
filter = new DebugFilter(source, message, pattern, complete, occurrences);
} else throw new IllegalArgumentException("unknown LogLevel " + clazz);
return filter.intercept(new AbstractFunction0<T>() {
public T apply() {
return run();
}
}, p.system());
}
public EventFilter<T> message(String msg) {
message = msg;
pattern = false;
complete = true;
return this;
}
public EventFilter<T> startsWith(String msg) {
message = msg;
pattern = false;
complete = false;
return this;
}
public EventFilter<T> matches(String regex) {
message = regex;
pattern = true;
return this;
}
public EventFilter<T> from(String source) {
this.source = source;
return this;
}
public EventFilter<T> occurrences(int number) {
occurrences = number;
return this;
}
}
}

View file

@ -132,4 +132,9 @@ object TestActorRef {
"\nOR try to change: 'actorOf(Props[MyActor]' to 'actorOf(Props(new MyActor)'.", exception)
}
}), name)
/**
* Java API
*/
def create[T <: Actor](system: ActorSystem, props: Props, name: String): TestActorRef[T] = apply(props, name)(system)
}

View file

@ -13,12 +13,24 @@ import scala.annotation.tailrec
import akka.actor.ActorSystem
import akka.util.Timeout
import akka.util.BoxedType
import scala.annotation.varargs
import akka.japi.PurePartialFunction
object TestActor {
type Ignore = Option[PartialFunction[AnyRef, Boolean]]
trait AutoPilot {
def run(sender: ActorRef, msg: Any): Option[AutoPilot]
abstract class AutoPilot {
def run(sender: ActorRef, msg: Any): AutoPilot
def noAutoPilot: AutoPilot = NoAutoPilot
def keepRunning: AutoPilot = KeepRunning
}
case object NoAutoPilot extends AutoPilot {
def run(sender: ActorRef, msg: Any): AutoPilot = this
}
case object KeepRunning extends AutoPilot {
def run(sender: ActorRef, msg: Any): AutoPilot = sys.error("must not call")
}
case class SetIgnore(i: Ignore)
@ -42,15 +54,18 @@ class TestActor(queue: BlockingDeque[TestActor.Message]) extends Actor {
var ignore: Ignore = None
var autopilot: Option[AutoPilot] = None
var autopilot: AutoPilot = NoAutoPilot
def receive = {
case SetIgnore(ign) ignore = ign
case x @ Watch(ref) context.watch(ref); queue.offerLast(RealMessage(x, self))
case x @ UnWatch(ref) context.unwatch(ref); queue.offerLast(RealMessage(x, self))
case SetAutoPilot(pilot) autopilot = Some(pilot)
case SetAutoPilot(pilot) autopilot = pilot
case x: AnyRef
autopilot = autopilot.flatMap(_.run(sender, x))
autopilot = autopilot.run(sender, x) match {
case KeepRunning autopilot
case other other
}
val observe = ignore map (ignoreFunc if (ignoreFunc isDefinedAt x) !ignoreFunc(x) else true) getOrElse true
if (observe) queue.offerLast(RealMessage(x, sender))
}
@ -130,20 +145,20 @@ trait TestKitBase {
* Have the testActor watch someone (i.e. `context.watch(...)`). Waits until
* the Watch message is received back using expectMsg.
*/
def watch(ref: ActorRef) {
def watch(ref: ActorRef): ActorRef = {
val msg = TestActor.Watch(ref)
testActor ! msg
expectMsg(msg)
expectMsg(msg).ref
}
/**
* Have the testActor stop watching someone (i.e. `context.unwatch(...)`). Waits until
* the Watch message is received back using expectMsg.
*/
def unwatch(ref: ActorRef) {
def unwatch(ref: ActorRef): ActorRef = {
val msg = TestActor.UnWatch(ref)
testActor ! msg
expectMsg(msg)
expectMsg(msg).ref
}
/**

View file

@ -4,13 +4,14 @@
package akka.testkit;
import org.junit.Test;
import akka.actor.Actor;
import akka.actor.Props;
public class TestActorRefJavaCompile {
public void shouldBeAbleToCompileWhenUsingApply() {
//Just a dummy call to make sure it compiles
TestActorRef ref = TestActorRef.apply(new Props(), null);
TestActorRef<Actor> ref = TestActorRef.apply(new Props(), null);
ref.toString();
}
}

View file

@ -44,10 +44,10 @@ class TestProbeSpec extends AkkaSpec with DefaultTimeout {
//#autopilot
val probe = TestProbe()
probe.setAutoPilot(new TestActor.AutoPilot {
def run(sender: ActorRef, msg: Any): Option[TestActor.AutoPilot] =
def run(sender: ActorRef, msg: Any): TestActor.AutoPilot =
msg match {
case "stop" None
case x testActor.tell(x, sender); Some(this)
case "stop" TestActor.NoAutoPilot
case x testActor.tell(x, sender); TestActor.KeepRunning
}
})
//#autopilot

View file

@ -320,7 +320,8 @@ object AkkaBuild extends Build {
settings = defaultSettings ++ Sphinx.settings ++ Seq(
unmanagedSourceDirectories in Test <<= baseDirectory { _ ** "code" get },
libraryDependencies ++= Dependencies.docs,
unmanagedSourceDirectories in ScalariformKeys.format in Test <<= unmanagedSourceDirectories in Test
unmanagedSourceDirectories in ScalariformKeys.format in Test <<= unmanagedSourceDirectories in Test,
testOptions += Tests.Argument(TestFrameworks.JUnit, "-v")
)
)
@ -511,7 +512,7 @@ object Dependencies {
val tutorials = Seq(Test.scalatest, Test.junit)
val docs = Seq(Test.scalatest, Test.junit, Test.specs2)
val docs = Seq(Test.scalatest, Test.junit, Test.specs2, Test.junitIntf)
val zeroMQ = Seq(protobuf, Dependency.zeroMQ, Test.scalatest, Test.junit)
}
@ -559,6 +560,7 @@ object Dependency {
val specs2 = "org.specs2" % "specs2_2.9.1" % "1.9" % "test" // Modified BSD / ApacheV2
val tinybundles = "org.ops4j.pax.tinybundles" % "tinybundles" % "1.0.0" % "test" // ApacheV2
val log4j = "log4j" % "log4j" % "1.2.14" % "test" // ApacheV2
val junitIntf = "com.novocode" % "junit-interface" % "0.8" % "test" // MIT
}
}