Merge pull request #1395 from akka/wip-3217-remotedeathwatchspec-failure-ban

Wait on shutdown of extra actor systems in tests. #3217
This commit is contained in:
Björn Antonsson 2013-05-07 02:03:05 -07:00
commit ddade2c59e
84 changed files with 762 additions and 845 deletions

View file

@ -4,35 +4,27 @@
package akka.actor; package akka.actor;
import akka.actor.ActorSystem;
import akka.event.Logging; import akka.event.Logging;
import akka.event.Logging.LoggerInitialized; import akka.event.Logging.LoggerInitialized;
import akka.japi.Creator; import akka.japi.Creator;
import akka.routing.CurrentRoutees; import akka.routing.CurrentRoutees;
import akka.routing.FromConfig; import akka.routing.FromConfig;
import akka.routing.NoRouter; import akka.routing.NoRouter;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import org.junit.AfterClass; import org.junit.ClassRule;
import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.*; import static org.junit.Assert.*;
public class JavaAPI { public class JavaAPI {
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("JAvaAPI", AkkaSpec.testConf());
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void beforeAll() {
system = ActorSystem.create("JavaAPI", AkkaSpec.testConf());
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
// compilation tests // compilation tests
@SuppressWarnings("unused") @SuppressWarnings("unused")
public void mustCompile() { public void mustCompile() {

View file

@ -3,14 +3,10 @@
*/ */
package akka.actor; package akka.actor;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.*;
import org.junit.Test;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
import com.typesafe.config.Config;
import static org.junit.Assert.*; import static org.junit.Assert.*;
@ -47,20 +43,13 @@ public class JavaExtension {
} }
} }
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("JavaExtension",
ConfigFactory.parseString("akka.extensions = [ \"akka.actor.JavaExtension$TestExtensionId\" ]")
.withFallback(AkkaSpec.testConf()));
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void beforeAll() {
Config c = ConfigFactory.parseString("akka.extensions = [ \"akka.actor.JavaExtension$TestExtensionId\" ]")
.withFallback(AkkaSpec.testConf());
system = ActorSystem.create("JavaExtension", c);
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
@Test @Test
public void mustBeAccessible() { public void mustBeAccessible() {

View file

@ -1,26 +1,18 @@
package akka.actor; package akka.actor;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
public class StashJavaAPI { public class StashJavaAPI {
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("StashJavaAPI", ActorWithBoundedStashSpec.testConf());
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void beforeAll() {
system = ActorSystem.create("StashJavaAPI",
ConfigFactory.parseString(ActorWithStashSpec.testConf()));
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
@Test @Test
public void mustBeAbleToUseStash() { public void mustBeAbleToUseStash() {

View file

@ -1,16 +1,14 @@
package akka.dispatch; package akka.dispatch;
import akka.util.Timeout; import akka.testkit.AkkaJUnitActorSystemResource;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.japi.*; import akka.japi.*;
import org.junit.ClassRule;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.Future; import scala.concurrent.Future;
import scala.concurrent.Promise; import scala.concurrent.Promise;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
import akka.testkit.TestKitExtension;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.*; import static org.junit.Assert.*;
@ -26,23 +24,13 @@ import akka.testkit.AkkaSpec;
public class JavaFutureTests { public class JavaFutureTests {
private static ActorSystem system; @ClassRule
private static Timeout t; public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("JavaFutureTests", AkkaSpec.testConf());
private final ActorSystem system = actorSystemResource.getSystem();
private final Duration timeout = Duration.create(5, TimeUnit.SECONDS); private final Duration timeout = Duration.create(5, TimeUnit.SECONDS);
@BeforeClass
public static void beforeAll() {
system = ActorSystem.create("JavaFutureTests", AkkaSpec.testConf());
t = TestKitExtension.get(system).DefaultTimeout();
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
@Test @Test
public void mustBeAbleToMapAFuture() throws Exception { public void mustBeAbleToMapAFuture() throws Exception {

View file

@ -147,7 +147,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
} }
"allow valid names" in { "allow valid names" in {
ActorSystem("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789-").shutdown() shutdown(ActorSystem("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789-"))
} }
"support extensions" in { "support extensions" in {
@ -199,14 +199,14 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
val system = ActorSystem() val system = ActorSystem()
system.isTerminated must be(false) system.isTerminated must be(false)
system.shutdown() system.shutdown()
system.awaitTermination() system.awaitTermination(10 seconds)
system.isTerminated must be(true) system.isTerminated must be(true)
} }
"throw RejectedExecutionException when shutdown" in { "throw RejectedExecutionException when shutdown" in {
val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf) val system2 = ActorSystem("AwaitTermination", AkkaSpec.testConf)
system2.shutdown() system2.shutdown()
system2.awaitTermination(5 seconds) system2.awaitTermination(10 seconds)
intercept[RejectedExecutionException] { intercept[RejectedExecutionException] {
system2.registerOnTermination { println("IF YOU SEE THIS THEN THERE'S A BUG HERE") } system2.registerOnTermination { println("IF YOU SEE THIS THEN THERE'S A BUG HERE") }
@ -275,7 +275,7 @@ class ActorSystemSpec extends AkkaSpec(ActorSystemSpec.config) with ImplicitSend
val t = probe.expectMsg(Terminated(a)(existenceConfirmed = true, addressTerminated = false)) val t = probe.expectMsg(Terminated(a)(existenceConfirmed = true, addressTerminated = false))
t.existenceConfirmed must be(true) t.existenceConfirmed must be(true)
t.addressTerminated must be(false) t.addressTerminated must be(false)
system.shutdown() shutdown(system)
} }
"shut down when /user escalates" in { "shut down when /user escalates" in {

View file

@ -132,7 +132,7 @@ class DeployerSpec extends AkkaSpec(DeployerSpec.deployerConf) {
} }
""", ConfigParseOptions.defaults).withFallback(AkkaSpec.testConf) """, ConfigParseOptions.defaults).withFallback(AkkaSpec.testConf)
ActorSystem("invalid", invalidDeployerConf).shutdown() shutdown(ActorSystem("invalid", invalidDeployerConf))
} }
} }

View file

@ -295,7 +295,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
} }
} }
} finally { } finally {
fsmEventSystem.shutdown() TestKit.shutdownActorSystem(fsmEventSystem)
} }
} }

View file

@ -104,7 +104,7 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
expectMsgAllOf(m, Logging.Debug(sys.deadLetters.path.toString, sys.deadLetters.getClass, "unhandled message from " + sys.deadLetters + ": 42")) expectMsgAllOf(m, Logging.Debug(sys.deadLetters.path.toString, sys.deadLetters.getClass, "unhandled message from " + sys.deadLetters + ": 42"))
sys.eventStream.unsubscribe(testActor) sys.eventStream.unsubscribe(testActor)
} finally { } finally {
sys.shutdown() shutdown(sys)
} }
} }

View file

@ -99,8 +99,7 @@ class LoggerSpec extends WordSpec with MustMatchers {
probe.expectNoMsg(0.5.seconds.dilated) probe.expectNoMsg(0.5.seconds.dilated)
} }
} finally { } finally {
system.shutdown() TestKit.shutdownActorSystem(system)
system.awaitTermination(5.seconds.dilated)
} }
} }
out out
@ -139,8 +138,7 @@ class LoggerSpec extends WordSpec with MustMatchers {
probe1.expectMsg("log it") probe1.expectMsg("log it")
probe2.expectMsg("log it") probe2.expectMsg("log it")
} finally { } finally {
system.shutdown() TestKit.shutdownActorSystem(system)
system.awaitTermination(5.seconds.dilated)
} }
} }
} }
@ -165,8 +163,7 @@ class LoggerSpec extends WordSpec with MustMatchers {
try { try {
SerializationExtension(s).serialize(Warning("foo", classOf[String])) SerializationExtension(s).serialize(Warning("foo", classOf[String]))
} finally { } finally {
s.shutdown() TestKit.shutdownActorSystem(s)
s.awaitTermination(5.seconds.dilated)
} }
} }
} }

View file

@ -47,9 +47,9 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
} }
override def afterAll { override def afterAll {
appLogging.shutdown() TestKit.shutdownActorSystem(appLogging)
appAuto.shutdown() TestKit.shutdownActorSystem(appAuto)
appLifecycle.shutdown() TestKit.shutdownActorSystem(appLifecycle)
} }
"A LoggingReceive" must { "A LoggingReceive" must {

View file

@ -590,7 +590,7 @@ class RoutingSpec extends AkkaSpec(RoutingSpec.config) with DefaultTimeout with
try { try {
sys.actorOf(Props.empty.withRouter(FromConfig), "routed") sys.actorOf(Props.empty.withRouter(FromConfig), "routed")
} finally { } finally {
sys.shutdown() shutdown(sys)
} }
} }

View file

@ -186,7 +186,7 @@ class SerializeSpec extends AkkaSpec(SerializationTests.serializeConf) {
(deadLetters eq a.deadLetters) must be(true) (deadLetters eq a.deadLetters) must be(true)
} }
} finally { } finally {
a.shutdown() shutdown(a)
} }
} }

View file

@ -4,53 +4,50 @@
package akka.camel; package akka.camel;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec;
import org.junit.ClassRule;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration; import scala.concurrent.duration.FiniteDuration;
import akka.actor.ActorRef;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.actor.Props; import akka.actor.Props;
import akka.testkit.JavaTestKit; import akka.testkit.JavaTestKit;
import akka.util.Timeout; import akka.util.Timeout;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.ExecutionContext; import scala.concurrent.ExecutionContext;
import org.junit.AfterClass;
import org.junit.Test; import org.junit.Test;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import akka.testkit.AkkaSpec;
import static org.junit.Assert.*; import static org.junit.Assert.*;
/**
*
*/
public class ConsumerJavaTestBase { public class ConsumerJavaTestBase {
static ActorSystem system = ActorSystem.create("test", AkkaSpec.testConf()); @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("ConsumerJavaTest", AkkaSpec.testConf());
@AfterClass private final ActorSystem system = actorSystemResource.getSystem();
public static void tearDownAfterClass() {
system.shutdown(); @Test
public void shouldHandleExceptionThrownByActorAndGenerateCustomResponse() throws Exception {
new JavaTestKit(system) {{
String result = new EventFilter<String>(Exception.class) {
protected String run() {
FiniteDuration duration = Duration.create(1, TimeUnit.SECONDS);
Timeout timeout = new Timeout(duration);
Camel camel = CamelExtension.get(system);
ExecutionContext executionContext = system.dispatcher();
try {
Await.result(
camel.activationFutureFor(system.actorOf(Props.create(SampleErrorHandlingConsumer.class), "sample-error-handling-consumer"), timeout, executionContext),
duration);
return camel.template().requestBody("direct:error-handler-test-java", "hello", String.class);
}
catch (Exception e) {
return e.getMessage();
}
}
}.occurrences(1).exec();
assertEquals("error: hello", result);
}};
} }
@Test
public void shouldHandleExceptionThrownByActorAndGenerateCustomResponse() throws Exception {
new JavaTestKit(system) {{
String result = new EventFilter<String>(Exception.class) {
protected String run() {
FiniteDuration duration = Duration.create(1, TimeUnit.SECONDS);
Timeout timeout = new Timeout(duration);
Camel camel = CamelExtension.get(system);
ExecutionContext executionContext = system.dispatcher();
try {
Await.result(
camel.activationFutureFor(system.actorOf(Props.create(SampleErrorHandlingConsumer.class), "sample-error-handling-consumer"), timeout, executionContext),
duration);
return camel.template().requestBody("direct:error-handler-test-java", "hello", String.class);
}
catch (Exception e) {
return e.getMessage();
}
}
}.occurrences(1).exec();
assertEquals("error: hello", result);
}};
}
} }

View file

@ -4,7 +4,9 @@ import akka.actor.*;
import akka.camel.internal.component.CamelPath; import akka.camel.internal.component.CamelPath;
import akka.camel.javaapi.UntypedConsumerActor; import akka.camel.javaapi.UntypedConsumerActor;
import akka.camel.javaapi.UntypedProducerActor; import akka.camel.javaapi.UntypedProducerActor;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.util.Timeout; import akka.util.Timeout;
import org.junit.*;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.ExecutionContext; import scala.concurrent.ExecutionContext;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
@ -14,25 +16,20 @@ import org.apache.camel.Exchange;
import org.apache.camel.Predicate; import org.apache.camel.Predicate;
import org.apache.camel.builder.RouteBuilder; import org.apache.camel.builder.RouteBuilder;
import org.apache.camel.component.mock.MockEndpoint; import org.apache.camel.component.mock.MockEndpoint;
import org.junit.Before;
import org.junit.After;
import org.junit.Test;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
public class CustomRouteTestBase { public class CustomRouteTestBase {
private static Camel camel;
private static ActorSystem system;
@Before @Rule
public void before() { public AkkaJUnitActorSystemResource actorSystemResource =
system = ActorSystem.create("test"); new AkkaJUnitActorSystemResource("CustomRouteTest");
camel = (Camel) CamelExtension.get(system);
}
@After private final ActorSystem system = actorSystemResource.getSystem();
public void after() { private Camel camel = (Camel) CamelExtension.get(system);
system.shutdown();
public static class MyActor extends UntypedActor {
@Override public void onReceive(Object o) {}
} }
@Test @Test
@ -98,7 +95,7 @@ public class CustomRouteTestBase {
camel.activationFutureFor(system.actorOf(Props.create(TestAckConsumer.class, "direct:testConsumerAckFromUri","mock:mockAckUri"), "testConsumerAckUri"), camel.activationFutureFor(system.actorOf(Props.create(TestAckConsumer.class, "direct:testConsumerAckFromUri","mock:mockAckUri"), "testConsumerAckUri"),
timeout, executionContext), timeout, executionContext),
duration); duration);
camel.context().addRoutes(new CustomRouteBuilder("direct:testAckFromUri","akka://test/user/testConsumerAckUri?autoAck=false")); camel.context().addRoutes(new CustomRouteBuilder("direct:testAckFromUri","akka://CustomRouteTest/user/testConsumerAckUri?autoAck=false"));
camel.template().sendBody("direct:testAckFromUri", "test"); camel.template().sendBody("direct:testAckFromUri", "test");
assertMockEndpoint(mockEndpoint); assertMockEndpoint(mockEndpoint);
system.stop(consumer); system.stop(consumer);

View file

@ -6,11 +6,10 @@ package akka.camel;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.dispatch.Mapper; import akka.dispatch.Mapper;
import akka.japi.Function; import akka.testkit.AkkaJUnitActorSystemResource;
import org.apache.camel.NoTypeConversionAvailableException; import org.apache.camel.NoTypeConversionAvailableException;
import org.apache.camel.converter.stream.InputStreamCache; import org.apache.camel.converter.stream.InputStreamCache;
import org.junit.AfterClass; import org.junit.ClassRule;
import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import java.io.InputStream; import java.io.InputStream;
@ -22,112 +21,106 @@ import static org.junit.Assert.assertEquals;
* *
*/ */
public class MessageJavaTestBase { public class MessageJavaTestBase {
static Camel camel; private Map<String,Object> empty = new HashMap<String, Object>();
private static ActorSystem system;
private Map<String,Object> empty = new HashMap<String, Object>();
@BeforeClass @ClassRule
public static void setUpBeforeClass() { public static AkkaJUnitActorSystemResource actorSystemResource =
system = ActorSystem.create("test"); new AkkaJUnitActorSystemResource("MessageJavaTest");
camel = (Camel) CamelExtension.get(system);
private final ActorSystem system = actorSystemResource.getSystem();
private Camel camel = (Camel) CamelExtension.get(system);
CamelMessage message(Object body){ return new CamelMessage(body, new HashMap<String, Object>()); }
CamelMessage message(Object body, Map<String, Object> headers){ return new CamelMessage(body, headers); }
@Test public void shouldConvertDoubleBodyToString() {
assertEquals("1.4", message("1.4", empty).getBodyAs(String.class,camel.context()));
}
@Test(expected=NoTypeConversionAvailableException.class)
public void shouldThrowExceptionWhenConvertingDoubleBodyToInputStream() {
message(1.4).getBodyAs(InputStream.class,camel.context());
}
@Test public void shouldConvertDoubleHeaderToString() {
CamelMessage message = message("test" , createMap("test", 1.4));
assertEquals("1.4", message.getHeaderAs("test", String.class,camel.context()));
}
@Test public void shouldReturnSubsetOfHeaders() {
CamelMessage message = message("test" , createMap("A", "1", "B", "2"));
assertEquals(createMap("B", "2"), message.getHeaders(createSet("B")));
}
@Test(expected=UnsupportedOperationException.class)
public void shouldReturnSubsetOfHeadersUnmodifiable() {
CamelMessage message = message("test" , createMap("A", "1", "B", "2"));
message.getHeaders(createSet("B")).put("x", "y");
}
@Test public void shouldReturnAllHeaders() {
CamelMessage message = message("test" , createMap("A", "1", "B", "2"));
assertEquals(createMap("A", "1", "B", "2"), message.getHeaders());
}
@Test(expected=UnsupportedOperationException.class)
public void shouldReturnAllHeadersUnmodifiable() {
CamelMessage message = message("test" , createMap("A", "1", "B", "2"));
message.getHeaders().put("x", "y");
}
@Test public void shouldTransformBodyAndPreserveHeaders() {
assertEquals(
message("ab", createMap("A", "1")),
message("a" , createMap("A", "1")).mapBody(new TestTransformer()));
}
@Test public void shouldConvertBodyAndPreserveHeaders() {
assertEquals(
message("1.4", createMap("A", "1")),
message(1.4 , createMap("A", "1")).withBodyAs(String.class,camel.context()));
}
@Test public void shouldSetBodyAndPreserveHeaders() {
assertEquals(
message("test2" , createMap("A", "1")),
message("test1" , createMap("A", "1")).withBody("test2"));
}
@Test public void shouldSetHeadersAndPreserveBody() {
assertEquals(
message("test1" , createMap("C", "3")),
message("test1" , createMap("A", "1")).withHeaders(createMap("C", "3")));
}
@Test
public void shouldBeAbleToReReadStreamCacheBody() throws Exception {
CamelMessage msg = new CamelMessage(new InputStreamCache("test1".getBytes("utf-8")), empty);
assertEquals("test1", msg.getBodyAs(String.class, camel.context()));
// re-read
assertEquals("test1", msg.getBodyAs(String.class, camel.context()));
}
private static Set<String> createSet(String... entries) {
HashSet<String> set = new HashSet<String>();
set.addAll(Arrays.asList(entries));
return set;
}
private static Map<String, Object> createMap(Object... pairs) {
HashMap<String, Object> map = new HashMap<String, Object>();
for (int i = 0; i < pairs.length; i += 2) {
map.put((String)pairs[i], pairs[i+1]);
} }
return map;
}
@AfterClass private static class TestTransformer extends Mapper<String, String> {
public static void cleanup(){ @Override
system.shutdown(); public String apply(String param) {
} return param + "b";
CamelMessage message(Object body){ return new CamelMessage(body, new HashMap<String, Object>()); }
CamelMessage message(Object body, Map<String, Object> headers){ return new CamelMessage(body, headers); }
@Test public void shouldConvertDoubleBodyToString() {
assertEquals("1.4", message("1.4", empty).getBodyAs(String.class,camel.context()));
}
@Test(expected=NoTypeConversionAvailableException.class)
public void shouldThrowExceptionWhenConvertingDoubleBodyToInputStream() {
message(1.4).getBodyAs(InputStream.class,camel.context());
}
@Test public void shouldConvertDoubleHeaderToString() {
CamelMessage message = message("test" , createMap("test", 1.4));
assertEquals("1.4", message.getHeaderAs("test", String.class,camel.context()));
}
@Test public void shouldReturnSubsetOfHeaders() {
CamelMessage message = message("test" , createMap("A", "1", "B", "2"));
assertEquals(createMap("B", "2"), message.getHeaders(createSet("B")));
}
@Test(expected=UnsupportedOperationException.class)
public void shouldReturnSubsetOfHeadersUnmodifiable() {
CamelMessage message = message("test" , createMap("A", "1", "B", "2"));
message.getHeaders(createSet("B")).put("x", "y");
}
@Test public void shouldReturnAllHeaders() {
CamelMessage message = message("test" , createMap("A", "1", "B", "2"));
assertEquals(createMap("A", "1", "B", "2"), message.getHeaders());
}
@Test(expected=UnsupportedOperationException.class)
public void shouldReturnAllHeadersUnmodifiable() {
CamelMessage message = message("test" , createMap("A", "1", "B", "2"));
message.getHeaders().put("x", "y");
}
@Test public void shouldTransformBodyAndPreserveHeaders() {
assertEquals(
message("ab", createMap("A", "1")),
message("a" , createMap("A", "1")).mapBody(new TestTransformer()));
}
@Test public void shouldConvertBodyAndPreserveHeaders() {
assertEquals(
message("1.4", createMap("A", "1")),
message(1.4 , createMap("A", "1")).withBodyAs(String.class,camel.context()));
}
@Test public void shouldSetBodyAndPreserveHeaders() {
assertEquals(
message("test2" , createMap("A", "1")),
message("test1" , createMap("A", "1")).withBody("test2"));
}
@Test public void shouldSetHeadersAndPreserveBody() {
assertEquals(
message("test1" , createMap("C", "3")),
message("test1" , createMap("A", "1")).withHeaders(createMap("C", "3")));
}
@Test
public void shouldBeAbleToReReadStreamCacheBody() throws Exception {
CamelMessage msg = new CamelMessage(new InputStreamCache("test1".getBytes("utf-8")), empty);
assertEquals("test1", msg.getBodyAs(String.class, camel.context()));
// re-read
assertEquals("test1", msg.getBodyAs(String.class, camel.context()));
}
private static Set<String> createSet(String... entries) {
HashSet<String> set = new HashSet<String>();
set.addAll(Arrays.asList(entries));
return set;
}
private static Map<String, Object> createMap(Object... pairs) {
HashMap<String, Object> map = new HashMap<String, Object>();
for (int i = 0; i < pairs.length; i += 2) {
map.put((String)pairs[i], pairs[i+1]);
}
return map;
}
private static class TestTransformer extends Mapper<String, String> {
@Override
public String apply(String param) {
return param + "b";
}
} }
}
} }

View file

@ -8,13 +8,14 @@ import org.scalatest.WordSpec
import akka.actor.ActorSystem import akka.actor.ActorSystem
import scala.concurrent.duration.Duration import scala.concurrent.duration.Duration
import java.util.concurrent.TimeUnit._ import java.util.concurrent.TimeUnit._
import akka.testkit.TestKit
class CamelConfigSpec extends WordSpec with MustMatchers { class CamelConfigSpec extends WordSpec with MustMatchers {
val (settings, config) = { val (settings, config) = {
val system = ActorSystem("CamelConfigSpec") val system = ActorSystem("CamelConfigSpec")
val result = (CamelExtension(system).settings, system.settings.config) val result = (CamelExtension(system).settings, system.settings.config)
system.shutdown() TestKit.shutdownActorSystem(system)
result result
} }
"CamelConfigSpec" must { "CamelConfigSpec" must {

View file

@ -31,7 +31,7 @@ class ProducerFeatureTest extends TestKit(ActorSystem("test", AkkaSpec.testConf)
override protected def afterAll() { override protected def afterAll() {
super.afterAll() super.afterAll()
system.shutdown() shutdown(system)
} }
val camelContext = camel.context val camelContext = camel.context

View file

@ -15,7 +15,7 @@ import scala.reflect.ClassTag
import akka.actor.{ ActorRef, Props, ActorSystem, Actor } import akka.actor.{ ActorRef, Props, ActorSystem, Actor }
import scala.concurrent.Await import scala.concurrent.Await
import akka.util.Timeout import akka.util.Timeout
import akka.testkit.AkkaSpec import akka.testkit.{ TestKit, AkkaSpec }
private[camel] object TestSupport { private[camel] object TestSupport {
def start(actor: Actor, name: String)(implicit system: ActorSystem, timeout: Timeout): ActorRef = def start(actor: Actor, name: String)(implicit system: ActorSystem, timeout: Timeout): ActorRef =
@ -53,7 +53,7 @@ private[camel] object TestSupport {
abstract override protected def afterAll() { abstract override protected def afterAll() {
super.afterAll() super.afterAll()
system.shutdown() TestKit.shutdownActorSystem(system)
} }
} }
@ -68,7 +68,7 @@ private[camel] object TestSupport {
} }
override protected def afterEach() { override protected def afterEach() {
system.shutdown() TestKit.shutdownActorSystem(system)
super.afterEach() super.afterEach()
} }

View file

@ -11,7 +11,7 @@ import akka.camel.internal.ActivationProtocol._
class ActivationTrackerTest extends TestKit(ActorSystem("test")) with WordSpec with MustMatchers with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen { class ActivationTrackerTest extends TestKit(ActorSystem("test")) with WordSpec with MustMatchers with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen {
override protected def afterAll() { system.shutdown() } override protected def afterAll() { shutdown(system) }
var actor: TestProbe = _ var actor: TestProbe = _
var awaiting: Awaiting = _ var awaiting: Awaiting = _

View file

@ -379,7 +379,7 @@ private[camel] trait ActorProducerFixture extends MockitoSugar with BeforeAndAft
} }
override protected def afterAll() { override protected def afterAll() {
system.shutdown() shutdown(system)
} }
def msg(s: String) = CamelMessage(s, Map.empty) def msg(s: String) = CamelMessage(s, Map.empty)

View file

@ -251,8 +251,7 @@ abstract class ClusterDeathWatchSpec
endProbe.expectMsg(EndActor.EndAck) endProbe.expectMsg(EndActor.EndAck)
} finally { } finally {
endSystem.shutdown() shutdown(endSystem, 10 seconds)
endSystem.awaitTermination(10 seconds)
} }
// no barrier here, because it is not part of testConductor roles any more // no barrier here, because it is not part of testConductor roles any more

View file

@ -57,9 +57,9 @@ abstract class RestartFirstSeedNodeSpec
override def afterAll(): Unit = { override def afterAll(): Unit = {
runOn(seed1) { runOn(seed1) {
if (seed1System.isTerminated) if (seed1System.isTerminated)
restartedSeed1System.shutdown() shutdown(restartedSeed1System)
else else
seed1System.shutdown() shutdown(seed1System)
} }
super.afterAll() super.afterAll()
} }
@ -103,8 +103,7 @@ abstract class RestartFirstSeedNodeSpec
// shutdown seed1System // shutdown seed1System
runOn(seed1) { runOn(seed1) {
seed1System.shutdown() shutdown(seed1System, remaining)
seed1System.awaitTermination(remaining)
} }
runOn(seed2, seed3) { runOn(seed2, seed3) {
awaitMembersUp(2, canNotBePartOfMemberRing = Set(seedNodes.head)) awaitMembersUp(2, canNotBePartOfMemberRing = Set(seedNodes.head))

View file

@ -961,7 +961,7 @@ abstract class StressSpec
reportResult { reportResult {
val nextAS = val nextAS =
if (activeRoles contains myself) { if (activeRoles contains myself) {
previousAS foreach { _.shutdown() } previousAS foreach { as TestKit.shutdownActorSystem(as) }
val sys = ActorSystem(system.name, system.settings.config) val sys = ActorSystem(system.name, system.settings.config)
muteLog(sys) muteLog(sys)
Cluster(sys).joinSeedNodes(seedNodes.toIndexedSeq map address) Cluster(sys).joinSeedNodes(seedNodes.toIndexedSeq map address)
@ -990,7 +990,7 @@ abstract class StressSpec
} }
} }
loop(1, None, Set.empty) foreach { _.shutdown } loop(1, None, Set.empty) foreach { as TestKit.shutdownActorSystem(as) }
within(loopDuration) { within(loopDuration) {
runOn(usedRoles: _*) { runOn(usedRoles: _*) {
awaitMembersUp(nbrUsedRoles, timeout = remaining) awaitMembersUp(nbrUsedRoles, timeout = remaining)

View file

@ -185,8 +185,7 @@ abstract class UnreachableNodeJoinsAgainSpec
endProbe.expectMsg(EndActor.EndAck) endProbe.expectMsg(EndActor.EndAck)
} finally { } finally {
freshSystem.shutdown() shutdown(freshSystem)
freshSystem.awaitTermination(10 seconds)
} }
// no barrier here, because it is not part of testConductor roles any more // no barrier here, because it is not part of testConductor roles any more
} }

View file

@ -4,8 +4,8 @@
package akka.contrib.pattern; package akka.contrib.pattern;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import akka.actor.ActorRef; import akka.actor.ActorRef;
@ -17,17 +17,12 @@ import akka.event.LoggingAdapter;
public class DistributedPubSubMediatorTest { public class DistributedPubSubMediatorTest {
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("DistributedPubSubMediatorTest");
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void setup() {
system = ActorSystem.create();
}
@AfterClass
public static void teardown() {
system.shutdown();
}
@Test @Test
public void demonstrateUsage() { public void demonstrateUsage() {

View file

@ -4,8 +4,8 @@
package akka.contrib.pattern; package akka.contrib.pattern;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
@ -25,17 +25,11 @@ import akka.contrib.pattern.ReliableProxy;
public class ReliableProxyTest { public class ReliableProxyTest {
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("ReliableProxyTest");
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void setup() {
system = ActorSystem.create();
}
@AfterClass
public static void teardown() {
system.shutdown();
}
@Test @Test
public void demonstrateUsage() { public void demonstrateUsage() {

View file

@ -32,7 +32,7 @@ class TimerBasedThrottlerSpec extends TestKit(ActorSystem("TimerBasedThrottlerSp
with WordSpec with MustMatchers with BeforeAndAfterAll { with WordSpec with MustMatchers with BeforeAndAfterAll {
override def afterAll { override def afterAll {
system.shutdown() shutdown(system)
} }
"A throttler" must { "A throttler" must {

View file

@ -5,6 +5,7 @@ package docs.config
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.matchers.MustMatchers import org.scalatest.matchers.MustMatchers
import akka.testkit.TestKit
//#imports //#imports
import akka.actor.ActorSystem import akka.actor.ActorSystem
@ -28,6 +29,6 @@ class ConfigDocSpec extends WordSpec with MustMatchers {
val system = ActorSystem("MySystem", ConfigFactory.load(customConf)) val system = ActorSystem("MySystem", ConfigFactory.load(customConf))
//#custom-config //#custom-config
system.shutdown() TestKit.shutdownActorSystem(system)
} }
} }

View file

@ -17,6 +17,7 @@ import akka.actor.UntypedActor;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.actor.Props; import akka.actor.Props;
import akka.testkit.JavaTestKit;
import akka.testkit.TestProbe; import akka.testkit.TestProbe;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
@ -201,7 +202,7 @@ public class FSMDocTestBase {
@org.junit.After @org.junit.After
public void cleanup() { public void cleanup() {
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
} }

View file

@ -26,6 +26,7 @@ import akka.testkit.TestProbe;
import akka.testkit.ErrorFilter; import akka.testkit.ErrorFilter;
import akka.testkit.EventFilter; import akka.testkit.EventFilter;
import akka.testkit.TestEvent; import akka.testkit.TestEvent;
import akka.testkit.JavaTestKit;
import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.concurrent.TimeUnit.SECONDS;
import static akka.japi.Util.immutableSeq; import static akka.japi.Util.immutableSeq;
import akka.japi.Function; import akka.japi.Function;
@ -155,7 +156,8 @@ public class FaultHandlingTestBase {
@AfterClass @AfterClass
public static void cleanup() { public static void cleanup() {
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
system = null;
} }
@Test @Test

View file

@ -6,8 +6,8 @@ package docs.actor;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
@ -21,19 +21,12 @@ import akka.testkit.JavaTestKit;
public class InboxDocTest { public class InboxDocTest {
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("InboxDocTest", AkkaSpec.testConf());
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void beforeAll() {
system = ActorSystem.create("MySystem", AkkaSpec.testConf());
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
@Test @Test
public void demonstrateInbox() { public void demonstrateInbox() {
final JavaTestKit probe = new JavaTestKit(system); final JavaTestKit probe = new JavaTestKit(system);

View file

@ -1,96 +1,86 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package docs.actor; package docs.actor;
import akka.actor.*; import akka.actor.*;
import akka.japi.Procedure; import akka.japi.Procedure;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.JavaTestKit; import akka.testkit.JavaTestKit;
import org.junit.AfterClass; import org.junit.ClassRule;
import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import scala.Option; import scala.Option;
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
public class InitializationDocSpecJava { public class InitializationDocSpecJava {
static public class PreStartInitExample extends UntypedActor { static public class PreStartInitExample extends UntypedActor {
public void onReceive(Object message) throws Exception {} public void onReceive(Object message) throws Exception {}
//#preStartInit
@Override
public void preStart() {
// Initialize children here
}
// Overriding postRestart to disable the call to preStart()
// after restarts
@Override
public void postRestart(Throwable reason) {
}
// The default implementation of preRestart() stops all the children
// of the actor. To opt-out from stopping the children, we
// have to override preRestart()
@Override
public void preRestart(Throwable reason, Option<Object> message)
throws Exception {
// Keep the call to postStop(), but no stopping of children
postStop();
}
//#preStartInit
//#preStartInit
@Override
public void preStart() {
// Initialize children here
} }
public static class MessageInitExample extends UntypedActor { // Overriding postRestart to disable the call to preStart()
//#messageInit // after restarts
private String initializeMe = null; @Override
public void postRestart(Throwable reason) {
@Override
public void onReceive(Object message) throws Exception {
if (message.equals("init")) {
initializeMe = "Up and running";
getContext().become(new Procedure<Object>() {
@Override
public void apply(Object message) throws Exception {
if (message.equals("U OK?"))
getSender().tell(initializeMe, getSelf());
}
});
}
}
//#messageInit
} }
static ActorSystem system; // The default implementation of preRestart() stops all the children
// of the actor. To opt-out from stopping the children, we
@BeforeClass // have to override preRestart()
public static void setup() { @Override
system = ActorSystem.create("TestSystem"); public void preRestart(Throwable reason, Option<Object> message)
throws Exception {
// Keep the call to postStop(), but no stopping of children
postStop();
} }
//#preStartInit
@AfterClass }
public static void teardown() {
system.shutdown(); public static class MessageInitExample extends UntypedActor {
//#messageInit
private String initializeMe = null;
@Override
public void onReceive(Object message) throws Exception {
if (message.equals("init")) {
initializeMe = "Up and running";
getContext().become(new Procedure<Object>() {
@Override
public void apply(Object message) throws Exception {
if (message.equals("U OK?"))
getSender().tell(initializeMe, getSelf());
}
});
}
} }
//#messageInit
}
@Test @ClassRule
public void testIt() { public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("InitializationDocSpecJava");
new JavaTestKit(system) {{ private final ActorSystem system = actorSystemResource.getSystem();
ActorRef testactor = system.actorOf(Props.create(MessageInitExample.class), "testactor");
String probe = "U OK?";
testactor.tell(probe, getRef()); @Test
expectNoMsg(); public void testIt() {
testactor.tell("init", getRef()); new JavaTestKit(system) {{
testactor.tell(probe, getRef()); ActorRef testactor = system.actorOf(Props.create(MessageInitExample.class), "testactor");
expectMsgEquals("Up and running"); String probe = "U OK?";
}};
testactor.tell(probe, getRef());
expectNoMsg();
testactor.tell("init", getRef());
} testactor.tell(probe, getRef());
expectMsgEquals("Up and running");
}};
}
} }

View file

@ -17,26 +17,17 @@ import akka.actor.Cancellable;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.After; import org.junit.*;
import org.junit.Before;
import org.junit.Test;
public class SchedulerDocTestBase { public class SchedulerDocTestBase {
ActorSystem system; @Rule
ActorRef testActor; public AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("SchedulerDocTest", AkkaSpec.testConf());
@Before private final ActorSystem system = actorSystemResource.getSystem();
public void setUp() { private ActorRef testActor = system.actorOf(Props.create(MyUntypedActor.class));
system = ActorSystem.create("MySystem", AkkaSpec.testConf());
testActor = system.actorOf(Props.create(MyUntypedActor.class));
}
@After
public void tearDown() {
system.shutdown();
}
@Test @Test
public void scheduleOneOffTask() { public void scheduleOneOffTask() {

View file

@ -15,8 +15,8 @@ import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
//#import-gracefulStop //#import-gracefulStop
@ -82,18 +82,11 @@ import akka.util.Timeout;
public class UntypedActorDocTestBase { public class UntypedActorDocTestBase {
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("UntypedActorDocTest", AkkaSpec.testConf());
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void beforeAll() {
system = ActorSystem.create("MySystem", AkkaSpec.testConf());
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
@SuppressWarnings("unused") @SuppressWarnings("unused")
@Test @Test
@ -156,7 +149,7 @@ public class UntypedActorDocTestBase {
} }
}; };
} finally { } finally {
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
} }

View file

@ -9,8 +9,8 @@ import akka.actor.ActorRef;
//#imports //#imports
import org.junit.After; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.Before; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
@ -20,18 +20,12 @@ import akka.actor.UntypedActor;
public class DurableMailboxDocTestBase { public class DurableMailboxDocTestBase {
ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("DurableMailboxDocTest",
ConfigFactory.parseString(DurableMailboxDocSpec.config()).withFallback(AkkaSpec.testConf()));
@Before private final ActorSystem system = actorSystemResource.getSystem();
public void setUp() {
system = ActorSystem.create("MySystem",
ConfigFactory.parseString(DurableMailboxDocSpec.config()).withFallback(AkkaSpec.testConf()));
}
@After
public void tearDown() {
system.shutdown();
}
@Test @Test
public void configDefinedDispatcher() { public void configDefinedDispatcher() {

View file

@ -5,10 +5,7 @@ package docs.agent;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import akka.testkit.AkkaSpec;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;

View file

@ -6,6 +6,8 @@ package docs.camel;
import akka.camel.Camel; import akka.camel.Camel;
import akka.camel.CamelExtension; import akka.camel.CamelExtension;
import akka.camel.javaapi.UntypedConsumerActor; import akka.camel.javaapi.UntypedConsumerActor;
import akka.testkit.JavaTestKit;
import akka.testkit.TestKit;
import akka.util.Timeout; import akka.util.Timeout;
import scala.concurrent.Future; import scala.concurrent.Future;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
@ -38,7 +40,7 @@ public class ActivationTestBase {
Future<ActorRef> deactivationFuture = camel.deactivationFutureFor(producer, Future<ActorRef> deactivationFuture = camel.deactivationFutureFor(producer,
timeout, system.dispatcher()); timeout, system.dispatcher());
//#CamelDeactivation //#CamelDeactivation
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
public static class MyConsumer extends UntypedConsumerActor { public static class MyConsumer extends UntypedConsumerActor {

View file

@ -3,6 +3,7 @@ package docs.camel;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.camel.Camel; import akka.camel.Camel;
import akka.camel.CamelExtension; import akka.camel.CamelExtension;
import akka.testkit.JavaTestKit;
import org.apache.camel.CamelContext; import org.apache.camel.CamelContext;
import org.apache.camel.ProducerTemplate; import org.apache.camel.ProducerTemplate;
import org.junit.Test; import org.junit.Test;
@ -16,7 +17,7 @@ public class CamelExtensionTestBase {
CamelContext camelContext = camel.context(); CamelContext camelContext = camel.context();
ProducerTemplate producerTemplate = camel.template(); ProducerTemplate producerTemplate = camel.template();
//#CamelExtension //#CamelExtension
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
public void addActiveMQComponent() { public void addActiveMQComponent() {
//#CamelExtensionAddComponent //#CamelExtensionAddComponent
@ -26,7 +27,7 @@ public class CamelExtensionTestBase {
// camelContext.addComponent("activemq", ActiveMQComponent.activeMQComponent( // camelContext.addComponent("activemq", ActiveMQComponent.activeMQComponent(
// "vm://localhost?broker.persistent=false")); // "vm://localhost?broker.persistent=false"));
//#CamelExtensionAddComponent //#CamelExtensionAddComponent
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
} }

View file

@ -5,6 +5,7 @@ import akka.actor.ActorSystem;
import akka.actor.Props; import akka.actor.Props;
import akka.camel.Camel; import akka.camel.Camel;
import akka.camel.CamelExtension; import akka.camel.CamelExtension;
import akka.testkit.JavaTestKit;
public class CustomRouteTestBase { public class CustomRouteTestBase {
public void customRoute() throws Exception{ public void customRoute() throws Exception{
@ -15,6 +16,6 @@ public class CustomRouteTestBase {
camel.context().addRoutes(new CustomRouteBuilder(responder)); camel.context().addRoutes(new CustomRouteBuilder(responder));
//#CustomRoute //#CustomRoute
system.stop(responder); system.stop(responder);
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
} }

View file

@ -1,6 +1,7 @@
package docs.camel; package docs.camel;
import akka.actor.*; import akka.actor.*;
import akka.testkit.JavaTestKit;
public class OnRouteResponseTestBase { public class OnRouteResponseTestBase {
@ -17,6 +18,6 @@ public class OnRouteResponseTestBase {
//#RouteResponse //#RouteResponse
system.stop(receiver); system.stop(receiver);
system.stop(forwardResponse); system.stop(forwardResponse);
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
} }

View file

@ -3,6 +3,7 @@ package docs.camel;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import akka.testkit.JavaTestKit;
import scala.concurrent.Future; import scala.concurrent.Future;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
@ -18,7 +19,7 @@ public class ProducerTestBase {
ActorRef producer = system.actorOf(props, "jmsproducer"); ActorRef producer = system.actorOf(props, "jmsproducer");
producer.tell("<order amount=\"100\" currency=\"PLN\" itemId=\"12345\"/>", null); producer.tell("<order amount=\"100\" currency=\"PLN\" itemId=\"12345\"/>", null);
//#TellProducer //#TellProducer
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
@SuppressWarnings("unused") @SuppressWarnings("unused")
@ -30,7 +31,7 @@ public class ProducerTestBase {
Future<Object> future = Patterns.ask(producer, "some request", 1000); Future<Object> future = Patterns.ask(producer, "some request", 1000);
//#AskProducer //#AskProducer
system.stop(producer); system.stop(producer);
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
public void correlate(){ public void correlate(){
@ -44,6 +45,6 @@ public class ProducerTestBase {
"itemId=\"12345\"/>",headers), null); "itemId=\"12345\"/>",headers), null);
//#Correlate //#Correlate
system.stop(producer); system.stop(producer);
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
} }

View file

@ -16,7 +16,9 @@ import akka.event.LoggingAdapter;
//#imports-prio-mailbox //#imports-prio-mailbox
import akka.dispatch.PriorityGenerator; import akka.dispatch.PriorityGenerator;
import akka.dispatch.UnboundedPriorityMailbox; import akka.dispatch.UnboundedPriorityMailbox;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.JavaTestKit; import akka.testkit.JavaTestKit;
import akka.testkit.TestKit;
import com.typesafe.config.Config; import com.typesafe.config.Config;
//#imports-prio-mailbox //#imports-prio-mailbox
@ -35,8 +37,7 @@ import java.util.concurrent.ConcurrentLinkedQueue;
//#imports-required-mailbox //#imports-required-mailbox
import docs.actor.MyBoundedUntypedActor; import docs.actor.MyBoundedUntypedActor;
import org.junit.AfterClass; import org.junit.ClassRule;
import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
import scala.Option; import scala.Option;
import scala.concurrent.ExecutionContext; import scala.concurrent.ExecutionContext;
@ -48,20 +49,13 @@ import akka.testkit.AkkaSpec;
public class DispatcherDocTestBase { public class DispatcherDocTestBase {
static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("DispatcherDocTest", ConfigFactory.parseString(
DispatcherDocSpec.javaConfig()).withFallback(ConfigFactory.parseString(
DispatcherDocSpec.config())).withFallback(AkkaSpec.testConf()));
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void beforeAll() {
system = ActorSystem.create("MySystem",
ConfigFactory.parseString(DispatcherDocSpec.javaConfig()).withFallback(
ConfigFactory.parseString(DispatcherDocSpec.config())).withFallback(AkkaSpec.testConf()));
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
@SuppressWarnings("unused") @SuppressWarnings("unused")
@Test @Test

View file

@ -19,7 +19,7 @@ import akka.event.Logging.Debug;
//#imports-listener //#imports-listener
import org.junit.Test; import org.junit.Test;
import akka.testkit.JavaTestKit;
import scala.Option; import scala.Option;
//#imports-deadletter //#imports-deadletter
@ -37,7 +37,7 @@ public class LoggingDocTestBase {
ActorSystem system = ActorSystem.create("MySystem"); ActorSystem system = ActorSystem.create("MySystem");
ActorRef myActor = system.actorOf(Props.create(MyActor.class, this)); ActorRef myActor = system.actorOf(Props.create(MyActor.class, this));
myActor.tell("test", null); myActor.tell("test", null);
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
@Test @Test
@ -47,7 +47,7 @@ public class LoggingDocTestBase {
final ActorRef actor = system.actorOf(Props.create(DeadLetterActor.class)); final ActorRef actor = system.actorOf(Props.create(DeadLetterActor.class));
system.eventStream().subscribe(actor, DeadLetter.class); system.eventStream().subscribe(actor, DeadLetter.class);
//#deadletters //#deadletters
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
@Test @Test
@ -57,7 +57,7 @@ public class LoggingDocTestBase {
final Object[] args = new Object[] { "The", "brown", "fox", "jumps", 42 }; final Object[] args = new Object[] { "The", "brown", "fox", "jumps", 42 };
system.log().debug("five parameters: {}, {}, {}, {}, {}", args); system.log().debug("five parameters: {}, {}, {}, {}, {}", args);
//#array //#array
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
//#my-actor //#my-actor

View file

@ -51,8 +51,8 @@ import java.util.List;
import java.util.concurrent.ExecutorService; import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import org.junit.After; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.Before; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
@ -67,45 +67,39 @@ import static org.junit.Assert.*;
public class FutureDocTestBase { public class FutureDocTestBase {
ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("FutureDocTest", AkkaSpec.testConf());
@Before private final ActorSystem system = actorSystemResource.getSystem();
public void setUp() {
system = ActorSystem.create("MySystem", AkkaSpec.testConf());
}
@After
public void tearDown() {
system.shutdown();
}
public final static class PrintResult<T> extends OnSuccess<T> { public final static class PrintResult<T> extends OnSuccess<T> {
@Override public final void onSuccess(T t) { @Override public final void onSuccess(T t) {
// print t // print t
}
} }
}
public final static class Demo { public final static class Demo {
//#print-result //#print-result
public final static class PrintResult<T> extends OnSuccess<T> { public final static class PrintResult<T> extends OnSuccess<T> {
@Override public final void onSuccess(T t) { @Override public final void onSuccess(T t) {
System.out.println(t); System.out.println(t);
}
} }
//#print-result }
//#print-result
} }
@SuppressWarnings("unchecked") @Test public void useCustomExecutionContext() throws Exception { @SuppressWarnings("unchecked") @Test public void useCustomExecutionContext() throws Exception {
ExecutorService yourExecutorServiceGoesHere = Executors.newSingleThreadExecutor(); ExecutorService yourExecutorServiceGoesHere = Executors.newSingleThreadExecutor();
//#diy-execution-context //#diy-execution-context
ExecutionContext ec = ExecutionContext ec =
ExecutionContexts.fromExecutorService(yourExecutorServiceGoesHere); ExecutionContexts.fromExecutorService(yourExecutorServiceGoesHere);
//Use ec with your Futures //Use ec with your Futures
Future<String> f1 = Futures.successful("foo"); Future<String> f1 = Futures.successful("foo");
// Then you shut down the ExecutorService at the end of your application. // Then you shut down the ExecutorService at the end of your application.
yourExecutorServiceGoesHere.shutdown(); yourExecutorServiceGoesHere.shutdown();
//#diy-execution-context //#diy-execution-context
} }
@Test @Test
@ -134,8 +128,8 @@ public class FutureDocTestBase {
f.onSuccess(new PrintResult<String>(), system.dispatcher()); f.onSuccess(new PrintResult<String>(), system.dispatcher());
//#future-eval //#future-eval
String result = (String) Await.result(f, Duration.create(5, SECONDS)); String result = (String) Await.result(f, Duration.create(5, SECONDS));
assertEquals("HelloWorld", result); assertEquals("HelloWorld", result);
} }
@Test @Test
@ -504,13 +498,13 @@ public class FutureDocTestBase {
final ExecutionContext ec = system.dispatcher(); final ExecutionContext ec = system.dispatcher();
future.onComplete(new OnComplete<String>() { future.onComplete(new OnComplete<String>() {
public void onComplete(Throwable failure, String result) { public void onComplete(Throwable failure, String result) {
if (failure != null) { if (failure != null) {
//We got a failure, handle it here //We got a failure, handle it here
} else { } else {
// We got a result, do something with it // We got a result, do something with it
}
} }
}
}, ec); }, ec);
//#onComplete //#onComplete
} }

View file

@ -4,8 +4,8 @@
package docs.io; package docs.io;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
@ -27,6 +27,7 @@ public class UdpConnectedDocTest {
static public class Demo extends UntypedActor { static public class Demo extends UntypedActor {
ActorRef connectionActor = null; ActorRef connectionActor = null;
ActorRef handler = getSelf(); ActorRef handler = getSelf();
ActorSystem system = context().system();
@Override @Override
public void onReceive(Object msg) { public void onReceive(Object msg) {
@ -78,18 +79,6 @@ public class UdpConnectedDocTest {
} }
} }
static ActorSystem system;
@BeforeClass
static public void setup() {
system = ActorSystem.create("UdpConnectedDocTest");
}
@AfterClass
static public void teardown() {
system.shutdown();
}
@Test @Test
public void demonstrateConnect() { public void demonstrateConnect() {
} }

View file

@ -19,83 +19,73 @@ import java.util.ArrayList;
import java.util.List; import java.util.List;
//#imports //#imports
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
public class UdpDocTest { public class UdpDocTest {
static public class Demo extends UntypedActor { static public class Demo extends UntypedActor {
public void onReceive(Object message) { ActorSystem system = context().system();
//#manager
final ActorRef udp = Udp.get(system).manager();
//#manager
//#simplesend public void onReceive(Object message) {
udp.tell(UdpMessage.simpleSender(), getSelf()); //#manager
final ActorRef udp = Udp.get(system).manager();
//#manager
// ... or with socket options: //#simplesend
final List<Inet.SocketOption> options = new ArrayList<Inet.SocketOption>(); udp.tell(UdpMessage.simpleSender(), getSelf());
options.add(UdpSO.broadcast(true));
udp.tell(UdpMessage.simpleSender(), getSelf());
//#simplesend
ActorRef simpleSender = null; // ... or with socket options:
final List<Inet.SocketOption> options = new ArrayList<Inet.SocketOption>();
options.add(UdpSO.broadcast(true));
udp.tell(UdpMessage.simpleSender(), getSelf());
//#simplesend
//#simplesend-finish ActorRef simpleSender = null;
if (message instanceof Udp.SimpleSendReady) {
simpleSender = getSender();
}
//#simplesend-finish
final ByteString data = ByteString.empty(); //#simplesend-finish
if (message instanceof Udp.SimpleSendReady) {
simpleSender = getSender();
}
//#simplesend-finish
//#simplesend-send final ByteString data = ByteString.empty();
simpleSender.tell(UdpMessage.send(data, new InetSocketAddress("127.0.0.1", 7654)), getSelf());
//#simplesend-send
final ActorRef handler = getSelf(); //#simplesend-send
simpleSender.tell(UdpMessage.send(data, new InetSocketAddress("127.0.0.1", 7654)), getSelf());
//#simplesend-send
//#bind final ActorRef handler = getSelf();
udp.tell(UdpMessage.bind(handler, new InetSocketAddress("127.0.0.1", 9876)), getSelf());
//#bind
ActorRef udpWorker = null; //#bind
udp.tell(UdpMessage.bind(handler, new InetSocketAddress("127.0.0.1", 9876)), getSelf());
//#bind
//#bind-finish ActorRef udpWorker = null;
if (message instanceof Udp.Bound) {
udpWorker = getSender();
}
//#bind-finish
//#bind-receive //#bind-finish
if (message instanceof Udp.Received) { if (message instanceof Udp.Bound) {
final Udp.Received rcvd = (Udp.Received) message; udpWorker = getSender();
final ByteString payload = rcvd.data(); }
final InetSocketAddress sender = rcvd.sender(); //#bind-finish
}
//#bind-receive
//#bind-send //#bind-receive
udpWorker.tell(UdpMessage.send(data, new InetSocketAddress("127.0.0.1", 7654)), getSelf()); if (message instanceof Udp.Received) {
//#bind-send final Udp.Received rcvd = (Udp.Received) message;
} final ByteString payload = rcvd.data();
final InetSocketAddress sender = rcvd.sender();
}
//#bind-receive
//#bind-send
udpWorker.tell(UdpMessage.send(data, new InetSocketAddress("127.0.0.1", 7654)), getSelf());
//#bind-send
} }
}
static ActorSystem system; @Test
public void demonstrateConnect() {
@BeforeClass }
static public void setup() {
system = ActorSystem.create("IODocTest");
}
@AfterClass
static public void teardown() {
system.shutdown();
}
@Test
public void demonstrateConnect() {
}
} }

View file

@ -5,8 +5,8 @@
package docs.io.japi; package docs.io.japi;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
//#imports //#imports
@ -138,19 +138,13 @@ public class IODocTest {
} }
//#client //#client
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
@BeforeClass new AkkaJUnitActorSystemResource("IODocTest", AkkaSpec.testConf());
public static void setup() {
system = ActorSystem.create("IODocTest", AkkaSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem();
}
@AfterClass
public static void teardown() {
system.shutdown();
}
@Test @Test
public void testConnection() { public void testConnection() {
new JavaTestKit(system) { new JavaTestKit(system) {

View file

@ -7,8 +7,8 @@ package docs.io.japi;
import java.nio.ByteOrder; import java.nio.ByteOrder;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
@ -48,18 +48,12 @@ public class PipelineTest {
} }
final Context ctx = new Context(); final Context ctx = new Context();
//#byteorder //#byteorder
static ActorSystem system = null; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
@BeforeClass new AkkaJUnitActorSystemResource("PipelineTest");
public static void setup() {
system = ActorSystem.create("PipelineTest"); private final ActorSystem system = actorSystemResource.getSystem();
}
@AfterClass
public static void teardown() {
system.shutdown();
}
@Test @Test
public void demonstratePipeline() throws Exception { public void demonstratePipeline() throws Exception {

View file

@ -9,8 +9,8 @@ import java.net.InetSocketAddress;
import javax.net.ssl.SSLContext; import javax.net.ssl.SSLContext;
import javax.net.ssl.SSLEngine; import javax.net.ssl.SSLEngine;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import akka.actor.ActorContext; import akka.actor.ActorContext;
@ -184,17 +184,11 @@ public class SslDocTest {
} }
//#server //#server
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
@BeforeClass new AkkaJUnitActorSystemResource("SslDocTest", AkkaSpec.testConf());
public static void setup() {
system = ActorSystem.create("IODocTest", AkkaSpec.testConf()); private final ActorSystem system = actorSystemResource.getSystem();
}
@AfterClass
public static void teardown() {
system.shutdown();
}
@Test @Test
public void demonstrateSslClient() { public void demonstrateSslClient() {

View file

@ -3,8 +3,8 @@
*/ */
package docs.jrouting; package docs.jrouting;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import akka.testkit.JavaTestKit; import akka.testkit.JavaTestKit;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
@ -27,17 +27,11 @@ import akka.routing.ConsistentHashingRouter.ConsistentHashableEnvelope;
public class ConsistentHashingRouterDocTestBase { public class ConsistentHashingRouterDocTestBase {
static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
@BeforeClass new AkkaJUnitActorSystemResource("ConsistentHashingRouterDocTest");
public static void setup() {
system = ActorSystem.create(); private final ActorSystem system = actorSystemResource.getSystem();
}
@AfterClass
public static void teardown() {
system.shutdown();
}
static static
//#cache-actor //#cache-actor

View file

@ -14,9 +14,8 @@ import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import org.junit.After; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.Before; import org.junit.*;
import org.junit.Test;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.Future; import scala.concurrent.Future;
@ -38,17 +37,11 @@ import akka.util.Timeout;
public class CustomRouterDocTestBase { public class CustomRouterDocTestBase {
ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("CustomRouterDocTest", AkkaSpec.testConf());
@Before private final ActorSystem system = actorSystemResource.getSystem();
public void setUp() {
system = ActorSystem.create("MySystem", AkkaSpec.testConf());
}
@After
public void tearDown() {
system.shutdown();
}
public static class MyActor extends UntypedActor { public static class MyActor extends UntypedActor {
@Override public void onReceive(Object o) {} @Override public void onReceive(Object o) {}

View file

@ -5,8 +5,8 @@ package docs.jrouting;
import java.util.Arrays; import java.util.Arrays;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import akka.actor.ActorRef; import akka.actor.ActorRef;
@ -23,17 +23,11 @@ import docs.routing.RouterViaProgramDocSpec.Echo;
public class RouterViaProgramDocTestBase { public class RouterViaProgramDocTestBase {
static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
@BeforeClass new AkkaJUnitActorSystemResource("RouterViaProgramDocTest");
public static void setup() {
system = ActorSystem.create(); private final ActorSystem system = actorSystemResource.getSystem();
}
@AfterClass
public static void teardown() {
system.shutdown();
}
private static class JavaTestKitWithSelf extends JavaTestKit { private static class JavaTestKitWithSelf extends JavaTestKit {
public JavaTestKitWithSelf(ActorSystem system) { public JavaTestKitWithSelf(ActorSystem system) {

View file

@ -16,17 +16,11 @@ import java.util.concurrent.TimeUnit;
public class SchedulerPatternTest { public class SchedulerPatternTest {
static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("SchedulerPatternTest", AkkaSpec.testConf());
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void setUp() {
system = ActorSystem.create("SchedulerPatternTest", AkkaSpec.testConf());
}
@AfterClass
public static void tearDown() {
system.shutdown();
}
static static
//#schedule-constructor //#schedule-constructor

View file

@ -3,8 +3,8 @@
*/ */
package docs.remoting; package docs.remoting;
import org.junit.AfterClass; import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.BeforeClass; import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
@ -28,18 +28,12 @@ public class RemoteDeploymentDocTestBase {
getSender().tell(getSelf(), getSelf()); getSender().tell(getSelf(), getSelf());
} }
} }
static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
@BeforeClass new AkkaJUnitActorSystemResource("RemoteDeploymentDocTest");
public static void init() {
system = ActorSystem.create(); private final ActorSystem system = actorSystemResource.getSystem();
}
@AfterClass
public static void cleanup() {
system.shutdown();
}
@Test @Test
public void demonstrateDeployment() { public void demonstrateDeployment() {

View file

@ -3,6 +3,7 @@
*/ */
package docs.serialization; package docs.serialization;
import akka.testkit.JavaTestKit;
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.*; import static org.junit.Assert.*;
//#imports //#imports
@ -66,7 +67,7 @@ public class SerializationDocTestBase {
final ActorRef deserializedActorRef = extendedSystem.provider().resolveActorRef(identifier); final ActorRef deserializedActorRef = extendedSystem.provider().resolveActorRef(identifier);
// Then just use the ActorRef // Then just use the ActorRef
//#actorref-serializer //#actorref-serializer
extendedSystem.shutdown(); JavaTestKit.shutdownActorSystem(extendedSystem);
} }
static static
@ -187,6 +188,6 @@ public class SerializationDocTestBase {
assertEquals(original, back); assertEquals(original, back);
//#programmatic //#programmatic
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
} }

View file

@ -5,12 +5,12 @@ package docs.testkit;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.junit.AfterClass; import akka.testkit.*;
import org.junit.BeforeClass; import docs.actor.mailbox.DurableMailboxDocSpec;
import org.junit.ClassRule;
import org.junit.Test; import org.junit.Test;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
import com.typesafe.config.Config;
import akka.actor.ActorKilledException; import akka.actor.ActorKilledException;
import akka.actor.ActorRef; import akka.actor.ActorRef;
@ -22,15 +22,18 @@ import akka.actor.Terminated;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.Future; import scala.concurrent.Future;
import akka.testkit.CallingThreadDispatcher;
import akka.testkit.TestActor;
import akka.testkit.TestActor.AutoPilot; import akka.testkit.TestActor.AutoPilot;
import akka.testkit.TestActorRef;
import akka.testkit.JavaTestKit;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
public class TestKitDocTest { public class TestKitDocTest {
@ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("TestKitDocTest",
ConfigFactory.parseString("akka.loggers = [akka.testkit.TestEventListener]"));
private final ActorSystem system = actorSystemResource.getSystem();
//#test-actor-ref //#test-actor-ref
static class MyActor extends UntypedActor { static class MyActor extends UntypedActor {
public void onReceive(Object o) throws Exception { public void onReceive(Object o) throws Exception {
@ -43,23 +46,6 @@ public class TestKitDocTest {
public boolean testMe() { return true; } public boolean testMe() { return true; }
} }
//#test-actor-ref
private static ActorSystem system;
@BeforeClass
public static void setup() {
final Config config = ConfigFactory.parseString(
"akka.loggers = [akka.testkit.TestEventListener]");
system = ActorSystem.create("demoSystem", config);
}
@AfterClass
public static void cleanup() {
system.shutdown();
}
//#test-actor-ref
@Test @Test
public void demonstrateTestActorRef() { public void demonstrateTestActorRef() {
final Props props = Props.create(MyActor.class); final Props props = Props.create(MyActor.class);
@ -415,7 +401,7 @@ public class TestKitDocTest {
public void demonstrateEventFilter() { public void demonstrateEventFilter() {
//#test-event-filter //#test-event-filter
new JavaTestKit(system) {{ new JavaTestKit(system) {{
assertEquals("demoSystem", system.name()); assertEquals("TestKitDocTest", system.name());
final ActorRef victim = system.actorOf(Props.empty(), "victim"); final ActorRef victim = system.actorOf(Props.empty(), "victim");
final int result = new EventFilter<Integer>(ActorKilledException.class) { final int result = new EventFilter<Integer>(ActorKilledException.class) {
@ -423,7 +409,7 @@ public class TestKitDocTest {
victim.tell(Kill.getInstance(), null); victim.tell(Kill.getInstance(), null);
return 42; return 42;
} }
}.from("akka://demoSystem/user/victim").occurrences(1).exec(); }.from("akka://TestKitDocTest/user/victim").occurrences(1).exec();
assertEquals(42, result); assertEquals(42, result);
}}; }};

View file

@ -43,7 +43,8 @@ public class TestKitSampleTest {
@AfterClass @AfterClass
public static void teardown() { public static void teardown() {
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
system = null;
} }
@Test @Test

View file

@ -5,6 +5,8 @@
package docs.transactor; package docs.transactor;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import akka.testkit.JavaTestKit;
import org.junit.Test; import org.junit.Test;
//#imports //#imports
@ -18,83 +20,83 @@ import static java.util.concurrent.TimeUnit.SECONDS;
public class TransactorDocTest { public class TransactorDocTest {
@Test @Test
public void coordinatedExample() throws Exception { public void coordinatedExample() throws Exception {
//#coordinated-example //#coordinated-example
ActorSystem system = ActorSystem.create("CoordinatedExample"); ActorSystem system = ActorSystem.create("CoordinatedExample");
ActorRef counter1 = system.actorOf(Props.create(CoordinatedCounter.class)); ActorRef counter1 = system.actorOf(Props.create(CoordinatedCounter.class));
ActorRef counter2 = system.actorOf(Props.create(CoordinatedCounter.class)); ActorRef counter2 = system.actorOf(Props.create(CoordinatedCounter.class));
Timeout timeout = new Timeout(5, SECONDS); Timeout timeout = new Timeout(5, SECONDS);
counter1.tell(new Coordinated(new Increment(counter2), timeout), null); counter1.tell(new Coordinated(new Increment(counter2), timeout), null);
Integer count = (Integer) Await.result( Integer count = (Integer) Await.result(
ask(counter1, "GetCount", timeout), timeout.duration()); ask(counter1, "GetCount", timeout), timeout.duration());
//#coordinated-example //#coordinated-example
assertEquals(count, new Integer(1)); assertEquals(count, new Integer(1));
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
@Test @Test
public void coordinatedApi() { public void coordinatedApi() {
//#create-coordinated //#create-coordinated
Timeout timeout = new Timeout(5, SECONDS); Timeout timeout = new Timeout(5, SECONDS);
Coordinated coordinated = new Coordinated(timeout); Coordinated coordinated = new Coordinated(timeout);
//#create-coordinated //#create-coordinated
ActorSystem system = ActorSystem.create("CoordinatedApi"); ActorSystem system = ActorSystem.create("CoordinatedApi");
ActorRef actor = system.actorOf(Props.create(Coordinator.class)); ActorRef actor = system.actorOf(Props.create(Coordinator.class));
//#send-coordinated //#send-coordinated
actor.tell(new Coordinated(new Message(), timeout), null); actor.tell(new Coordinated(new Message(), timeout), null);
//#send-coordinated //#send-coordinated
//#include-coordinated //#include-coordinated
actor.tell(coordinated.coordinate(new Message()), null); actor.tell(coordinated.coordinate(new Message()), null);
//#include-coordinated //#include-coordinated
coordinated.await(); coordinated.await();
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
@Test @Test
public void counterTransactor() throws Exception { public void counterTransactor() throws Exception {
ActorSystem system = ActorSystem.create("CounterTransactor"); ActorSystem system = ActorSystem.create("CounterTransactor");
ActorRef counter = system.actorOf(Props.create(Counter.class)); ActorRef counter = system.actorOf(Props.create(Counter.class));
Timeout timeout = new Timeout(5, SECONDS); Timeout timeout = new Timeout(5, SECONDS);
Coordinated coordinated = new Coordinated(timeout); Coordinated coordinated = new Coordinated(timeout);
counter.tell(coordinated.coordinate(new Increment()), null); counter.tell(coordinated.coordinate(new Increment()), null);
coordinated.await(); coordinated.await();
Integer count = (Integer) Await.result(ask(counter, "GetCount", timeout), timeout.duration()); Integer count = (Integer) Await.result(ask(counter, "GetCount", timeout), timeout.duration());
assertEquals(count, new Integer(1)); assertEquals(count, new Integer(1));
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
@Test @Test
public void friendlyCounterTransactor() throws Exception { public void friendlyCounterTransactor() throws Exception {
ActorSystem system = ActorSystem.create("FriendlyCounterTransactor"); ActorSystem system = ActorSystem.create("FriendlyCounterTransactor");
ActorRef friend = system.actorOf(Props.create(Counter.class)); ActorRef friend = system.actorOf(Props.create(Counter.class));
ActorRef friendlyCounter = system.actorOf(Props.create(FriendlyCounter.class)); ActorRef friendlyCounter = system.actorOf(Props.create(FriendlyCounter.class));
Timeout timeout = new Timeout(5, SECONDS); Timeout timeout = new Timeout(5, SECONDS);
Coordinated coordinated = new Coordinated(timeout); Coordinated coordinated = new Coordinated(timeout);
friendlyCounter.tell(coordinated.coordinate(new Increment(friend)), null); friendlyCounter.tell(coordinated.coordinate(new Increment(friend)), null);
coordinated.await(); coordinated.await();
Integer count1 = (Integer) Await.result(ask(friendlyCounter, "GetCount", timeout), timeout.duration()); Integer count1 = (Integer) Await.result(ask(friendlyCounter, "GetCount", timeout), timeout.duration());
assertEquals(count1, new Integer(1)); assertEquals(count1, new Integer(1));
Integer count2 = (Integer) Await.result(ask(friend, "GetCount", timeout), timeout.duration()); Integer count2 = (Integer) Await.result(ask(friend, "GetCount", timeout), timeout.duration());
assertEquals(count2, new Integer(1)); assertEquals(count2, new Integer(1));
system.shutdown(); JavaTestKit.shutdownActorSystem(system);
} }
} }

View file

@ -30,6 +30,7 @@ import akka.actor.UntypedActor;
import akka.actor.Props; import akka.actor.Props;
import akka.event.Logging; import akka.event.Logging;
import akka.event.LoggingAdapter; import akka.event.LoggingAdapter;
import org.junit.*;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
import akka.serialization.SerializationExtension; import akka.serialization.SerializationExtension;
import akka.serialization.Serialization; import akka.serialization.Serialization;
@ -47,26 +48,16 @@ import java.text.SimpleDateFormat;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import akka.testkit.AkkaJUnitActorSystemResource;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
import org.junit.Assume;
public class ZeromqDocTestBase { public class ZeromqDocTestBase {
ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("ZeromqDocTest",
ConfigFactory.parseString("akka.loglevel=INFO").withFallback(AkkaSpec.testConf()));
@Before private final ActorSystem system = actorSystemResource.getSystem();
public void setUp() {
system = ActorSystem.create("ZeromqDocTest",
ConfigFactory.parseString("akka.loglevel=INFO").withFallback(AkkaSpec.testConf()));
}
@After
public void tearDown() {
system.shutdown();
}
@SuppressWarnings("unused") @SuppressWarnings("unused")
@Test @Test

View file

@ -267,7 +267,7 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
val system = ActorSystem("mySystem") val system = ActorSystem("mySystem")
val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), "myactor2") val myActor = system.actorOf(Props[MyActor].withDispatcher("my-dispatcher"), "myactor2")
//#system-actorOf //#system-actorOf
system.shutdown() shutdown(system)
} }
"creating actor with IndirectActorProducer" in { "creating actor with IndirectActorProducer" in {

View file

@ -28,7 +28,7 @@ class RemoteDeploymentDocSpec extends AkkaSpec("""
val other = ActorSystem("remote", system.settings.config) val other = ActorSystem("remote", system.settings.config)
val address = other.asInstanceOf[ExtendedActorSystem].provider.getExternalAddressFor(Address("akka.tcp", "s", "host", 1)).get val address = other.asInstanceOf[ExtendedActorSystem].provider.getExternalAddressFor(Address("akka.tcp", "s", "host", 1)).get
override def afterTermination() { other.shutdown() } override def afterTermination() { shutdown(other) }
"demonstrate programmatic deployment" in { "demonstrate programmatic deployment" in {
//#deploy //#deploy

View file

@ -64,7 +64,7 @@ package docs.serialization {
//#serialize-messages-config //#serialize-messages-config
val a = ActorSystem("system", config) val a = ActorSystem("system", config)
a.settings.SerializeAllMessages must be(true) a.settings.SerializeAllMessages must be(true)
a.shutdown() shutdown(a)
} }
"demonstrate configuration of serialize creators" in { "demonstrate configuration of serialize creators" in {
@ -79,7 +79,7 @@ package docs.serialization {
//#serialize-creators-config //#serialize-creators-config
val a = ActorSystem("system", config) val a = ActorSystem("system", config)
a.settings.SerializeAllCreators must be(true) a.settings.SerializeAllCreators must be(true)
a.shutdown() shutdown(a)
} }
"demonstrate configuration of serializers" in { "demonstrate configuration of serializers" in {
@ -97,7 +97,7 @@ package docs.serialization {
""") """)
//#serialize-serializers-config //#serialize-serializers-config
val a = ActorSystem("system", config) val a = ActorSystem("system", config)
a.shutdown() shutdown(a)
} }
"demonstrate configuration of serialization-bindings" in { "demonstrate configuration of serialization-bindings" in {
@ -126,7 +126,7 @@ package docs.serialization {
SerializationExtension(a).serializerFor(classOf[String]).getClass must equal(classOf[JavaSerializer]) SerializationExtension(a).serializerFor(classOf[String]).getClass must equal(classOf[JavaSerializer])
SerializationExtension(a).serializerFor(classOf[Customer]).getClass must equal(classOf[JavaSerializer]) SerializationExtension(a).serializerFor(classOf[Customer]).getClass must equal(classOf[JavaSerializer])
SerializationExtension(a).serializerFor(classOf[java.lang.Boolean]).getClass must equal(classOf[MyOwnSerializer]) SerializationExtension(a).serializerFor(classOf[java.lang.Boolean]).getClass must equal(classOf[MyOwnSerializer])
a.shutdown() shutdown(a)
} }
"demonstrate the programmatic API" in { "demonstrate the programmatic API" in {
@ -152,7 +152,7 @@ package docs.serialization {
back must equal(original) back must equal(original)
//#programmatic //#programmatic
system.shutdown() shutdown(system)
} }
"demonstrate serialization of ActorRefs" in { "demonstrate serialization of ActorRefs" in {

View file

@ -31,7 +31,7 @@ class MySpec(_system: ActorSystem) extends TestKit(_system) with ImplicitSender
import MySpec._ import MySpec._
override def afterAll { override def afterAll {
system.shutdown() TestKit.shutdownActorSystem(system)
} }
"An Echo actor" must { "An Echo actor" must {

View file

@ -45,7 +45,7 @@ class TestKitUsageSpec
system.actorOf(Props(classOf[SequencingActor], testActor, headList, tailList)) system.actorOf(Props(classOf[SequencingActor], testActor, headList, tailList))
override def afterAll { override def afterAll {
system.shutdown() shutdown(system)
} }
"An EchoActor" should { "An EchoActor" should {

View file

@ -264,7 +264,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
actor ! Kill actor ! Kill
} }
} finally { } finally {
system.shutdown() shutdown(system)
} }
//#event-filter //#event-filter
} }
@ -282,7 +282,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
try expectMsg("hello") catch { case NonFatal(e) system.shutdown(); throw e } try expectMsg("hello") catch { case NonFatal(e) system.shutdown(); throw e }
//#put-your-test-code-here //#put-your-test-code-here
system.shutdown() shutdown(system)
} }
//#test-kit-base //#test-kit-base
} }

View file

@ -161,7 +161,7 @@ class TransactorDocSpec extends AkkaSpec {
count must be === 1 count must be === 1
system.shutdown() shutdown(system)
} }
"coordinated api" in { "coordinated api" in {
@ -191,7 +191,7 @@ class TransactorDocSpec extends AkkaSpec {
coordinated.await() coordinated.await()
system.shutdown() shutdown(system)
} }
"counter transactor" in { "counter transactor" in {
@ -208,7 +208,7 @@ class TransactorDocSpec extends AkkaSpec {
underlyingCounter.count.single.get must be === 1 underlyingCounter.count.single.get must be === 1
system.shutdown() shutdown(system)
} }
"friendly counter transactor" in { "friendly counter transactor" in {
@ -229,6 +229,6 @@ class TransactorDocSpec extends AkkaSpec {
underlyingFriendlyCounter.count.single.get must be === 1 underlyingFriendlyCounter.count.single.get must be === 1
underlyingFriend.count.single.get must be === 1 underlyingFriend.count.single.get must be === 1
system.shutdown() shutdown(system)
} }
} }

View file

@ -80,7 +80,7 @@ abstract class DurableMailboxSpec(system: ActorSystem, val backendName: String)
protected def atStartup() {} protected def atStartup() {}
final override def afterAll { final override def afterAll {
system.shutdown() TestKit.shutdownActorSystem(system)
try system.awaitTermination(5 seconds) catch { try system.awaitTermination(5 seconds) catch {
case _: TimeoutException system.log.warning("Failed to stop [{}] within 5 seconds", system.name) case _: TimeoutException system.log.warning("Failed to stop [{}] within 5 seconds", system.name)
} }

View file

@ -268,14 +268,7 @@ abstract class MultiNodeSpec(val myself: RoleName, _system: ActorSystem, _roles:
} }
} }
} }
system.shutdown() shutdown(system)
try system.awaitTermination(shutdownTimeout) catch {
case _: TimeoutException
val msg = "Failed to stop [%s] within [%s] \n%s".format(system.name, shutdownTimeout,
system.asInstanceOf[ActorSystemImpl].printTree)
if (verifySystemShutdown) throw new RuntimeException(msg)
else system.log.warning(msg)
}
afterTermination() afterTermination()
} }

View file

@ -34,7 +34,7 @@ akka {
} }
override def afterTermination() { override def afterTermination() {
other.shutdown() shutdown(other)
} }
"receive Terminated when watched node is unknown host" in { "receive Terminated when watched node is unknown host" in {

View file

@ -71,7 +71,7 @@ akka.actor.deployment {
val otherSystem = ActorSystem("remote-sys", conf) val otherSystem = ActorSystem("remote-sys", conf)
override def afterTermination() { override def afterTermination() {
otherSystem.shutdown() shutdown(otherSystem)
} }
"A Remote Router" must { "A Remote Router" must {

View file

@ -96,7 +96,7 @@ class RemoteWatcherSpec extends AkkaSpec(
akka.remote.transport.ActorTransportAdapter.DisassociateUnderlying.getClass)(_)) akka.remote.transport.ActorTransportAdapter.DisassociateUnderlying.getClass)(_))
override def afterTermination() { override def afterTermination() {
remoteSystem.shutdown() shutdown(remoteSystem)
} }
val heartbeatMsgB = Heartbeat(remoteAddressUid) val heartbeatMsgB = Heartbeat(remoteAddressUid)

View file

@ -201,7 +201,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
val maxPayloadBytes = system.settings.config.getBytes("akka.remote.test.maximum-payload-bytes").toInt val maxPayloadBytes = system.settings.config.getBytes("akka.remote.test.maximum-payload-bytes").toInt
override def afterTermination() { override def afterTermination() {
remoteSystem.shutdown() shutdown(remoteSystem)
AssociationRegistry.clear() AssociationRegistry.clear()
} }
@ -258,8 +258,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
// then we shutdown all but one system to simulate broken connections // then we shutdown all but one system to simulate broken connections
moreSystems foreach { sys moreSystems foreach { sys
sys.shutdown() shutdown(sys)
sys.awaitTermination(5.seconds.dilated)
} }
1 to n foreach { x 1 to n foreach { x
@ -531,8 +530,7 @@ class RemotingSpec extends AkkaSpec(RemotingSpec.cfg) with ImplicitSender with D
expectMsg(3.seconds, ("pong", otherGuyRemoteTest)) expectMsg(3.seconds, ("pong", otherGuyRemoteTest))
}(otherSystem) }(otherSystem)
} finally { } finally {
otherSystem.shutdown() shutdown(otherSystem)
otherSystem.awaitTermination(5.seconds.dilated)
} }
} }
} }

View file

@ -129,8 +129,7 @@ abstract class Ticket1978CommunicationSpec(val cipherConfig: CipherConfig) exten
override def afterTermination() { override def afterTermination() {
if (cipherConfig.runTest) { if (cipherConfig.runTest) {
other.shutdown() shutdown(other)
other.awaitTermination()
} }
} }

View file

@ -38,7 +38,7 @@ akka.loglevel = DEBUG
val target2 = other.actorFor(RootActorPath(addr) / testActor.path.elements) val target2 = other.actorFor(RootActorPath(addr) / testActor.path.elements)
override def afterTermination() { override def afterTermination() {
other.shutdown() shutdown(other)
} }
// need to enable debug log-level without actually printing those messages // need to enable debug log-level without actually printing those messages

View file

@ -103,6 +103,6 @@ class AkkaProtocolStressTest extends AkkaSpec(configA) with ImplicitSender with
EventFilter.warning(pattern = "received dead letter.*(InboundPayload|Disassociate)"))) EventFilter.warning(pattern = "received dead letter.*(InboundPayload|Disassociate)")))
} }
override def afterTermination(): Unit = systemB.shutdown() override def afterTermination(): Unit = shutdown(systemB)
} }

View file

@ -136,7 +136,7 @@ abstract class SystemMessageDeliveryStressTest(msg: String, cfg: String)
EventFilter.warning(pattern = "received dead letter.*(InboundPayload|Disassociate)"))) EventFilter.warning(pattern = "received dead letter.*(InboundPayload|Disassociate)")))
} }
override def afterTermination(): Unit = systemB.shutdown() override def afterTermination(): Unit = shutdown(systemB)
} }

View file

@ -136,7 +136,7 @@ class ThrottlerTransportAdapterSpec extends AkkaSpec(configA) with ImplicitSende
EventFilter.warning(pattern = "received dead letter.*(InboundPayload|Disassociate)"))) EventFilter.warning(pattern = "received dead letter.*(InboundPayload|Disassociate)")))
} }
override def afterTermination(): Unit = systemB.shutdown() override def afterTermination(): Unit = shutdown(systemB)
} }
class ThrottlerTransportAdapterGenericSpec extends GenericTransportSpec(withAkkaProtocol = true) { class ThrottlerTransportAdapterGenericSpec extends GenericTransportSpec(withAkkaProtocol = true) {

View file

@ -4,7 +4,6 @@
package akka.testkit; package akka.testkit;
import akka.actor.Terminated; import akka.actor.Terminated;
import akka.japi.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;
@ -15,6 +14,8 @@ import akka.japi.Util;
import scala.concurrent.duration.Duration; import scala.concurrent.duration.Duration;
import scala.concurrent.duration.FiniteDuration; import scala.concurrent.duration.FiniteDuration;
import java.util.concurrent.TimeUnit;
/** /**
* Java API: Test kit for testing actors. Inheriting from this class enables * Java API: Test kit for testing actors. Inheriting from this class enables
* reception of replies from actors, which are queued by an internal actor and * reception of replies from actors, which are queued by an internal actor and
@ -42,6 +43,29 @@ import scala.concurrent.duration.FiniteDuration;
* *
*/ */
public class JavaTestKit { public class JavaTestKit {
/**
* Shut down an actor system and wait for termination.
* On failure debug output will be logged about the remaining actors in the system.
* <p>
*
* If verifySystemShutdown is true, then an exception will be thrown on failure.
*/
public static void shutdownActorSystem(ActorSystem actorSystem, Duration duration, Boolean verifySystemShutdown) {
boolean vss = verifySystemShutdown != null ? verifySystemShutdown : false;
Duration dur = duration != null ? duration : FiniteDuration.create(10, TimeUnit.SECONDS);
TestKit.shutdownActorSystem(actorSystem, dur, vss);
}
public static void shutdownActorSystem(ActorSystem actorSystem) {
shutdownActorSystem(actorSystem, null, null);
}
public void shutdownActorSystem(ActorSystem actorSystem, Duration duration) {
shutdownActorSystem(actorSystem, duration, null);
}
public void shutdownActorSystem(ActorSystem actorSystem, Boolean verifySystemShutdown) {
shutdownActorSystem(actorSystem, null, verifySystemShutdown);
}
private final TestProbe p; private final TestProbe p;
public JavaTestKit(ActorSystem system) { public JavaTestKit(ActorSystem system) {
@ -637,4 +661,28 @@ public class JavaTestKit {
} }
} }
/**
* Shut down an actor system and wait for termination.
* On failure debug output will be logged about the remaining actors in the system.
* <p>
*
* If verifySystemShutdown is true, then an exception will be thrown on failure.
*/
public void shutdown(ActorSystem actorSystem, Duration duration, Boolean verifySystemShutdown) {
boolean vss = verifySystemShutdown != null ? verifySystemShutdown : false;
Duration dur = duration != null ? duration :
TestKit.dilated(FiniteDuration.create(5, TimeUnit.SECONDS),
getSystem()).min(FiniteDuration.create(10, TimeUnit.SECONDS));
JavaTestKit.shutdownActorSystem(actorSystem, dur, vss);
}
public void shutdown(ActorSystem actorSystem) {
shutdown(actorSystem, null, null);
}
public void shutdown(ActorSystem actorSystem, Duration duration) {
shutdown(actorSystem, duration, null);
}
public void shutdown(ActorSystem actorSystem, Boolean verifySystemShutdown) {
shutdown(actorSystem, null, verifySystemShutdown);
}
} }

View file

@ -8,12 +8,17 @@ import scala.annotation.{ varargs, tailrec }
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.reflect.ClassTag import scala.reflect.ClassTag
import java.util.concurrent.{ BlockingDeque, LinkedBlockingDeque, TimeUnit, atomic } import java.util.concurrent._
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.actor._ import akka.actor._
import akka.actor.Actor._ import akka.actor.Actor._
import akka.util.{ Timeout, BoxedType } import akka.util.{ Timeout, BoxedType }
import scala.util.control.NonFatal import scala.util.control.NonFatal
import scala.Some
import java.util.concurrent.TimeUnit
import akka.actor.IllegalActorStateException
import akka.actor.DeadLetter
import akka.actor.Terminated
object TestActor { object TestActor {
type Ignore = Option[PartialFunction[Any, Boolean]] type Ignore = Option[PartialFunction[Any, Boolean]]
@ -645,6 +650,18 @@ trait TestKitBase {
} }
} }
/**
* Shut down an actor system and wait for termination.
* On failure debug output will be logged about the remaining actors in the system.
*
* If verifySystemShutdown is true, then an exception will be thrown on failure.
*/
def shutdown(actorSystem: ActorSystem,
duration: Duration = 5.seconds.dilated.min(10.seconds),
verifySystemShutdown: Boolean = false) {
TestKit.shutdownActorSystem(actorSystem, duration, verifySystemShutdown)
}
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)
} }
@ -727,6 +744,25 @@ object TestKit {
*/ */
def dilated(duration: Duration, system: ActorSystem): Duration = def dilated(duration: Duration, system: ActorSystem): Duration =
duration * TestKitExtension(system).TestTimeFactor duration * TestKitExtension(system).TestTimeFactor
/**
* Shut down an actor system and wait for termination.
* On failure debug output will be logged about the remaining actors in the system.
*
* If verifySystemShutdown is true, then an exception will be thrown on failure.
*/
def shutdownActorSystem(actorSystem: ActorSystem,
duration: Duration = 10 seconds,
verifySystemShutdown: Boolean = false): Unit = {
actorSystem.shutdown()
try actorSystem.awaitTermination(duration) catch {
case _: TimeoutException
val msg = "Failed to stop [%s] within [%s] \n%s".format(actorSystem.name, duration,
actorSystem.asInstanceOf[ActorSystemImpl].printTree)
if (verifySystemShutdown) throw new RuntimeException(msg)
else actorSystem.log.warning(msg)
}
}
} }
/** /**

View file

@ -0,0 +1,72 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.testkit;
import akka.actor.ActorSystem;
import com.typesafe.config.Config;
import org.junit.rules.ExternalResource;
/**
* This is a resource for creating an actor system before test start and shut it down afterwards.
*
* To use it on a class level add this to your test class:
*
* @ClassRule
* public static AkkaJUnitActorSystemResource actorSystemResource =
* new AkkaJUnitActorSystemResource(name, config);
*
* private final ActorSystem system = actorSystemResource.getSystem();
*
*
* To use it on a per test level add this to your test class:
*
* @Rule
* public AkkaJUnitActorSystemResource actorSystemResource =
* new AkkaJUnitActorSystemResource(name, config);
*
* private final ActorSystem system = actorSystemResource.getSystem();
*/
public class AkkaJUnitActorSystemResource extends ExternalResource {
private ActorSystem system = null;
private final String name;
private final Config config;
private ActorSystem createSystem(String name, Config config) {
if (config == null)
return ActorSystem.create(name);
else
return ActorSystem.create(name, config);
}
public AkkaJUnitActorSystemResource(String name, Config config) {
this.name = name;
this.config = config;
system = createSystem(name, config);
}
public AkkaJUnitActorSystemResource(String name) {
this(name, AkkaSpec.testConf());
}
@Override
protected void before() throws Throwable {
// Sometimes the ExternalResource seems to be reused, and
// we don't run the constructor again, so if that's the case
// then create the system here
if (system == null) {
system = createSystem(name, config);
}
}
@Override
protected void after() {
JavaTestKit.shutdownActorSystem(system);
system = null;
}
public ActorSystem getSystem() {
return system;
}
}

View file

@ -73,12 +73,7 @@ abstract class AkkaSpec(_system: ActorSystem)
final override def afterAll { final override def afterAll {
beforeTermination() beforeTermination()
system.shutdown() shutdown(system)
try system.awaitTermination(5 seconds) catch {
case _: TimeoutException
system.log.warning("Failed to stop [{}] within 5 seconds", system.name)
println(system.asInstanceOf[ActorSystemImpl].printTree)
}
afterTermination() afterTermination()
stopCoroner() stopCoroner()
} }

View file

@ -28,7 +28,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers {
a ! 42 a ! 42
} }
} finally { } finally {
system.shutdown() TestKit.shutdownActorSystem(system)
} }
} }
@ -43,7 +43,7 @@ class AkkaSpecSpec extends WordSpec with MustMatchers {
val ref = Seq(testActor, system.actorOf(Props.empty, "name")) val ref = Seq(testActor, system.actorOf(Props.empty, "name"))
} }
spec.ref foreach (_.isTerminated must not be true) spec.ref foreach (_.isTerminated must not be true)
system.shutdown() TestKit.shutdownActorSystem(system)
spec.awaitCond(spec.ref forall (_.isTerminated), 2 seconds) spec.awaitCond(spec.ref forall (_.isTerminated), 2 seconds)
} }
@ -86,15 +86,15 @@ class AkkaSpecSpec extends WordSpec with MustMatchers {
val latch = new TestLatch(1)(system) val latch = new TestLatch(1)(system)
system.registerOnTermination(latch.countDown()) system.registerOnTermination(latch.countDown())
system.shutdown() TestKit.shutdownActorSystem(system)
Await.ready(latch, 2 seconds) Await.ready(latch, 2 seconds)
Await.result(davyJones ? "Die!", timeout.duration) must be === "finally gone" Await.result(davyJones ? "Die!", timeout.duration) must be === "finally gone"
// this will typically also contain log messages which were sent after the logger shutdown // this will typically also contain log messages which were sent after the logger shutdown
locker must contain(DeadLetter(42, davyJones, probe.ref)) locker must contain(DeadLetter(42, davyJones, probe.ref))
} finally { } finally {
system.shutdown() TestKit.shutdownActorSystem(system)
otherSystem.shutdown() TestKit.shutdownActorSystem(otherSystem)
} }
} }
} }

View file

@ -6,10 +6,8 @@ package akka.transactor;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.junit.AfterClass; import akka.testkit.*;
import org.junit.BeforeClass; import org.junit.*;
import org.junit.Test;
import org.junit.Before;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.actor.ActorRef; import akka.actor.ActorRef;
@ -17,10 +15,7 @@ import akka.actor.Props;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.Future; import scala.concurrent.Future;
import static akka.pattern.Patterns.ask; import static akka.pattern.Patterns.ask;
import akka.testkit.AkkaSpec;
import akka.testkit.EventFilter;
import akka.testkit.ErrorFilter;
import akka.testkit.TestEvent;
import akka.util.Timeout; import akka.util.Timeout;
import static akka.japi.Util.immutableSeq; import static akka.japi.Util.immutableSeq;
@ -33,18 +28,12 @@ import scala.collection.JavaConverters;
import scala.collection.immutable.Seq; import scala.collection.immutable.Seq;
public class UntypedCoordinatedIncrementTest { public class UntypedCoordinatedIncrementTest {
private static ActorSystem system;
@BeforeClass @ClassRule
public static void beforeAll() { public static AkkaJUnitActorSystemResource actorSystemResource =
system = ActorSystem.create("UntypedCoordinatedIncrementTest", AkkaSpec.testConf()); new AkkaJUnitActorSystemResource("UntypedCoordinatedIncrementTest", AkkaSpec.testConf());
}
@AfterClass private final ActorSystem system = actorSystemResource.getSystem();
public static void afterAll() {
system.shutdown();
system = null;
}
List<ActorRef> counters; List<ActorRef> counters;
ActorRef failer; ActorRef failer;

View file

@ -6,10 +6,8 @@ package akka.transactor;
import static org.junit.Assert.*; import static org.junit.Assert.*;
import org.junit.AfterClass; import akka.testkit.*;
import org.junit.BeforeClass; import org.junit.*;
import org.junit.Test;
import org.junit.Before;
import akka.actor.ActorSystem; import akka.actor.ActorSystem;
import akka.actor.ActorRef; import akka.actor.ActorRef;
@ -17,10 +15,7 @@ import akka.actor.Props;
import scala.concurrent.Await; import scala.concurrent.Await;
import scala.concurrent.Future; import scala.concurrent.Future;
import static akka.pattern.Patterns.ask; import static akka.pattern.Patterns.ask;
import akka.testkit.AkkaSpec;
import akka.testkit.EventFilter;
import akka.testkit.ErrorFilter;
import akka.testkit.TestEvent;
import akka.util.Timeout; import akka.util.Timeout;
import static akka.japi.Util.immutableSeq; import static akka.japi.Util.immutableSeq;
@ -34,18 +29,11 @@ import scala.collection.immutable.Seq;
public class UntypedTransactorTest { public class UntypedTransactorTest {
private static ActorSystem system; @ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("UntypedTransactorTest", AkkaSpec.testConf());
@BeforeClass private final ActorSystem system = actorSystemResource.getSystem();
public static void beforeAll() {
system = ActorSystem.create("UntypedTransactorTest", AkkaSpec.testConf());
}
@AfterClass
public static void afterAll() {
system.shutdown();
system = null;
}
List<ActorRef> counters; List<ActorRef> counters;
ActorRef failer; ActorRef failer;