Typed Java testkit, #24551

* Java specific testkit
* Separate all the APIs
* Dilate the default timeouts
* TestKit -> ActorTestKit to be consistent with BehaviorTestKit
* Only have stuff in packages expected for this module
* Auto testkit-system-naming that works (verified)
* Separate actually working apis for manual timer
* More docs
* Separate apis for BehaviorTestKit effects
* Moved the example tests into the testkit, added samples for JUnit/ScalaTest
* TestKitJunitResource: No need to explicitly provide class, also overload for just custom config
This commit is contained in:
Johan Andrén 2018-02-22 12:25:04 +01:00 committed by Patrik Nordwall
parent d6000df367
commit ede094a2c6
95 changed files with 1974 additions and 973 deletions

View file

@ -1,54 +0,0 @@
/**
* Copyright (C) 2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package akka.actor.typed;
//#manual-scheduling-simple
import java.util.concurrent.TimeUnit;
import static com.typesafe.config.ConfigFactory.parseString;
import scala.concurrent.duration.Duration;
import akka.actor.typed.javadsl.Behaviors;
import org.junit.Test;
import akka.testkit.typed.TestKit;
import akka.testkit.typed.javadsl.ExplicitlyTriggeredScheduler;
import akka.testkit.typed.javadsl.TestProbe;
public class ManualTimerTest extends TestKit {
ExplicitlyTriggeredScheduler scheduler;
public ManualTimerTest() {
super(parseString("akka.scheduler.implementation = \"akka.testkit.typed.javadsl.ExplicitlyTriggeredScheduler\""));
this.scheduler = (ExplicitlyTriggeredScheduler) system().scheduler();
}
static final class Tick {}
static final class Tock {}
@Test
public void testScheduleNonRepeatedTicks() {
TestProbe<Tock> probe = TestProbe.create(system());
Behavior<Tick> behavior = Behaviors.withTimers(timer -> {
timer.startSingleTimer("T", new Tick(), Duration.create(10, TimeUnit.MILLISECONDS));
return Behaviors.immutable( (ctx, tick) -> {
probe.ref().tell(new Tock());
return Behaviors.same();
});
});
spawn(behavior);
scheduler.expectNoMessageFor(Duration.create(9, TimeUnit.MILLISECONDS), probe);
scheduler.timePasses(Duration.create(2, TimeUnit.MILLISECONDS));
probe.expectMessageClass(Tock.class);
scheduler.expectNoMessageFor(Duration.create(10, TimeUnit.SECONDS), probe);
}
}
//#manual-scheduling-simple

View file

@ -3,11 +3,11 @@
*/
package akka.actor.typed.javadsl;
import akka.actor.ActorSystem;
import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.AkkaSpec;
import akka.testkit.typed.javadsl.TestKitJunitResource;
import akka.testkit.typed.javadsl.TestProbe;
import akka.util.Timeout;
import org.junit.ClassRule;
@ -19,10 +19,7 @@ import java.util.concurrent.TimeUnit;
public class ActorContextAskTest extends JUnitSuite {
@ClassRule
public static AkkaJUnitActorSystemResource actorSystemResource = new AkkaJUnitActorSystemResource("ActorSelectionTest",
AkkaSpec.testConf());
private final ActorSystem system = actorSystemResource.getSystem();
public static final TestKitJunitResource testKit = new TestKitJunitResource(AkkaSpec.testConf());
static class Ping {
final ActorRef<Pong> respondTo;
@ -39,10 +36,8 @@ public class ActorContextAskTest extends JUnitSuite {
return Behaviors.same();
});
final ActorRef<Ping> pingPong = Adapter.spawnAnonymous(system, pingPongBehavior);
final TestProbe<Object> probe = TestProbe.create(Adapter.toTyped(system));
final ActorRef<Ping> pingPong = testKit.spawn(pingPongBehavior);
final TestProbe<Object> probe = testKit.createTestProbe();
final Behavior<Object> snitch = Behaviors.setup((ActorContext<Object> ctx) -> {
ctx.ask(Pong.class,
@ -60,7 +55,7 @@ public class ActorContextAskTest extends JUnitSuite {
});
});
Adapter.spawnAnonymous(system, snitch);
testKit.spawn(snitch);
probe.expectMessageClass(Pong.class);
}

View file

@ -7,9 +7,9 @@ import akka.actor.typed.Behavior;
import akka.event.Logging;
import akka.japi.pf.PFBuilder;
import akka.testkit.CustomEventFilter;
import akka.testkit.typed.TestKit;
import akka.testkit.typed.javadsl.TestKitJunitResource;
import com.typesafe.config.ConfigFactory;
import org.junit.AfterClass;
import org.junit.ClassRule;
import org.junit.Test;
import org.scalatest.junit.JUnitSuite;
import scala.concurrent.duration.FiniteDuration;
@ -20,17 +20,13 @@ import java.util.concurrent.TimeUnit;
public class ActorLoggingTest extends JUnitSuite {
private final static TestKit testKit = new TestKit("ActorLoggingTest",
@ClassRule
public static final TestKitJunitResource testKit = new TestKitJunitResource(
ConfigFactory.parseString(
"akka.loglevel = INFO\n" +
"akka.loggers = [\"akka.testkit.TestEventListener\"]"
));
@AfterClass
public static void tearDown() {
testKit.shutdown();
}
interface Protocol {
String getTransactionId();
}

View file

@ -7,6 +7,9 @@ import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit;
import akka.Done;
import akka.testkit.AkkaSpec;
import akka.testkit.typed.javadsl.TestKitJunitResource;
import org.junit.ClassRule;
import org.scalatest.junit.JUnitSuite;
import scala.concurrent.Await;
import scala.concurrent.duration.Duration;
@ -14,19 +17,29 @@ import akka.util.Timeout;
import org.junit.Test;
import akka.actor.typed.*;
import static akka.actor.typed.javadsl.Behaviors.*;
public class WatchTest extends JUnitSuite {
static interface Message {}
static final class RunTest<T> implements Message {
private final ActorRef<T> replyTo;
public RunTest(ActorRef<T> replyTo) {
@ClassRule
public static TestKitJunitResource testKit = new TestKitJunitResource();
interface Message { }
static final class RunTest implements Message {
private final ActorRef<Done> replyTo;
public RunTest(ActorRef<Done> replyTo) {
this.replyTo = replyTo;
}
}
static final class Stop {}
static final class CustomTerminationMessage implements Message {}
static final class Stop {
}
static final class CustomTerminationMessage implements Message {
}
// final FiniteDuration fiveSeconds = FiniteDuration.create(5, TimeUnit.SECONDS);
final Timeout timeout = new Timeout(Duration.create(5, TimeUnit.SECONDS));
@ -36,7 +49,7 @@ public class WatchTest extends JUnitSuite {
return stopped();
});
private Behavior<RunTest<Done>> waitingForTermination(ActorRef<Done> replyWhenTerminated) {
private Behavior<RunTest> waitingForTermination(ActorRef<Done> replyWhenTerminated) {
return immutable(
(ctx, msg) -> unhandled(),
(ctx, sig) -> {
@ -63,43 +76,34 @@ public class WatchTest extends JUnitSuite {
@Test
public void shouldWatchTerminatingActor() throws Exception {
Behavior<RunTest<Done>> root = immutable((ctx, msg) -> {
Behavior<RunTest> exiting = Behaviors.immutable(RunTest.class)
.onMessage(RunTest.class, (ctx, msg) -> {
ActorRef<Stop> watched = ctx.spawn(exitingActor, "exitingActor");
ctx.watch(watched);
watched.tell(new Stop());
return waitingForTermination(msg.replyTo);
});
ActorSystem<RunTest<Done>> system = ActorSystem.create(root, "sysname");
try {
// Not sure why this does not compile without an explicit cast?
// system.tell(new RunTest());
CompletionStage<Done> result = AskPattern.ask((ActorRef<RunTest<Done>>)system, (ActorRef<Done> ref) -> new RunTest<Done>(ref), timeout, system.scheduler());
}).build();
ActorRef<RunTest> exitingRef = testKit.spawn(exiting);
CompletionStage<Done> result = AskPattern.ask(exitingRef, RunTest::new, timeout, testKit.scheduler());
result.toCompletableFuture().get(3, TimeUnit.SECONDS);
} finally {
Await.ready(system.terminate(), Duration.create(10, TimeUnit.SECONDS));
}
}
@Test
public void shouldWatchWithCustomMessage() throws Exception {
Behavior<Message> root = immutable((ctx, msg) -> {
if (msg instanceof RunTest) {
Behavior<Message> exiting = Behaviors.immutable(Message.class)
.onMessage(RunTest.class, (ctx, msg) -> {
ActorRef<Stop> watched = ctx.spawn(exitingActor, "exitingActor");
ctx.watchWith(watched, new CustomTerminationMessage());
watched.tell(new Stop());
return waitingForMessage(((RunTest<Done>) msg).replyTo);
} else {
return unhandled();
}
});
ActorSystem<Message> system = ActorSystem.create(root, "sysname");
try {
return waitingForMessage(msg.replyTo);
}).build();
ActorRef<Message> exitingRef = testKit.spawn(exiting);
// Not sure why this does not compile without an explicit cast?
// system.tell(new RunTest());
CompletionStage<Done> result = AskPattern.ask((ActorRef<Message>)system, (ActorRef<Done> ref) -> new RunTest<Done>(ref), timeout, system.scheduler());
CompletionStage<Done> result = AskPattern.ask(exitingRef, RunTest::new, timeout, testKit.scheduler());
result.toCompletableFuture().get(3, TimeUnit.SECONDS);
} finally {
Await.ready(system.terminate(), Duration.create(10, TimeUnit.SECONDS));
}
}
}

View file

@ -11,8 +11,8 @@ import akka.Done;
import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior;
import akka.actor.typed.javadsl.Behaviors;
import akka.testkit.typed.BehaviorTestkit;
import akka.testkit.typed.TestInbox;
import akka.testkit.typed.javadsl.TestInbox;
import akka.testkit.typed.javadsl.BehaviorTestKit;
import org.junit.Test;
import org.scalatest.junit.JUnitSuite;
@ -178,23 +178,23 @@ public class StashDocTest extends JUnitSuite {
}
};
final DataAccess dataAccess = new DataAccess("17", db);
BehaviorTestkit<DataAccess.Command> testKit = BehaviorTestkit.create(dataAccess.behavior());
TestInbox<String> getInbox = TestInbox.apply("getInbox");
testKit.run(new DataAccess.Get(getInbox.ref()));
DataAccess.Command initialStateMsg = testKit.selfInbox().receiveMsg();
BehaviorTestKit<DataAccess.Command> testKit = BehaviorTestKit.create(dataAccess.behavior());
TestInbox<String> getInbox = TestInbox.create("getInbox");
testKit.run(new DataAccess.Get(getInbox.getRef()));
DataAccess.Command initialStateMsg = testKit.selfInbox().receiveMessage();
testKit.run(initialStateMsg);
getInbox.expectMsg("TheValue");
getInbox.expectMessage("TheValue");
TestInbox<Done> saveInbox = TestInbox.apply("saveInbox");
testKit.run(new DataAccess.Save("UpdatedValue", saveInbox.ref()));
testKit.run(new DataAccess.Get(getInbox.ref()));
DataAccess.Command saveSuccessMsg = testKit.selfInbox().receiveMsg();
TestInbox<Done> saveInbox = TestInbox.create("saveInbox");
testKit.run(new DataAccess.Save("UpdatedValue", saveInbox.getRef()));
testKit.run(new DataAccess.Get(getInbox.getRef()));
DataAccess.Command saveSuccessMsg = testKit.selfInbox().receiveMessage();
testKit.run(saveSuccessMsg);
saveInbox.expectMsg(Done.getInstance());
getInbox.expectMsg("UpdatedValue");
saveInbox.expectMessage(Done.getInstance());
getInbox.expectMessage("UpdatedValue");
testKit.run(new DataAccess.Get(getInbox.ref()));
getInbox.expectMsg("UpdatedValue");
testKit.run(new DataAccess.Get(getInbox.getRef()));
getInbox.expectMessage("UpdatedValue");
}
}

View file

@ -4,10 +4,11 @@
package akka.actor.typed
import akka.actor.typed.scaladsl.Behaviors._
import akka.actor.typed.scaladsl.{ Behaviors, AskPattern }
import akka.actor.typed.scaladsl.{ AskPattern, Behaviors }
import akka.actor.{ ActorInitializationException, DeadLetterSuppression, InvalidMessageException }
import akka.testkit.AkkaSpec
import akka.testkit.TestEvent.Mute
import akka.util.Timeout
import com.typesafe.config.ConfigFactory
import org.scalactic.CanEqual
@ -317,6 +318,8 @@ object ActorContextSpec {
abstract class ActorContextSpec extends TypedAkkaSpec {
import ActorContextSpec._
implicit val timeout = Timeout(3.seconds)
val config = ConfigFactory.parseString(
"""
akka {

View file

@ -9,8 +9,7 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.Behaviors._
import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.util.Timeout
import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.ScalaFutures
@ -25,9 +24,10 @@ object AskSpec {
final case class Stop(replyTo: ActorRef[Unit]) extends Msg
}
class AskSpec
extends TestKit("AskSpec", ConfigFactory.parseString("akka.loggers = [ akka.testkit.TestEventListener ]"))
with TypedAkkaSpec with ScalaFutures {
class AskSpec extends ActorTestKit
with TypedAkkaSpecWithShutdown with ScalaFutures {
override def name = "AskSpec"
override def config = ConfigFactory.parseString("akka.loggers = [ akka.testkit.TestEventListener ]")
import AskSpec._

View file

@ -4,13 +4,13 @@
package akka.actor.typed
import akka.actor.typed.scaladsl.{ Behaviors SActor }
import akka.actor.typed.javadsl.{ Behaviors JActor, ActorContext JActorContext }
import akka.actor.typed.javadsl.{ ActorContext JActorContext, Behaviors JActor }
import akka.japi.function.{ Function F1e, Function2 F2, Procedure2 P2 }
import akka.japi.pf.{ FI, PFBuilder }
import java.util.function.{ Function F1 }
import akka.Done
import akka.testkit.typed.{ BehaviorTestkit, TestInbox }
import akka.testkit.typed.scaladsl.{ BehaviorTestKit, TestInbox }
object BehaviorSpec {
sealed trait Command {
@ -69,12 +69,12 @@ object BehaviorSpec {
case class Init(behv: Behavior[Command], inbox: TestInbox[Event], aux: Aux) {
def mkCtx(): Setup = {
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
inbox.receiveAll()
Setup(testkit, inbox, aux)
}
}
case class Setup(testKit: BehaviorTestkit[Command], inbox: TestInbox[Event], aux: Aux)
case class Setup(testKit: BehaviorTestKit[Command], inbox: TestInbox[Event], aux: Aux)
def init(): Init = {
val inbox = TestInbox[Event]("evt")

View file

@ -3,13 +3,13 @@
*/
package akka.actor.typed
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.Behaviors.BehaviorDecorators
import akka.testkit.typed.{ BehaviorTestkit, TestInbox, TestKit, TestKitSettings }
import akka.testkit.typed.TestKitSettings
import akka.testkit.typed.scaladsl._
import scala.util.control.NoStackTrace
object DeferredSpec {
sealed trait Command
case object Ping extends Command
@ -26,7 +26,7 @@ object DeferredSpec {
})
}
class DeferredSpec extends TestKit with TypedAkkaSpec {
class DeferredSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import DeferredSpec._
implicit val testSettings = TestKitSettings(system)
@ -135,9 +135,9 @@ class DeferredStubbedSpec extends TypedAkkaSpec {
inbox.ref ! Started
target(inbox.ref)
}
BehaviorTestkit(behv)
BehaviorTestKit(behv)
// it's supposed to be created immediately (not waiting for first message)
inbox.receiveMsg() should ===(Started)
inbox.receiveMessage() should ===(Started)
}
"must stop when exception from factory" in {
@ -148,9 +148,9 @@ class DeferredStubbedSpec extends TypedAkkaSpec {
throw exc
}
intercept[RuntimeException] {
BehaviorTestkit(behv)
BehaviorTestKit(behv)
} should ===(exc)
inbox.receiveMsg() should ===(Started)
inbox.receiveMessage() should ===(Started)
}
}

View file

@ -77,63 +77,63 @@ class StubbedSupervisionSpec extends WordSpec with Matchers {
import SupervisionSpec._
def mkTestkit(behv: Behavior[Command]): BehaviorTestkit[Command] =
BehaviorTestkit(behv)
def mkTestkit(behv: Behavior[Command]): BehaviorTestKit[Command] =
BehaviorTestKit(behv)
"A restarter (stubbed)" must {
"receive message" in {
val inbox = TestInbox[Event]("evt")
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Throwable](SupervisorStrategy.restart)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
testkit.run(Ping)
inbox.receiveMsg() should ===(Pong)
inbox.receiveMessage() should ===(Pong)
}
"stop when no supervise" in {
val inbox = TestInbox[Event]("evt")
val behv = targetBehavior(inbox.ref)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
intercept[Exc3] {
testkit.run(Throw(new Exc3))
}
inbox.receiveMsg() should ===(GotSignal(PostStop))
inbox.receiveMessage() should ===(GotSignal(PostStop))
}
"stop when unhandled exception" in {
val inbox = TestInbox[Event]("evt")
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](SupervisorStrategy.restart)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
intercept[Exc3] {
testkit.run(Throw(new Exc3))
}
inbox.receiveMsg() should ===(GotSignal(PostStop))
inbox.receiveMessage() should ===(GotSignal(PostStop))
}
"restart when handled exception" in {
val inbox = TestInbox[Event]("evt")
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](SupervisorStrategy.restart)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
testkit.run(IncrementState)
testkit.run(GetState)
inbox.receiveMsg() should ===(State(1, Map.empty))
inbox.receiveMessage() should ===(State(1, Map.empty))
testkit.run(Throw(new Exc2))
inbox.receiveMsg() should ===(GotSignal(PreRestart))
inbox.receiveMessage() should ===(GotSignal(PreRestart))
testkit.run(GetState)
inbox.receiveMsg() should ===(State(0, Map.empty))
inbox.receiveMessage() should ===(State(0, Map.empty))
}
"resume when handled exception" in {
val inbox = TestInbox[Event]("evt")
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](SupervisorStrategy.resume)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
testkit.run(IncrementState)
testkit.run(GetState)
inbox.receiveMsg() should ===(State(1, Map.empty))
inbox.receiveMessage() should ===(State(1, Map.empty))
testkit.run(Throw(new Exc2))
testkit.run(GetState)
inbox.receiveMsg() should ===(State(1, Map.empty))
inbox.receiveMessage() should ===(State(1, Map.empty))
}
"support nesting to handle different exceptions" in {
@ -144,34 +144,34 @@ class StubbedSupervisionSpec extends WordSpec with Matchers {
targetBehavior(inbox.ref)
).onFailure[Exc2](SupervisorStrategy.resume)
).onFailure[Exc3](SupervisorStrategy.restart)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
testkit.run(IncrementState)
testkit.run(GetState)
inbox.receiveMsg() should ===(State(1, Map.empty))
inbox.receiveMessage() should ===(State(1, Map.empty))
// resume
testkit.run(Throw(new Exc2))
testkit.run(GetState)
inbox.receiveMsg() should ===(State(1, Map.empty))
inbox.receiveMessage() should ===(State(1, Map.empty))
// restart
testkit.run(Throw(new Exc3))
inbox.receiveMsg() should ===(GotSignal(PreRestart))
inbox.receiveMessage() should ===(GotSignal(PreRestart))
testkit.run(GetState)
inbox.receiveMsg() should ===(State(0, Map.empty))
inbox.receiveMessage() should ===(State(0, Map.empty))
// stop
intercept[Exc1] {
testkit.run(Throw(new Exc1))
}
inbox.receiveMsg() should ===(GotSignal(PostStop))
inbox.receiveMessage() should ===(GotSignal(PostStop))
}
"not catch fatal error" in {
val inbox = TestInbox[Event]()
val behv = Behaviors.supervise(targetBehavior(inbox.ref))
.onFailure[Throwable](SupervisorStrategy.restart)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
intercept[StackOverflowError] {
testkit.run(Throw(new StackOverflowError))
}
@ -182,15 +182,15 @@ class StubbedSupervisionSpec extends WordSpec with Matchers {
val inbox = TestInbox[Event]("evt")
val strategy = SupervisorStrategy.restartWithLimit(maxNrOfRetries = 2, withinTimeRange = 1.minute)
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](strategy)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
testkit.run(Throw(new Exc1))
inbox.receiveMsg() should ===(GotSignal(PreRestart))
inbox.receiveMessage() should ===(GotSignal(PreRestart))
testkit.run(Throw(new Exc1))
inbox.receiveMsg() should ===(GotSignal(PreRestart))
inbox.receiveMessage() should ===(GotSignal(PreRestart))
intercept[Exc1] {
testkit.run(Throw(new Exc1))
}
inbox.receiveMsg() should ===(GotSignal(PostStop))
inbox.receiveMessage() should ===(GotSignal(PostStop))
}
"reset retry limit after withinTimeRange" in {
@ -198,21 +198,21 @@ class StubbedSupervisionSpec extends WordSpec with Matchers {
val withinTimeRange = 2.seconds
val strategy = SupervisorStrategy.restartWithLimit(maxNrOfRetries = 2, withinTimeRange)
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](strategy)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
testkit.run(Throw(new Exc1))
inbox.receiveMsg() should ===(GotSignal(PreRestart))
inbox.receiveMessage() should ===(GotSignal(PreRestart))
testkit.run(Throw(new Exc1))
inbox.receiveMsg() should ===(GotSignal(PreRestart))
inbox.receiveMessage() should ===(GotSignal(PreRestart))
Thread.sleep((2.seconds + 100.millis).toMillis)
testkit.run(Throw(new Exc1))
inbox.receiveMsg() should ===(GotSignal(PreRestart))
inbox.receiveMessage() should ===(GotSignal(PreRestart))
testkit.run(Throw(new Exc1))
inbox.receiveMsg() should ===(GotSignal(PreRestart))
inbox.receiveMessage() should ===(GotSignal(PreRestart))
intercept[Exc1] {
testkit.run(Throw(new Exc1))
}
inbox.receiveMsg() should ===(GotSignal(PostStop))
inbox.receiveMessage() should ===(GotSignal(PostStop))
}
"stop at first exception when restart retries limit is 0" in {
@ -220,11 +220,11 @@ class StubbedSupervisionSpec extends WordSpec with Matchers {
val strategy = SupervisorStrategy.restartWithLimit(maxNrOfRetries = 0, withinTimeRange = 1.minute)
val behv = supervise(targetBehavior(inbox.ref))
.onFailure[Exc1](strategy)
val testkit = BehaviorTestkit(behv)
val testkit = BehaviorTestKit(behv)
intercept[Exc1] {
testkit.run(Throw(new Exc1))
}
inbox.receiveMsg() should ===(GotSignal(PostStop))
inbox.receiveMessage() should ===(GotSignal(PostStop))
}
"create underlying deferred behavior immediately" in {
@ -235,15 +235,17 @@ class StubbedSupervisionSpec extends WordSpec with Matchers {
}).onFailure[Exc1](SupervisorStrategy.restart)
mkTestkit(behv)
// it's supposed to be created immediately (not waiting for first message)
inbox.receiveMsg() should ===(Started)
inbox.receiveMessage() should ===(Started)
}
}
}
class SupervisionSpec extends TestKit("SupervisionSpec", ConfigFactory.parseString(
class SupervisionSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
override def config = ConfigFactory.parseString(
"""
akka.loggers = [akka.testkit.TestEventListener]
""")) with TypedAkkaSpecWithShutdown {
""")
import SupervisionSpec._
private val nameCounter = Iterator.from(0)

View file

@ -13,12 +13,10 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.TimerScheduler
import akka.testkit.TimingTest
import akka.testkit.typed.TestKitSettings
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl._
import akka.testkit.typed.scaladsl.{ ActorTestKit, _ }
import org.scalatest.WordSpecLike
class TimerSpec extends TestKit("TimerSpec")
with WordSpecLike {
class TimerSpec extends ActorTestKit with WordSpecLike with TypedAkkaSpecWithShutdown {
sealed trait Command
case class Tick(n: Int) extends Command
@ -37,8 +35,6 @@ class TimerSpec extends TestKit("TimerSpec")
class Exc extends RuntimeException("simulated exc") with NoStackTrace
implicit val testSettings = TestKitSettings(system)
val interval = 1.second
def target(monitor: ActorRef[Event], timer: TimerScheduler[Command], bumpCount: Int): Behavior[Command] = {

View file

@ -1,7 +1,6 @@
package akka.actor.typed
import akka.testkit.typed.{ TestInbox, TestKit }
import akka.util.Timeout
import akka.testkit.typed.scaladsl.{ TestInbox, ActorTestKit }
import org.scalactic.TypeCheckedTripleEquals
import org.scalatest.concurrent.{ Eventually, ScalaFutures }
import org.scalatest.time.Span
@ -17,7 +16,6 @@ trait TypedAkkaSpec extends WordSpecLike with Matchers with BeforeAndAfterAll wi
with TypeCheckedTripleEquals with Eventually {
implicit override val patienceConfig: PatienceConfig = PatienceConfig(3.seconds, Span(100, org.scalatest.time.Millis))
implicit val timeout = Timeout(3.seconds)
def assertEmpty(inboxes: TestInbox[_]*): Unit = {
inboxes foreach (i withClue(s"inbox $i had messages")(i.hasMessages should be(false)))
@ -26,11 +24,11 @@ trait TypedAkkaSpec extends WordSpecLike with Matchers with BeforeAndAfterAll wi
}
/**
* Helper that also shuts down the actor system if using [[TestKit]]
* Helper that also shuts down the actor system if using [[ActorTestKit]]
*/
trait TypedAkkaSpecWithShutdown extends TypedAkkaSpec {
self: TestKit
override protected def afterAll(): Unit = shutdown()
self: ActorTestKit
override protected def afterAll(): Unit = shutdownTestKit()
}
class TestException(msg: String) extends RuntimeException(msg) with NoStackTrace

View file

@ -7,11 +7,10 @@ import akka.Done
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import scala.concurrent._
import scala.concurrent.duration._
import akka.testkit.typed.TestKit
import com.typesafe.config.ConfigFactory
object WatchSpec {
@ -32,8 +31,10 @@ object WatchSpec {
case class StartWatchingWith(watchee: ActorRef[Stop.type], msg: CustomTerminationMessage) extends Message
}
class WatchSpec extends TestKit("WordSpec", WatchSpec.config)
class WatchSpec extends ActorTestKit
with TypedAkkaSpecWithShutdown {
override def config = WatchSpec.config
implicit def untypedSystem = system.toUntyped
import WatchSpec._

View file

@ -7,7 +7,7 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.{ ActorRef, TypedAkkaSpecWithShutdown }
import akka.serialization.{ JavaSerializer, SerializationExtension }
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.ActorTestKit
import com.typesafe.config.ConfigFactory
object ActorRefSerializationSpec {
@ -24,7 +24,9 @@ object ActorRefSerializationSpec {
case class MessageWrappingActorRef(s: String, ref: ActorRef[Unit]) extends java.io.Serializable
}
class ActorRefSerializationSpec extends TestKit(ActorRefSerializationSpec.config) with TypedAkkaSpecWithShutdown {
class ActorRefSerializationSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
override def config = ActorRefSerializationSpec.config
val serialization = SerializationExtension(system.toUntyped)

View file

@ -7,7 +7,7 @@ package internal
import akka.Done
import akka.actor.InvalidMessageException
import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.TestInbox
import akka.testkit.typed.scaladsl.TestInbox
import org.scalatest._
import org.scalatest.concurrent.{ Eventually, ScalaFutures }

View file

@ -5,15 +5,15 @@ package akka.actor.typed.receptionist
import akka.actor.typed._
import akka.actor.typed.receptionist.Receptionist._
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.AskPattern._
import akka.testkit.typed.{ BehaviorTestkit, TestInbox, TestKit, TestKitSettings }
import akka.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.TestKitSettings
import akka.testkit.typed.scaladsl.{ BehaviorTestKit, TestInbox, ActorTestKit, TestProbe }
import org.scalatest.concurrent.Eventually
import scala.concurrent.Future
class LocalReceptionistSpec extends TestKit with TypedAkkaSpecWithShutdown with Eventually {
class LocalReceptionistSpec extends ActorTestKit with TypedAkkaSpecWithShutdown with Eventually {
trait ServiceA
val ServiceKeyA = ServiceKey[ServiceA]("service-a")
@ -42,50 +42,50 @@ class LocalReceptionistSpec extends TestKit with TypedAkkaSpecWithShutdown with
"A local receptionist" must {
"register a service" in {
val testkit = BehaviorTestkit(receptionistBehavior)
val testkit = BehaviorTestKit(receptionistBehavior)
val a = TestInbox[ServiceA]("a")
val r = TestInbox[Registered[_]]("r")
testkit.run(Register(ServiceKeyA, a.ref)(r.ref))
testkit.retrieveEffect() // watching however that is implemented
r.receiveMsg() should be(Registered(ServiceKeyA, a.ref))
r.receiveMessage() should be(Registered(ServiceKeyA, a.ref))
val q = TestInbox[Listing[ServiceA]]("q")
testkit.run(Find(ServiceKeyA)(q.ref))
testkit.retrieveAllEffects() should be(Nil)
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref)))
q.receiveMessage() should be(Listing(ServiceKeyA, Set(a.ref)))
assertEmpty(a, r, q)
}
"register two services" in {
val testkit = BehaviorTestkit(receptionistBehavior)
val testkit = BehaviorTestKit(receptionistBehavior)
val a = TestInbox[ServiceA]("a")
val r = TestInbox[Registered[_]]("r")
testkit.run(Register(ServiceKeyA, a.ref)(r.ref))
r.receiveMsg() should be(Registered(ServiceKeyA, a.ref))
r.receiveMessage() should be(Registered(ServiceKeyA, a.ref))
val b = TestInbox[ServiceB]("b")
testkit.run(Register(ServiceKeyB, b.ref)(r.ref))
r.receiveMsg() should be(Registered(ServiceKeyB, b.ref))
r.receiveMessage() should be(Registered(ServiceKeyB, b.ref))
val q = TestInbox[Listing[_]]("q")
testkit.run(Find(ServiceKeyA)(q.ref))
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref)))
q.receiveMessage() should be(Listing(ServiceKeyA, Set(a.ref)))
testkit.run(Find(ServiceKeyB)(q.ref))
q.receiveMsg() should be(Listing(ServiceKeyB, Set(b.ref)))
q.receiveMessage() should be(Listing(ServiceKeyB, Set(b.ref)))
assertEmpty(a, b, r, q)
}
"register two services with the same key" in {
val testkit = BehaviorTestkit(receptionistBehavior)
val testkit = BehaviorTestKit(receptionistBehavior)
val a1 = TestInbox[ServiceA]("a1")
val r = TestInbox[Registered[_]]("r")
testkit.run(Register(ServiceKeyA, a1.ref)(r.ref))
r.receiveMsg() should be(Registered(ServiceKeyA, a1.ref))
r.receiveMessage() should be(Registered(ServiceKeyA, a1.ref))
val a2 = TestInbox[ServiceA]("a2")
testkit.run(Register(ServiceKeyA, a2.ref)(r.ref))
r.receiveMsg() should be(Registered(ServiceKeyA, a2.ref))
r.receiveMessage() should be(Registered(ServiceKeyA, a2.ref))
val q = TestInbox[Listing[_]]("q")
testkit.run(Find(ServiceKeyA)(q.ref))
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a1.ref, a2.ref)))
q.receiveMessage() should be(Listing(ServiceKeyA, Set(a1.ref, a2.ref)))
testkit.run(Find(ServiceKeyB)(q.ref))
q.receiveMsg() should be(Listing(ServiceKeyB, Set.empty[ActorRef[ServiceB]]))
q.receiveMessage() should be(Listing(ServiceKeyB, Set.empty[ActorRef[ServiceB]]))
assertEmpty(a1, a2, r, q)
}

View file

@ -8,9 +8,11 @@ import akka.actor.typed.internal.ActorRefSerializationSpec
import akka.actor.typed.internal.receptionist.ServiceKeySerializer
import akka.actor.typed.scaladsl.adapter._
import akka.serialization.SerializationExtension
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.ActorTestKit
class ServiceKeySerializationSpec extends TestKit(ActorRefSerializationSpec.config) with TypedAkkaSpecWithShutdown {
class ServiceKeySerializationSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
override def config = ActorRefSerializationSpec.config
val serialization = SerializationExtension(system.toUntyped)

View file

@ -6,8 +6,7 @@ package akka.actor.typed.scaladsl
import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.{ ActorRef, PostStop, Props, TypedAkkaSpecWithShutdown }
import akka.testkit.EventFilter
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory
import scala.concurrent.TimeoutException
@ -30,7 +29,9 @@ object ActorContextAskSpec {
""")
}
class ActorContextAskSpec extends TestKit(ActorContextAskSpec.config) with TypedAkkaSpecWithShutdown {
class ActorContextAskSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
override def config = ActorContextAskSpec.config
implicit val untyped = system.toUntyped // FIXME no typed event filter yet

View file

@ -5,19 +5,21 @@ package akka.actor.typed.scaladsl
import akka.actor.typed.{ LogMarker, TestException, TypedAkkaSpec, scaladsl }
import akka.testkit.EventFilter
import akka.testkit.typed.TestKit
import com.typesafe.config.ConfigFactory
import akka.actor.typed.scaladsl.adapter._
import akka.event.Logging
import akka.event.Logging.{ LogEventWithCause, LogEventWithMarker }
import akka.testkit.typed.scaladsl.ActorTestKit
import scala.util.control.NoStackTrace
class ActorLoggingSpec extends TestKit(ConfigFactory.parseString(
class ActorLoggingSpec extends ActorTestKit with TypedAkkaSpec {
override def config = ConfigFactory.parseString(
"""
akka.loglevel = DEBUG
akka.loggers = ["akka.testkit.TestEventListener"]
""")) with TypedAkkaSpec {
""")
val marker = LogMarker("marker")
val cause = new TestException("böö")

View file

@ -6,10 +6,9 @@ package scaladsl
import akka.Done
import akka.NotUsed
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
final class GracefulStopSpec extends TestKit with TypedAkkaSpecWithShutdown {
final class GracefulStopSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
"Graceful stop" must {

View file

@ -4,10 +4,9 @@
package akka.actor.typed
package scaladsl
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
class ImmutablePartialSpec extends TestKit with TypedAkkaSpecWithShutdown {
class ImmutablePartialSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
"An immutable partial" must {

View file

@ -8,7 +8,6 @@ import scala.concurrent.duration._
import scala.reflect.ClassTag
import scala.util.Failure
import scala.util.Success
import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.ActorRef
import akka.actor.typed.PostStop
@ -16,8 +15,7 @@ import akka.actor.typed.Props
import akka.actor.typed.TestException
import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.testkit.EventFilter
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory
object MessageAdapterSpec {
@ -36,8 +34,9 @@ object MessageAdapterSpec {
""")
}
class MessageAdapterSpec extends TestKit(MessageAdapterSpec.config) with TypedAkkaSpecWithShutdown {
class MessageAdapterSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
override def config = MessageAdapterSpec.config
implicit val untyped = system.toUntyped // FIXME no typed event filter yet
"Message adapters" must {

View file

@ -5,10 +5,9 @@ package akka.actor.typed
package scaladsl
import akka.Done
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
final class OnSignalSpec extends TestKit with TypedAkkaSpecWithShutdown {
final class OnSignalSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
"An Actor.OnSignal behavior" must {
"must correctly install the signal handler" in {

View file

@ -4,10 +4,9 @@
package akka.actor.typed.scaladsl
import akka.actor.typed.Behavior
import akka.testkit.typed.EffectfulActorContext
import akka.testkit.typed.TestInbox
import org.scalatest.Matchers
import org.scalatest.WordSpec
import akka.testkit.typed.internal.EffectfulActorContext
import akka.testkit.typed.scaladsl.TestInbox
import org.scalatest.{ Matchers, WordSpec }
class StashBufferSpec extends WordSpec with Matchers {
@ -95,7 +94,7 @@ class StashBufferSpec extends WordSpec with Matchers {
}
buffer.unstashAll(ctx, behavior(""))
valueInbox.expectMsg("m1m2m3")
valueInbox.expectMessage("m1m2m3")
buffer.isEmpty should ===(true)
}
@ -118,7 +117,7 @@ class StashBufferSpec extends WordSpec with Matchers {
}
buffer.unstashAll(ctx, behavior(""))
valueInbox.expectMsg("m1m2m3")
valueInbox.expectMessage("m1m2m3")
buffer.isEmpty should ===(true)
}
@ -147,7 +146,7 @@ class StashBufferSpec extends WordSpec with Matchers {
// the call is made, not unstash new messages added to the buffer while
// unstashing.
val b2 = buffer.unstashAll(ctx, behavior(""))
valueInbox.expectMsg("m1m3")
valueInbox.expectMessage("m1m3")
buffer.size should ===(1)
buffer.head should ===("m2")

View file

@ -4,8 +4,7 @@
package akka.actor.typed
package scaladsl
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
object StashSpec {
sealed trait Command
@ -186,7 +185,7 @@ class MutableStashSpec extends StashSpec {
def behaviorUnderTest: Behavior[Command] = Behaviors.mutable(ctx new MutableStash(ctx))
}
abstract class StashSpec extends TestKit with TypedAkkaSpecWithShutdown {
abstract class StashSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import StashSpec._
def testQualifier: String

View file

@ -5,14 +5,16 @@ package docs.akka.typed
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ DeathPactException, SupervisorStrategy, TypedAkkaSpecWithShutdown }
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.ActorTestKit
import com.typesafe.config.ConfigFactory
class FaultToleranceDocSpec extends TestKit(ConfigFactory.parseString(
class FaultToleranceDocSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
override def config = ConfigFactory.parseString(
"""
# silenced to not put noise in test logs
akka.loglevel = OFF
""")) with TypedAkkaSpecWithShutdown {
""")
"Bubbling of failures" must {

View file

@ -6,13 +6,13 @@ package docs.akka.typed
//#imports
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorSystem, Logger, PostStop }
import akka.testkit.typed.scaladsl.ActorTestKit
import scala.concurrent.Await
import scala.concurrent.duration._
//#imports
import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.testkit.typed.TestKit
object GracefulStopDocSpec {
@ -67,7 +67,7 @@ object GracefulStopDocSpec {
}
class GracefulStopDocSpec extends TestKit with TypedAkkaSpecWithShutdown {
class GracefulStopDocSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import GracefulStopDocSpec._

View file

@ -8,15 +8,14 @@ import java.net.URI
import akka.NotUsed
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, TypedAkkaSpecWithShutdown }
import akka.actor.typed.scaladsl.{ ActorContext, Behaviors, TimerScheduler }
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.util.Timeout
import scala.concurrent.Future
import scala.concurrent.duration._
import scala.util.{ Failure, Success }
class InteractionPatternsSpec extends TestKit with TypedAkkaSpecWithShutdown {
class InteractionPatternsSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
"The interaction patterns docs" must {
@ -387,7 +386,7 @@ class InteractionPatternsSpec extends TestKit with TypedAkkaSpecWithShutdown {
val result: Future[Cookies] = cookieActorRef ? (ref GiveMeCookies(ref))
// the response callback will be executed on this execution context
import system.executionContext
implicit val ec = system.executionContext
result.onComplete {
case Success(cookies) println("Yay, cookies!")

View file

@ -11,7 +11,7 @@ import akka.NotUsed
import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Terminated }
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.ActorTestKit
import scala.concurrent.duration._
import scala.concurrent.{ Await, Future }
@ -100,7 +100,7 @@ object IntroSpec {
}
class IntroSpec extends TestKit with TypedAkkaSpecWithShutdown {
class IntroSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import IntroSpec._

View file

@ -10,7 +10,8 @@ import java.nio.charset.StandardCharsets
import akka.actor.typed._
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.ActorContext
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.ActorTestKit
import scala.concurrent.duration._
import scala.concurrent.Await
//#imports
@ -87,7 +88,7 @@ object MutableIntroSpec {
}
class MutableIntroSpec extends TestKit with TypedAkkaSpecWithShutdown {
class MutableIntroSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import MutableIntroSpec._

View file

@ -6,13 +6,11 @@ package docs.akka.typed
import scala.concurrent.Future
import scala.util.Failure
import scala.util.Success
import akka.Done
import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.BehaviorTestkit
import akka.testkit.typed.TestInbox
import akka.testkit.typed.scaladsl.{ BehaviorTestKit, TestInbox }
import org.scalatest.Matchers
import org.scalatest.WordSpec
@ -109,23 +107,23 @@ class StashDocSpec extends WordSpec with Matchers {
override def save(id: String, value: String): Future[Done] = Future.successful(Done)
override def load(id: String): Future[String] = Future.successful("TheValue")
}
val testKit = BehaviorTestkit(DataAccess.behavior(id = "17", db))
val testKit = BehaviorTestKit(DataAccess.behavior(id = "17", db))
val getInbox = TestInbox[String]()
testKit.run(DataAccess.Get(getInbox.ref))
val initialStateMsg = testKit.selfInbox().receiveMsg()
val initialStateMsg = testKit.selfInbox().receiveMessage()
testKit.run(initialStateMsg)
getInbox.expectMsg("TheValue")
getInbox.expectMessage("TheValue")
val saveInbox = TestInbox[Done]()
testKit.run(DataAccess.Save("UpdatedValue", saveInbox.ref))
testKit.run(DataAccess.Get(getInbox.ref))
val saveSuccessMsg = testKit.selfInbox().receiveMsg()
val saveSuccessMsg = testKit.selfInbox().receiveMessage()
testKit.run(saveSuccessMsg)
saveInbox.expectMsg(Done)
getInbox.expectMsg("UpdatedValue")
saveInbox.expectMessage(Done)
getInbox.expectMessage("UpdatedValue")
testKit.run(DataAccess.Get(getInbox.ref))
getInbox.expectMsg("UpdatedValue")
getInbox.expectMessage("UpdatedValue")
}
}

View file

@ -7,9 +7,9 @@ import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.actor.typed.internal.adapter.ActorSystemAdapter
import akka.cluster.sharding.typed.internal.ShardingSerializer
import akka.serialization.SerializationExtension
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.ActorTestKit
class ShardingSerializerSpec extends TestKit with TypedAkkaSpecWithShutdown {
class ShardingSerializerSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
"The typed ShardingSerializer" must {

View file

@ -11,8 +11,7 @@ import akka.cluster.sharding.typed.ClusterShardingSettings
import akka.cluster.typed.Cluster
import akka.cluster.typed.Join
import akka.persistence.typed.scaladsl.PersistentBehaviors
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory
object ClusterShardingPersistenceSpec {
@ -61,10 +60,12 @@ object ClusterShardingPersistenceSpec {
}
class ClusterShardingPersistenceSpec extends TestKit("ClusterShardingPersistenceSPec", ClusterShardingPersistenceSpec.config)
class ClusterShardingPersistenceSpec extends ActorTestKit
with TypedAkkaSpecWithShutdown {
import ClusterShardingPersistenceSpec._
override def config = ClusterShardingPersistenceSpec.config
val sharding = ClusterSharding(system)
"Typed cluster sharding with persistent actor" must {

View file

@ -21,8 +21,7 @@ import akka.cluster.typed.Cluster
import akka.cluster.typed.Join
import akka.cluster.typed.Leave
import akka.serialization.SerializerWithStringManifest
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.util.Timeout
import com.typesafe.config.ConfigFactory
import org.scalatest.time.Span
@ -120,16 +119,18 @@ object ClusterShardingSpec {
}
class ClusterShardingSpec extends TestKit("ClusterShardingSpec", ClusterShardingSpec.config) with TypedAkkaSpecWithShutdown {
class ClusterShardingSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import ClusterShardingSpec._
override def config = ClusterShardingSpec.config
val sharding = ClusterSharding(system)
val system2 = ActorSystem(Behaviors.ignore[Any], name = system.name, config = system.settings.config)
val sharding2 = ClusterSharding(system2)
override def afterAll(): Unit = {
TestKit.shutdown(system2, 5.seconds)
ActorTestKit.shutdown(system2, 5.seconds)
super.afterAll()
}

View file

@ -12,7 +12,7 @@ import akka.cluster.Cluster
import akka.cluster.ddata.typed.scaladsl.Replicator._
import akka.cluster.ddata.{ GCounter, GCounterKey, ReplicatedData }
import akka.testkit.typed.scaladsl._
import akka.testkit.typed.{ TestKit, TestKitSettings }
import akka.testkit.typed.TestKitSettings
import akka.util.Timeout
import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.Eventually
@ -116,7 +116,9 @@ object ReplicatorSpec {
}
class ReplicatorSpec extends TestKit(ReplicatorSpec.config) with TypedAkkaSpecWithShutdown with Eventually {
class ReplicatorSpec extends ActorTestKit with TypedAkkaSpecWithShutdown with Eventually {
override def config = ReplicatorSpec.config
import ReplicatorSpec._

View file

@ -4,20 +4,15 @@
package akka.cluster.typed
import akka.actor.InvalidMessageException
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.{ PostStop, Terminated }
import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.TestInbox
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, PostStop, Terminated }
import akka.testkit.typed.scaladsl.TestInbox
import com.typesafe.config.ConfigFactory
import org.scalatest._
import org.scalatest.concurrent.Eventually
import org.scalatest.concurrent.ScalaFutures
import org.scalatest.concurrent.{ Eventually, ScalaFutures }
import scala.concurrent.{ Future, Promise }
import scala.concurrent.duration._
import scala.concurrent.Future
import scala.concurrent.Promise
import scala.util.control.NonFatal
class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll

View file

@ -7,8 +7,8 @@ import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.actor.typed.scaladsl.adapter._
import akka.cluster.ClusterEvent._
import akka.cluster.MemberStatus
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.{ TestKit, TestKitSettings }
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.testkit.typed.TestKitSettings
import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.ScalaFutures
@ -32,7 +32,9 @@ object ClusterApiSpec {
""")
}
class ClusterApiSpec extends TestKit("ClusterApiSpec", ClusterApiSpec.config) with TypedAkkaSpecWithShutdown with ScalaFutures {
class ClusterApiSpec extends ActorTestKit with TypedAkkaSpecWithShutdown with ScalaFutures {
override def config = ClusterApiSpec.config
val testSettings = TestKitSettings(system)
val clusterNode1 = Cluster(system)

View file

@ -8,8 +8,8 @@ import java.nio.charset.StandardCharsets
import akka.actor.ExtendedActorSystem
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.testkit.typed.{ TestKit, TestKitSettings }
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.TestKitSettings
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.actor.typed.{ ActorRef, ActorRefResolver, Props, TypedAkkaSpecWithShutdown }
import akka.serialization.SerializerWithStringManifest
import com.typesafe.config.ConfigFactory
@ -83,9 +83,11 @@ object ClusterSingletonApiSpec {
}
}
class ClusterSingletonApiSpec extends TestKit("ClusterSingletonApiSpec", ClusterSingletonApiSpec.config) with TypedAkkaSpecWithShutdown {
class ClusterSingletonApiSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import ClusterSingletonApiSpec._
override def config = ClusterSingletonApiSpec.config
implicit val testSettings = TestKitSettings(system)
val clusterNode1 = Cluster(system)
val untypedSystem1 = system.toUntyped

View file

@ -7,8 +7,7 @@ package akka.cluster.typed
import akka.actor.typed.{ ActorRef, Behavior, Props, TypedAkkaSpecWithShutdown }
import akka.persistence.typed.scaladsl.PersistentBehaviors
import akka.persistence.typed.scaladsl.PersistentBehaviors.{ CommandHandler, Effect }
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory
object ClusterSingletonPersistenceSpec {
@ -51,10 +50,12 @@ object ClusterSingletonPersistenceSpec {
}
class ClusterSingletonPersistenceSpec extends TestKit(ClusterSingletonPersistenceSpec.config) with TypedAkkaSpecWithShutdown {
class ClusterSingletonPersistenceSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import ClusterSingletonPersistenceSpec._
import akka.actor.typed.scaladsl.adapter._
override def config = ClusterSingletonPersistenceSpec.config
implicit val s = system
implicit val untypedSystem = system.toUntyped

View file

@ -11,8 +11,7 @@ import akka.actor.typed.receptionist.{ Receptionist, ServiceKey }
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, ActorRefResolver, ActorSystem, TypedAkkaSpecWithShutdown }
import akka.serialization.SerializerWithStringManifest
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import akka.actor.typed.scaladsl.adapter._
import akka.util.Timeout
import com.typesafe.config.ConfigFactory
@ -83,10 +82,12 @@ object RemoteContextAskSpec {
}
class RemoteContextAskSpec extends TestKit(RemoteContextAskSpec.config) with TypedAkkaSpecWithShutdown {
class RemoteContextAskSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import RemoteContextAskSpec._
override def config = RemoteContextAskSpec.config
"Asking another actor through the ActorContext across remoting" must {
"work" in {

View file

@ -6,9 +6,9 @@ package akka.cluster.typed
import akka.actor.typed.ActorSystem
import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
object RemoteDeployNotAllowedSpec {
@ -42,7 +42,9 @@ object RemoteDeployNotAllowedSpec {
""").withFallback(config)
}
class RemoteDeployNotAllowedSpec extends TestKit(RemoteDeployNotAllowedSpec.config) with TypedAkkaSpecWithShutdown {
class RemoteDeployNotAllowedSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
override def config = RemoteDeployNotAllowedSpec.config
"Typed cluster" must {
@ -93,7 +95,7 @@ class RemoteDeployNotAllowedSpec extends TestKit(RemoteDeployNotAllowedSpec.conf
system2 ! SpawnAnonymous
probe.expectMessageType[Exception].getMessage should ===("Remote deployment not allowed for typed actors")
} finally {
TestKit.shutdown(system2, 5.seconds)
ActorTestKit.shutdown(system2, 5.seconds)
}
}
}

View file

@ -13,8 +13,8 @@ import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.cluster.Cluster
import akka.serialization.SerializerWithStringManifest
import akka.testkit.typed.{ TestKit, TestKitSettings }
import akka.testkit.typed.scaladsl.TestProbe
import akka.testkit.typed.TestKitSettings
import akka.testkit.typed.scaladsl.{ ActorTestKit, TestProbe }
import com.typesafe.config.ConfigFactory
import scala.concurrent.Await
@ -85,9 +85,11 @@ object ClusterReceptionistSpec {
val PingKey = ServiceKey[PingProtocol]("pingy")
}
class ClusterReceptionistSpec extends TestKit("ClusterReceptionistSpec", ClusterReceptionistSpec.config)
class ClusterReceptionistSpec extends ActorTestKit
with TypedAkkaSpecWithShutdown {
override def config = ClusterReceptionistSpec.config
import ClusterReceptionistSpec._
implicit val testSettings = TestKitSettings(system)

View file

@ -41,27 +41,27 @@ The following demonstrates how to test:
The examples below require the following imports:
Scala
: @@snip [BasicSyncTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/sync/BasicSyncTestingSpec.scala) { #imports }
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #imports }
Java
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #imports }
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #imports }
Each of the tests are testing an actor that based on the message executes a different effect to be tested:
Scala
: @@snip [BasicSyncTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/sync/BasicSyncTestingSpec.scala) { #under-test }
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #under-test }
Java
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #under-test }
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #under-test }
For creating a child actor a noop actor is created:
Scala
: @@snip [BasicSyncTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/sync/BasicSyncTestingSpec.scala) { #child }
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #child }
Java
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #child }
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #child }
All of the tests make use of the `BehaviorTestkit` to avoid the need for a real `ActorContext`. Some of the tests
make use of the `TestInbox` which allows the creation of an `ActorRef` that can be used for synchronous testing, similar to the
@ -73,18 +73,18 @@ make use of the `TestInbox` which allows the creation of an `ActorRef` that can
With a name:
Scala
: @@snip [BasicSyncTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/sync/BasicSyncTestingSpec.scala) { #test-child }
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-child }
Java
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #test-child }
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-child }
Anonymously:
Scala
: @@snip [BasicSyncTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/sync/BasicSyncTestingSpec.scala) { #test-anonymous-child }
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-anonymous-child }
Java
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #test-anonymous-child }
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-anonymous-child }
### Sending messages
@ -92,27 +92,27 @@ For testing sending a message a `TestInbox` is created that provides an `ActorRe
messages that have been sent to it.
Scala
: @@snip [BasicSyncTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/sync/BasicSyncTestingSpec.scala) { #test-message }
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-message }
Java
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #test-message }
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-message }
Another use case is sending a message to a child actor you can do this by looking up the 'TestInbox' for
a child actor from the 'BehaviorTestKit':
Scala
: @@snip [BasicSyncTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/sync/BasicSyncTestingSpec.scala) { #test-child-message }
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-child-message }
Java
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #test-child-message }
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-child-message }
For anonymous children the actor names are generated in a deterministic way:
Scala
: @@snip [BasicSyncTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/sync/BasicSyncTestingSpec.scala) { #test-child-message-anonymous }
: @@snip [SyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-child-message-anonymous }
Java
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #test-child-message-anonymous }
: @@snip [SyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-child-message-anonymous }
### Testing other effects
@ -140,29 +140,30 @@ the same in that a single procedure drives the test.
Actor under test:
Scala
: @@snip [BasicAsyncTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/async/BasicAsyncTestingSpec.scala) { #under-test }
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #under-test }
Java
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.java) { #under-test }
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #under-test }
@scala[Tests extend `ActorTestKit`. This provides access to]@java[Tests create an instance of `ActorTestKit`. This provides access to]
Tests extend `TestKit` or include the `TestKitBase`. This provides access to
* An ActorSystem
* Methods for spawning Actors. These are created under the root guardian
* Methods for creating system actors
* A hook to shut down the ActorSystem from the test suite
Scala
: @@snip [BasicTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/async/BasicAsyncTestingSpec.scala) { #test-header }
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-header }
Java
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.java) { #test-header }
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-header }
Your test is responsible for shutting down the `ActorSystem` e.g. using `BeforeAndAfterAll` when using ScalaTest
Scala
: @@snip [BasicTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/async/BasicAsyncTestingSpec.scala) { #test-shutdown }
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-shutdown }
Java
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.java) { #test-shutdown }
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-shutdown }
The following demonstrates:
@ -171,18 +172,43 @@ The following demonstrates:
* Verifying that the actor under test responds via the `TestProbe`
Scala
: @@snip [BasicTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/async/BasicAsyncTestingSpec.scala) { #test-spawn }
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-spawn }
Java
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.java) { #test-spawn }
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-spawn }
Actors can also be spawned anonymously:
Scala
: @@snip [BasicTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/async/BasicAsyncTestingSpec.scala) { #test-spawn-anonymous }
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AsyncTestingExampleSpec.scala) { #test-spawn-anonymous }
Java
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.java) { #test-spawn-anonymous }
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/AsyncTestingExampleTest.java) { #test-spawn-anonymous }
### Test framework integration
@@@ div { .group-java }
If you are using JUnit you can use `akka.testkit.typed.javadsl.TestKitJunitResource` to have the async test kit automatically
shutdown when the test is complete.
Note that the dependency on JUnit is marked as optional from the test kit module, so your project must explicitly include
a dependency on JUnit to use this.
@@@
@@@ div { .group-scala }
It often makes sense to introduce a common base class for all tests using the async test kit, here is an example how to
hook it into a ScalaTest test suite.
@@@
Scala
: @@snip [AsyncTestingExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/AbstractActorSpec.scala) { #scalatest-glue }
Java
: @@snip [AsyncTestingExampleTest.java]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/JunitIntegrationExampleTest.java) { #junit-integration }
### Controlling the scheduler
@ -193,7 +219,7 @@ Making such tests more reliable by using generous timeouts make the tests take a
For such situations, we provide a scheduler where you can manually, explicitly advance the clock.
Scala
: @@snip [ManualTimerSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/akka/actor/typed/ManualTimerSpec.scala) { #manual-scheduling-simple }
: @@snip [ManualTimerExampleSpec.scala]($akka$/akka-testkit-typed/src/test/scala/akka/testkit/typed/scaladsl/ManualTimerExampleSpec.scala) { #manual-scheduling-simple }
Java
: @@snip [ManualTimerTest.scala]($akka$/akka-actor-typed-tests/src/test/java/akka/actor/typed/ManualTimerTest.java) { #manual-scheduling-simple }
: @@snip [ManualTimerExampleTest.scala]($akka$/akka-testkit-typed/src/test/java/akka/testkit/typed/javadsl/ManualTimerExampleTest.java) { #manual-scheduling-simple }

View file

@ -5,7 +5,7 @@ package akka.persistence.typed.javadsl;
import akka.actor.Scheduler;
import akka.actor.typed.ActorRef;
import akka.testkit.typed.TestInbox;
import akka.testkit.typed.javadsl.TestInbox;
import akka.util.Timeout;
import java.util.*;
@ -208,7 +208,7 @@ public class PersistentActorCompileOnlyTest {
}
}
static ActorRef<Request> sideEffectProcessor = new TestInbox<Request>().ref();
static ActorRef<Request> sideEffectProcessor = TestInbox.<Request>create().getRef();
static Timeout timeout = new Timeout(1, TimeUnit.SECONDS);
private static void performSideEffect(ActorRef<AcknowledgeSideEffect> sender, int correlationId, String data, Scheduler scheduler) {

View file

@ -8,8 +8,12 @@ import akka.actor.typed.javadsl.Behaviors;
import akka.japi.Pair;
import akka.japi.function.Function3;
import akka.persistence.typed.scaladsl.PersistentActorSpec;
import akka.testkit.typed.TestKit;
import akka.persistence.typed.scaladsl.PersistentActorSpec$;
import akka.testkit.AkkaJUnitActorSystemResource;
import akka.testkit.typed.javadsl.TestKitJunitResource;
import akka.testkit.typed.scaladsl.ActorTestKit;
import akka.testkit.typed.javadsl.TestProbe;
import org.junit.ClassRule;
import org.junit.Test;
import scala.concurrent.duration.FiniteDuration;
@ -19,14 +23,17 @@ import java.util.concurrent.TimeUnit;
import static java.util.Collections.singletonList;
import static org.junit.Assert.assertEquals;
public class PersistentActorTest extends TestKit {
public class PersistentActorTest {
@ClassRule
public static final TestKitJunitResource testKit = new TestKitJunitResource(PersistentActorSpec$.MODULE$.config());
static final Incremented timeoutEvent = new Incremented(100);
static final State emptyState = new State(0, Collections.emptyList());
static final Incremented terminatedEvent = new Incremented(10);
public PersistentActorTest() {
super(PersistentActorSpec.config());
super();
}
interface Command {
@ -145,14 +152,14 @@ public class PersistentActorTest extends TestKit {
private PersistentBehavior<Command, Incremented, State> counter(String persistenceId, ActorRef<Pair<State, Incremented>> probe) {
ActorRef<String> loggingProbe = TestProbe.create(String.class, system()).ref();
ActorRef<String> loggingProbe = TestProbe.create(String.class, testKit.system()).ref();
return counter(persistenceId, probe, loggingProbe, (s, i, l) -> false);
}
private PersistentBehavior<Command, Incremented, State> counter(String persistenceId) {
return counter(persistenceId,
TestProbe.<Pair<State, Incremented>>create(system()).ref(),
TestProbe.<String>create(system()).ref(),
TestProbe.<Pair<State, Incremented>>create(testKit.system()).ref(),
TestProbe.<String>create(testKit.system()).ref(),
(s, i, l) -> false);
}
@ -161,8 +168,8 @@ public class PersistentActorTest extends TestKit {
Function3<State, Incremented, Long, Boolean> snapshot
) {
return counter(persistenceId,
TestProbe.<Pair<State, Incremented>>create(system()).ref(),
TestProbe.<String>create(system()).ref(), snapshot);
testKit.<Pair<State, Incremented>>createTestProbe().ref(),
testKit.<String>createTestProbe().ref(), snapshot);
}
private PersistentBehavior<Command, Incremented, State> counter(
@ -176,7 +183,7 @@ public class PersistentActorTest extends TestKit {
String persistentId,
ActorRef<Pair<State, Incremented>> eventProbe,
Function3<State, Incremented, Long, Boolean> snapshot) {
return counter(persistentId, eventProbe, TestProbe.<String>create(system()).ref(), snapshot);
return counter(persistentId, eventProbe, testKit.<String>createTestProbe().ref(), snapshot);
}
private PersistentBehavior<Command, Incremented, State> counter(
@ -252,8 +259,8 @@ public class PersistentActorTest extends TestKit {
@Test
public void persistEvents() {
ActorRef<Command> c = spawn(counter("c2"));
TestProbe<State> probe = TestProbe.create(system());
ActorRef<Command> c = testKit.spawn(counter("c2"));
TestProbe<State> probe = testKit.createTestProbe();
c.tell(Increment.instance);
c.tell(new GetValue(probe.ref()));
probe.expectMessage(new State(1, singletonList(0)));
@ -261,15 +268,15 @@ public class PersistentActorTest extends TestKit {
@Test
public void replyStoredEvents() {
ActorRef<Command> c = spawn(counter("c2"));
TestProbe<State> probe = TestProbe.create(system());
ActorRef<Command> c = testKit.spawn(counter("c2"));
TestProbe<State> probe = testKit.createTestProbe();
c.tell(Increment.instance);
c.tell(Increment.instance);
c.tell(Increment.instance);
c.tell(new GetValue(probe.ref()));
probe.expectMessage(new State(3, Arrays.asList(0, 1, 2)));
ActorRef<Command> c2 = spawn(counter("c2"));
ActorRef<Command> c2 = testKit.spawn(counter("c2"));
c2.tell(new GetValue(probe.ref()));
probe.expectMessage(new State(3, Arrays.asList(0, 1, 2)));
c2.tell(Increment.instance);
@ -279,8 +286,8 @@ public class PersistentActorTest extends TestKit {
@Test
public void handleTerminatedSignal() {
TestProbe<Pair<State, Incremented>> eventHandlerProbe = TestProbe.create(system());
ActorRef<Command> c = spawn(counter("c2", eventHandlerProbe.ref()));
TestProbe<Pair<State, Incremented>> eventHandlerProbe = testKit.createTestProbe();
ActorRef<Command> c = testKit.spawn(counter("c2", eventHandlerProbe.ref()));
c.tell(Increment.instance);
c.tell(new IncrementLater());
eventHandlerProbe.expectMessage(Pair.create(emptyState, new Incremented(1)));
@ -289,17 +296,17 @@ public class PersistentActorTest extends TestKit {
@Test
public void handleReceiveTimeout() {
TestProbe<Pair<State, Incremented>> eventHandlerProbe = TestProbe.create(system());
ActorRef<Command> c = spawn(counter("c1", eventHandlerProbe.ref()));
TestProbe<Pair<State, Incremented>> eventHandlerProbe = testKit.createTestProbe();
ActorRef<Command> c = testKit.spawn(counter("c1", eventHandlerProbe.ref()));
c.tell(new Increment100OnTimeout());
eventHandlerProbe.expectMessage(Pair.create(emptyState, timeoutEvent));
}
@Test
public void chainableSideEffectsWithEvents() {
TestProbe<Pair<State, Incremented>> eventHandlerProbe = TestProbe.create(system());
TestProbe<String> loggingProbe = TestProbe.create(system());
ActorRef<Command> c = spawn(counter("c1", eventHandlerProbe.ref(), loggingProbe.ref()));
TestProbe<Pair<State, Incremented>> eventHandlerProbe = testKit.createTestProbe();
TestProbe<String> loggingProbe = testKit.createTestProbe();
ActorRef<Command> c = testKit.spawn(counter("c1", eventHandlerProbe.ref(), loggingProbe.ref()));
c.tell(new EmptyEventsListAndThenLog());
loggingProbe.expectMessage(loggingOne);
}
@ -307,17 +314,17 @@ public class PersistentActorTest extends TestKit {
@Test
public void snapshot() {
PersistentBehavior<Command, Incremented, State> snapshoter = counter("c11", (s, e, l) -> s.value % 2 == 0);
ActorRef<Command> c = spawn(snapshoter);
ActorRef<Command> c = testKit.spawn(snapshoter);
c.tell(Increment.instance);
c.tell(Increment.instance);
c.tell(Increment.instance);
TestProbe<State> probe = TestProbe.create(system());
TestProbe<State> probe = testKit.createTestProbe();
c.tell(new GetValue(probe.ref()));
probe.expectMessage(new State(3, Arrays.asList(0, 1, 2)));
TestProbe<Pair<State, Incremented>> eventProbe = TestProbe.create(system());
TestProbe<Pair<State, Incremented>> eventProbe = testKit.createTestProbe();
snapshoter = counter("c11", eventProbe.ref(), (s, e, l) -> s.value % 2 == 0);
ActorRef<Command> c2 = spawn(snapshoter);
ActorRef<Command> c2 = testKit.spawn(snapshoter);
// First 2 are snapshot
eventProbe.expectMessage(Pair.create(new State(2, Arrays.asList(0, 1)), new Incremented(1)));
c2.tell(new GetValue(probe.ref()));
@ -326,8 +333,8 @@ public class PersistentActorTest extends TestKit {
@Test
public void stopThenLog() {
TestProbe<State> probe = TestProbe.create(system());
ActorRef<Command> c = spawn(counter("c12"));
TestProbe<State> probe = testKit.createTestProbe();
ActorRef<Command> c = testKit.spawn(counter("c12"));
c.tell(new StopThenLog());
probe.expectTerminated(c, FiniteDuration.create(1, TimeUnit.SECONDS));
}

View file

@ -8,7 +8,7 @@ import akka.actor.typed.{ ActorRef, ActorSystem, SupervisorStrategy, Terminated,
import akka.persistence.snapshot.SnapshotStore
import akka.persistence.typed.scaladsl.PersistentBehaviors._
import akka.persistence.{ SelectedSnapshot, SnapshotMetadata, SnapshotSelectionCriteria }
import akka.testkit.typed.{ TestKit, TestKitSettings }
import akka.testkit.typed.TestKitSettings
import akka.testkit.typed.scaladsl._
import com.typesafe.config.ConfigFactory
import org.scalatest.concurrent.Eventually
@ -146,12 +146,12 @@ object PersistentActorSpec {
}
class PersistentActorSpec extends TestKit(PersistentActorSpec.config) with Eventually with TypedAkkaSpecWithShutdown {
class PersistentActorSpec extends ActorTestKit with Eventually with TypedAkkaSpecWithShutdown {
override def config = PersistentActorSpec.config
import PersistentActorSpec._
implicit val testSettings = TestKitSettings(system)
"A typed persistent actor" must {
"persist an event" in {

View file

@ -11,8 +11,7 @@ import akka.stream.scaladsl.Keep
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.stream.typed.ActorMaterializer
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl._
import akka.testkit.typed.scaladsl.{ ActorTestKit, _ }
object ActorSourceSinkSpec {
@ -23,7 +22,7 @@ object ActorSourceSinkSpec {
case object Failed extends AckProto
}
class ActorSourceSinkSpec extends TestKit with TypedAkkaSpecWithShutdown {
class ActorSourceSinkSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import ActorSourceSinkSpec._
implicit val mat = ActorMaterializer()

View file

@ -4,14 +4,13 @@
package akka.stream.typed.scaladsl
import scala.concurrent.Future
import akka.actor.typed.ActorRef
import akka.actor.typed.TypedAkkaSpecWithShutdown
import akka.actor.typed.scaladsl.Behaviors
import akka.stream.scaladsl.Sink
import akka.stream.scaladsl.Source
import akka.stream.typed.ActorMaterializer
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.ActorTestKit
object CustomGuardianAndMaterializerSpec {
@ -22,7 +21,7 @@ object CustomGuardianAndMaterializerSpec {
case object Failed extends GuardianProtocol
}
class CustomGuardianAndMaterializerSpec extends TestKit with TypedAkkaSpecWithShutdown {
class CustomGuardianAndMaterializerSpec extends ActorTestKit with TypedAkkaSpecWithShutdown {
import CustomGuardianAndMaterializerSpec._
val guardian = Behaviors.immutable[GuardianProtocol] {

View file

@ -18,7 +18,14 @@ akka.actor.typed.test {
# will be dilated by the timefactor.
expect-no-message-default = 100ms
# The timeout that is added as an implicit by DefaultTimeout trait
# The timeout that is added as an implicit by DefaultTimeout trait, will be dilated by the timefactor.
default-timeout = 5s
# Default timeout for shutting down the actor system (used when no explicit timeout specified),
# will be dilated by the timefactor.
system-shutdown-default=10s
# Throw an exception on shutdown if the timeout is hit, if false an error is printed to stdout instead.
throw-on-shutdown-timeout=true
}

View file

@ -1,220 +0,0 @@
/**
* Copyright (C) 2014-2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.testkit.typed
import java.util.concurrent.ConcurrentLinkedQueue
import akka.actor.typed.internal.ControlledExecutor
import akka.actor.typed.{ ActorRef, Behavior, PostStop, Props, Signal }
import akka.annotation.{ ApiMayChange, InternalApi }
import scala.annotation.tailrec
import scala.collection.immutable
import scala.concurrent.duration.{ Duration, FiniteDuration }
import scala.language.existentials
import scala.util.control.Exception.Catcher
import scala.util.control.NonFatal
/**
* All tracked effects must extend implement this type. It is deliberately
* not sealed in order to allow extensions.
*/
abstract class Effect
// TODO offer a better Java API for default params that are rarely used e.g. props
@ApiMayChange
object Effect {
abstract class SpawnedEffect extends Effect
@SerialVersionUID(1L) final case class Spawned(behavior: Behavior[_], childName: String, props: Props = Props.empty) extends SpawnedEffect
@SerialVersionUID(1L) final case class SpawnedAnonymous(behavior: Behavior[_], props: Props = Props.empty) extends SpawnedEffect
@SerialVersionUID(1L) final case object SpawnedAdapter extends SpawnedEffect
@SerialVersionUID(1L) final case class Stopped(childName: String) extends Effect
@SerialVersionUID(1L) final case class Watched[T](other: ActorRef[T]) extends Effect
@SerialVersionUID(1L) final case class Unwatched[T](other: ActorRef[T]) extends Effect
@SerialVersionUID(1L) final case class ReceiveTimeoutSet[T](d: Duration, msg: T) extends Effect
@SerialVersionUID(1L) final case class Scheduled[U](delay: FiniteDuration, target: ActorRef[U], msg: U) extends Effect
@SerialVersionUID(1L) case object NoEffects extends Effect
}
/**
* INTERNAL API
*/
@InternalApi private[akka] class EffectfulActorContext[T](name: String) extends StubbedActorContext[T](name) {
import Effect._
import akka.{ actor a }
/**
* INTERNAL API
*/
@InternalApi private[akka] val effectQueue = new ConcurrentLinkedQueue[Effect]
override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] = {
val ref = super.spawnAnonymous(behavior, props)
effectQueue.offer(SpawnedAnonymous(behavior, props))
ref
}
override def spawnMessageAdapter[U](f: U T): ActorRef[U] = {
spawnMessageAdapter(f, "")
}
override def spawnMessageAdapter[U](f: U T, name: String): ActorRef[U] = {
val ref = super.spawnMessageAdapter(f, name)
effectQueue.offer(SpawnedAdapter)
ref
}
override def spawn[U](behavior: Behavior[U], name: String, props: Props = Props.empty): ActorRef[U] = {
effectQueue.offer(Spawned(behavior, name, props))
super.spawn(behavior, name, props)
}
override def stop[U](child: ActorRef[U]): Unit = {
effectQueue.offer(Stopped(child.path.name))
super.stop(child)
}
override def watch[U](other: ActorRef[U]): Unit = {
effectQueue.offer(Watched(other))
super.watch(other)
}
override def unwatch[U](other: ActorRef[U]): Unit = {
effectQueue.offer(Unwatched(other))
super.unwatch(other)
}
override def setReceiveTimeout(d: FiniteDuration, msg: T): Unit = {
effectQueue.offer(ReceiveTimeoutSet(d, msg))
super.setReceiveTimeout(d, msg)
}
override def cancelReceiveTimeout(): Unit = {
effectQueue.offer(ReceiveTimeoutSet(Duration.Undefined, null))
super.cancelReceiveTimeout()
}
override def schedule[U](delay: FiniteDuration, target: ActorRef[U], msg: U): a.Cancellable = {
effectQueue.offer(Scheduled(delay, target, msg))
super.schedule(delay, target, msg)
}
}
@ApiMayChange
object BehaviorTestkit {
def apply[T](initialBehavior: Behavior[T], name: String): BehaviorTestkit[T] =
new BehaviorTestkit[T](name, initialBehavior)
def apply[T](initialBehavior: Behavior[T]): BehaviorTestkit[T] =
apply(initialBehavior, "testkit")
/**
* JAVA API
*/
def create[T](initialBehavior: Behavior[T], name: String): BehaviorTestkit[T] =
new BehaviorTestkit[T](name, initialBehavior)
/**
* JAVA API
*/
def create[T](initialBehavior: Behavior[T]): BehaviorTestkit[T] =
apply(initialBehavior, "testkit")
}
/**
* Used for testing [[Behavior]]s. Stores all effects e.g. Spawning of children,
* watching and offers access to what effects have taken place.
*/
@ApiMayChange
class BehaviorTestkit[T] private (_name: String, _initialBehavior: Behavior[T]) {
import Effect._
// really this should be private, make so when we port out tests that need it
private[akka] val ctx = new EffectfulActorContext[T](_name)
/**
* Requests the oldest [[Effect]] or [[NoEffects]] if no effects
* have taken place. The effect is consumed, subsequent calls won't
* will not include this effect.
*/
def retrieveEffect(): Effect = ctx.effectQueue.poll() match {
case null NoEffects
case x x
}
def childInbox[U](name: String): TestInbox[U] = {
val inbox = ctx.childInbox[U](name)
assert(inbox.isDefined, s"Child not created: $name. Children created: [${ctx.childrenNames.mkString(",")}]")
inbox.get
}
def selfInbox(): TestInbox[T] = ctx.selfInbox
/**
* Requests all the effects. The effects are consumed, subsequent calls will only
* see new effects.
*/
def retrieveAllEffects(): immutable.Seq[Effect] = {
@tailrec def rec(acc: List[Effect]): List[Effect] = ctx.effectQueue.poll() match {
case null acc.reverse
case x rec(x :: acc)
}
rec(Nil)
}
/**
* Asserts that the oldest effect is the expectedEffect. Removing it from
* further assertions.
*/
def expectEffect(expectedEffect: Effect): Unit = {
ctx.effectQueue.poll() match {
case null assert(assertion = false, s"expected: $expectedEffect but no effects were recorded")
case effect assert(expectedEffect == effect, s"expected: $expectedEffect but found $effect")
}
}
private var current = Behavior.validateAsInitial(Behavior.start(_initialBehavior, ctx))
private var currentUncanonical = _initialBehavior
def currentBehavior: Behavior[T] = current
/**
* Returns the current behavior as it was returned from processing the previous message.
* For example if [[Behavior.unhandled]] is returned it will be kept here, but not in
* [[currentBehavior]].
*/
def returnedBehavior: Behavior[T] = currentUncanonical
def isAlive: Boolean = Behavior.isAlive(current)
private def handleException: Catcher[Unit] = {
case NonFatal(e)
try Behavior.canonicalize(Behavior.interpretSignal(current, ctx, PostStop), current, ctx) // TODO why canonicalize here?
catch {
case NonFatal(_) /* ignore, real is logging */
}
throw e
}
/**
* Send the msg to the behavior and record any [[Effect]]s
*/
def run(msg: T): Unit = {
try {
currentUncanonical = Behavior.interpretMessage(current, ctx, msg)
current = Behavior.canonicalize(currentUncanonical, current, ctx)
ctx.executionContext match {
case controlled: ControlledExecutor controlled.runAll()
case _
}
} catch handleException
}
/**
* Send the signal to the beheavior and record any [[Effect]]s
*/
def signal(signal: Signal): Unit = {
try {
currentUncanonical = Behavior.interpretSignal(current, ctx, signal)
current = Behavior.canonicalize(currentUncanonical, current, ctx)
} catch handleException
}
}

View file

@ -0,0 +1,19 @@
/**
* Copyright (C) 2014-2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.testkit.typed
import akka.annotation.DoNotInherit
/**
* All tracked effects for the [[akka.testkit.typed.scaladsl.BehaviorTestKit]] and
* [[akka.testkit.typed.javadsl.BehaviorTestKit]] must extend this type.
*
* Factories/types for effects are available through [[akka.testkit.typed.scaladsl.Effects]]
* and [[akka.testkit.typed.javadsl.Effects]]
*
* Not for user extension
*/
@DoNotInherit
abstract class Effect private[akka] ()

View file

@ -18,5 +18,5 @@ object FishingOutcome {
case object Continue extends FishingOutcome
case object ContinueAndIgnore extends FishingOutcome
case object Complete extends FishingOutcome
case class Fail(error: String) extends FishingOutcome
final case class Fail(error: String) extends FishingOutcome
}

View file

@ -1,125 +0,0 @@
package akka.testkit.typed
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }
import akka.annotation.{ ApiMayChange, DoNotInherit }
import akka.testkit.typed.TestKit._
import akka.util.Timeout
import com.typesafe.config.Config
import scala.concurrent.duration._
import scala.concurrent.{ Await, TimeoutException }
import scala.util.control.NoStackTrace
/**
* Exception without stack trace to use for verifying exceptions in tests
*/
final case class TE(message: String) extends RuntimeException(message) with NoStackTrace
object TestKit {
private[akka] sealed trait TestKitCommand
private[akka] case class SpawnActor[T](name: String, behavior: Behavior[T], replyTo: ActorRef[ActorRef[T]], props: Props) extends TestKitCommand
private[akka] case class SpawnActorAnonymous[T](behavior: Behavior[T], replyTo: ActorRef[ActorRef[T]], props: Props) extends TestKitCommand
private val testKitGuardian = Behaviors.immutable[TestKitCommand] {
case (ctx, SpawnActor(name, behavior, reply, props))
reply ! ctx.spawn(behavior, name, props)
Behaviors.same
case (ctx, SpawnActorAnonymous(behavior, reply, props))
reply ! ctx.spawnAnonymous(behavior, props)
Behaviors.same
}
private def getCallerName(clazz: Class[_]): String = {
val s = (Thread.currentThread.getStackTrace map (_.getClassName) drop 1)
.dropWhile(_ matches "(java.lang.Thread|.*\\.Abstract.*)")
val reduced = s.lastIndexWhere(_ == clazz.getName) match {
case -1 s
case z s drop (z + 1)
}
reduced.head.replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_")
}
def shutdown(
system: ActorSystem[_],
duration: Duration,
verifySystemShutdown: Boolean = false): Unit = {
system.terminate()
try Await.ready(system.whenTerminated, duration) catch {
case _: TimeoutException
val msg = "Failed to stop [%s] within [%s] \n%s".format(system.name, duration,
system.printTree)
if (verifySystemShutdown) throw new RuntimeException(msg)
else println(msg)
}
}
}
/**
* Testkit for typed actors. Extending this removes some boiler plate when testing
* typed actors.
*
* If a test can't extend then use the [[TestKitBase]] trait
*/
@ApiMayChange
class TestKit(name: String, config: Option[Config]) extends TestKitBase {
def this() = this(TestKit.getCallerName(classOf[TestKit]), None)
def this(name: String) = this(name, None)
def this(config: Config) = this(TestKit.getCallerName(classOf[TestKit]), Some(config))
def this(name: String, config: Config) = this(name, Some(config))
import TestKit._
implicit val system: ActorSystem[TestKitCommand] = ActorSystem(testKitGuardian, name, config = config)
}
@ApiMayChange
trait TestKitBase {
def system: ActorSystem[TestKitCommand]
implicit def testkitSettings = TestKitSettings(system)
implicit def scheduler = system.scheduler
private val childName: Iterator[String] = Iterator.from(0).map(_.toString)
// FIXME testkit config
private val timeoutDuration = 5.seconds
implicit private val timeout = Timeout(timeoutDuration)
def shutdown(): Unit = {
TestKit.shutdown(system, timeoutDuration)
}
/**
* Spawn the given behavior. This is created as a child of the test kit
* guardian
*/
def spawn[T](behavior: Behavior[T]): ActorRef[T] =
spawn(behavior, Props.empty)
/**
* Spawn the given behavior. This is created as a child of the test kit
* guardian
*/
def spawn[T](behavior: Behavior[T], props: Props): ActorRef[T] =
Await.result(system ? (SpawnActorAnonymous(behavior, _, props)), timeoutDuration)
/**
* Spawn the given behavior. This is created as a child of the test kit
* guardian
*/
def spawn[T](behavior: Behavior[T], name: String): ActorRef[T] =
spawn(behavior, name, Props.empty)
/**
* Spawn the given behavior. This is created as a child of the test kit
* guardian
*/
def spawn[T](behavior: Behavior[T], name: String, props: Props): ActorRef[T] =
Await.result(system ? (SpawnActor(name, behavior, _, props)), timeoutDuration)
def systemActor[T](behavior: Behavior[T], name: String): ActorRef[T] =
Await.result(system.systemActorOf(behavior, name), timeoutDuration)
def systemActor[T](behavior: Behavior[T]): ActorRef[T] =
Await.result(system.systemActorOf(behavior, childName.next()), timeoutDuration)
}

View file

@ -4,16 +4,24 @@
package akka.testkit.typed
import com.typesafe.config.Config
import scala.concurrent.duration.FiniteDuration
import akka.util.Timeout
import akka.actor.typed.ActorSystem
import scala.util.control.NoStackTrace
/**
* Exception without stack trace to use for verifying exceptions in tests
*/
final case class TE(message: String) extends RuntimeException(message) with NoStackTrace
object TestKitSettings {
/**
* Reads configuration settings from `akka.actor.typed.test` section.
*/
def apply(system: ActorSystem[_]): TestKitSettings =
apply(system.settings.config)
apply(system.settings.config.getConfig("akka.actor.typed.test"))
/**
* Reads configuration settings from given `Config` that
@ -21,15 +29,41 @@ object TestKitSettings {
*/
def apply(config: Config): TestKitSettings =
new TestKitSettings(config)
/**
* Java API: Reads configuration settings from `akka.actor.typed.test` section.
*/
def create(system: ActorSystem[_]): TestKitSettings =
apply(system)
/**
* Reads configuration settings from given `Config` that
* must have the same layout as the `akka.actor.typed.test` section.
*/
def create(config: Config): TestKitSettings =
new TestKitSettings(config)
}
class TestKitSettings(val config: Config) {
final class TestKitSettings(val config: Config) {
import akka.util.Helpers._
val TestTimeFactor = config.getDouble("akka.actor.typed.test.timefactor").
requiring(tf !tf.isInfinite && tf > 0, "akka.actor.typed.test.timefactor must be positive finite double")
val SingleExpectDefaultTimeout: FiniteDuration = config.getMillisDuration("akka.actor.typed.test.single-expect-default")
val ExpectNoMessageDefaultTimeout: FiniteDuration = config.getMillisDuration("akka.actor.typed.test.expect-no-message-default")
val DefaultTimeout: Timeout = Timeout(config.getMillisDuration("akka.actor.typed.test.default-timeout"))
val TestTimeFactor = config.getDouble("timefactor").
requiring(tf !tf.isInfinite && tf > 0, "timefactor must be positive finite double")
/** dilated with `TestTimeFactor` */
val SingleExpectDefaultTimeout: FiniteDuration = dilated(config.getMillisDuration("single-expect-default"))
/** dilated with `TestTimeFactor` */
val ExpectNoMessageDefaultTimeout: FiniteDuration = dilated(config.getMillisDuration("expect-no-message-default"))
/** dilated with `TestTimeFactor` */
val DefaultTimeout: Timeout = Timeout(dilated(config.getMillisDuration("default-timeout")))
/** dilated with `TestTimeFactor` */
val DefaultActorSystemShutdownTimeout: FiniteDuration = dilated(config.getMillisDuration("system-shutdown-default"))
val ThrowOnShutdownTimeout: Boolean = config.getBoolean("throw-on-shutdown-timeout")
/**
* Scale the `duration` with the configured `TestTimeFactor`
*/
def dilated(duration: FiniteDuration): FiniteDuration = (duration * TestTimeFactor).asInstanceOf[FiniteDuration]
}

View file

@ -1,14 +1,13 @@
/**
* Copyright (C) 2016-2018 Lightbend Inc. <http://www.lightbend.com/>
*/
package akka.actor.typed
package internal
package akka.testkit.typed.internal
import java.util.concurrent.{ CompletionStage, ThreadFactory }
import akka.actor.typed.internal.ActorRefImpl
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, DispatcherSelector, Dispatchers, Extension, ExtensionId, Logger, Props, Settings, Terminated }
import akka.annotation.InternalApi
import akka.event.{ BusLogging, DefaultLoggingFilter, Logging }
import akka.testkit.typed.StubbedLogger
import akka.util.Timeout
import akka.{ actor a, event e }
import com.typesafe.config.ConfigFactory
@ -19,7 +18,7 @@ import scala.concurrent._
/**
* INTERNAL API
*/
@InternalApi private[akka] class ActorSystemStub(val name: String)
@InternalApi private[akka] final class ActorSystemStub(val name: String)
extends ActorSystem[Nothing] with ActorRef[Nothing] with ActorRefImpl[Nothing] {
override val path: a.ActorPath = a.RootActorPath(a.Address("akka", name)) / "user"

View file

@ -0,0 +1,97 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.internal
import java.util
import akka.actor.typed.{ Behavior, PostStop, Signal }
import akka.annotation.InternalApi
import akka.testkit.typed.Effect
import akka.testkit.typed.scaladsl.Effects._
import scala.annotation.tailrec
import scala.collection.JavaConverters._
import scala.collection.immutable
import scala.util.control.Exception.Catcher
import scala.util.control.NonFatal
/**
* INTERNAL API
*/
@InternalApi
private[akka] final class BehaviorTestKitImpl[T](_name: String, _initialBehavior: Behavior[T])
extends akka.testkit.typed.javadsl.BehaviorTestKit[T]
with akka.testkit.typed.scaladsl.BehaviorTestKit[T] {
// really this should be private, make so when we port out tests that need it
private[akka] val ctx = new EffectfulActorContext[T](_name)
private var currentUncanonical = _initialBehavior
private var current = Behavior.validateAsInitial(Behavior.start(_initialBehavior, ctx))
override def retrieveEffect(): Effect = ctx.effectQueue.poll() match {
case null NoEffects
case x x
}
override def childInbox[U](name: String): TestInboxImpl[U] = {
val inbox = ctx.childInbox[U](name)
assert(inbox.isDefined, s"Child not created: $name. Children created: [${ctx.childrenNames.mkString(",")}]")
inbox.get
}
override def selfInbox(): TestInboxImpl[T] = ctx.selfInbox
override def retrieveAllEffects(): immutable.Seq[Effect] = {
@tailrec def rec(acc: List[Effect]): List[Effect] = ctx.effectQueue.poll() match {
case null acc.reverse
case x rec(x :: acc)
}
rec(Nil)
}
def getEffect(): Effect = retrieveEffect()
def getAllEffects(): util.List[Effect] = retrieveAllEffects().asJava
override def expectEffect(expectedEffect: Effect): Unit = {
ctx.effectQueue.poll() match {
case null assert(assertion = false, s"expected: $expectedEffect but no effects were recorded")
case effect assert(expectedEffect == effect, s"expected: $expectedEffect but found $effect")
}
}
def returnedBehavior: Behavior[T] = currentUncanonical
def currentBehavior: Behavior[T] = current
def isAlive: Boolean = Behavior.isAlive(current)
private def handleException: Catcher[Unit] = {
case NonFatal(e)
try Behavior.canonicalize(Behavior.interpretSignal(current, ctx, PostStop), current, ctx) // TODO why canonicalize here?
catch {
case NonFatal(_) /* ignore, real is logging */
}
throw e
}
override def run(msg: T): Unit = {
try {
currentUncanonical = Behavior.interpretMessage(current, ctx, msg)
current = Behavior.canonicalize(currentUncanonical, current, ctx)
ctx.executionContext match {
case controlled: ControlledExecutor controlled.runAll()
case _
}
} catch handleException
}
override def signal(signal: Signal): Unit = {
try {
currentUncanonical = Behavior.interpretSignal(current, ctx, signal)
current = Behavior.canonicalize(currentUncanonical, current, ctx)
} catch handleException
}
}

View file

@ -1,13 +1,19 @@
/**
* Copyright (C) 2016-2018 Lightbend Inc. <http://www.lightbend.com/>
*/
package akka.actor.typed.internal
package akka.testkit.typed.internal
import java.util.LinkedList
import akka.annotation.InternalApi
import scala.concurrent.ExecutionContextExecutor
class ControlledExecutor extends ExecutionContextExecutor {
/**
* INTERNAL API
*/
@InternalApi
private[akka] final class ControlledExecutor extends ExecutionContextExecutor {
private val tasks = new LinkedList[Runnable]
def queueSize: Int = tasks.size()

View file

@ -1,11 +1,12 @@
/**
* Copyright (C) 2016-2018 Lightbend Inc. <http://www.lightbend.com/>
*/
package akka.actor.typed
package internal
package akka.testkit.typed.internal
import java.util.concurrent.ConcurrentLinkedQueue
import akka.actor.typed.ActorRef
import akka.actor.typed.internal.{ ActorRefImpl, SystemMessage }
import akka.annotation.InternalApi
import akka.{ actor a }
@ -14,7 +15,7 @@ import scala.annotation.tailrec
/**
* INTERNAL API
*/
@InternalApi private[akka] class DebugRef[T](override val path: a.ActorPath, override val isLocal: Boolean)
@InternalApi private[akka] final class DebugRef[T](override val path: a.ActorPath, override val isLocal: Boolean)
extends ActorRef[T] with ActorRefImpl[T] {
private val q = new ConcurrentLinkedQueue[Either[SystemMessage, T]]

View file

@ -0,0 +1,69 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.internal
import java.util.concurrent.ConcurrentLinkedQueue
import akka.actor.Cancellable
import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.annotation.InternalApi
import akka.testkit.typed.Effect
import akka.testkit.typed.scaladsl.Effects._
import scala.concurrent.duration.{ Duration, FiniteDuration }
/**
* INTERNAL API
*/
@InternalApi private[akka] final class EffectfulActorContext[T](name: String) extends StubbedActorContext[T](name) {
private[akka] val effectQueue = new ConcurrentLinkedQueue[Effect]
override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] = {
val ref = super.spawnAnonymous(behavior, props)
effectQueue.offer(SpawnedAnonymous(behavior, props))
ref
}
override def spawnMessageAdapter[U](f: U T): ActorRef[U] = {
val ref = super.spawnMessageAdapter(f)
effectQueue.offer(SpawnedAdapter)
ref
}
override def spawnMessageAdapter[U](f: U T, name: String): ActorRef[U] = {
val ref = super.spawnMessageAdapter(f, name)
effectQueue.offer(SpawnedAdapter)
ref
}
override def spawn[U](behavior: Behavior[U], name: String, props: Props = Props.empty): ActorRef[U] = {
effectQueue.offer(Spawned(behavior, name, props))
super.spawn(behavior, name, props)
}
override def stop[U](child: ActorRef[U]): Unit = {
effectQueue.offer(Stopped(child.path.name))
super.stop(child)
}
override def watch[U](other: ActorRef[U]): Unit = {
effectQueue.offer(Watched(other))
super.watch(other)
}
override def unwatch[U](other: ActorRef[U]): Unit = {
effectQueue.offer(Unwatched(other))
super.unwatch(other)
}
override def setReceiveTimeout(d: FiniteDuration, msg: T): Unit = {
effectQueue.offer(ReceiveTimeoutSet(d, msg))
super.setReceiveTimeout(d, msg)
}
override def cancelReceiveTimeout(): Unit = {
effectQueue.offer(ReceiveTimeoutSet(Duration.Undefined, null))
super.cancelReceiveTimeout()
}
override def schedule[U](delay: FiniteDuration, target: ActorRef[U], msg: U): Cancellable = {
effectQueue.offer(Scheduled(delay, target, msg))
super.schedule(delay, target, msg)
}
}

View file

@ -1,26 +1,28 @@
package akka.testkit.typed
package akka.testkit.typed.internal
import akka.actor.InvalidMessageException
import akka.{ actor untyped }
import akka.actor.typed._
import akka.actor.typed.internal._
import akka.actor.typed.internal.adapter.LoggerAdapterImpl
import akka.actor.{ ActorPath, InvalidMessageException }
import akka.annotation.InternalApi
import akka.event.Logging
import akka.event.Logging.LogLevel
import akka.util.{ Helpers, OptionVal }
import akka.{ actor a }
import akka.{ actor untyped }
import scala.collection.immutable.TreeMap
import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.FiniteDuration
import akka.annotation.InternalApi
import akka.actor.typed.internal._
import akka.actor.typed.internal.adapter.LoggerAdapterImpl
import akka.event.Logging.{ Info, LogEvent, LogLevel }
import akka.event.{ Logging, LoggingAdapter }
/**
* INTERNAL API
*
* A local synchronous ActorRef that invokes the given function for every message send.
* This reference cannot watch other references.
*/
@InternalApi
private[akka] final class FunctionRef[-T](
_path: a.ActorPath,
_path: ActorPath,
send: (T, FunctionRef[T]) Unit,
_terminate: FunctionRef[T] Unit)
extends ActorRef[T] with ActorRefImpl[T] {
@ -74,8 +76,6 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String, cause: Op
* An [[ActorContext]] for synchronous execution of a [[Behavior]] that
* provides only stubs for the effects an Actor can perform and replaces
* created child Actors by a synchronous Inbox (see `Inbox.sync`).
*
* See [[BehaviorTestkit]] for more advanced uses.
*/
@InternalApi private[akka] class StubbedActorContext[T](
val name: String) extends ActorContextImpl[T] {
@ -83,11 +83,11 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String, cause: Op
/**
* INTERNAL API
*/
@InternalApi private[akka] val selfInbox = TestInbox[T](name)
@InternalApi private[akka] val selfInbox = new TestInboxImpl[T](name)
override val self = selfInbox.ref
override val system = new ActorSystemStub("StubbedActorContext")
private var _children = TreeMap.empty[String, TestInbox[_]]
private var _children = TreeMap.empty[String, TestInboxImpl[_]]
private val childName = Iterator from 0 map (Helpers.base64(_))
private val loggingAdapter = new StubbedLogger
@ -97,7 +97,7 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String, cause: Op
override def child(name: String): Option[ActorRef[Nothing]] = _children get name map (_.ref)
override def spawnAnonymous[U](behavior: Behavior[U], props: Props = Props.empty): ActorRef[U] = {
val i = TestInbox[U](childName.next())
val i = new TestInboxImpl[U](childName.next())
_children += i.ref.path.name i
i.ref
}
@ -106,7 +106,7 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String, cause: Op
case Some(_) throw untyped.InvalidActorNameException(s"actor name $name is already taken")
case None
// FIXME correct child path for the Inbox ref
val i = TestInbox[U](name)
val i = new TestInboxImpl[U](name)
_children += name i
i.ref
}
@ -142,7 +142,7 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String, cause: Op
@InternalApi private[akka] def internalSpawnMessageAdapter[U](f: U T, name: String): ActorRef[U] = {
val n = if (name != "") s"${childName.next()}-$name" else childName.next()
val i = TestInbox[U](n)
val i = new TestInboxImpl[U](n)
_children += i.ref.path.name i
new FunctionRef[U](
@ -155,8 +155,8 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String, cause: Op
* Retrieve the inbox representing the given child actor. The passed ActorRef must be one that was returned
* by one of the spawn methods earlier.
*/
def childInbox[U](child: ActorRef[U]): TestInbox[U] = {
val inbox = _children(child.path.name).asInstanceOf[TestInbox[U]]
def childInbox[U](child: ActorRef[U]): TestInboxImpl[U] = {
val inbox = _children(child.path.name).asInstanceOf[TestInboxImpl[U]]
if (inbox.ref != child) throw new IllegalArgumentException(s"$child is not a child of $this")
inbox
}
@ -164,7 +164,7 @@ final case class CapturedLogEvent(logLevel: LogLevel, message: String, cause: Op
/**
* Retrieve the inbox representing the child actor with the given name.
*/
def childInbox[U](name: String): Option[TestInbox[U]] = _children.get(name).map(_.asInstanceOf[TestInbox[U]])
def childInbox[U](name: String): Option[TestInboxImpl[U]] = _children.get(name).map(_.asInstanceOf[TestInboxImpl[U]])
/**
* Remove the given inbox from the list of children, for example after

View file

@ -1,46 +1,40 @@
/**
* Copyright (C) 2014-2018 Lightbend Inc. <https://www.lightbend.com>
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed
package akka.testkit.typed.internal
import java.util.concurrent.{ ConcurrentLinkedQueue, ThreadLocalRandom }
import akka.actor.{ Address, RootActorPath }
import akka.actor.typed.ActorRef
import akka.annotation.ApiMayChange
import akka.actor.{ Address, RootActorPath }
import akka.annotation.InternalApi
import scala.annotation.tailrec
import scala.collection.immutable
/**
* Utility for use as an [[ActorRef]] when synchronously testing [[akka.actor.typed.Behavior]]
* to be used along with [[BehaviorTestkit]]. If you plan to use a real [[akka.actor.typed.ActorSystem]]
* then use [[akka.testkit.typed.scaladsl.TestProbe]] for asynchronous testing.
* INTERNAL API
*/
@ApiMayChange
class TestInbox[T](name: String) {
def this() = this("inbox")
@InternalApi
private[akka] final class TestInboxImpl[T](name: String)
extends akka.testkit.typed.javadsl.TestInbox[T]
with akka.testkit.typed.scaladsl.TestInbox[T] {
private val q = new ConcurrentLinkedQueue[T]
val ref: ActorRef[T] = {
override val ref: ActorRef[T] = {
val uid = ThreadLocalRandom.current().nextInt()
val path = RootActorPath(Address("akka.actor.typed.inbox", "anonymous")).child(name).withUid(uid)
new FunctionRef[T](path, (msg, self) q.add(msg), (self) ())
}
override def getRef() = ref
/**
* Get and remove the oldest message
*/
def receiveMsg(): T = q.poll() match {
override def receiveMessage(): T = q.poll() match {
case null throw new NoSuchElementException(s"polling on an empty inbox: $name")
case x x
}
/**
* Assert and remove the the oldest message.
*/
def expectMsg(expectedMessage: T): TestInbox[T] = {
override def expectMessage(expectedMessage: T): TestInboxImpl[T] = {
q.poll() match {
case null assert(assertion = false, s"expected msg: $expectedMessage but no messages were received")
case message assert(message == expectedMessage, s"expected: $expectedMessage but received $message")
@ -48,7 +42,7 @@ class TestInbox[T](name: String) {
this
}
def receiveAll(): immutable.Seq[T] = {
override protected def internalReceiveAll(): immutable.Seq[T] = {
@tailrec def rec(acc: List[T]): List[T] = q.poll() match {
case null acc.reverse
case x rec(x :: acc)
@ -59,10 +53,4 @@ class TestInbox[T](name: String) {
def hasMessages: Boolean = q.peek() != null
// TODO expectNoMsg etc
}
@ApiMayChange
object TestInbox {
def apply[T](name: String = "inbox"): TestInbox[T] = new TestInbox(name)
}

View file

@ -0,0 +1,71 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.internal
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }
import akka.annotation.InternalApi
import scala.concurrent.{ Await, TimeoutException }
import scala.concurrent.duration.Duration
/**
* INTERNAL API
*/
@InternalApi
private[akka] object ActorTestKitGuardian {
sealed trait TestKitCommand
final case class SpawnActor[T](name: String, behavior: Behavior[T], replyTo: ActorRef[ActorRef[T]], props: Props) extends TestKitCommand
final case class SpawnActorAnonymous[T](behavior: Behavior[T], replyTo: ActorRef[ActorRef[T]], props: Props) extends TestKitCommand
val testKitGuardian: Behavior[TestKitCommand] = Behaviors.immutable[TestKitCommand] {
case (ctx, SpawnActor(name, behavior, reply, props))
reply ! ctx.spawn(behavior, name, props)
Behaviors.same
case (ctx, SpawnActorAnonymous(behavior, reply, props))
reply ! ctx.spawnAnonymous(behavior, props)
Behaviors.same
}
}
/**
* INTERNAL API
*/
@InternalApi
private[akka] object TestKitUtils {
// common internal utility impls for Java and Scala
private val TestKitRegex = """akka\.testkit\.typed\.(?:javadsl|scaladsl)\.ActorTestKit(?:\$.*)?""".r
def testNameFromCallStack(classToStartFrom: Class[_]): String = {
val startFrom = classToStartFrom.getName
val filteredStack = Thread.currentThread.getStackTrace.toIterator
.map(_.getClassName)
// drop until we find the first occurence of classToStartFrom
.dropWhile(!_.startsWith(startFrom))
// then continue to the next entry after classToStartFrom that makes sense
.dropWhile {
case `startFrom` true
case str if str.startsWith(startFrom + "$") true // lambdas inside startFrom etc
case TestKitRegex() true // testkit internals
case _ false
}
// sanitize for actor system name
filteredStack.next().replaceFirst(""".*\.""", "").replaceAll("[^a-zA-Z_0-9]", "_")
}
def shutdown(
system: ActorSystem[_],
timeout: Duration,
throwIfShutdownTimesOut: Boolean): Unit = {
system.terminate()
try Await.ready(system.whenTerminated, timeout) catch {
case _: TimeoutException
val msg = "Failed to stop [%s] within [%s] \n%s".format(system.name, timeout, system.printTree)
if (throwIfShutdownTimesOut) throw new RuntimeException(msg)
else println(msg)
}
}
}

View file

@ -19,7 +19,7 @@ import akka.util.{ BoxedType, Timeout }
import scala.annotation.tailrec
import scala.concurrent.Await
import scala.concurrent.duration._
import scala.util.control.NonFatal
import scala.util.control.{ NoStackTrace, NonFatal }
@InternalApi
private[akka] object TestProbeImpl {

View file

@ -0,0 +1,165 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.javadsl
import java.util.concurrent.TimeUnit
import akka.actor.Scheduler
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }
import akka.testkit.typed.TestKitSettings
import akka.testkit.typed.internal.TestKitUtils
import akka.testkit.typed.scaladsl.{ ActorTestKit ScalaTestKit }
import akka.util.Timeout
import com.typesafe.config.Config
import scala.concurrent.duration.Duration
object ActorTestKit {
/**
* Create a testkit named from the class that is calling this method
*/
def create(): ActorTestKit = new ActorTestKit(new ScalaTestKit {})
/**
* Create a testkit named with this test class
*/
def create(testClass: Class[_]): ActorTestKit = new ActorTestKit(new ScalaTestKit {
override def name = TestKitUtils.testNameFromCallStack(testClass)
})
/**
* Create a testkit named with this test class, and use a custom config for the actor system
*/
def create(testClass: Class[_], customConfig: Config) = new ActorTestKit(new ScalaTestKit {
override def name = TestKitUtils.testNameFromCallStack(testClass)
override def config = customConfig
})
/**
* Create a testkit named with this test class, and use a custom config for the actor system,
* and a custom [[akka.testkit.typed.TestKitSettings]]
*/
def create(testClass: Class[_], customConfig: Config, settings: TestKitSettings) = new ActorTestKit(new ScalaTestKit {
override def name = TestKitUtils.testNameFromCallStack(testClass)
override def config = customConfig
override def testkitSettings: TestKitSettings = settings
})
/**
* Shutdown the given actor system and wait up to `duration` for shutdown to complete.
* @param throwIfShutdownTimesOut Fail the test if the system fails to shut down, if false
* an error is printed to stdout when the system did not shutdown but
* no exception is thrown.
*/
def shutdown(system: ActorSystem[_], duration: Duration, throwIfShutdownTimesOut: Boolean): Unit = {
TestKitUtils.shutdown(system, duration, throwIfShutdownTimesOut)
}
/**
* Shutdown the given [[akka.actor.typed.ActorSystem]] and block until it shuts down,
* if more time than `system-shutdown-default` passes an exception is thrown
* (can be configured with `throw-on-shutdown-timeout`).
*/
def shutdown(system: ActorSystem[_], duration: Duration): Unit = {
val settings = TestKitSettings.create(system)
shutdown(system, duration, settings.ThrowOnShutdownTimeout)
}
/**
* Shutdown the given [[akka.actor.typed.ActorSystem]] and block until it shuts down,
* if more time than `system-shutdown-default` passes an exception is thrown
* (can be configured with `throw-on-shutdown-timeout`).
*/
def shutdown(system: ActorSystem[_]): Unit = {
val settings = TestKitSettings.create(system)
shutdown(
system,
settings.DefaultActorSystemShutdownTimeout,
settings.ThrowOnShutdownTimeout
)
}
}
/**
* Java API: Test kit for asynchronous testing of typed actors.
* Provides a typed actor system started on creation, that can be used for multiple test cases and is
* shut down when `shutdown` is called.
*
* The actor system has a custom guardian that allows for spawning arbitrary actors using the `spawn` methods.
*
* Designed to work with any test framework, but framework glue code that calls `shutdownTestKit` after all tests has
* run needs to be provided by the user.
*
* Use `TestKit.create` factories to construct manually or [[TestKitJunitResource]] to use together with JUnit tests
*
* For synchronous testing of a `Behavior` see [[BehaviorTestKit]]
*/
final class ActorTestKit protected (delegate: akka.testkit.typed.scaladsl.ActorTestKit) {
/**
* The default timeout as specified with the config/[[akka.testkit.typed.TestKitSettings]]
*/
def timeout: Timeout = delegate.timeout
/**
* The actor system running for this testkit. Interaction with the user guardian is done through methods on the testkit
* which is why it is typed to `Void`.
*/
def system: ActorSystem[Void] = delegate.system.asInstanceOf[ActorSystem[Void]]
/**
* The scheduler of the testkit actor system
*/
def scheduler: Scheduler = delegate.scheduler
/**
* Spawn a new auto-named actor under the testkit user guardian and return the ActorRef for the spawned actor
*/
def spawn[T](behavior: Behavior[T]): ActorRef[T] = delegate.spawn(behavior)
/**
* Spawn a new named actor under the testkit user guardian and return the ActorRef for the spawned actor,
* note that spawning actors with the same name in multiple test cases will cause failures.
*/
def spawn[T](behavior: Behavior[T], name: String): ActorRef[T] = delegate.spawn(behavior, name)
/**
* Spawn a new auto-named actor under the testkit user guardian with the given props
* and return the ActorRef for the spawned actor
*/
def spawn[T](behavior: Behavior[T], props: Props): ActorRef[T] = delegate.spawn(behavior, props)
/**
* Spawn a new named actor under the testkit user guardian with the given props and return the ActorRef
* for the spawned actor, note that spawning actors with the same name in multiple test cases will cause failures.
*/
def spawn[T](behavior: Behavior[T], name: String, props: Props): ActorRef[T] = delegate.spawn(behavior, name, props)
/**
* Shortcut for creating a new test probe for the testkit actor system
* @tparam M the type of messages the probe should accept
*/
def createTestProbe[M](): TestProbe[M] = TestProbe.create(system)
/**
* Shortcut for creating a new test probe for the testkit actor system
* @tparam M the type of messages the probe should accept
*/
def createTestProbe[M](clazz: Class[M]): TestProbe[M] = TestProbe.create(clazz, system)
/**
* Shortcut for creating a new named test probe for the testkit actor system
* @tparam M the type of messages the probe should accept
*/
def createTestProbe[M](name: String): TestProbe[M] = TestProbe.create(name, system)
/**
* Shortcut for creating a new named test probe for the testkit actor system
* @tparam M the type of messages the probe should accept
*/
def createTestProbe[M](name: String, clazz: Class[M]): TestProbe[M] = TestProbe.create(name, clazz, system)
/**
* Terminate the actor system and the testkit
*/
def shutdownTestKit(): Unit = delegate.shutdownTestKit()
}

View file

@ -0,0 +1,91 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.javadsl
import akka.actor.typed.{ Behavior, Signal }
import akka.annotation.DoNotInherit
import akka.testkit.typed.Effect
import akka.testkit.typed.internal.BehaviorTestKitImpl
object BehaviorTestKit {
/**
* JAVA API
*/
def create[T](initialBehavior: Behavior[T], name: String): BehaviorTestKit[T] =
new BehaviorTestKitImpl[T](name, initialBehavior)
/**
* JAVA API
*/
def create[T](initialBehavior: Behavior[T]): BehaviorTestKit[T] =
new BehaviorTestKitImpl[T]("testkit", initialBehavior)
}
/**
* Used for synchronous testing [[akka.actor.typed.Behavior]]s. Stores all effects e.g. Spawning of children,
* watching and offers access to what effects have taken place.
*
* Not for user extension or instantiation. See `BehaviorTestKit.create` factory methods
*
* For asynchronous testing of `Behavior`s running see [[ActorTestKit]]
*/
@DoNotInherit
abstract class BehaviorTestKit[T] {
/**
* Requests the oldest [[Effect]] or [[akka.testkit.typed.javadsl.Effects.noEffects]] if no effects
* have taken place. The effect is consumed, subsequent calls won't
* will not include this effect.
*/
def getEffect(): Effect
/**
* Get the child inbox for the child with the given name, or fail if there is no child with the given name
* spawned
*/
def childInbox[U](name: String): TestInbox[U]
/**
* The self inbox contains messages the behavior sent to `ctx.self`
*/
def selfInbox(): TestInbox[T]
/**
* Requests all the effects. The effects are consumed, subsequent calls will only
* see new effects.
*/
def getAllEffects(): java.util.List[Effect]
/**
* Asserts that the oldest effect is the expectedEffect. Removing it from
* further assertions.
*/
def expectEffect(expectedEffect: Effect): Unit
/**
* The current behavior, can change any time `run` is called
*/
def currentBehavior: Behavior[T]
/**
* Returns the current behavior as it was returned from processing the previous message.
* For example if [[Behavior.unhandled]] is returned it will be kept here, but not in
* [[currentBehavior]].
*/
def returnedBehavior: Behavior[T]
/**
* Is the current behavior alive or stopped
*/
def isAlive: Boolean
/**
* Send the msg to the behavior and record any [[Effect]]s
*/
def run(msg: T): Unit
/**
* Send the signal to the beheavior and record any [[Effect]]s
*/
def signal(signal: Signal): Unit
}

View file

@ -0,0 +1,73 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.javadsl
import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.testkit.typed.Effect
import scala.concurrent.duration.{ Duration, FiniteDuration }
/**
* Factories for behavior effects for [[BehaviorTestKit]], each effect has a suitable equals and can be used to compare
* actual effects to expected ones.
*/
object Effects {
import akka.testkit.typed.scaladsl.Effects._
/**
* The behavior spawned a named child with the given behavior with no specific props
*/
def spawned[T](behavior: Behavior[T], childName: String): Effect = Spawned(behavior, childName)
/**
* The behavior spawned a named child with the given behavior and specific props
*/
def spawned[T](behavior: Behavior[T], childName: String, props: Props): Effect = Spawned(behavior, childName, props)
/**
* The behavior spawned an anonymous child with the given behavior with no specific props
*/
def spawnedAnonymous(behavior: Behavior[_]): Effect = SpawnedAnonymous(behavior)
/**
* The behavior spawned an anonymous child with the given behavior with specific props
*/
def spawnedAnonymous(behavior: Behavior[_], props: Props): Effect = SpawnedAnonymous(behavior, props)
/**
* The behavior spawned an anonymous adapter, through `ctx.spawnMessageAdapter`
*/
def spawnedAdapter: Effect = SpawnedAdapter
/**
* The behavior spawned a named adapter, through `ctx.spawnMessageAdapter`
*/
def spawnedNamedAdapter(name: String): Effect = SpawnedNamedAdapter(name)
/**
* The behavior stopped `childName`
*/
def stopped(childName: String): Effect = Stopped(childName)
/**
* The behavior started watching `other`, through `ctx.watch(other)`
*/
def watched[T](other: ActorRef[T]): Effect = Watched(other)
/**
* The behavior started watching `other`, through `ctx.unwatch(other)`
*/
def unwatched[T](other: ActorRef[T]): Effect = Unwatched(other)
/**
* The behavior set a new receive timeout, with `msg` as timeout notification
*/
def receiveTimeoutSet[T](d: Duration, msg: T): Effect = ReceiveTimeoutSet(d, msg)
/**
* The behavior used `ctx.schedule` to schedule `msg` to be sent to `target` after `delay`
* FIXME what about events scheduled through the scheduler?
*/
def scheduled[U](delay: FiniteDuration, target: ActorRef[U], msg: U): Effect =
Scheduled(delay, target, msg)
/**
* Used to represent an empty list of effects - in other words, the behavior didn't do anything observable
*/
def noEffects(): Effect = NoEffects
}

View file

@ -1,22 +0,0 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.testkit.typed.javadsl
import java.util.concurrent.ThreadFactory
import akka.event.LoggingAdapter
import com.typesafe.config.Config
import scala.annotation.varargs
import scala.concurrent.duration.{ Duration, FiniteDuration }
class ExplicitlyTriggeredScheduler(config: Config, log: LoggingAdapter, tf: ThreadFactory) extends akka.testkit.ExplicitlyTriggeredScheduler(config, log, tf) {
@varargs
def expectNoMessageFor(duration: FiniteDuration, on: TestProbe[_]*): Unit = {
timePasses(duration)
on.foreach(_.expectNoMessage(Duration.Zero))
}
}

View file

@ -0,0 +1,58 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.javadsl
import akka.actor.typed.ActorSystem
import com.typesafe.config.Config
import scala.annotation.varargs
import scala.concurrent.duration.{ Duration, FiniteDuration }
/**
* Manual time allows you to do async tests while controlling the scheduler of the system.
*
* To use it you need to configure the `ActorSystem`/`ActorTestKit` with [[ManualTime.config]] and access the
* scheduler control through [[ManualTime.get()]]
*/
object ManualTime {
/**
* Config that needs to be in place for the actor system to use the manual
*/
def config(): Config = akka.testkit.typed.scaladsl.ManualTime.config
/**
* Access the manual scheduler, note that you need to setup the actor system/testkit with [[config()]] for this to
* work.
*/
def get[A](system: ActorSystem[A]): ManualTime =
system.scheduler match {
case sc: akka.testkit.ExplicitlyTriggeredScheduler new ManualTime(sc)
case _ throw new IllegalArgumentException("ActorSystem not configured with explicitly triggered scheduler, " +
"make sure to include akka.testkit.typed.javadsl.ManualTime.config() when setting up the test")
}
}
/**
* Not for user instantiation, see [[ManualTime#get]]
*/
final class ManualTime(delegate: akka.testkit.ExplicitlyTriggeredScheduler) {
/**
* Advance the clock by the specified duration, executing all outstanding jobs on the calling thread before returning.
*
* We will not add a dilation factor to this amount, since the scheduler API also does not apply dilation.
* If you want the amount of time passed to be dilated, apply the dilation before passing the delay to
* this method.
*/
def timePasses(amount: FiniteDuration): Unit = delegate.timePasses(amount)
@varargs
def expectNoMessageFor(duration: FiniteDuration, on: TestProbe[_]*): Unit = {
delegate.timePasses(duration)
on.foreach(_.expectNoMessage(Duration.Zero))
}
}

View file

@ -0,0 +1,56 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.javadsl
import akka.actor.typed.ActorRef
import akka.annotation.DoNotInherit
import akka.testkit.typed.internal.TestInboxImpl
import scala.collection.JavaConverters._
import scala.collection.immutable
object TestInbox {
def create[T](name: String): TestInbox[T] = new TestInboxImpl(name)
def create[T](): TestInbox[T] = new TestInboxImpl[T]("inbox")
}
/**
* Utility for use as an [[ActorRef]] when *synchronously* testing [[akka.actor.typed.Behavior]]
* with [[akka.testkit.typed.javadsl.BehaviorTestKit]].
*
* If you plan to use a real [[akka.actor.typed.ActorSystem]] then use [[akka.testkit.typed.javadsl.TestProbe]]
* for asynchronous testing.
*
* Use `TestInbox.create` factory methods to create instances
*
* Not for user extension
*/
@DoNotInherit
abstract class TestInbox[T] {
/**
* The actor ref of the inbox
*/
def getRef(): ActorRef[T]
/**
* Get and remove the oldest message
*/
def receiveMessage(): T
/**
* Assert and remove the the oldest message.
*/
def expectMessage(expectedMessage: T): TestInbox[T]
/**
* Collect all messages in the inbox and clear it out
*/
def getAllReceived(): java.util.List[T] = internalReceiveAll().asJava
protected def internalReceiveAll(): immutable.Seq[T]
def hasMessages: Boolean
// TODO expectNoMsg etc
}

View file

@ -0,0 +1,97 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.javadsl
import akka.actor.Scheduler
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }
import akka.util.Timeout
import com.typesafe.config.Config
import org.junit.Rule
import org.junit.rules.ExternalResource
/**
* A Junit external resource for the testkit, making it possible to have Junit manage the lifecycle of the testkit.
* The testkit will be automatically shut down when the test completes fails.
*
* Note that Junit is not provided as a transitive dependency of the testkit module but must be added explicitly
* to your project to use this.
*
* Example:
* {{{
* public class MyActorTest {
* @ClassRule
* public static final TestKitResource testKit = new TestKitResource();
*
* @Test
* public void testBlah() throws Exception {
* // spawn actors etc using the testKit
* ActorRef<Message> ref = testKit.spawn(behavior);
* }
* }
* }}}
*/
class TestKitJunitResource(_kit: ActorTestKit) extends ExternalResource {
def this() = this(ActorTestKit.create(classOf[TestKitJunitResource]))
def this(customConfig: Config) = this(ActorTestKit.create(classOf[TestKitJunitResource], customConfig))
def this(testClass: Class[_]) = this(ActorTestKit.create())
def this(testClass: Class[_], customConfig: Config) = this(ActorTestKit.create(testClass, customConfig))
@Rule
final val testKit = _kit
// delegates of the TestKit api for minimum fuss
/**
* See corresponding method on [[ActorTestKit]]
*/
def system: ActorSystem[Void] = testKit.system
/**
* See corresponding method on [[ActorTestKit]]
*/
def timeout: Timeout = testKit.timeout
/**
* See corresponding method on [[ActorTestKit]]
*/
def scheduler: Scheduler = testKit.scheduler
/**
* See corresponding method on [[ActorTestKit]]
*/
def spawn[T](behavior: Behavior[T]): ActorRef[T] = testKit.spawn(behavior)
/**
* See corresponding method on [[ActorTestKit]]
*/
def spawn[T](behavior: Behavior[T], name: String): ActorRef[T] = testKit.spawn(behavior, name)
/**
* See corresponding method on [[ActorTestKit]]
*/
def spawn[T](behavior: Behavior[T], props: Props): ActorRef[T] = testKit.spawn(behavior, props)
/**
* See corresponding method on [[ActorTestKit]]
*/
def spawn[T](behavior: Behavior[T], name: String, props: Props): ActorRef[T] = testKit.spawn(behavior, name, props)
/**
* See corresponding method on [[ActorTestKit]]
*/
def createTestProbe[M](): TestProbe[M] = testKit.createTestProbe[M]()
/**
* See corresponding method on [[ActorTestKit]]
*/
def createTestProbe[M](clazz: Class[M]): TestProbe[M] = testKit.createTestProbe(clazz)
/**
* See corresponding method on [[ActorTestKit]]
*/
def createTestProbe[M](name: String, clazz: Class[M]): TestProbe[M] = testKit.createTestProbe(name, clazz)
/**
* See corresponding method on [[ActorTestKit]]
*/
def createTestProbe[M](name: String): TestProbe[M] = testKit.createTestProbe(name)
override def after(): Unit = {
testKit.shutdownTestKit()
}
}

View file

@ -70,6 +70,11 @@ abstract class TestProbe[M] {
*/
def ref: ActorRef[M]
/**
* ActorRef for this TestProbe
*/
def getRef(): ActorRef[M] = ref
/**
* Obtain time remaining for execution of the innermost enclosing `within`
* block or missing that it returns the properly dilated default for this

View file

@ -0,0 +1,130 @@
package akka.testkit.typed.scaladsl
import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, Props }
import akka.annotation.{ ApiMayChange, InternalApi }
import akka.testkit.typed.TestKitSettings
import akka.testkit.typed.internal.{ ActorTestKitGuardian, TestKitUtils }
import com.typesafe.config.{ Config, ConfigFactory }
import scala.concurrent.Await
import scala.concurrent.duration._
object ActorTestKit {
/**
* Shutdown the given [[akka.actor.typed.ActorSystem]] and block until it shuts down,
* if more time than `TestKitSettings.DefaultActorSystemShutdownTimeout` passes an exception is thrown
*/
def shutdown(system: ActorSystem[_]): Unit = {
val settings = TestKitSettings(system)
TestKitUtils.shutdown(
system,
settings.DefaultActorSystemShutdownTimeout,
settings.ThrowOnShutdownTimeout
)
}
/**
* Shutdown the given [[akka.actor.typed.ActorSystem]] and block until it shuts down
* or the `duration` hits. If the timeout hits `verifySystemShutdown` decides
*/
def shutdown(
system: ActorSystem[_],
timeout: Duration,
throwIfShutdownFails: Boolean = false): Unit =
TestKitUtils.shutdown(system, timeout, throwIfShutdownFails)
// place holder for no custom config specified to avoid the boilerplate
// of an option for config in the trait
private val noConfigSet = ConfigFactory.parseString("")
}
/**
* Testkit for asynchronous testing of typed actors, meant for mixing into the test class.
*
* Provides a typed actor system started on creation, used for all test cases and shut down when `shutdown` is called.
*
* The actor system has a custom guardian that allows for spawning arbitrary actors using the `spawn` methods.
*
* Designed to work with any test framework, but framework glue code that calls shutdown after all tests has
* run needs to be provided by the user.
*
* For synchronous testing of a `Behavior` see [[BehaviorTestKit]]
*/
@ApiMayChange
trait ActorTestKit {
/**
* Actor system name based on the test it is mixed into, override to customize, or pass to constructor
* if using [[ActorTestKit]] rather than [[ActorTestKit]]
*/
protected def name: String = TestKitUtils.testNameFromCallStack(classOf[ActorTestKit])
/**
* Configuration the actor system is created with, override to customize, or pass to constructor
* if using [[ActorTestKit]] rather than [[ActorTestKit]]
*/
def config: Config = ActorTestKit.noConfigSet
/**
* TestKit settings used in the tests, override or provide custom config to customize
*/
protected implicit def testkitSettings = TestKitSettings(system)
private val internalSystem: ActorSystem[ActorTestKitGuardian.TestKitCommand] =
if (config eq ActorTestKit.noConfigSet) ActorSystem(ActorTestKitGuardian.testKitGuardian, name)
else ActorSystem(ActorTestKitGuardian.testKitGuardian, name, config)
implicit final def system: ActorSystem[Nothing] = internalSystem
implicit def scheduler = system.scheduler
private val childName: Iterator[String] = Iterator.from(0).map(_.toString)
implicit val timeout = testkitSettings.DefaultTimeout
final def shutdownTestKit(): Unit = {
ActorTestKit.shutdown(
system,
testkitSettings.DefaultActorSystemShutdownTimeout,
testkitSettings.ThrowOnShutdownTimeout
)
}
/**
* Spawn the given behavior. This is created as a child of the test kit
* guardian
*/
final def spawn[T](behavior: Behavior[T]): ActorRef[T] =
spawn(behavior, Props.empty)
/**
* Spawn the given behavior. This is created as a child of the test kit
* guardian
*/
final def spawn[T](behavior: Behavior[T], props: Props): ActorRef[T] =
Await.result(internalSystem ? (ActorTestKitGuardian.SpawnActorAnonymous(behavior, _, props)), timeout.duration)
/**
* Spawn the given behavior. This is created as a child of the test kit
* guardian
*/
final def spawn[T](behavior: Behavior[T], name: String): ActorRef[T] =
spawn(behavior, name, Props.empty)
/**
* Spawn the given behavior. This is created as a child of the test kit
* guardian
*/
final def spawn[T](behavior: Behavior[T], name: String, props: Props): ActorRef[T] =
Await.result(internalSystem ? (ActorTestKitGuardian.SpawnActor(name, behavior, _, props)), timeout.duration)
// FIXME needed for Akka internal tests but, users shouldn't spawn system actors?
@InternalApi
private[akka] def systemActor[T](behavior: Behavior[T], name: String): ActorRef[T] =
Await.result(system.systemActorOf(behavior, name), timeout.duration)
@InternalApi
private[akka] def systemActor[T](behavior: Behavior[T]): ActorRef[T] =
Await.result(system.systemActorOf(behavior, childName.next()), timeout.duration)
}

View file

@ -0,0 +1,91 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.scaladsl
import akka.actor.typed.{ Behavior, Signal }
import akka.annotation.DoNotInherit
import akka.testkit.typed.Effect
import akka.testkit.typed.internal.BehaviorTestKitImpl
import scala.collection.immutable
object BehaviorTestKit {
def apply[T](initialBehavior: Behavior[T], name: String): BehaviorTestKit[T] =
new BehaviorTestKitImpl[T](name, initialBehavior)
def apply[T](initialBehavior: Behavior[T]): BehaviorTestKit[T] =
apply(initialBehavior, "testkit")
}
/**
* Used for synchronous testing [[akka.actor.typed.Behavior]]s. Stores all effects e.g. Spawning of children,
* watching and offers access to what effects have taken place.
*
* For asynchronous testing of `Behavior`s running see [[ActorTestKit]]
*
* Not for user extension. See `BehaviorTestKit.apply` factory methods
*/
@DoNotInherit
trait BehaviorTestKit[T] {
// FIXME it is weird that this is public but it is used in BehaviorSpec, could we avoid that?
private[akka] def ctx: akka.actor.typed.ActorContext[T]
/**
* Requests the oldest [[Effect]] or [[akka.testkit.typed.scaladsl.Effects.NoEffects]] if no effects
* have taken place. The effect is consumed, subsequent calls won't
* will not include this effect.
*/
def retrieveEffect(): Effect
/**
* Get the child inbox for the child with the given name, or fail if there is no child with the given name
* spawned
*/
def childInbox[U](name: String): TestInbox[U]
/**
* The self inbox contains messages the behavior sent to `ctx.self`
*/
def selfInbox(): TestInbox[T]
/**
* Requests all the effects. The effects are consumed, subsequent calls will only
* see new effects.
*/
def retrieveAllEffects(): immutable.Seq[Effect]
/**
* Asserts that the oldest effect is the expectedEffect. Removing it from
* further assertions.
*/
def expectEffect(expectedEffect: Effect): Unit
/**
* The current behavior, can change any time `run` is called
*/
def currentBehavior: Behavior[T]
/**
* Returns the current behavior as it was returned from processing the previous message.
* For example if [[Behavior.unhandled]] is returned it will be kept here, but not in
* [[currentBehavior]].
*/
def returnedBehavior: Behavior[T]
/**
* Is the current behavior alive or stopped
*/
def isAlive: Boolean
/**
* Send the msg to the behavior and record any [[Effect]]s
*/
def run(msg: T): Unit
/**
* Send the signal to the beheavior and record any [[Effect]]s
*/
def signal(signal: Signal): Unit
}

View file

@ -0,0 +1,61 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.scaladsl
import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.testkit.typed.Effect
import scala.concurrent.duration.{ Duration, FiniteDuration }
/**
* Types for behavior effects for [[BehaviorTestKit]], each effect has a suitable equals and can be used to compare
* actual effects to expected ones.
*/
object Effects {
/**
* The behavior spawned a named child with the given behavior (and optionally specific props)
*/
final case class Spawned[T](behavior: Behavior[T], childName: String, props: Props = Props.empty) extends Effect
/**
* The behavior spawned an anonymous child with the given behavior (and optionally specific props)
*/
final case class SpawnedAnonymous[T](behavior: Behavior[T], props: Props = Props.empty) extends Effect
/**
* The behavior spawned an anonymous adapter, through `ctx.spawnMessageAdapter`
*/
final case object SpawnedAdapter extends Effect
/**
* The behavior spawned a named adapter, through `ctx.spawnMessageAdapter`
*/
final case class SpawnedNamedAdapter(name: String) extends Effect
/**
* The behavior stopped `childName`
*/
final case class Stopped(childName: String) extends Effect
/**
* The behavior started watching `other`, through `ctx.watch(other)`
*/
final case class Watched[T](other: ActorRef[T]) extends Effect
/**
* The behavior started watching `other`, through `ctx.unwatch(other)`
*/
final case class Unwatched[T](other: ActorRef[T]) extends Effect
/**
* The behavior set a new receive timeout, with `msg` as timeout notification
*/
final case class ReceiveTimeoutSet[T](d: Duration, msg: T) extends Effect
/**
* The behavior used `ctx.schedule` to schedule `msg` to be sent to `target` after `delay`
* FIXME what about events scheduled through the scheduler?
*/
final case class Scheduled[U](delay: FiniteDuration, target: ActorRef[U], msg: U) extends Effect
/**
* Used to represent an empty list of effects - in other words, the behavior didn't do anything observable
*/
case object NoEffects extends Effect
}

View file

@ -1,22 +0,0 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.testkit.typed.scaladsl
import java.util.concurrent.ThreadFactory
import akka.event.LoggingAdapter
import com.typesafe.config.Config
import scala.annotation.varargs
import scala.concurrent.duration.{ Duration, FiniteDuration }
class ExplicitlyTriggeredScheduler(config: Config, log: LoggingAdapter, tf: ThreadFactory) extends akka.testkit.ExplicitlyTriggeredScheduler(config, log, tf) {
@varargs
def expectNoMessageFor(duration: FiniteDuration, on: TestProbe[_]*): Unit = {
timePasses(duration)
on.foreach(_.expectNoMessage(Duration.Zero))
}
}

View file

@ -1,12 +1,54 @@
package akka.testkit.typed.scaladsl
import akka.actor.typed.ActorSystem
import com.typesafe.config.{ Config, ConfigFactory }
import akka.testkit.typed._
import scala.annotation.varargs
import scala.concurrent.duration.{ Duration, FiniteDuration }
/**
* Manual time allows you to do async tests while controlling the scheduler of the system.
*
* To use it you need to configure the `ActorSystem`/`ActorTestKit` with [[ManualTime.config]] and access the
* scheduler control through [[ManualTime.apply()]]
*/
object ManualTime {
val config: Config = ConfigFactory.parseString("""akka.scheduler.implementation = "akka.testkit.typed.scaladsl.ExplicitlyTriggeredScheduler"""")
/**
* Config needed to use the `ExplicitlyTriggeredScheduler`
*/
val config: Config = ConfigFactory.parseString("""akka.scheduler.implementation = "akka.testkit.ExplicitlyTriggeredScheduler"""")
/**
* Access the manual scheduler, note that you need to setup the actor system/testkit with [[config()]] for this to
* work.
*/
def apply()(implicit system: ActorSystem[_]): ManualTime =
system.scheduler match {
case sc: akka.testkit.ExplicitlyTriggeredScheduler new ManualTime(sc)
case _ throw new IllegalArgumentException("ActorSystem not configured with explicitly triggered scheduler, " +
"make sure to include akka.testkit.typed.scaladsl.ManualTime.config() when setting up the test")
}
trait ManualTime { self: TestKit
override val scheduler: ExplicitlyTriggeredScheduler = self.system.scheduler.asInstanceOf[ExplicitlyTriggeredScheduler]
}
/**
* Not for user instantiation, see [[ManualTime#apply]]
*/
final class ManualTime(delegate: akka.testkit.ExplicitlyTriggeredScheduler) {
/**
* Advance the clock by the specified duration, executing all outstanding jobs on the calling thread before returning.
*
* We will not add a dilation factor to this amount, since the scheduler API also does not apply dilation.
* If you want the amount of time passed to be dilated, apply the dilation before passing the delay to
* this method.
*/
def timePasses(amount: FiniteDuration): Unit = delegate.timePasses(amount)
@varargs
def expectNoMessageFor(duration: FiniteDuration, on: TestProbe[_]*): Unit = {
delegate.timePasses(duration)
on.foreach(_.expectNoMessage(Duration.Zero))
}
}

View file

@ -0,0 +1,55 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.scaladsl
import akka.actor.typed.ActorRef
import akka.annotation.{ ApiMayChange, DoNotInherit }
import akka.testkit.typed.internal.TestInboxImpl
import scala.collection.immutable
@ApiMayChange
object TestInbox {
def apply[T](name: String = "inbox"): TestInbox[T] = new TestInboxImpl(name)
}
/**
* Utility for use as an [[ActorRef]] when *synchronously* testing [[akka.actor.typed.Behavior]]
* with [[akka.testkit.typed.javadsl.BehaviorTestKit]].
*
* If you plan to use a real [[akka.actor.typed.ActorSystem]] then use [[akka.testkit.typed.javadsl.TestProbe]]
* for asynchronous testing.
*
* Use factory `apply` in companion to create instances
*
* Not for user extension
*/
@DoNotInherit
trait TestInbox[T] {
/**
* The actor ref of the inbox
*/
def ref: ActorRef[T]
/**
* Get and remove the oldest message
*/
def receiveMessage(): T
/**
* Assert and remove the the oldest message.
*/
def expectMessage(expectedMessage: T): TestInbox[T]
/**
* Collect all messages in the inbox and clear it out
*/
def receiveAll(): immutable.Seq[T] = internalReceiveAll()
protected def internalReceiveAll(): immutable.Seq[T]
def hasMessages: Boolean
// TODO expectNoMsg etc
}

View file

@ -16,19 +16,19 @@ object FishingOutcomes {
/**
* Complete fishing and return all messages up until this
*/
val complete = FishingOutcome.Complete
val complete: FishingOutcome = FishingOutcome.Complete
/**
* Consume this message, collect it into the result, and continue with the next message
*/
val continue = FishingOutcome.Continue
val continue: FishingOutcome = FishingOutcome.Continue
/**
* Consume this message, but do not collect it into the result, and continue with the next message
*/
val continueAndIgnore = FishingOutcome.ContinueAndIgnore
val continueAndIgnore: FishingOutcome = FishingOutcome.ContinueAndIgnore
/**
* Fail fishing with a custom error message
*/
def fail(msg: String) = FishingOutcome.Fail(msg)
def fail(msg: String): FishingOutcome = FishingOutcome.Fail(msg)
}
object TestProbe {

View file

@ -27,8 +27,7 @@ package object scaladsl {
*
*/
implicit class TestDuration(val duration: FiniteDuration) extends AnyVal {
def dilated(implicit settings: TestKitSettings): FiniteDuration =
(duration * settings.TestTimeFactor).asInstanceOf[FiniteDuration]
def dilated(implicit settings: TestKitSettings): FiniteDuration = settings.dilated(duration)
}
}

View file

@ -0,0 +1,36 @@
package akka.testkit.typed.javadsl;
import akka.Done;
import akka.actor.typed.javadsl.Behaviors;
import org.junit.ClassRule;
import org.junit.Test;
import org.scalatest.junit.JUnitSuite;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import static org.junit.Assert.assertEquals;
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
public class ActorTestKitTest extends JUnitSuite {
@ClassRule
public static TestKitJunitResource testKit = new TestKitJunitResource();
@Test
public void systemNameShouldComeFromTest() {
assertEquals("ActorTestKitTest", testKit.system().name());
}
@Test
public void testKitShouldSpawnActor() throws Exception {
final CompletableFuture<Done> started = new CompletableFuture<>();
testKit.spawn(Behaviors.setup((ctx) -> {
started.complete(Done.getInstance());
return Behaviors.same();
}));
started.get(3, TimeUnit.SECONDS);
}
}

View file

@ -1,18 +1,19 @@
/**
* Copyright (C) 2017-2018 Lightbend Inc. <https://www.lightbend.com>
*/
package jdocs.akka.typed.testing.async;
package akka.testkit.typed.javadsl;
import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior;
import akka.actor.typed.javadsl.Behaviors;
import akka.testkit.typed.javadsl.ActorTestKit;
import akka.testkit.typed.javadsl.TestProbe;
import akka.testkit.typed.TestKit;
import org.junit.AfterClass;
import org.junit.Test;
//#test-header
public class BasicAsyncTestingTest extends TestKit {
public class AsyncTestingExampleTest {
final static ActorTestKit testKit = ActorTestKit.create(AsyncTestingExampleTest.class);
//#test-header
//#under-test
@ -42,15 +43,15 @@ public class BasicAsyncTestingTest extends TestKit {
//#test-shutdown
@AfterClass
public void cleanup() {
this.shutdown();
testKit.shutdownTestKit();
}
//#test-shutdown
@Test
public void testVerifyingAResponse() {
//#test-spawn
TestProbe<Pong> probe = TestProbe.create(system());
ActorRef<Ping> pinger = spawn(echoActor, "ping");
TestProbe<Pong> probe = testKit.createTestProbe();
ActorRef<Ping> pinger = testKit.spawn(echoActor, "ping");
pinger.tell(new Ping("hello", probe.ref()));
probe.expectMessage(new Pong("hello"));
//#test-spawn
@ -59,8 +60,8 @@ public class BasicAsyncTestingTest extends TestKit {
@Test
public void testVerifyingAResponseAnonymous() {
//#test-spawn-anonymous
TestProbe<Pong> probe = TestProbe.create(system());
ActorRef<Ping> pinger = spawn(echoActor);
TestProbe<Pong> probe = testKit.createTestProbe();
ActorRef<Ping> pinger = testKit.spawn(echoActor);
pinger.tell(new Ping("hello", probe.ref()));
probe.expectMessage(new Pong("hello"));
//#test-spawn-anonymous

View file

@ -0,0 +1,23 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.javadsl;
// #junit-integration
import akka.testkit.typed.javadsl.TestKitJunitResource;
import org.junit.ClassRule;
import org.junit.Test;
public class JunitIntegrationExampleTest {
@ClassRule
public static final TestKitJunitResource testKit = new TestKitJunitResource();
@Test
public void testSomething() {
TestProbe<String> probe = testKit.createTestProbe();
// ... assertions etc.
}
}
// #junit-integration

View file

@ -0,0 +1,55 @@
/**
* Copyright (C) 2017 Lightbend Inc. <http://www.lightbend.com/>
*/
package akka.testkit.typed.javadsl;
//#manual-scheduling-simple
import java.util.concurrent.TimeUnit;
import akka.actor.typed.Behavior;
import akka.testkit.typed.javadsl.ManualTime;
import akka.testkit.typed.javadsl.TestKitJunitResource;
import org.junit.ClassRule;
import org.scalatest.junit.JUnitSuite;
import scala.concurrent.duration.Duration;
import akka.actor.typed.javadsl.Behaviors;
import org.junit.Test;
import akka.testkit.typed.javadsl.TestProbe;
public class ManualTimerExampleTest extends JUnitSuite {
@ClassRule
public static final TestKitJunitResource testKit = new TestKitJunitResource(ManualTime.config());
private final ManualTime manualTime = ManualTime.get(testKit.system());
static final class Tick {}
static final class Tock {}
@Test
public void testScheduleNonRepeatedTicks() {
TestProbe<Tock> probe = testKit.createTestProbe();
Behavior<Tick> behavior = Behaviors.withTimers(timer -> {
timer.startSingleTimer("T", new Tick(), Duration.create(10, TimeUnit.MILLISECONDS));
return Behaviors.immutable( (ctx, tick) -> {
probe.ref().tell(new Tock());
return Behaviors.same();
});
});
testKit.spawn(behavior);
manualTime.expectNoMessageFor(Duration.create(9, TimeUnit.MILLISECONDS), probe);
manualTime.timePasses(Duration.create(2, TimeUnit.MILLISECONDS));
probe.expectMessageClass(Tock.class);
manualTime.expectNoMessageFor(Duration.create(10, TimeUnit.SECONDS), probe);
}
}
//#manual-scheduling-simple

View file

@ -1,14 +1,14 @@
package jdocs.akka.typed.testing.sync;
package akka.testkit.typed.javadsl;
//#imports
import akka.actor.typed.*;
import akka.actor.typed.javadsl.*;
import akka.testkit.typed.*;
import akka.testkit.typed.javadsl.*;
//#imports
import org.junit.Test;
import org.scalatest.junit.JUnitSuite;
public class BasicSyncTestingTest extends JUnitSuite {
public class SyncTestingExampleTest extends JUnitSuite {
//#child
public static Behavior<String> childActor = Behaviors.immutable((ctx, msg) -> Behaviors.same());
@ -66,49 +66,49 @@ public class BasicSyncTestingTest extends JUnitSuite {
@Test
public void testSpawning() {
//#test-child
BehaviorTestkit<Command> test = BehaviorTestkit.create(myBehavior);
BehaviorTestKit<Command> test = BehaviorTestKit.create(myBehavior);
test.run(new CreateAChild("child"));
test.expectEffect(new Effect.Spawned(childActor, "child", Props.empty()));
test.expectEffect(Effects.spawned(childActor, "child", Props.empty()));
//#test-child
}
@Test
public void testSpawningAnonymous() {
//#test-anonymous-child
BehaviorTestkit<Command> test = BehaviorTestkit.create(myBehavior);
BehaviorTestKit<Command> test = BehaviorTestKit.create(myBehavior);
test.run(new CreateAnAnonymousChild());
test.expectEffect(new Effect.SpawnedAnonymous(childActor, Props.empty()));
test.expectEffect(Effects.spawnedAnonymous(childActor, Props.empty()));
//#test-anonymous-child
}
@Test
public void testRecodingMessageSend() {
//#test-message
BehaviorTestkit<Command> test = BehaviorTestkit.create(myBehavior);
TestInbox<String> inbox = new TestInbox<String>();
test.run(new SayHello(inbox.ref()));
inbox.expectMsg("hello");
BehaviorTestKit<Command> test = BehaviorTestKit.create(myBehavior);
TestInbox<String> inbox = TestInbox.create();
test.run(new SayHello(inbox.getRef()));
inbox.expectMessage("hello");
//#test-message
}
@Test
public void testMessageToChild() {
//#test-child-message
BehaviorTestkit<Command> testKit = BehaviorTestkit.create(myBehavior);
BehaviorTestKit<Command> testKit = BehaviorTestKit.create(myBehavior);
testKit.run(new SayHelloToChild("child"));
TestInbox<String> childInbox = testKit.childInbox("child");
childInbox.expectMsg("hello");
childInbox.expectMessage("hello");
//#test-child-message
}
@Test
public void testMessageToAnonymousChild() {
//#test-child-message-anonymous
BehaviorTestkit<Command> testKit = BehaviorTestkit.create(myBehavior);
BehaviorTestKit<Command> testKit = BehaviorTestKit.create(myBehavior);
testKit.run(new SayHelloToAnonymousChild());
// Anonymous actors are created as: $a $b etc
TestInbox<String> childInbox = testKit.childInbox("$a");
childInbox.expectMsg("hello stranger");
childInbox.expectMessage("hello stranger");
//#test-child-message-anonymous
}
}

View file

@ -0,0 +1,16 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.scaladsl
// NOTE: do not optimize import, unused import is here on purpose for docs
//#scalatest-glue
import akka.testkit.typed.scaladsl.ActorTestKit
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
abstract class AbstractActorSpec extends WordSpec with ActorTestKit with Matchers with BeforeAndAfterAll {
override protected def afterAll(): Unit = {
shutdownTestKit()
}
}
//#scalatest-glue

View file

@ -0,0 +1,66 @@
/**
* Copyright (C) 2009-2018 Lightbend Inc. <http://www.lightbend.com>
*/
package akka.testkit.typed.scaladsl
import akka.actor.typed.Terminated
import akka.actor.typed.scaladsl.Behaviors
import org.scalatest.concurrent.ScalaFutures
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
import scala.concurrent.Promise
// can be mixed into any spec style,
class ActorTestKitSpec extends WordSpec with Matchers with ActorTestKit with ScalaFutures {
"the Scala testkit" should {
"generate a default name from the test class" in {
system.name should ===("ActorTestKitSpec")
}
"spawn an actor" in {
val sawMessage = Promise[Boolean]()
val ref = spawn(Behaviors.setup[AnyRef] { ctx
sawMessage.trySuccess(true)
Behaviors.empty
})
sawMessage.future.futureValue should ===(true)
}
"spawn a named actor" in {
val spawnedWithName = Promise[String]()
val ref = spawn(Behaviors.setup[AnyRef] { ctx
spawnedWithName.trySuccess(ctx.self.path.name)
Behaviors.empty
}, "name")
spawnedWithName.future.futureValue should ===("name")
}
"stop the actor system" in {
// usually done in test framework hook method but we want to assert
shutdownTestKit()
system.whenTerminated.futureValue shouldBe a[Terminated]
}
}
}
// derivate classes should also work fine (esp the naming part
trait MyBaseSpec extends WordSpec with ActorTestKit with Matchers with BeforeAndAfterAll {
override protected def afterAll(): Unit = {
shutdownTestKit()
}
}
class MyConcreteDerivateSpec extends MyBaseSpec {
"A derivate test" should {
"generate a default name from the test class" in {
system.name should ===("MyConcreteDerivateSpec")
}
}
}

View file

@ -1,12 +1,10 @@
package docs.akka.typed.testing.async
package akka.testkit.typed.scaladsl
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed._
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl._
import org.scalatest._
import akka.actor.typed.scaladsl.Behaviors
import org.scalatest.{ BeforeAndAfterAll, WordSpec }
object BasicAsyncTestingSpec {
object AsyncTestingExampleSpec {
//#under-test
case class Ping(msg: String, response: ActorRef[Pong])
case class Pong(msg: String)
@ -22,11 +20,10 @@ object BasicAsyncTestingSpec {
}
//#test-header
class BasicAsyncTestingSpec extends TestKit("BasicTestingSpec")
with WordSpecLike with BeforeAndAfterAll {
class AsyncTestingExampleSpec extends WordSpec with ActorTestKit with BeforeAndAfterAll {
//#test-header
import BasicAsyncTestingSpec._
import AsyncTestingExampleSpec._
"A testkit" must {
"support verifying a response" in {
@ -49,6 +46,6 @@ class BasicAsyncTestingSpec extends TestKit("BasicTestingSpec")
}
//#test-shutdown
override def afterAll(): Unit = shutdown()
override def afterAll(): Unit = shutdownTestKit()
//#test-shutdown
}

View file

@ -2,16 +2,16 @@
* Copyright (C) 2014-2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.testkit.typed
package akka.testkit.typed.scaladsl
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ Behavior, Props }
import akka.testkit.typed.BehaviorTestkitSpec.Father._
import akka.testkit.typed.BehaviorTestkitSpec.{ Child, Father }
import akka.testkit.typed.Effect.{ Spawned, SpawnedAdapter, SpawnedAnonymous }
import akka.testkit.typed.scaladsl.Effects.{ Spawned, SpawnedAdapter, SpawnedAnonymous }
import akka.testkit.typed.scaladsl.BehaviorTestKitSpec.{ Child, Father }
import akka.testkit.typed.scaladsl.BehaviorTestKitSpec.Father._
import org.scalatest.{ Matchers, WordSpec }
object BehaviorTestkitSpec {
object BehaviorTestKitSpec {
object Father {
case class Reproduce(times: Int)
@ -78,20 +78,20 @@ object BehaviorTestkitSpec {
}
class BehaviorTestkitSpec extends WordSpec with Matchers {
class BehaviorTestKitSpec extends WordSpec with Matchers {
private val props = Props.empty
"BehaviorTestkit's spawn" must {
"create children when no props specified" in {
val testkit = BehaviorTestkit[Father.Command](Father.init())
val testkit = BehaviorTestKit[Father.Command](Father.init())
testkit.run(SpawnChildren(2))
val effects = testkit.retrieveAllEffects()
effects should contain only (Spawned(Child.initial, "child0"), Spawned(Child.initial, "child1", Props.empty))
}
"create children when props specified and record effects" in {
val testkit = BehaviorTestkit[Father.Command](Father.init())
val testkit = BehaviorTestKit[Father.Command](Father.init())
testkit.run(SpawnChildrenWithProps(2, props))
val effects = testkit.retrieveAllEffects()
effects should contain only (Spawned(Child.initial, "child0", props), Spawned(Child.initial, "child1", props))
@ -100,14 +100,14 @@ class BehaviorTestkitSpec extends WordSpec with Matchers {
"BehaviorTestkit's spawnAnonymous" must {
"create children when no props specified and record effects" in {
val testkit = BehaviorTestkit[Father.Command](Father.init())
val testkit = BehaviorTestKit[Father.Command](Father.init())
testkit.run(SpawnAnonymous(2))
val effects = testkit.retrieveAllEffects()
effects shouldBe Seq(SpawnedAnonymous(Child.initial, Props.empty), SpawnedAnonymous(Child.initial, Props.empty))
}
"create children when props specified and record effects" in {
val testkit = BehaviorTestkit[Father.Command](Father.init())
val testkit = BehaviorTestKit[Father.Command](Father.init())
testkit.run(SpawnAnonymousWithProps(2, props))
val effects = testkit.retrieveAllEffects()
@ -117,14 +117,14 @@ class BehaviorTestkitSpec extends WordSpec with Matchers {
"BehaviorTestkit's spawnMessageAdapter" must {
"create adapters without name and record effects" in {
val testkit = BehaviorTestkit[Father.Command](Father.init())
val testkit = BehaviorTestKit[Father.Command](Father.init())
testkit.run(SpawnAdapter)
val effects = testkit.retrieveAllEffects()
effects shouldBe Seq(SpawnedAdapter)
}
"create adapters with name and record effects" in {
val testkit = BehaviorTestkit[Father.Command](Father.init())
val testkit = BehaviorTestKit[Father.Command](Father.init())
testkit.run(SpawnAdapterWithName("adapter"))
val effects = testkit.retrieveAllEffects()
effects shouldBe Seq(SpawnedAdapter)
@ -133,7 +133,7 @@ class BehaviorTestkitSpec extends WordSpec with Matchers {
"BehaviorTestkit's run" can {
"run behaviors with messages without canonicalization" in {
val testkit = BehaviorTestkit[Father.Command](Father.init())
val testkit = BehaviorTestKit[Father.Command](Father.init())
testkit.run(SpawnAdapterWithName("adapter"))
testkit.currentBehavior should not be Behavior.same
testkit.returnedBehavior shouldBe Behavior.same

View file

@ -1,16 +1,16 @@
package akka.actor.typed
package akka.testkit.typed.scaladsl
//#manual-scheduling-simple
import scala.concurrent.duration._
import akka.actor.typed.scaladsl.Behaviors
import org.scalatest.WordSpecLike
import akka.testkit.typed.TestKit
import akka.testkit.typed.scaladsl.{ ManualTime, TestProbe }
import scala.concurrent.duration._
class ManualTimerSpec extends TestKit(ManualTime.config) with ManualTime with WordSpecLike {
class ManualTimerExampleSpec extends AbstractActorSpec {
override def config = ManualTime.config
val manualTime = ManualTime()
"A timer" must {
"schedule non-repeated ticks" in {
@ -28,12 +28,12 @@ class ManualTimerSpec extends TestKit(ManualTime.config) with ManualTime with Wo
spawn(behavior)
scheduler.expectNoMessageFor(9.millis, probe)
manualTime.expectNoMessageFor(9.millis, probe)
scheduler.timePasses(2.millis)
manualTime.timePasses(2.millis)
probe.expectMessage(Tock)
scheduler.expectNoMessageFor(10.seconds, probe)
manualTime.expectNoMessageFor(10.seconds, probe)
}
//#manual-scheduling-simple
@ -53,9 +53,9 @@ class ManualTimerSpec extends TestKit(ManualTime.config) with ManualTime with Wo
spawn(behavior)
for (_ Range(0, 5)) {
scheduler.expectNoMessageFor(9.millis, probe)
manualTime.expectNoMessageFor(9.millis, probe)
scheduler.timePasses(1.milli)
manualTime.timePasses(1.milli)
probe.expectMessage(Tock)
}
}
@ -78,7 +78,7 @@ class ManualTimerSpec extends TestKit(ManualTime.config) with ManualTime with Wo
probe.ref ! Tock(n)
Behaviors.same
case SlowThenBump(nextCount)
scheduler.timePasses(interval)
manualTime.timePasses(interval)
timer.startPeriodicTimer("T", Tick(nextCount), interval)
Behaviors.same
}
@ -87,14 +87,14 @@ class ManualTimerSpec extends TestKit(ManualTime.config) with ManualTime with Wo
val ref = spawn(behavior)
scheduler.timePasses(11.millis)
manualTime.timePasses(11.millis)
probe.expectMessage(Tock(1))
// next Tock(1) enqueued in mailboxed, but should be discarded because of new timer
ref ! SlowThenBump(2)
scheduler.expectNoMessageFor(interval, probe)
manualTime.expectNoMessageFor(interval, probe)
scheduler.timePasses(interval)
manualTime.timePasses(interval)
probe.expectMessage(Tock(2))
}

View file

@ -1,14 +1,13 @@
package docs.akka.typed.testing.sync
package akka.testkit.typed.scaladsl
//#imports
import akka.actor.typed._
import akka.actor.typed.scaladsl._
import akka.testkit.typed._
import akka.testkit.typed.Effect._
import akka.testkit.typed.scaladsl.Effects._
//#imports
import org.scalatest.{ Matchers, WordSpec }
object BasicSyncTestingSpec {
object SyncTestingExampleSpec {
//#child
val childActor = Behaviors.immutable[String] { (_, _)
Behaviors.same[String]
@ -46,15 +45,15 @@ object BasicSyncTestingSpec {
}
class BasicSyncTestingSpec extends WordSpec with Matchers {
class SyncTestingExampleSpec extends WordSpec with Matchers {
import BasicSyncTestingSpec._
import SyncTestingExampleSpec._
"Typed actor synchronous testing" must {
"record spawning" in {
//#test-child
val testKit = BehaviorTestkit(myBehavior)
val testKit = BehaviorTestKit(myBehavior)
testKit.run(CreateChild("child"))
testKit.expectEffect(Spawned(childActor, "child"))
//#test-child
@ -62,7 +61,7 @@ class BasicSyncTestingSpec extends WordSpec with Matchers {
"record spawning anonymous" in {
//#test-anonymous-child
val testKit = BehaviorTestkit(myBehavior)
val testKit = BehaviorTestKit(myBehavior)
testKit.run(CreateAnonymousChild)
testKit.expectEffect(SpawnedAnonymous(childActor))
//#test-anonymous-child
@ -70,29 +69,29 @@ class BasicSyncTestingSpec extends WordSpec with Matchers {
"record message sends" in {
//#test-message
val testKit = BehaviorTestkit(myBehavior)
val testKit = BehaviorTestKit(myBehavior)
val inbox = TestInbox[String]()
testKit.run(SayHello(inbox.ref))
inbox.expectMsg("hello")
inbox.expectMessage("hello")
//#test-message
}
"send a message to a spawned child" in {
//#test-child-message
val testKit = BehaviorTestkit(myBehavior)
val testKit = BehaviorTestKit(myBehavior)
testKit.run(SayHelloToChild("child"))
val childInbox = testKit.childInbox[String]("child")
childInbox.expectMsg("hello")
childInbox.expectMessage("hello")
//#test-child-message
}
"send a message to an anonymous spawned child" in {
//#test-child-message-anonymous
val testKit = BehaviorTestkit(myBehavior)
val testKit = BehaviorTestKit(myBehavior)
testKit.run(SayHelloToAnonymousChild)
// Anonymous actors are created as: $a $b etc
val childInbox = testKit.childInbox[String](s"$$a")
childInbox.expectMsg("hello stranger")
childInbox.expectMessage("hello stranger")
//#test-child-message-anonymous
}
}

View file

@ -4,11 +4,11 @@
package akka.testkit.typed.scaladsl
import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.typed.TestKit
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpecLike }
import scala.concurrent.duration._
class TestProbeSpec extends TestKit with WordSpecLike with Matchers with BeforeAndAfterAll {
class TestProbeSpec extends AbstractActorSpec {
def compileOnlyApiTest(): Unit = {
val probe = TestProbe[AnyRef]()
@ -111,7 +111,4 @@ class TestProbeSpec extends TestKit with WordSpecLike with Matchers with BeforeA
}
override protected def afterAll(): Unit = {
shutdown()
}
}

View file

@ -440,6 +440,7 @@ lazy val streamTyped = akkaModule("akka-stream-typed")
lazy val typedTestkit = akkaModule("akka-testkit-typed")
.dependsOn(actorTyped, testkit % "compile->compile;test->test")
.settings(AutomaticModuleName.settings("akka.testkit.typed"))
.settings(Dependencies.typedTestkit)
.disablePlugins(MimaPlugin)
lazy val actorTypedTests = akkaModule("akka-actor-typed-tests")

View file

@ -65,7 +65,6 @@ object Dependencies {
val lmdb = "org.lmdbjava" % "lmdbjava" % "0.6.0" // ApacheV2, OpenLDAP Public License
// For akka-http-testkit-java
val junit = "junit" % "junit" % junitVersion // Common Public License 1.0
// For Java 8 Conversions
@ -118,6 +117,8 @@ object Dependencies {
val levelDB = "org.iq80.leveldb" % "leveldb" % "0.10" % "optional;provided" // ApacheV2
val levelDBNative = "org.fusesource.leveldbjni" % "leveldbjni-all" % "1.8" % "optional;provided" // New BSD
val junit = Compile.junit % "optional;provided;test"
}
}
@ -133,6 +134,8 @@ object Dependencies {
val actorTests = l ++= Seq(Test.junit, Test.scalatest.value, Test.commonsCodec, Test.commonsMath,
Test.mockito, Test.scalacheck.value, Test.jimfs)
val typedTestkit = l ++= Seq(Provided.junit)
val remote = l ++= Seq(netty, aeronDriver, aeronClient, Test.junit, Test.scalatest.value, Test.jimfs)
val remoteTests = l ++= Seq(Test.junit, Test.scalatest.value, Test.scalaXml)