Typed async testkit (#24203)
* Typed async testkit * Pull out useful parts from internal TypedSpec into a public TestKit * Port internal tests to use the external testkit
This commit is contained in:
parent
c7cbebb534
commit
bd2a3de88a
44 changed files with 716 additions and 836 deletions
|
|
@ -4,7 +4,6 @@
|
||||||
package jdocs.akka.typed.testing.async;
|
package jdocs.akka.typed.testing.async;
|
||||||
|
|
||||||
import akka.actor.typed.ActorRef;
|
import akka.actor.typed.ActorRef;
|
||||||
import akka.actor.typed.ActorSystem;
|
|
||||||
import akka.actor.typed.Behavior;
|
import akka.actor.typed.Behavior;
|
||||||
import akka.actor.typed.javadsl.Actor;
|
import akka.actor.typed.javadsl.Actor;
|
||||||
import akka.testkit.typed.javadsl.TestProbe;
|
import akka.testkit.typed.javadsl.TestProbe;
|
||||||
|
|
@ -15,7 +14,7 @@ import org.junit.Test;
|
||||||
//#test-header
|
//#test-header
|
||||||
public class BasicAsyncTestingTest extends TestKit {
|
public class BasicAsyncTestingTest extends TestKit {
|
||||||
public BasicAsyncTestingTest() {
|
public BasicAsyncTestingTest() {
|
||||||
super(ActorSystem.create(Actor.empty(), "BasicAsyncTestingTest"));
|
super("BasicAsyncTestingTest");
|
||||||
}
|
}
|
||||||
//#test-header
|
//#test-header
|
||||||
|
|
||||||
|
|
@ -52,11 +51,21 @@ public class BasicAsyncTestingTest extends TestKit {
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testVerifyingAResponse() {
|
public void testVerifyingAResponse() {
|
||||||
//#test
|
//#test-spawn
|
||||||
TestProbe<Pong> probe = new TestProbe<>(system(), testkitSettings());
|
TestProbe<Pong> probe = new TestProbe<>(system(), testkitSettings());
|
||||||
ActorRef<Ping> pinger = actorOf(echoActor, "ping");
|
ActorRef<Ping> pinger = spawn(echoActor, "ping");
|
||||||
pinger.tell(new Ping("hello", probe.ref()));
|
pinger.tell(new Ping("hello", probe.ref()));
|
||||||
probe.expectMsg(new Pong("hello"));
|
probe.expectMsg(new Pong("hello"));
|
||||||
//#test
|
//#test-spawn
|
||||||
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testVerifyingAResponseAnonymous() {
|
||||||
|
//#test-spawn-anonymous
|
||||||
|
TestProbe<Pong> probe = new TestProbe<>(system(), testkitSettings());
|
||||||
|
ActorRef<Ping> pinger = spawn(echoActor);
|
||||||
|
pinger.tell(new Ping("hello", probe.ref()));
|
||||||
|
probe.expectMsg(new Pong("hello"));
|
||||||
|
//#test-spawn-anonymous
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -3,13 +3,19 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor.typed
|
package akka.actor.typed
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import akka.actor.typed.scaladsl.Actor._
|
||||||
import scala.concurrent.Future
|
import akka.actor.typed.scaladsl.{ Actor, AskPattern }
|
||||||
|
import akka.actor.{ ActorInitializationException, DeadLetterSuppression, InvalidMessageException }
|
||||||
|
import akka.testkit.AkkaSpec
|
||||||
|
import akka.testkit.TestEvent.Mute
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import akka.actor.{ DeadLetterSuppression, InvalidMessageException }
|
import org.scalactic.CanEqual
|
||||||
import akka.actor.typed.scaladsl.Actor
|
|
||||||
|
|
||||||
|
import scala.concurrent.{ Await, Future }
|
||||||
|
import scala.concurrent.duration._
|
||||||
import scala.language.existentials
|
import scala.language.existentials
|
||||||
|
import scala.reflect.ClassTag
|
||||||
|
import scala.util.control.{ NoStackTrace, NonFatal }
|
||||||
|
|
||||||
object ActorContextSpec {
|
object ActorContextSpec {
|
||||||
|
|
||||||
|
|
@ -227,8 +233,8 @@ object ActorContextSpec {
|
||||||
case BecomeInert(replyTo) ⇒
|
case BecomeInert(replyTo) ⇒
|
||||||
replyTo ! BecameInert
|
replyTo ! BecameInert
|
||||||
Actor.immutable[Command] {
|
Actor.immutable[Command] {
|
||||||
case (_, Ping(replyTo)) ⇒
|
case (_, Ping(r)) ⇒
|
||||||
replyTo ! Pong2
|
r ! Pong2
|
||||||
Actor.same
|
Actor.same
|
||||||
case (_, Throw(ex)) ⇒
|
case (_, Throw(ex)) ⇒
|
||||||
throw ex
|
throw ex
|
||||||
|
|
@ -257,21 +263,136 @@ object ActorContextSpec {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
sealed abstract class Start
|
||||||
|
case object Start extends Start
|
||||||
|
|
||||||
|
sealed trait GuardianCommand
|
||||||
|
case class RunTest[T](name: String, behavior: Behavior[T], replyTo: ActorRef[Status], timeout: FiniteDuration) extends GuardianCommand
|
||||||
|
case class Terminate(reply: ActorRef[Status]) extends GuardianCommand
|
||||||
|
case class Create[T](behavior: Behavior[T], name: String)(val replyTo: ActorRef[ActorRef[T]]) extends GuardianCommand
|
||||||
|
|
||||||
|
sealed trait Status
|
||||||
|
case object Success extends Status
|
||||||
|
case class Failed(thr: Throwable) extends Status
|
||||||
|
case object Timedout extends Status
|
||||||
|
|
||||||
|
class SimulatedException(message: String) extends RuntimeException(message) with NoStackTrace
|
||||||
|
|
||||||
|
def guardian(outstanding: Map[ActorRef[_], ActorRef[Status]] = Map.empty): Behavior[GuardianCommand] =
|
||||||
|
Actor.immutable[GuardianCommand] {
|
||||||
|
case (ctx, r: RunTest[t]) ⇒
|
||||||
|
val test = ctx.spawn(r.behavior, r.name)
|
||||||
|
ctx.schedule(r.timeout, r.replyTo, Timedout)
|
||||||
|
ctx.watch(test)
|
||||||
|
guardian(outstanding + ((test, r.replyTo)))
|
||||||
|
case (_, Terminate(reply)) ⇒
|
||||||
|
reply ! Success
|
||||||
|
stopped
|
||||||
|
case (ctx, c: Create[t]) ⇒
|
||||||
|
c.replyTo ! ctx.spawn(c.behavior, c.name)
|
||||||
|
same
|
||||||
|
} onSignal {
|
||||||
|
case (ctx, t @ Terminated(test)) ⇒
|
||||||
|
outstanding get test match {
|
||||||
|
case Some(reply) ⇒
|
||||||
|
if (t.failure eq null) reply ! Success
|
||||||
|
else reply ! Failed(t.failure)
|
||||||
|
guardian(outstanding - test)
|
||||||
|
case None ⇒ same
|
||||||
|
}
|
||||||
|
case _ ⇒ same
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
abstract class ActorContextSpec extends TypedSpec(ConfigFactory.parseString(
|
abstract class ActorContextSpec extends TypedAkkaSpec {
|
||||||
"""|akka {
|
import ActorContextSpec._
|
||||||
|
|
||||||
|
val config = ConfigFactory.parseString(
|
||||||
|
"""|akka {
|
||||||
| loglevel = WARNING
|
| loglevel = WARNING
|
||||||
| actor.debug {
|
| actor.debug {
|
||||||
| lifecycle = off
|
| lifecycle = off
|
||||||
| autoreceive = off
|
| autoreceive = off
|
||||||
| }
|
| }
|
||||||
| typed.loggers = ["akka.testkit.typed.TestEventListener"]
|
| typed.loggers = ["akka.testkit.typed.TestEventListener"]
|
||||||
|}""".stripMargin)) {
|
|}""".stripMargin)
|
||||||
|
|
||||||
import ActorContextSpec._
|
implicit lazy val system: ActorSystem[GuardianCommand] =
|
||||||
|
ActorSystem(guardian(), AkkaSpec.getCallerName(classOf[ActorContextSpec]), config = Some(config withFallback AkkaSpec.testConf))
|
||||||
|
|
||||||
val expectTimeout = 3.seconds
|
val expectTimeout = 3.seconds
|
||||||
|
import AskPattern._
|
||||||
|
|
||||||
|
implicit def scheduler = system.scheduler
|
||||||
|
|
||||||
|
lazy val blackhole = await(system ? Create(immutable[Any] { case _ ⇒ same }, "blackhole"))
|
||||||
|
|
||||||
|
override def afterAll(): Unit = {
|
||||||
|
Await.result(system.terminate, timeout.duration)
|
||||||
|
}
|
||||||
|
|
||||||
|
// TODO remove after basing on ScalaTest 3 with async support
|
||||||
|
import akka.testkit._
|
||||||
|
|
||||||
|
def await[T](f: Future[T]): T = Await.result(f, timeout.duration * 1.1)
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Run an Actor-based test. The test procedure is most conveniently
|
||||||
|
* formulated using the [[StepWise]] behavior type.
|
||||||
|
*/
|
||||||
|
def runTest[T: ClassTag](name: String)(behavior: Behavior[T])(implicit system: ActorSystem[GuardianCommand]): Future[Status] =
|
||||||
|
system ? (RunTest(name, behavior, _, timeout.duration))
|
||||||
|
|
||||||
|
// TODO remove after basing on ScalaTest 3 with async support
|
||||||
|
def sync(f: Future[Status])(implicit system: ActorSystem[GuardianCommand]): Unit = {
|
||||||
|
def unwrap(ex: Throwable): Throwable = ex match {
|
||||||
|
case ActorInitializationException(_, _, ex) ⇒ ex
|
||||||
|
case other ⇒ other
|
||||||
|
}
|
||||||
|
|
||||||
|
try await(f) match {
|
||||||
|
case Success ⇒ ()
|
||||||
|
case Failed(ex) ⇒
|
||||||
|
unwrap(ex) match {
|
||||||
|
case ex2: SimulatedException ⇒
|
||||||
|
throw ex2
|
||||||
|
case _ ⇒
|
||||||
|
println(system.printTree)
|
||||||
|
throw unwrap(ex)
|
||||||
|
}
|
||||||
|
case Timedout ⇒
|
||||||
|
println(system.printTree)
|
||||||
|
fail("test timed out")
|
||||||
|
} catch {
|
||||||
|
case ex: SimulatedException ⇒
|
||||||
|
throw ex
|
||||||
|
case NonFatal(ex) ⇒
|
||||||
|
println(system.printTree)
|
||||||
|
throw ex
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
def muteExpectedException[T <: Exception: ClassTag](
|
||||||
|
message: String = null,
|
||||||
|
source: String = null,
|
||||||
|
start: String = "",
|
||||||
|
pattern: String = null,
|
||||||
|
occurrences: Int = Int.MaxValue)(implicit system: ActorSystem[GuardianCommand]): EventFilter = {
|
||||||
|
val filter = EventFilter(message, source, start, pattern, occurrences)
|
||||||
|
system.eventStream.publish(Mute(filter))
|
||||||
|
filter
|
||||||
|
}
|
||||||
|
|
||||||
|
// for ScalaTest === compare of Class objects
|
||||||
|
implicit def classEqualityConstraint[A, B]: CanEqual[Class[A], Class[B]] =
|
||||||
|
new CanEqual[Class[A], Class[B]] {
|
||||||
|
def areEqual(a: Class[A], b: Class[B]) = a == b
|
||||||
|
}
|
||||||
|
|
||||||
|
implicit def setEqualityConstraint[A, T <: Set[_ <: A]]: CanEqual[Set[A], T] =
|
||||||
|
new CanEqual[Set[A], T] {
|
||||||
|
def areEqual(a: Set[A], b: T) = a == b
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The name for the set of tests to be instantiated, used for keeping the test case actors’ names unique.
|
* The name for the set of tests to be instantiated, used for keeping the test case actors’ names unique.
|
||||||
|
|
@ -286,7 +407,7 @@ abstract class ActorContextSpec extends TypedSpec(ConfigFactory.parseString(
|
||||||
private def mySuite: String = suite + "Adapted"
|
private def mySuite: String = suite + "Adapted"
|
||||||
|
|
||||||
def setup(name: String, wrapper: Option[Behavior[Command] ⇒ Behavior[Command]] = None, ignorePostStop: Boolean = true)(
|
def setup(name: String, wrapper: Option[Behavior[Command] ⇒ Behavior[Command]] = None, ignorePostStop: Boolean = true)(
|
||||||
proc: (scaladsl.ActorContext[Event], StepWise.Steps[Event, ActorRef[Command]]) ⇒ StepWise.Steps[Event, _]): Future[TypedSpec.Status] =
|
proc: (scaladsl.ActorContext[Event], StepWise.Steps[Event, ActorRef[Command]]) ⇒ StepWise.Steps[Event, _]): Future[Status] =
|
||||||
runTest(s"$mySuite-$name")(StepWise[Event] { (ctx, startWith) ⇒
|
runTest(s"$mySuite-$name")(StepWise[Event] { (ctx, startWith) ⇒
|
||||||
val b = behavior(ctx, ignorePostStop)
|
val b = behavior(ctx, ignorePostStop)
|
||||||
val props = wrapper.map(_(b)).getOrElse(b)
|
val props = wrapper.map(_(b)).getOrElse(b)
|
||||||
|
|
@ -671,7 +792,7 @@ abstract class ActorContextSpec extends TypedSpec(ConfigFactory.parseString(
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
import ActorContextSpec._
|
import akka.actor.typed.ActorContextSpec._
|
||||||
|
|
||||||
class NormalActorContextSpec extends ActorContextSpec {
|
class NormalActorContextSpec extends ActorContextSpec {
|
||||||
override def suite = "normal"
|
override def suite = "normal"
|
||||||
|
|
@ -705,4 +826,3 @@ class TapActorContextSpec extends ActorContextSpec {
|
||||||
override def behavior(ctx: scaladsl.ActorContext[Event], ignorePostStop: Boolean): Behavior[Command] =
|
override def behavior(ctx: scaladsl.ActorContext[Event], ignorePostStop: Boolean): Behavior[Command] =
|
||||||
Actor.tap((_, _) ⇒ (), (_, _) ⇒ (), subject(ctx.self, ignorePostStop))
|
Actor.tap((_, _) ⇒ (), (_, _) ⇒ (), subject(ctx.self, ignorePostStop))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,13 +3,15 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor.typed
|
package akka.actor.typed
|
||||||
|
|
||||||
import scala.concurrent.ExecutionContext
|
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
||||||
import scala.concurrent.duration._
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import org.scalatest.concurrent.ScalaFutures
|
|
||||||
import akka.util.Timeout
|
|
||||||
import akka.pattern.AskTimeoutException
|
|
||||||
import akka.actor.typed.scaladsl.Actor._
|
import akka.actor.typed.scaladsl.Actor._
|
||||||
import akka.actor.typed.scaladsl.AskPattern._
|
import akka.actor.typed.scaladsl.AskPattern._
|
||||||
|
import akka.pattern.AskTimeoutException
|
||||||
|
import akka.testkit.typed.TestKit
|
||||||
|
import org.scalatest.concurrent.ScalaFutures
|
||||||
|
|
||||||
|
import scala.concurrent.ExecutionContext
|
||||||
|
|
||||||
object AskSpec {
|
object AskSpec {
|
||||||
sealed trait Msg
|
sealed trait Msg
|
||||||
|
|
@ -17,7 +19,8 @@ object AskSpec {
|
||||||
final case class Stop(replyTo: ActorRef[Unit]) extends Msg
|
final case class Stop(replyTo: ActorRef[Unit]) extends Msg
|
||||||
}
|
}
|
||||||
|
|
||||||
class AskSpec extends TypedSpec with ScalaFutures {
|
class AskSpec extends TestKit("AskSpec") with TypedAkkaSpec with ScalaFutures {
|
||||||
|
|
||||||
import AskSpec._
|
import AskSpec._
|
||||||
|
|
||||||
implicit def executor: ExecutionContext =
|
implicit def executor: ExecutionContext =
|
||||||
|
|
@ -26,43 +29,38 @@ class AskSpec extends TypedSpec with ScalaFutures {
|
||||||
val behavior: Behavior[Msg] = immutable[Msg] {
|
val behavior: Behavior[Msg] = immutable[Msg] {
|
||||||
case (_, foo: Foo) ⇒
|
case (_, foo: Foo) ⇒
|
||||||
foo.replyTo ! "foo"
|
foo.replyTo ! "foo"
|
||||||
same
|
Actor.same
|
||||||
case (_, Stop(r)) ⇒
|
case (_, Stop(r)) ⇒
|
||||||
r ! ()
|
r ! ()
|
||||||
stopped
|
Actor.stopped
|
||||||
}
|
}
|
||||||
|
|
||||||
"Ask pattern" must {
|
"Ask pattern" must {
|
||||||
"must fail the future if the actor is already terminated" in {
|
"must fail the future if the actor is already terminated" in {
|
||||||
val fut = for {
|
val ref = spawn(behavior)
|
||||||
ref ← system ? TypedSpec.Create(behavior, "test1")
|
(ref ? Stop).futureValue
|
||||||
_ ← ref ? Stop
|
val answer = ref ? Foo("bar")
|
||||||
answer ← ref.?(Foo("bar"))(Timeout(1.second), implicitly)
|
answer.recover { case _: AskTimeoutException ⇒ "ask" }.futureValue should ===("ask")
|
||||||
} yield answer
|
|
||||||
fut.recover { case _: AskTimeoutException ⇒ "" }.futureValue should ===("")
|
|
||||||
}
|
}
|
||||||
|
|
||||||
"must succeed when the actor is alive" in {
|
"must succeed when the actor is alive" in {
|
||||||
val fut = for {
|
val ref = spawn(behavior)
|
||||||
ref ← system ? TypedSpec.Create(behavior, "test2")
|
val response = ref ? Foo("bar")
|
||||||
answer ← ref ? Foo("bar")
|
response.futureValue should ===("foo")
|
||||||
} yield answer
|
|
||||||
fut.futureValue should ===("foo")
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/** See issue #19947 (MatchError with adapted ActorRef) */
|
/** See issue #19947 (MatchError with adapted ActorRef) */
|
||||||
"must fail the future if the actor doesn't exist" in {
|
"must fail the future if the actor doesn't exist" in {
|
||||||
val noSuchActor: ActorRef[Msg] = system match {
|
val noSuchActor: ActorRef[Msg] = system match {
|
||||||
case adaptedSys: akka.actor.typed.internal.adapter.ActorSystemAdapter[_] ⇒
|
case adaptedSys: ActorSystemAdapter[_] ⇒
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
adaptedSys.untyped.provider.resolveActorRef("/foo/bar")
|
adaptedSys.untyped.provider.resolveActorRef("/foo/bar")
|
||||||
case _ ⇒
|
case _ ⇒
|
||||||
fail("this test must only run in an adapted actor system")
|
fail("this test must only run in an adapted actor system")
|
||||||
}
|
}
|
||||||
val fut = for {
|
|
||||||
answer ← noSuchActor.?(Foo("bar"))(Timeout(1.second), implicitly)
|
val answer = noSuchActor ? Foo("bar")
|
||||||
} yield answer
|
answer.recover { case _: AskTimeoutException ⇒ "ask" }.futureValue should ===("ask")
|
||||||
(fut.recover { case _: AskTimeoutException ⇒ "" }).futureValue should ===("")
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -61,9 +61,8 @@ object BehaviorSpec {
|
||||||
override def next = StateA
|
override def next = StateA
|
||||||
}
|
}
|
||||||
|
|
||||||
trait Common extends TypedSpec {
|
trait Common extends TypedAkkaSpec {
|
||||||
type Aux >: Null <: AnyRef
|
type Aux >: Null <: AnyRef
|
||||||
def system: ActorSystem[TypedSpec.Command]
|
|
||||||
def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux)
|
def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux)
|
||||||
def checkAux(signal: Signal, aux: Aux): Unit = ()
|
def checkAux(signal: Signal, aux: Aux): Unit = ()
|
||||||
def checkAux(command: Command, aux: Aux): Unit = ()
|
def checkAux(command: Command, aux: Aux): Unit = ()
|
||||||
|
|
@ -339,11 +338,11 @@ object BehaviorSpec {
|
||||||
|
|
||||||
import BehaviorSpec._
|
import BehaviorSpec._
|
||||||
|
|
||||||
class FullBehaviorSpec extends TypedSpec with Messages with BecomeWithLifecycle with Stoppable {
|
class FullBehaviorSpec extends TypedAkkaSpec with Messages with BecomeWithLifecycle with Stoppable {
|
||||||
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = mkFull(monitor) → null
|
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = mkFull(monitor) → null
|
||||||
}
|
}
|
||||||
|
|
||||||
class ImmutableBehaviorSpec extends TypedSpec with Messages with BecomeWithLifecycle with Stoppable {
|
class ImmutableBehaviorSpec extends Messages with BecomeWithLifecycle with Stoppable {
|
||||||
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) → null
|
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) → null
|
||||||
private def behv(monitor: ActorRef[Event], state: State): Behavior[Command] = {
|
private def behv(monitor: ActorRef[Event], state: State): Behavior[Command] = {
|
||||||
SActor.immutable[Command] {
|
SActor.immutable[Command] {
|
||||||
|
|
@ -375,7 +374,7 @@ class ImmutableBehaviorSpec extends TypedSpec with Messages with BecomeWithLifec
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class ImmutableWithSignalScalaBehaviorSpec extends TypedSpec with Messages with BecomeWithLifecycle with Stoppable {
|
class ImmutableWithSignalScalaBehaviorSpec extends TypedAkkaSpec with Messages with BecomeWithLifecycle with Stoppable {
|
||||||
|
|
||||||
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor) → null
|
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor) → null
|
||||||
|
|
||||||
|
|
@ -411,7 +410,7 @@ class ImmutableWithSignalScalaBehaviorSpec extends TypedSpec with Messages with
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class ImmutableScalaBehaviorSpec extends TypedSpec with Messages with Become with Stoppable {
|
class ImmutableScalaBehaviorSpec extends Messages with Become with Stoppable {
|
||||||
|
|
||||||
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) → null
|
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) → null
|
||||||
|
|
||||||
|
|
@ -442,7 +441,7 @@ class ImmutableScalaBehaviorSpec extends TypedSpec with Messages with Become wit
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class MutableScalaBehaviorSpec extends TypedSpec with Messages with Become with Stoppable {
|
class MutableScalaBehaviorSpec extends Messages with Become with Stoppable {
|
||||||
|
|
||||||
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor) → null
|
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor) → null
|
||||||
|
|
||||||
|
|
@ -518,7 +517,7 @@ class RestarterScalaBehaviorSpec extends ImmutableWithSignalScalaBehaviorSpec wi
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class ImmutableWithSignalJavaBehaviorSpec extends TypedSpec with Messages with BecomeWithLifecycle with Stoppable {
|
class ImmutableWithSignalJavaBehaviorSpec extends Messages with BecomeWithLifecycle with Stoppable {
|
||||||
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor) → null
|
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor) → null
|
||||||
def behv(monitor: ActorRef[Event], state: State = StateA): Behavior[Command] =
|
def behv(monitor: ActorRef[Event], state: State = StateA): Behavior[Command] =
|
||||||
JActor.immutable(
|
JActor.immutable(
|
||||||
|
|
@ -550,7 +549,7 @@ class ImmutableWithSignalJavaBehaviorSpec extends TypedSpec with Messages with B
|
||||||
}))
|
}))
|
||||||
}
|
}
|
||||||
|
|
||||||
class ImmutableJavaBehaviorSpec extends TypedSpec with Messages with Become with Stoppable {
|
class ImmutableJavaBehaviorSpec extends Messages with Become with Stoppable {
|
||||||
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) → null
|
override def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) = behv(monitor, StateA) → null
|
||||||
def behv(monitor: ActorRef[Event], state: State): Behavior[Command] =
|
def behv(monitor: ActorRef[Event], state: State): Behavior[Command] =
|
||||||
JActor.immutable {
|
JActor.immutable {
|
||||||
|
|
|
||||||
|
|
@ -7,7 +7,7 @@ import scala.concurrent.duration._
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.Actor.BehaviorDecorators
|
import akka.actor.typed.scaladsl.Actor.BehaviorDecorators
|
||||||
import akka.testkit.typed.{ BehaviorTestkit, TestInbox, TestKitSettings }
|
import akka.testkit.typed.{ BehaviorTestkit, TestInbox, TestKit, TestKitSettings }
|
||||||
import akka.testkit.typed.scaladsl._
|
import akka.testkit.typed.scaladsl._
|
||||||
|
|
||||||
object DeferredSpec {
|
object DeferredSpec {
|
||||||
|
|
@ -26,7 +26,7 @@ object DeferredSpec {
|
||||||
})
|
})
|
||||||
}
|
}
|
||||||
|
|
||||||
class DeferredSpec extends TypedSpec with StartSupport {
|
class DeferredSpec extends TestKit with TypedAkkaSpec {
|
||||||
|
|
||||||
import DeferredSpec._
|
import DeferredSpec._
|
||||||
implicit val testSettings = TestKitSettings(system)
|
implicit val testSettings = TestKitSettings(system)
|
||||||
|
|
@ -39,7 +39,7 @@ class DeferredSpec extends TypedSpec with StartSupport {
|
||||||
target(probe.ref)
|
target(probe.ref)
|
||||||
}
|
}
|
||||||
probe.expectNoMsg(100.millis) // not yet
|
probe.expectNoMsg(100.millis) // not yet
|
||||||
start(behv)
|
spawn(behv)
|
||||||
// it's supposed to be created immediately (not waiting for first message)
|
// it's supposed to be created immediately (not waiting for first message)
|
||||||
probe.expectMsg(Started)
|
probe.expectMsg(Started)
|
||||||
}
|
}
|
||||||
|
|
@ -58,7 +58,7 @@ class DeferredSpec extends TypedSpec with StartSupport {
|
||||||
Actor.stopped
|
Actor.stopped
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
start(behv)
|
spawn(behv)
|
||||||
probe.expectMsg(Started)
|
probe.expectMsg(Started)
|
||||||
probe.expectMsg(Pong)
|
probe.expectMsg(Pong)
|
||||||
}
|
}
|
||||||
|
|
@ -74,7 +74,7 @@ class DeferredSpec extends TypedSpec with StartSupport {
|
||||||
Actor.stopped
|
Actor.stopped
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
start(behv)
|
spawn(behv)
|
||||||
probe.expectMsg(Pong)
|
probe.expectMsg(Pong)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -86,7 +86,7 @@ class DeferredSpec extends TypedSpec with StartSupport {
|
||||||
target(probe.ref)
|
target(probe.ref)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
start(behv)
|
spawn(behv)
|
||||||
probe.expectMsg(Started)
|
probe.expectMsg(Started)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -99,7 +99,7 @@ class DeferredSpec extends TypedSpec with StartSupport {
|
||||||
case m ⇒ m
|
case m ⇒ m
|
||||||
}
|
}
|
||||||
probe.expectNoMsg(100.millis) // not yet
|
probe.expectNoMsg(100.millis) // not yet
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
// it's supposed to be created immediately (not waiting for first message)
|
// it's supposed to be created immediately (not waiting for first message)
|
||||||
probe.expectMsg(Started)
|
probe.expectMsg(Started)
|
||||||
ref ! Ping
|
ref ! Ping
|
||||||
|
|
@ -115,7 +115,7 @@ class DeferredSpec extends TypedSpec with StartSupport {
|
||||||
target(probe.ref)
|
target(probe.ref)
|
||||||
})
|
})
|
||||||
probe.expectNoMsg(100.millis) // not yet
|
probe.expectNoMsg(100.millis) // not yet
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
// it's supposed to be created immediately (not waiting for first message)
|
// it's supposed to be created immediately (not waiting for first message)
|
||||||
probe.expectMsg(Started)
|
probe.expectMsg(Started)
|
||||||
ref ! Ping
|
ref ! Ping
|
||||||
|
|
@ -123,23 +123,19 @@ class DeferredSpec extends TypedSpec with StartSupport {
|
||||||
probe.expectMsg(Pong)
|
probe.expectMsg(Pong)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class DeferredStubbedSpec extends TypedSpec {
|
class DeferredStubbedSpec extends TypedAkkaSpec {
|
||||||
|
|
||||||
import DeferredSpec._
|
import DeferredSpec._
|
||||||
|
|
||||||
def mkCtx(behv: Behavior[Command]): BehaviorTestkit[Command] =
|
|
||||||
BehaviorTestkit(behv, "ctx")
|
|
||||||
|
|
||||||
"must create underlying deferred behavior immediately" in {
|
"must create underlying deferred behavior immediately" in {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val behv = Actor.deferred[Command] { _ ⇒
|
val behv = Actor.deferred[Command] { _ ⇒
|
||||||
inbox.ref ! Started
|
inbox.ref ! Started
|
||||||
target(inbox.ref)
|
target(inbox.ref)
|
||||||
}
|
}
|
||||||
mkCtx(behv)
|
BehaviorTestkit(behv)
|
||||||
// it's supposed to be created immediately (not waiting for first message)
|
// it's supposed to be created immediately (not waiting for first message)
|
||||||
inbox.receiveMsg() should ===(Started)
|
inbox.receiveMsg() should ===(Started)
|
||||||
}
|
}
|
||||||
|
|
@ -152,7 +148,7 @@ class DeferredStubbedSpec extends TypedSpec {
|
||||||
throw exc
|
throw exc
|
||||||
}
|
}
|
||||||
intercept[RuntimeException] {
|
intercept[RuntimeException] {
|
||||||
mkCtx(behv)
|
BehaviorTestkit(behv)
|
||||||
} should ===(exc)
|
} should ===(exc)
|
||||||
inbox.receiveMsg() should ===(Started)
|
inbox.receiveMsg() should ===(Started)
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -43,7 +43,7 @@ object InstanceCountingExtension extends ExtensionId[DummyExtension1] {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class ExtensionsSpec extends TypedSpecSetup {
|
class ExtensionsSpec extends TypedAkkaSpec {
|
||||||
|
|
||||||
"The extensions subsystem" must {
|
"The extensions subsystem" must {
|
||||||
"return the same instance for the same id" in
|
"return the same instance for the same id" in
|
||||||
|
|
|
||||||
|
|
@ -1,104 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2014-2017 Lightbend Inc. <http://www.lightbend.com>
|
|
||||||
*/
|
|
||||||
package akka.actor.typed
|
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import com.typesafe.config.ConfigFactory
|
|
||||||
import org.junit.runner.RunWith
|
|
||||||
import akka.actor.typed.scaladsl.Actor._
|
|
||||||
import akka.util.Timeout
|
|
||||||
|
|
||||||
@RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
|
||||||
class PerformanceSpec extends TypedSpec(
|
|
||||||
ConfigFactory.parseString(
|
|
||||||
"""
|
|
||||||
# increase this if you do real benchmarking
|
|
||||||
akka.actor.typed.PerformanceSpec.iterations=100000
|
|
||||||
""")) {
|
|
||||||
|
|
||||||
override def setTimeout = Timeout(20.seconds)
|
|
||||||
|
|
||||||
case class Ping(x: Int, pong: ActorRef[Pong], report: ActorRef[Pong])
|
|
||||||
case class Pong(x: Int, ping: ActorRef[Ping], report: ActorRef[Pong])
|
|
||||||
|
|
||||||
def behavior(pairs: Int, pings: Int, count: Int, executor: String) =
|
|
||||||
StepWise[Pong] { (ctx, startWith) ⇒
|
|
||||||
startWith {
|
|
||||||
|
|
||||||
val pinger = immutable[Ping] { (ctx, msg) ⇒
|
|
||||||
if (msg.x == 0) {
|
|
||||||
msg.report ! Pong(0, ctx.self, msg.report)
|
|
||||||
same
|
|
||||||
} else {
|
|
||||||
msg.pong ! Pong(msg.x - 1, ctx.self, msg.report)
|
|
||||||
same
|
|
||||||
}
|
|
||||||
} // FIXME .withDispatcher(executor)
|
|
||||||
|
|
||||||
val ponger = immutable[Pong] { (ctx, msg) ⇒
|
|
||||||
msg.ping ! Ping(msg.x, ctx.self, msg.report)
|
|
||||||
same
|
|
||||||
} // FIXME .withDispatcher(executor)
|
|
||||||
|
|
||||||
val actors =
|
|
||||||
for (i ← 1 to pairs)
|
|
||||||
yield (ctx.spawn(pinger, s"pinger-$i"), ctx.spawn(ponger, s"ponger-$i"))
|
|
||||||
|
|
||||||
val start = Deadline.now
|
|
||||||
|
|
||||||
for {
|
|
||||||
(ping, pong) ← actors
|
|
||||||
_ ← 1 to pings
|
|
||||||
} ping ! Ping(count, pong, ctx.self)
|
|
||||||
|
|
||||||
start
|
|
||||||
}.expectMultipleMessages(10.seconds, pairs * pings) { (msgs, start) ⇒
|
|
||||||
val stop = Deadline.now
|
|
||||||
|
|
||||||
val rate = 2L * count * pairs * pings / (stop - start).toMillis
|
|
||||||
info(s"messaging rate was $rate/ms")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
val iterations = system.settings.config.getInt("akka.actor.typed.PerformanceSpec.iterations")
|
|
||||||
|
|
||||||
"An immutable behaviour" must {
|
|
||||||
"when warming up" in {
|
|
||||||
sync(runTest("01")(behavior(1, 1, iterations, "dispatcher-1")))
|
|
||||||
}
|
|
||||||
|
|
||||||
"when using a single message on a single thread" in {
|
|
||||||
sync(runTest("02")(behavior(1, 1, iterations, "dispatcher-1")))
|
|
||||||
}
|
|
||||||
|
|
||||||
"when using a 10 messages on a single thread" in {
|
|
||||||
sync(runTest("03")(behavior(1, 10, iterations, "dispatcher-1")))
|
|
||||||
}
|
|
||||||
|
|
||||||
"when using a single message on two threads" in {
|
|
||||||
sync(runTest("04")(behavior(1, 1, iterations, "dispatcher-2")))
|
|
||||||
}
|
|
||||||
|
|
||||||
"when using a 10 messages on two threads" in {
|
|
||||||
sync(runTest("05")(behavior(1, 10, iterations, "dispatcher-2")))
|
|
||||||
}
|
|
||||||
|
|
||||||
"when using 4 pairs with a single message" in {
|
|
||||||
sync(runTest("06")(behavior(4, 1, iterations, "dispatcher-8")))
|
|
||||||
}
|
|
||||||
|
|
||||||
"when using 4 pairs with 10 messages" in {
|
|
||||||
sync(runTest("07")(behavior(4, 10, iterations, "dispatcher-8")))
|
|
||||||
}
|
|
||||||
|
|
||||||
"when using 8 pairs with a single message" in {
|
|
||||||
sync(runTest("08")(behavior(8, 1, iterations, "dispatcher-8")))
|
|
||||||
}
|
|
||||||
|
|
||||||
"when using 8 pairs with 10 messages" in {
|
|
||||||
sync(runTest("09")(behavior(8, 10, iterations, "dispatcher-8")))
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
@ -3,7 +3,7 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor.typed
|
package akka.actor.typed
|
||||||
|
|
||||||
class PropsSpec extends TypedSpecSetup {
|
class PropsSpec extends TypedAkkaSpec {
|
||||||
|
|
||||||
val dispatcherFirst = DispatcherDefault(DispatcherFromConfig("pool"))
|
val dispatcherFirst = DispatcherDefault(DispatcherFromConfig("pool"))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,19 +3,22 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor.typed
|
package akka.actor.typed
|
||||||
|
|
||||||
|
import akka.actor.typed.scaladsl.Actor
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import akka.actor.typed.scaladsl.Actor._
|
import akka.actor.typed.scaladsl.Actor._
|
||||||
import akka.testkit.typed.{ BehaviorTestkit, TestInbox, TestKitSettings }
|
import akka.testkit.typed.{ BehaviorTestkit, TestInbox, TestKit, TestKitSettings }
|
||||||
|
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
import akka.testkit.typed.scaladsl._
|
import akka.testkit.typed.scaladsl._
|
||||||
|
import org.scalatest.{ Matchers, WordSpec, fixture }
|
||||||
|
|
||||||
object RestarterSpec {
|
object SupervisionSpec {
|
||||||
|
|
||||||
sealed trait Command
|
sealed trait Command
|
||||||
case object Ping extends Command
|
case object Ping extends Command
|
||||||
case class Throw(e: Throwable) extends Command
|
case class Throw(e: Throwable) extends Command
|
||||||
case object NextState extends Command
|
case object IncrementState extends Command
|
||||||
case object GetState extends Command
|
case object GetState extends Command
|
||||||
case class CreateChild[T](behavior: Behavior[T], name: String) extends Command
|
case class CreateChild[T](behavior: Behavior[T], name: String) extends Command
|
||||||
|
|
||||||
|
|
@ -29,28 +32,28 @@ object RestarterSpec {
|
||||||
class Exc2 extends Exc1("exc-2")
|
class Exc2 extends Exc1("exc-2")
|
||||||
class Exc3(msg: String = "exc-3") extends RuntimeException(msg) with NoStackTrace
|
class Exc3(msg: String = "exc-3") extends RuntimeException(msg) with NoStackTrace
|
||||||
|
|
||||||
def target(monitor: ActorRef[Event], state: State = State(0, Map.empty)): Behavior[Command] =
|
def targetBehavior(monitor: ActorRef[Event], state: State = State(0, Map.empty)): Behavior[Command] =
|
||||||
immutable[Command] { (ctx, cmd) ⇒
|
immutable[Command] { (ctx, cmd) ⇒
|
||||||
cmd match {
|
cmd match {
|
||||||
case Ping ⇒
|
case Ping ⇒
|
||||||
monitor ! Pong
|
monitor ! Pong
|
||||||
same
|
Actor.same
|
||||||
case NextState ⇒
|
case IncrementState ⇒
|
||||||
target(monitor, state.copy(n = state.n + 1))
|
targetBehavior(monitor, state.copy(n = state.n + 1))
|
||||||
case GetState ⇒
|
case GetState ⇒
|
||||||
val reply = state.copy(children = ctx.children.map(c ⇒ c.path.name → c.upcast[Command]).toMap)
|
val reply = state.copy(children = ctx.children.map(c ⇒ c.path.name → c.upcast[Command]).toMap)
|
||||||
monitor ! reply
|
monitor ! reply
|
||||||
same
|
Actor.same
|
||||||
case CreateChild(childBehv, childName) ⇒
|
case CreateChild(childBehv, childName) ⇒
|
||||||
ctx.spawn(childBehv, childName)
|
ctx.spawn(childBehv, childName)
|
||||||
same
|
Actor.same
|
||||||
case Throw(e) ⇒
|
case Throw(e) ⇒
|
||||||
throw e
|
throw e
|
||||||
}
|
}
|
||||||
} onSignal {
|
} onSignal {
|
||||||
case (ctx, sig) ⇒
|
case (_, sig) ⇒
|
||||||
monitor ! GotSignal(sig)
|
monitor ! GotSignal(sig)
|
||||||
same
|
Actor.same
|
||||||
}
|
}
|
||||||
|
|
||||||
class FailingConstructor(monitor: ActorRef[Event]) extends MutableBehavior[Command] {
|
class FailingConstructor(monitor: ActorRef[Event]) extends MutableBehavior[Command] {
|
||||||
|
|
@ -59,71 +62,71 @@ object RestarterSpec {
|
||||||
|
|
||||||
override def onMessage(msg: Command): Behavior[Command] = {
|
override def onMessage(msg: Command): Behavior[Command] = {
|
||||||
monitor ! Pong
|
monitor ! Pong
|
||||||
same
|
Actor.same
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class RestarterSpec extends TypedSpec {
|
class StubbedSupervisionSpec extends WordSpec with Matchers {
|
||||||
|
|
||||||
import RestarterSpec._
|
import SupervisionSpec._
|
||||||
|
|
||||||
def mkCtx(behv: Behavior[Command]): BehaviorTestkit[Command] =
|
def mkTestkit(behv: Behavior[Command]): BehaviorTestkit[Command] =
|
||||||
BehaviorTestkit(behv, "ctx")
|
BehaviorTestkit(behv)
|
||||||
|
|
||||||
"A restarter" must {
|
"A restarter (stubbed)" must {
|
||||||
"receive message" in {
|
"receive message" in {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val behv = supervise(target(inbox.ref)).onFailure[Throwable](SupervisorStrategy.restart)
|
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Throwable](SupervisorStrategy.restart)
|
||||||
val ctx = mkCtx(behv)
|
val testkit = BehaviorTestkit(behv)
|
||||||
ctx.run(Ping)
|
testkit.run(Ping)
|
||||||
inbox.receiveMsg() should ===(Pong)
|
inbox.receiveMsg() should ===(Pong)
|
||||||
}
|
}
|
||||||
|
|
||||||
"stop when no supervise" in {
|
"stop when no supervise" in {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val behv = target(inbox.ref)
|
val behv = targetBehavior(inbox.ref)
|
||||||
val ctx = mkCtx(behv)
|
val testkit = BehaviorTestkit(behv)
|
||||||
intercept[Exc3] {
|
intercept[Exc3] {
|
||||||
ctx.run(Throw(new Exc3))
|
testkit.run(Throw(new Exc3))
|
||||||
}
|
}
|
||||||
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
||||||
}
|
}
|
||||||
|
|
||||||
"stop when unhandled exception" in {
|
"stop when unhandled exception" in {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val behv = supervise(target(inbox.ref)).onFailure[Exc1](SupervisorStrategy.restart)
|
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](SupervisorStrategy.restart)
|
||||||
val ctx = mkCtx(behv)
|
val testkit = BehaviorTestkit(behv)
|
||||||
intercept[Exc3] {
|
intercept[Exc3] {
|
||||||
ctx.run(Throw(new Exc3))
|
testkit.run(Throw(new Exc3))
|
||||||
}
|
}
|
||||||
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
||||||
}
|
}
|
||||||
|
|
||||||
"restart when handled exception" in {
|
"restart when handled exception" in {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val behv = supervise(target(inbox.ref)).onFailure[Exc1](SupervisorStrategy.restart)
|
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](SupervisorStrategy.restart)
|
||||||
val ctx = mkCtx(behv)
|
val testkit = BehaviorTestkit(behv)
|
||||||
ctx.run(NextState)
|
testkit.run(IncrementState)
|
||||||
ctx.run(GetState)
|
testkit.run(GetState)
|
||||||
inbox.receiveMsg() should ===(State(1, Map.empty))
|
inbox.receiveMsg() should ===(State(1, Map.empty))
|
||||||
|
|
||||||
ctx.run(Throw(new Exc2))
|
testkit.run(Throw(new Exc2))
|
||||||
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
||||||
ctx.run(GetState)
|
testkit.run(GetState)
|
||||||
inbox.receiveMsg() should ===(State(0, Map.empty))
|
inbox.receiveMsg() should ===(State(0, Map.empty))
|
||||||
}
|
}
|
||||||
|
|
||||||
"resume when handled exception" in {
|
"resume when handled exception" in {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val behv = supervise(target(inbox.ref)).onFailure[Exc1](SupervisorStrategy.resume)
|
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](SupervisorStrategy.resume)
|
||||||
val ctx = mkCtx(behv)
|
val testkit = BehaviorTestkit(behv)
|
||||||
ctx.run(NextState)
|
testkit.run(IncrementState)
|
||||||
ctx.run(GetState)
|
testkit.run(GetState)
|
||||||
inbox.receiveMsg() should ===(State(1, Map.empty))
|
inbox.receiveMsg() should ===(State(1, Map.empty))
|
||||||
|
|
||||||
ctx.run(Throw(new Exc2))
|
testkit.run(Throw(new Exc2))
|
||||||
ctx.run(GetState)
|
testkit.run(GetState)
|
||||||
inbox.receiveMsg() should ===(State(1, Map.empty))
|
inbox.receiveMsg() should ===(State(1, Map.empty))
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -132,38 +135,39 @@ class RestarterSpec extends TypedSpec {
|
||||||
val behv =
|
val behv =
|
||||||
supervise(
|
supervise(
|
||||||
supervise(
|
supervise(
|
||||||
target(inbox.ref)
|
targetBehavior(inbox.ref)
|
||||||
).onFailure[Exc2](SupervisorStrategy.resume)
|
).onFailure[Exc2](SupervisorStrategy.resume)
|
||||||
).onFailure[Exc3](SupervisorStrategy.restart)
|
).onFailure[Exc3](SupervisorStrategy.restart)
|
||||||
val ctx = mkCtx(behv)
|
val testkit = BehaviorTestkit(behv)
|
||||||
ctx.run(NextState)
|
testkit.run(IncrementState)
|
||||||
ctx.run(GetState)
|
testkit.run(GetState)
|
||||||
inbox.receiveMsg() should ===(State(1, Map.empty))
|
inbox.receiveMsg() should ===(State(1, Map.empty))
|
||||||
|
|
||||||
// resume
|
// resume
|
||||||
ctx.run(Throw(new Exc2))
|
testkit.run(Throw(new Exc2))
|
||||||
ctx.run(GetState)
|
testkit.run(GetState)
|
||||||
inbox.receiveMsg() should ===(State(1, Map.empty))
|
inbox.receiveMsg() should ===(State(1, Map.empty))
|
||||||
|
|
||||||
// restart
|
// restart
|
||||||
ctx.run(Throw(new Exc3))
|
testkit.run(Throw(new Exc3))
|
||||||
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
||||||
ctx.run(GetState)
|
testkit.run(GetState)
|
||||||
inbox.receiveMsg() should ===(State(0, Map.empty))
|
inbox.receiveMsg() should ===(State(0, Map.empty))
|
||||||
|
|
||||||
// stop
|
// stop
|
||||||
intercept[Exc1] {
|
intercept[Exc1] {
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
}
|
}
|
||||||
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
||||||
}
|
}
|
||||||
|
|
||||||
"not catch fatal error" in {
|
"not catch fatal error" in {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]()
|
||||||
val behv = supervise(target(inbox.ref)).onFailure[Throwable](SupervisorStrategy.restart)
|
val behv = Actor.supervise(targetBehavior(inbox.ref))
|
||||||
val ctx = mkCtx(behv)
|
.onFailure[Throwable](SupervisorStrategy.restart)
|
||||||
|
val testkit = BehaviorTestkit(behv)
|
||||||
intercept[StackOverflowError] {
|
intercept[StackOverflowError] {
|
||||||
ctx.run(Throw(new StackOverflowError))
|
testkit.run(Throw(new StackOverflowError))
|
||||||
}
|
}
|
||||||
inbox.receiveAll() should ===(Nil)
|
inbox.receiveAll() should ===(Nil)
|
||||||
}
|
}
|
||||||
|
|
@ -171,14 +175,14 @@ class RestarterSpec extends TypedSpec {
|
||||||
"stop after restart retries limit" in {
|
"stop after restart retries limit" in {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val strategy = SupervisorStrategy.restartWithLimit(maxNrOfRetries = 2, withinTimeRange = 1.minute)
|
val strategy = SupervisorStrategy.restartWithLimit(maxNrOfRetries = 2, withinTimeRange = 1.minute)
|
||||||
val behv = supervise(target(inbox.ref)).onFailure[Exc1](strategy)
|
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](strategy)
|
||||||
val ctx = mkCtx(behv)
|
val testkit = BehaviorTestkit(behv)
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
||||||
intercept[Exc1] {
|
intercept[Exc1] {
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
}
|
}
|
||||||
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
||||||
}
|
}
|
||||||
|
|
@ -187,20 +191,20 @@ class RestarterSpec extends TypedSpec {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val withinTimeRange = 2.seconds
|
val withinTimeRange = 2.seconds
|
||||||
val strategy = SupervisorStrategy.restartWithLimit(maxNrOfRetries = 2, withinTimeRange)
|
val strategy = SupervisorStrategy.restartWithLimit(maxNrOfRetries = 2, withinTimeRange)
|
||||||
val behv = supervise(target(inbox.ref)).onFailure[Exc1](strategy)
|
val behv = supervise(targetBehavior(inbox.ref)).onFailure[Exc1](strategy)
|
||||||
val ctx = mkCtx(behv)
|
val testkit = BehaviorTestkit(behv)
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
||||||
Thread.sleep((2.seconds + 100.millis).toMillis)
|
Thread.sleep((2.seconds + 100.millis).toMillis)
|
||||||
|
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
inbox.receiveMsg() should ===(GotSignal(PreRestart))
|
||||||
intercept[Exc1] {
|
intercept[Exc1] {
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
}
|
}
|
||||||
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
||||||
}
|
}
|
||||||
|
|
@ -208,10 +212,11 @@ class RestarterSpec extends TypedSpec {
|
||||||
"stop at first exception when restart retries limit is 0" in {
|
"stop at first exception when restart retries limit is 0" in {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val strategy = SupervisorStrategy.restartWithLimit(maxNrOfRetries = 0, withinTimeRange = 1.minute)
|
val strategy = SupervisorStrategy.restartWithLimit(maxNrOfRetries = 0, withinTimeRange = 1.minute)
|
||||||
val behv = supervise(target(inbox.ref)).onFailure[Exc1](strategy)
|
val behv = supervise(targetBehavior(inbox.ref))
|
||||||
val ctx = mkCtx(behv)
|
.onFailure[Exc1](strategy)
|
||||||
|
val testkit = BehaviorTestkit(behv)
|
||||||
intercept[Exc1] {
|
intercept[Exc1] {
|
||||||
ctx.run(Throw(new Exc1))
|
testkit.run(Throw(new Exc1))
|
||||||
}
|
}
|
||||||
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
inbox.receiveMsg() should ===(GotSignal(PostStop))
|
||||||
}
|
}
|
||||||
|
|
@ -220,34 +225,38 @@ class RestarterSpec extends TypedSpec {
|
||||||
val inbox = TestInbox[Event]("evt")
|
val inbox = TestInbox[Event]("evt")
|
||||||
val behv = supervise(deferred[Command] { _ ⇒
|
val behv = supervise(deferred[Command] { _ ⇒
|
||||||
inbox.ref ! Started
|
inbox.ref ! Started
|
||||||
target(inbox.ref)
|
targetBehavior(inbox.ref)
|
||||||
}).onFailure[Exc1](SupervisorStrategy.restart)
|
}).onFailure[Exc1](SupervisorStrategy.restart)
|
||||||
mkCtx(behv)
|
mkTestkit(behv)
|
||||||
// it's supposed to be created immediately (not waiting for first message)
|
// it's supposed to be created immediately (not waiting for first message)
|
||||||
inbox.receiveMsg() should ===(Started)
|
inbox.receiveMsg() should ===(Started)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class RestarterStubbedSpec extends TypedSpec with StartSupport {
|
class SupervisionSpec extends TestKit("SupervisionSpec") with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
import RestarterSpec._
|
import SupervisionSpec._
|
||||||
|
private val nameCounter = Iterator.from(0)
|
||||||
|
private def nextName(prefix: String = "a"): String = s"$prefix-${nameCounter.next()}"
|
||||||
|
private val waitTime = 50.millis
|
||||||
|
|
||||||
implicit val testSettings = TestKitSettings(system)
|
implicit val testSettings = TestKitSettings(system)
|
||||||
|
|
||||||
"A restart (subbed)" must {
|
"A supervised actor" must {
|
||||||
"receive message" in {
|
"receive message" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]("evt")
|
||||||
val behv = supervise(target(probe.ref)).onFailure[Throwable](SupervisorStrategy.restart)
|
val behv = Actor.supervise(targetBehavior(probe.ref))
|
||||||
val ref = start(behv)
|
.onFailure[Throwable](SupervisorStrategy.restart)
|
||||||
|
val ref = spawn(behv)
|
||||||
ref ! Ping
|
ref ! Ping
|
||||||
probe.expectMsg(Pong)
|
probe.expectMsg(Pong)
|
||||||
}
|
}
|
||||||
|
|
||||||
"stop when no supervise" in {
|
"stop when not supervised" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]("evt")
|
||||||
val behv = target(probe.ref)
|
val behv = targetBehavior(probe.ref)
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
ref ! Throw(new Exc3)
|
ref ! Throw(new Exc3)
|
||||||
|
|
||||||
probe.expectMsg(GotSignal(PostStop))
|
probe.expectMsg(GotSignal(PostStop))
|
||||||
|
|
@ -255,17 +264,19 @@ class RestarterStubbedSpec extends TypedSpec with StartSupport {
|
||||||
|
|
||||||
"stop when unhandled exception" in {
|
"stop when unhandled exception" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]("evt")
|
||||||
val behv = supervise(target(probe.ref)).onFailure[Exc1](SupervisorStrategy.restart)
|
val behv = Actor.supervise(targetBehavior(probe.ref))
|
||||||
val ref = start(behv)
|
.onFailure[Exc1](SupervisorStrategy.restart)
|
||||||
|
val ref = spawn(behv)
|
||||||
ref ! Throw(new Exc3)
|
ref ! Throw(new Exc3)
|
||||||
probe.expectMsg(GotSignal(PostStop))
|
probe.expectMsg(GotSignal(PostStop))
|
||||||
}
|
}
|
||||||
|
|
||||||
"restart when handled exception" in {
|
"restart when handled exception" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]("evt")
|
||||||
val behv = supervise(target(probe.ref)).onFailure[Exc1](SupervisorStrategy.restart)
|
val behv = Actor.supervise(targetBehavior(probe.ref))
|
||||||
val ref = start(behv)
|
.onFailure[Exc1](SupervisorStrategy.restart)
|
||||||
ref ! NextState
|
val ref = spawn(behv)
|
||||||
|
ref ! IncrementState
|
||||||
ref ! GetState
|
ref ! GetState
|
||||||
probe.expectMsg(State(1, Map.empty))
|
probe.expectMsg(State(1, Map.empty))
|
||||||
|
|
||||||
|
|
@ -276,29 +287,31 @@ class RestarterStubbedSpec extends TypedSpec with StartSupport {
|
||||||
}
|
}
|
||||||
|
|
||||||
"NOT stop children when restarting" in {
|
"NOT stop children when restarting" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val parentProbe = TestProbe[Event]("evt")
|
||||||
val behv = supervise(target(probe.ref)).onFailure[Exc1](SupervisorStrategy.restart)
|
val behv = Actor.supervise(targetBehavior(parentProbe.ref))
|
||||||
val ref = start(behv)
|
.onFailure[Exc1](SupervisorStrategy.restart)
|
||||||
|
val ref = spawn(behv)
|
||||||
|
|
||||||
val childProbe = TestProbe[Event]("childEvt")
|
val childProbe = TestProbe[Event]("childEvt")
|
||||||
val childName = nextName()
|
val childName = nextName()
|
||||||
ref ! CreateChild(target(childProbe.ref), childName)
|
ref ! CreateChild(targetBehavior(childProbe.ref), childName)
|
||||||
ref ! GetState
|
ref ! GetState
|
||||||
probe.expectMsgType[State].children.keySet should contain(childName)
|
parentProbe.expectMsgType[State].children.keySet should contain(childName)
|
||||||
|
|
||||||
ref ! Throw(new Exc1)
|
ref ! Throw(new Exc1)
|
||||||
probe.expectMsg(GotSignal(PreRestart))
|
parentProbe.expectMsg(GotSignal(PreRestart))
|
||||||
ref ! GetState
|
ref ! GetState
|
||||||
// TODO document this difference compared to classic actors, and that
|
// TODO document this difference compared to classic actors, and that
|
||||||
// children can be stopped if needed in PreRestart
|
// children can be stopped if needed in PreRestart
|
||||||
probe.expectMsgType[State].children.keySet should contain(childName)
|
parentProbe.expectMsgType[State].children.keySet should contain(childName)
|
||||||
|
childProbe.expectNoMsg(waitTime)
|
||||||
}
|
}
|
||||||
|
|
||||||
"resume when handled exception" in {
|
"resume when handled exception" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]("evt")
|
||||||
val behv = supervise(target(probe.ref)).onFailure[Exc1](SupervisorStrategy.resume)
|
val behv = supervise(targetBehavior(probe.ref)).onFailure[Exc1](SupervisorStrategy.resume)
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
ref ! NextState
|
ref ! IncrementState
|
||||||
ref ! GetState
|
ref ! GetState
|
||||||
probe.expectMsg(State(1, Map.empty))
|
probe.expectMsg(State(1, Map.empty))
|
||||||
|
|
||||||
|
|
@ -309,16 +322,18 @@ class RestarterStubbedSpec extends TypedSpec with StartSupport {
|
||||||
|
|
||||||
"support nesting to handle different exceptions" in {
|
"support nesting to handle different exceptions" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]("evt")
|
||||||
val behv = supervise(
|
val behv = Actor.supervise(
|
||||||
supervise(target(probe.ref)).onFailure[Exc2](SupervisorStrategy.resume)
|
Actor.supervise(targetBehavior(probe.ref))
|
||||||
|
.onFailure[Exc2](SupervisorStrategy.resume)
|
||||||
).onFailure[Exc3](SupervisorStrategy.restart)
|
).onFailure[Exc3](SupervisorStrategy.restart)
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
ref ! NextState
|
ref ! IncrementState
|
||||||
ref ! GetState
|
ref ! GetState
|
||||||
probe.expectMsg(State(1, Map.empty))
|
probe.expectMsg(State(1, Map.empty))
|
||||||
|
|
||||||
// resume
|
// resume
|
||||||
ref ! Throw(new Exc2)
|
ref ! Throw(new Exc2)
|
||||||
|
probe.expectNoMsg(waitTime)
|
||||||
ref ! GetState
|
ref ! GetState
|
||||||
probe.expectMsg(State(1, Map.empty))
|
probe.expectMsg(State(1, Map.empty))
|
||||||
|
|
||||||
|
|
@ -337,16 +352,17 @@ class RestarterStubbedSpec extends TypedSpec with StartSupport {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]("evt")
|
||||||
val startedProbe = TestProbe[Event]("started")
|
val startedProbe = TestProbe[Event]("started")
|
||||||
val minBackoff = 1.seconds
|
val minBackoff = 1.seconds
|
||||||
val strategy = SupervisorStrategy.restartWithBackoff(minBackoff, 10.seconds, 0.0)
|
val strategy = SupervisorStrategy
|
||||||
|
.restartWithBackoff(minBackoff, 10.seconds, 0.0)
|
||||||
.withResetBackoffAfter(10.seconds)
|
.withResetBackoffAfter(10.seconds)
|
||||||
val behv = supervise(deferred[Command] { _ ⇒
|
val behv = Actor.supervise(Actor.deferred[Command] { _ ⇒
|
||||||
startedProbe.ref ! Started
|
startedProbe.ref ! Started
|
||||||
target(probe.ref)
|
targetBehavior(probe.ref)
|
||||||
}).onFailure[Exception](strategy)
|
}).onFailure[Exception](strategy)
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
|
|
||||||
startedProbe.expectMsg(Started)
|
startedProbe.expectMsg(Started)
|
||||||
ref ! NextState
|
ref ! IncrementState
|
||||||
ref ! Throw(new Exc1)
|
ref ! Throw(new Exc1)
|
||||||
probe.expectMsg(GotSignal(PreRestart))
|
probe.expectMsg(GotSignal(PreRestart))
|
||||||
ref ! Ping // dropped due to backoff
|
ref ! Ping // dropped due to backoff
|
||||||
|
|
@ -358,7 +374,7 @@ class RestarterStubbedSpec extends TypedSpec with StartSupport {
|
||||||
probe.expectMsg(State(0, Map.empty))
|
probe.expectMsg(State(0, Map.empty))
|
||||||
|
|
||||||
// one more time
|
// one more time
|
||||||
ref ! NextState
|
ref ! IncrementState
|
||||||
ref ! Throw(new Exc1)
|
ref ! Throw(new Exc1)
|
||||||
probe.expectMsg(GotSignal(PreRestart))
|
probe.expectMsg(GotSignal(PreRestart))
|
||||||
ref ! Ping // dropped due to backoff
|
ref ! Ping // dropped due to backoff
|
||||||
|
|
@ -375,10 +391,10 @@ class RestarterStubbedSpec extends TypedSpec with StartSupport {
|
||||||
val minBackoff = 1.seconds
|
val minBackoff = 1.seconds
|
||||||
val strategy = SupervisorStrategy.restartWithBackoff(minBackoff, 10.seconds, 0.0)
|
val strategy = SupervisorStrategy.restartWithBackoff(minBackoff, 10.seconds, 0.0)
|
||||||
.withResetBackoffAfter(100.millis)
|
.withResetBackoffAfter(100.millis)
|
||||||
val behv = supervise(target(probe.ref)).onFailure[Exc1](strategy)
|
val behv = supervise(targetBehavior(probe.ref)).onFailure[Exc1](strategy)
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
|
|
||||||
ref ! NextState
|
ref ! IncrementState
|
||||||
ref ! Throw(new Exc1)
|
ref ! Throw(new Exc1)
|
||||||
probe.expectMsg(GotSignal(PreRestart))
|
probe.expectMsg(GotSignal(PreRestart))
|
||||||
ref ! Ping // dropped due to backoff
|
ref ! Ping // dropped due to backoff
|
||||||
|
|
@ -389,7 +405,7 @@ class RestarterStubbedSpec extends TypedSpec with StartSupport {
|
||||||
|
|
||||||
// one more time after the reset timeout
|
// one more time after the reset timeout
|
||||||
probe.expectNoMsg(strategy.resetBackoffAfter + 100.millis)
|
probe.expectNoMsg(strategy.resetBackoffAfter + 100.millis)
|
||||||
ref ! NextState
|
ref ! IncrementState
|
||||||
ref ! Throw(new Exc1)
|
ref ! Throw(new Exc1)
|
||||||
probe.expectMsg(GotSignal(PreRestart))
|
probe.expectMsg(GotSignal(PreRestart))
|
||||||
ref ! Ping // dropped due to backoff
|
ref ! Ping // dropped due to backoff
|
||||||
|
|
@ -404,18 +420,19 @@ class RestarterStubbedSpec extends TypedSpec with StartSupport {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]("evt")
|
||||||
val behv = supervise(deferred[Command] { _ ⇒
|
val behv = supervise(deferred[Command] { _ ⇒
|
||||||
probe.ref ! Started
|
probe.ref ! Started
|
||||||
target(probe.ref)
|
targetBehavior(probe.ref)
|
||||||
}).onFailure[Exception](SupervisorStrategy.restart)
|
}).onFailure[Exception](SupervisorStrategy.restart)
|
||||||
probe.expectNoMsg(100.millis) // not yet
|
probe.expectNoMsg(100.millis) // not yet
|
||||||
start(behv)
|
spawn(behv)
|
||||||
// it's supposed to be created immediately (not waiting for first message)
|
// it's supposed to be created immediately (not waiting for first message)
|
||||||
probe.expectMsg(Started)
|
probe.expectMsg(Started)
|
||||||
}
|
}
|
||||||
|
|
||||||
"stop when exception from MutableBehavior constructor" in {
|
"stop when exception from MutableBehavior constructor" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]("evt")
|
||||||
val behv = supervise(mutable[Command](_ ⇒ new FailingConstructor(probe.ref))).onFailure[Exception](SupervisorStrategy.restart)
|
val behv = supervise(mutable[Command](_ ⇒ new FailingConstructor(probe.ref)))
|
||||||
val ref = start(behv)
|
.onFailure[Exception](SupervisorStrategy.restart)
|
||||||
|
val ref = spawn(behv)
|
||||||
probe.expectMsg(Started)
|
probe.expectMsg(Started)
|
||||||
ref ! Ping
|
ref ! Ping
|
||||||
probe.expectNoMsg(100.millis)
|
probe.expectNoMsg(100.millis)
|
||||||
|
|
@ -9,16 +9,15 @@ import java.util.concurrent.atomic.AtomicInteger
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.util.control.NoStackTrace
|
import scala.util.control.NoStackTrace
|
||||||
|
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.TimerScheduler
|
import akka.actor.typed.scaladsl.TimerScheduler
|
||||||
import akka.testkit.typed.TestKitSettings
|
import akka.testkit.typed.TestKitSettings
|
||||||
|
import akka.testkit.typed.TestKit
|
||||||
import akka.testkit.typed.scaladsl._
|
import akka.testkit.typed.scaladsl._
|
||||||
|
import org.scalatest.WordSpecLike
|
||||||
|
|
||||||
class TimerSpec extends TypedSpec(
|
class TimerSpec extends TestKit("TimerSpec")
|
||||||
"""
|
with WordSpecLike {
|
||||||
#akka.loglevel = DEBUG
|
|
||||||
""") with StartSupport {
|
|
||||||
|
|
||||||
sealed trait Command
|
sealed trait Command
|
||||||
case class Tick(n: Int) extends Command
|
case class Tick(n: Int) extends Command
|
||||||
|
|
@ -87,7 +86,7 @@ class TimerSpec extends TypedSpec(
|
||||||
target(probe.ref, timer, 1)
|
target(probe.ref, timer, 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
probe.expectMsg(Tock(1))
|
probe.expectMsg(Tock(1))
|
||||||
probe.expectNoMsg(100.millis)
|
probe.expectNoMsg(100.millis)
|
||||||
|
|
||||||
|
|
@ -102,7 +101,7 @@ class TimerSpec extends TypedSpec(
|
||||||
target(probe.ref, timer, 1)
|
target(probe.ref, timer, 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
probe.within((interval * 4) - 100.millis) {
|
probe.within((interval * 4) - 100.millis) {
|
||||||
probe.expectMsg(Tock(1))
|
probe.expectMsg(Tock(1))
|
||||||
probe.expectMsg(Tock(1))
|
probe.expectMsg(Tock(1))
|
||||||
|
|
@ -120,7 +119,7 @@ class TimerSpec extends TypedSpec(
|
||||||
target(probe.ref, timer, 1)
|
target(probe.ref, timer, 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
probe.expectMsg(Tock(1))
|
probe.expectMsg(Tock(1))
|
||||||
val latch = new CountDownLatch(1)
|
val latch = new CountDownLatch(1)
|
||||||
// next Tock(1) enqueued in mailboxed, but should be discarded because of new timer
|
// next Tock(1) enqueued in mailboxed, but should be discarded because of new timer
|
||||||
|
|
@ -140,7 +139,7 @@ class TimerSpec extends TypedSpec(
|
||||||
target(probe.ref, timer, 1)
|
target(probe.ref, timer, 1)
|
||||||
}
|
}
|
||||||
|
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
probe.expectMsg(Tock(1))
|
probe.expectMsg(Tock(1))
|
||||||
ref ! Cancel
|
ref ! Cancel
|
||||||
probe.expectNoMsg(dilatedInterval + 100.millis)
|
probe.expectNoMsg(dilatedInterval + 100.millis)
|
||||||
|
|
@ -157,7 +156,7 @@ class TimerSpec extends TypedSpec(
|
||||||
target(probe.ref, timer, 1)
|
target(probe.ref, timer, 1)
|
||||||
}).onFailure[Exception](SupervisorStrategy.restart)
|
}).onFailure[Exception](SupervisorStrategy.restart)
|
||||||
|
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
probe.expectMsg(Tock(1))
|
probe.expectMsg(Tock(1))
|
||||||
|
|
||||||
val latch = new CountDownLatch(1)
|
val latch = new CountDownLatch(1)
|
||||||
|
|
@ -180,7 +179,7 @@ class TimerSpec extends TypedSpec(
|
||||||
target(probe.ref, timer, 1)
|
target(probe.ref, timer, 1)
|
||||||
}).onFailure[Exception](SupervisorStrategy.restart)
|
}).onFailure[Exception](SupervisorStrategy.restart)
|
||||||
|
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
probe.expectMsg(Tock(1))
|
probe.expectMsg(Tock(1))
|
||||||
// change state so that we see that the restart starts over again
|
// change state so that we see that the restart starts over again
|
||||||
ref ! Bump
|
ref ! Bump
|
||||||
|
|
@ -200,23 +199,23 @@ class TimerSpec extends TypedSpec(
|
||||||
}
|
}
|
||||||
|
|
||||||
"cancel timers when stopped from exception" in {
|
"cancel timers when stopped from exception" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]()
|
||||||
val behv = Actor.withTimers[Command] { timer ⇒
|
val behv = Actor.withTimers[Command] { timer ⇒
|
||||||
timer.startPeriodicTimer("T", Tick(1), interval)
|
timer.startPeriodicTimer("T", Tick(1), interval)
|
||||||
target(probe.ref, timer, 1)
|
target(probe.ref, timer, 1)
|
||||||
}
|
}
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
ref ! Throw(new Exc)
|
ref ! Throw(new Exc)
|
||||||
probe.expectMsg(GotPostStop(false))
|
probe.expectMsg(GotPostStop(false))
|
||||||
}
|
}
|
||||||
|
|
||||||
"cancel timers when stopped voluntarily" in {
|
"cancel timers when stopped voluntarily" in {
|
||||||
val probe = TestProbe[Event]("evt")
|
val probe = TestProbe[Event]()
|
||||||
val behv = Actor.withTimers[Command] { timer ⇒
|
val behv = Actor.withTimers[Command] { timer ⇒
|
||||||
timer.startPeriodicTimer("T", Tick(1), interval)
|
timer.startPeriodicTimer("T", Tick(1), interval)
|
||||||
target(probe.ref, timer, 1)
|
target(probe.ref, timer, 1)
|
||||||
}
|
}
|
||||||
val ref = start(behv)
|
val ref = spawn(behv)
|
||||||
ref ! End
|
ref ! End
|
||||||
probe.expectMsg(GotPostStop(false))
|
probe.expectMsg(GotPostStop(false))
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -0,0 +1,31 @@
|
||||||
|
package akka.actor.typed
|
||||||
|
|
||||||
|
import akka.testkit.typed.{ TestInbox, TestKit }
|
||||||
|
import akka.util.Timeout
|
||||||
|
import org.scalactic.TypeCheckedTripleEquals
|
||||||
|
import org.scalatest.concurrent.ScalaFutures
|
||||||
|
import org.scalatest.time.Span
|
||||||
|
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpecLike }
|
||||||
|
|
||||||
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper trait to include standard traits for typed tests
|
||||||
|
*/
|
||||||
|
trait TypedAkkaSpec extends WordSpecLike with Matchers with BeforeAndAfterAll with ScalaFutures with TypeCheckedTripleEquals {
|
||||||
|
implicit val akkaPatience = 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)))
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Helper that also shuts down the actor system if using [[TestKit]]
|
||||||
|
*/
|
||||||
|
trait TypedAkkaSpecWithShutdown extends TypedAkkaSpec {
|
||||||
|
self: TestKit ⇒
|
||||||
|
override protected def afterAll(): Unit = shutdown()
|
||||||
|
}
|
||||||
|
|
@ -1,228 +0,0 @@
|
||||||
/**
|
|
||||||
* Copyright (C) 2014-2017 Lightbend Inc. <http://www.lightbend.com>
|
|
||||||
*/
|
|
||||||
package akka.actor.typed
|
|
||||||
|
|
||||||
import org.scalatest.{ BeforeAndAfterAll, Matchers, WordSpec }
|
|
||||||
import akka.testkit.AkkaSpec
|
|
||||||
|
|
||||||
import scala.concurrent.Await
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import scala.concurrent.Future
|
|
||||||
import com.typesafe.config.Config
|
|
||||||
import com.typesafe.config.ConfigFactory
|
|
||||||
import akka.util.Timeout
|
|
||||||
|
|
||||||
import scala.reflect.ClassTag
|
|
||||||
import akka.actor.ActorInitializationException
|
|
||||||
|
|
||||||
import language.existentials
|
|
||||||
import akka.testkit.TestEvent.Mute
|
|
||||||
import akka.actor.typed.scaladsl.Actor._
|
|
||||||
import org.scalatest.concurrent.ScalaFutures
|
|
||||||
import org.scalactic.TypeCheckedTripleEquals
|
|
||||||
import org.scalactic.CanEqual
|
|
||||||
|
|
||||||
import scala.util.control.NonFatal
|
|
||||||
import akka.actor.typed.scaladsl.AskPattern
|
|
||||||
|
|
||||||
import scala.util.control.NoStackTrace
|
|
||||||
import akka.testkit.typed.{ TestInbox, TestKitSettings }
|
|
||||||
import org.scalatest.time.Span
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Helper class for writing tests for typed Actors with ScalaTest.
|
|
||||||
*/
|
|
||||||
class TypedSpecSetup extends WordSpec with Matchers with BeforeAndAfterAll with ScalaFutures with TypeCheckedTripleEquals {
|
|
||||||
|
|
||||||
// TODO hook this up with config like in akka-testkit/AkkaSpec?
|
|
||||||
implicit val akkaPatience = PatienceConfig(3.seconds, Span(100, org.scalatest.time.Millis))
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
trait StartSupport {
|
|
||||||
implicit def system: ActorSystem[TypedSpec.Command]
|
|
||||||
private implicit def timeout: Timeout = Timeout(1.minute)
|
|
||||||
private implicit def scheduler = system.scheduler
|
|
||||||
|
|
||||||
private val nameCounter = Iterator.from(0)
|
|
||||||
|
|
||||||
def nextName(prefix: String = "a"): String = s"$prefix-${nameCounter.next()}"
|
|
||||||
|
|
||||||
def start[T](behv: Behavior[T]): ActorRef[T] = {
|
|
||||||
import akka.actor.typed.scaladsl.AskPattern._
|
|
||||||
import akka.testkit.typed.scaladsl._
|
|
||||||
implicit val testSettings = TestKitSettings(system)
|
|
||||||
Await.result(system ? TypedSpec.Create(behv, nextName()), 3.seconds.dilated)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
/**
|
|
||||||
* Helper class for writing tests against both ActorSystemImpl and ActorSystemAdapter.
|
|
||||||
*/
|
|
||||||
abstract class TypedSpec(val config: Config) extends TypedSpecSetup {
|
|
||||||
|
|
||||||
import TypedSpec._
|
|
||||||
import AskPattern._
|
|
||||||
|
|
||||||
def this() = this(ConfigFactory.empty)
|
|
||||||
|
|
||||||
def this(config: String) = this(ConfigFactory.parseString(config))
|
|
||||||
|
|
||||||
// extension point
|
|
||||||
def setTimeout: Timeout = Timeout(1.minute)
|
|
||||||
|
|
||||||
implicit lazy val system: ActorSystem[TypedSpec.Command] = {
|
|
||||||
val sys = ActorSystem(guardian(), AkkaSpec.getCallerName(classOf[TypedSpec]), config = Some(config withFallback AkkaSpec.testConf))
|
|
||||||
sys
|
|
||||||
}
|
|
||||||
|
|
||||||
trait AdaptedSystem {
|
|
||||||
def system: ActorSystem[TypedSpec.Command] = TypedSpec.this.system
|
|
||||||
}
|
|
||||||
|
|
||||||
implicit val timeout = setTimeout
|
|
||||||
implicit def scheduler = system.scheduler
|
|
||||||
|
|
||||||
lazy val blackhole = await(system ? Create(immutable[Any] { case _ ⇒ same }, "blackhole"))
|
|
||||||
|
|
||||||
override def afterAll(): Unit = {
|
|
||||||
Await.result(system.terminate, timeout.duration)
|
|
||||||
}
|
|
||||||
|
|
||||||
// TODO remove after basing on ScalaTest 3 with async support
|
|
||||||
import akka.testkit._
|
|
||||||
|
|
||||||
def await[T](f: Future[T]): T = Await.result(f, timeout.duration * 1.1)
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Run an Actor-based test. The test procedure is most conveniently
|
|
||||||
* formulated using the [[StepWise]] behavior type.
|
|
||||||
*/
|
|
||||||
def runTest[T: ClassTag](name: String)(behavior: Behavior[T])(implicit system: ActorSystem[Command]): Future[Status] =
|
|
||||||
system ? (RunTest(name, behavior, _, timeout.duration))
|
|
||||||
|
|
||||||
// TODO remove after basing on ScalaTest 3 with async support
|
|
||||||
def sync(f: Future[Status])(implicit system: ActorSystem[Command]): Unit = {
|
|
||||||
def unwrap(ex: Throwable): Throwable = ex match {
|
|
||||||
case ActorInitializationException(_, _, ex) ⇒ ex
|
|
||||||
case other ⇒ other
|
|
||||||
}
|
|
||||||
|
|
||||||
try await(f) match {
|
|
||||||
case Success ⇒ ()
|
|
||||||
case Failed(ex) ⇒
|
|
||||||
unwrap(ex) match {
|
|
||||||
case ex2: TypedSpec.SimulatedException ⇒
|
|
||||||
throw ex2
|
|
||||||
case _ ⇒
|
|
||||||
println(system.printTree)
|
|
||||||
throw unwrap(ex)
|
|
||||||
}
|
|
||||||
case Timedout ⇒
|
|
||||||
println(system.printTree)
|
|
||||||
fail("test timed out")
|
|
||||||
} catch {
|
|
||||||
case ex: TypedSpec.SimulatedException ⇒
|
|
||||||
throw ex
|
|
||||||
case NonFatal(ex) ⇒
|
|
||||||
println(system.printTree)
|
|
||||||
throw ex
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
def muteExpectedException[T <: Exception: ClassTag](
|
|
||||||
message: String = null,
|
|
||||||
source: String = null,
|
|
||||||
start: String = "",
|
|
||||||
pattern: String = null,
|
|
||||||
occurrences: Int = Int.MaxValue)(implicit system: ActorSystem[Command]): EventFilter = {
|
|
||||||
val filter = EventFilter(message, source, start, pattern, occurrences)
|
|
||||||
system.eventStream.publish(Mute(filter))
|
|
||||||
filter
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Group assertion that ensures that the given inboxes are empty.
|
|
||||||
*/
|
|
||||||
def assertEmpty(inboxes: TestInbox[_]*): Unit = {
|
|
||||||
inboxes foreach (i ⇒ withClue(s"inbox $i had messages")(i.hasMessages should be(false)))
|
|
||||||
}
|
|
||||||
|
|
||||||
// for ScalaTest === compare of Class objects
|
|
||||||
implicit def classEqualityConstraint[A, B]: CanEqual[Class[A], Class[B]] =
|
|
||||||
new CanEqual[Class[A], Class[B]] {
|
|
||||||
def areEqual(a: Class[A], b: Class[B]) = a == b
|
|
||||||
}
|
|
||||||
|
|
||||||
implicit def setEqualityConstraint[A, T <: Set[_ <: A]]: CanEqual[Set[A], T] =
|
|
||||||
new CanEqual[Set[A], T] {
|
|
||||||
def areEqual(a: Set[A], b: T) = a == b
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
object TypedSpec {
|
|
||||||
|
|
||||||
sealed abstract class Start
|
|
||||||
case object Start extends Start
|
|
||||||
|
|
||||||
sealed trait Command
|
|
||||||
case class RunTest[T](name: String, behavior: Behavior[T], replyTo: ActorRef[Status], timeout: FiniteDuration) extends Command
|
|
||||||
case class Terminate(reply: ActorRef[Status]) extends Command
|
|
||||||
case class Create[T](behavior: Behavior[T], name: String)(val replyTo: ActorRef[ActorRef[T]]) extends Command
|
|
||||||
|
|
||||||
sealed trait Status
|
|
||||||
case object Success extends Status
|
|
||||||
case class Failed(thr: Throwable) extends Status
|
|
||||||
case object Timedout extends Status
|
|
||||||
|
|
||||||
class SimulatedException(message: String) extends RuntimeException(message) with NoStackTrace
|
|
||||||
|
|
||||||
def guardian(outstanding: Map[ActorRef[_], ActorRef[Status]] = Map.empty): Behavior[Command] =
|
|
||||||
immutable[Command] {
|
|
||||||
case (ctx, r: RunTest[t]) ⇒
|
|
||||||
val test = ctx.spawn(r.behavior, r.name)
|
|
||||||
ctx.schedule(r.timeout, r.replyTo, Timedout)
|
|
||||||
ctx.watch(test)
|
|
||||||
guardian(outstanding + ((test, r.replyTo)))
|
|
||||||
case (_, Terminate(reply)) ⇒
|
|
||||||
reply ! Success
|
|
||||||
stopped
|
|
||||||
case (ctx, c: Create[t]) ⇒
|
|
||||||
c.replyTo ! ctx.spawn(c.behavior, c.name)
|
|
||||||
same
|
|
||||||
} onSignal {
|
|
||||||
case (ctx, t @ Terminated(test)) ⇒
|
|
||||||
outstanding get test match {
|
|
||||||
case Some(reply) ⇒
|
|
||||||
if (t.failure eq null) reply ! Success
|
|
||||||
else reply ! Failed(t.failure)
|
|
||||||
guardian(outstanding - test)
|
|
||||||
case None ⇒ same
|
|
||||||
}
|
|
||||||
case _ ⇒ same
|
|
||||||
}
|
|
||||||
|
|
||||||
def getCallerName(clazz: Class[_]): String = {
|
|
||||||
val s = (Thread.currentThread.getStackTrace map (_.getClassName) drop 1)
|
|
||||||
.dropWhile(_ matches "(java.lang.Thread|.*TypedSpec.?$)")
|
|
||||||
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]", "_")
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
class TypedSpecSpec extends TypedSpec {
|
|
||||||
|
|
||||||
"A TypedSpec" must {
|
|
||||||
"must report failures" in {
|
|
||||||
a[TypedSpec.SimulatedException] must be thrownBy {
|
|
||||||
sync(runTest("failure")(StepWise[String]((ctx, startWith) ⇒
|
|
||||||
startWith {
|
|
||||||
throw new TypedSpec.SimulatedException("expected")
|
|
||||||
})))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
@ -3,63 +3,68 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor.typed
|
package akka.actor.typed
|
||||||
|
|
||||||
import scala.concurrent._
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import scala.concurrent.duration._
|
|
||||||
import akka.actor.typed.scaladsl.Actor._
|
|
||||||
import akka.actor.typed.scaladsl.AskPattern._
|
|
||||||
|
|
||||||
class WatchSpec extends TypedSpec {
|
import scala.concurrent._
|
||||||
|
import akka.testkit.typed.TestKit
|
||||||
|
|
||||||
|
object WatchSpec {
|
||||||
|
case object Stop
|
||||||
|
|
||||||
|
val terminatorBehavior =
|
||||||
|
Actor.immutable[Stop.type] {
|
||||||
|
case (_, Stop) ⇒ Actor.stopped
|
||||||
|
}
|
||||||
|
|
||||||
|
sealed trait Message
|
||||||
|
case object CustomTerminationMessage extends Message
|
||||||
|
case class StartWatchingWith(watchee: ActorRef[Stop.type], msg: CustomTerminationMessage.type) extends Message
|
||||||
|
}
|
||||||
|
|
||||||
|
class WatchSpec extends TestKit("WordSpec")
|
||||||
|
with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
|
import WatchSpec._
|
||||||
|
|
||||||
"Actor monitoring" must {
|
"Actor monitoring" must {
|
||||||
|
|
||||||
"get notified of actor termination" in {
|
"get notified of actor termination" in {
|
||||||
case object Stop
|
|
||||||
case class StartWatching(watchee: ActorRef[Stop.type])
|
case class StartWatching(watchee: ActorRef[Stop.type])
|
||||||
|
val terminator = systemActor(terminatorBehavior)
|
||||||
|
val receivedTerminationSignal: Promise[ActorRef[Nothing]] = Promise()
|
||||||
|
|
||||||
val terminator = Await.result(system ? TypedSpec.Create(immutable[Stop.type] {
|
val watcher = systemActor(Actor.immutable[StartWatching] {
|
||||||
case (ctx, `Stop`) ⇒ stopped
|
case (ctx, StartWatching(watchee)) ⇒
|
||||||
}, "t1"), 3.seconds /*.dilated*/ )
|
ctx.watch(watchee)
|
||||||
|
Actor.same
|
||||||
val receivedTerminationSignal: Promise[Unit] = Promise()
|
|
||||||
|
|
||||||
val watcher = Await.result(system ? TypedSpec.Create(immutable[StartWatching] {
|
|
||||||
case (ctx, StartWatching(watchee)) ⇒ ctx.watch(watchee); same
|
|
||||||
}.onSignal {
|
}.onSignal {
|
||||||
case (ctx, Terminated(_)) ⇒ receivedTerminationSignal.success(()); stopped
|
case (_, Terminated(stopped)) ⇒
|
||||||
}, "w1"), 3.seconds /*.dilated*/ )
|
receivedTerminationSignal.success(stopped)
|
||||||
|
Actor.stopped
|
||||||
|
})
|
||||||
|
|
||||||
watcher ! StartWatching(terminator)
|
watcher ! StartWatching(terminator)
|
||||||
terminator ! Stop
|
terminator ! Stop
|
||||||
|
|
||||||
Await.result(receivedTerminationSignal.future, 3.seconds /*.dilated*/ )
|
receivedTerminationSignal.future.futureValue shouldEqual terminator
|
||||||
}
|
}
|
||||||
|
|
||||||
"get notified of actor termination with a custom message" in {
|
"get notified of actor termination with a custom message" in {
|
||||||
case object Stop
|
val terminator = systemActor(terminatorBehavior)
|
||||||
|
val receivedTerminationSignal: Promise[Message] = Promise()
|
||||||
|
|
||||||
sealed trait Message
|
val watcher = systemActor(Actor.immutable[Message] {
|
||||||
case object CustomTerminationMessage extends Message
|
|
||||||
case class StartWatchingWith(watchee: ActorRef[Stop.type], msg: CustomTerminationMessage.type) extends Message
|
|
||||||
|
|
||||||
val terminator = Await.result(system ? TypedSpec.Create(immutable[Stop.type] {
|
|
||||||
case (ctx, `Stop`) ⇒ stopped
|
|
||||||
}, "t2"), 3.seconds /*.dilated*/ )
|
|
||||||
|
|
||||||
val receivedTerminationSignal: Promise[Unit] = Promise()
|
|
||||||
|
|
||||||
val watcher = Await.result(system ? TypedSpec.Create(immutable[Message] {
|
|
||||||
case (ctx, StartWatchingWith(watchee, msg)) ⇒
|
case (ctx, StartWatchingWith(watchee, msg)) ⇒
|
||||||
ctx.watchWith(watchee, msg)
|
ctx.watchWith(watchee, msg)
|
||||||
same
|
Actor.same
|
||||||
case (ctx, `CustomTerminationMessage`) ⇒
|
case (_, msg) ⇒
|
||||||
receivedTerminationSignal.success(())
|
receivedTerminationSignal.success(msg)
|
||||||
stopped
|
Actor.stopped
|
||||||
}, "w2"), 3.seconds /*.dilated*/ )
|
})
|
||||||
|
|
||||||
watcher ! StartWatchingWith(terminator, CustomTerminationMessage)
|
watcher ! StartWatchingWith(terminator, CustomTerminationMessage)
|
||||||
terminator ! Stop
|
terminator ! Stop
|
||||||
|
|
||||||
Await.result(receivedTerminationSignal.future, 3.seconds /*.dilated*/ )
|
receivedTerminationSignal.future.futureValue shouldEqual CustomTerminationMessage
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -3,12 +3,11 @@
|
||||||
*/
|
*/
|
||||||
package akka.actor.typed.internal
|
package akka.actor.typed.internal
|
||||||
|
|
||||||
import akka.actor.typed.TypedSpec
|
import akka.actor.typed.TypedAkkaSpecWithShutdown
|
||||||
import akka.actor.typed.TypedSpec.{ Create ⇒ TCreate }
|
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.AskPattern._
|
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
|
import akka.testkit.typed.TestKit
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
object MiscMessageSerializerSpec {
|
object MiscMessageSerializerSpec {
|
||||||
|
|
@ -23,7 +22,7 @@ object MiscMessageSerializerSpec {
|
||||||
""")
|
""")
|
||||||
}
|
}
|
||||||
|
|
||||||
class MiscMessageSerializerSpec extends TypedSpec(MiscMessageSerializerSpec.config) {
|
class MiscMessageSerializerSpec extends TestKit(MiscMessageSerializerSpec.config) with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
val serialization = SerializationExtension(system.toUntyped)
|
val serialization = SerializationExtension(system.toUntyped)
|
||||||
|
|
||||||
|
|
@ -40,8 +39,7 @@ class MiscMessageSerializerSpec extends TypedSpec(MiscMessageSerializerSpec.conf
|
||||||
}
|
}
|
||||||
|
|
||||||
"must serialize and deserialize typed actor refs" in {
|
"must serialize and deserialize typed actor refs" in {
|
||||||
val ref = (system ? TCreate(Actor.empty[Unit], "some-actor")).futureValue
|
val ref = spawn(Actor.empty[Unit])
|
||||||
println(ref.getClass)
|
|
||||||
checkSerialization(ref)
|
checkSerialization(ref)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -7,15 +7,13 @@ import akka.actor.typed._
|
||||||
import akka.actor.typed.receptionist.Receptionist._
|
import akka.actor.typed.receptionist.Receptionist._
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.AskPattern._
|
import akka.actor.typed.scaladsl.AskPattern._
|
||||||
import akka.testkit.typed.BehaviorTestkit
|
import akka.testkit.typed.{ BehaviorTestkit, TestInbox, TestKit, TestKitSettings }
|
||||||
import akka.testkit.typed.TestInbox
|
|
||||||
import akka.testkit.typed.TestKitSettings
|
|
||||||
import akka.testkit.typed.scaladsl.TestProbe
|
import akka.testkit.typed.scaladsl.TestProbe
|
||||||
import org.scalatest.concurrent.Eventually
|
import org.scalatest.concurrent.Eventually
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.Future
|
||||||
|
|
||||||
class LocalReceptionistSpec extends TypedSpec with Eventually with StartSupport {
|
class LocalReceptionistSpec extends TestKit with TypedAkkaSpecWithShutdown with Eventually {
|
||||||
|
|
||||||
trait ServiceA
|
trait ServiceA
|
||||||
val ServiceKeyA = Receptionist.ServiceKey[ServiceA]("service-a")
|
val ServiceKeyA = Receptionist.ServiceKey[ServiceA]("service-a")
|
||||||
|
|
@ -33,77 +31,77 @@ class LocalReceptionistSpec extends TypedSpec with Eventually with StartSupport
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
import akka.actor.typed.internal.receptionist.ReceptionistImpl.{ localOnlyBehavior ⇒ behavior }
|
import akka.actor.typed.internal.receptionist.ReceptionistImpl.{ localOnlyBehavior ⇒ receptionistBehavior }
|
||||||
|
|
||||||
implicit val testSettings = TestKitSettings(system)
|
implicit val testSettings = TestKitSettings(system)
|
||||||
|
|
||||||
abstract class TestSetup {
|
abstract class TestSetup {
|
||||||
val receptionist = start(behavior)
|
val receptionist = spawn(receptionistBehavior)
|
||||||
}
|
}
|
||||||
|
|
||||||
"A local receptionist" must {
|
"A local receptionist" must {
|
||||||
|
|
||||||
"must register a service" in {
|
"register a service" in {
|
||||||
val ctx = new BehaviorTestkit("register", behavior)
|
val testkit = BehaviorTestkit(receptionistBehavior)
|
||||||
val a = TestInbox[ServiceA]("a")
|
val a = TestInbox[ServiceA]("a")
|
||||||
val r = TestInbox[Registered[_]]("r")
|
val r = TestInbox[Registered[_]]("r")
|
||||||
ctx.run(Register(ServiceKeyA, a.ref)(r.ref))
|
testkit.run(Register(ServiceKeyA, a.ref)(r.ref))
|
||||||
ctx.retrieveEffect() // watching however that is implemented
|
testkit.retrieveEffect() // watching however that is implemented
|
||||||
r.receiveMsg() should be(Registered(ServiceKeyA, a.ref))
|
r.receiveMsg() should be(Registered(ServiceKeyA, a.ref))
|
||||||
val q = TestInbox[Listing[ServiceA]]("q")
|
val q = TestInbox[Listing[ServiceA]]("q")
|
||||||
ctx.run(Find(ServiceKeyA)(q.ref))
|
testkit.run(Find(ServiceKeyA)(q.ref))
|
||||||
ctx.retrieveAllEffects() should be(Nil)
|
testkit.retrieveAllEffects() should be(Nil)
|
||||||
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref)))
|
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref)))
|
||||||
assertEmpty(a, r, q)
|
assertEmpty(a, r, q)
|
||||||
}
|
}
|
||||||
|
|
||||||
"must register two services" in {
|
"register two services" in {
|
||||||
val ctx = new BehaviorTestkit("registertwo", behavior)
|
val testkit = BehaviorTestkit(receptionistBehavior)
|
||||||
val a = TestInbox[ServiceA]("a")
|
val a = TestInbox[ServiceA]("a")
|
||||||
val r = TestInbox[Registered[_]]("r")
|
val r = TestInbox[Registered[_]]("r")
|
||||||
ctx.run(Register(ServiceKeyA, a.ref)(r.ref))
|
testkit.run(Register(ServiceKeyA, a.ref)(r.ref))
|
||||||
r.receiveMsg() should be(Registered(ServiceKeyA, a.ref))
|
r.receiveMsg() should be(Registered(ServiceKeyA, a.ref))
|
||||||
val b = TestInbox[ServiceB]("b")
|
val b = TestInbox[ServiceB]("b")
|
||||||
ctx.run(Register(ServiceKeyB, b.ref)(r.ref))
|
testkit.run(Register(ServiceKeyB, b.ref)(r.ref))
|
||||||
r.receiveMsg() should be(Registered(ServiceKeyB, b.ref))
|
r.receiveMsg() should be(Registered(ServiceKeyB, b.ref))
|
||||||
val q = TestInbox[Listing[_]]("q")
|
val q = TestInbox[Listing[_]]("q")
|
||||||
ctx.run(Find(ServiceKeyA)(q.ref))
|
testkit.run(Find(ServiceKeyA)(q.ref))
|
||||||
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref)))
|
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a.ref)))
|
||||||
ctx.run(Find(ServiceKeyB)(q.ref))
|
testkit.run(Find(ServiceKeyB)(q.ref))
|
||||||
q.receiveMsg() should be(Listing(ServiceKeyB, Set(b.ref)))
|
q.receiveMsg() should be(Listing(ServiceKeyB, Set(b.ref)))
|
||||||
assertEmpty(a, b, r, q)
|
assertEmpty(a, b, r, q)
|
||||||
}
|
}
|
||||||
|
|
||||||
"must register two services with the same key" in {
|
"register two services with the same key" in {
|
||||||
val ctx = new BehaviorTestkit("registertwosame", behavior)
|
val testkit = BehaviorTestkit(receptionistBehavior)
|
||||||
val a1 = TestInbox[ServiceA]("a1")
|
val a1 = TestInbox[ServiceA]("a1")
|
||||||
val r = TestInbox[Registered[_]]("r")
|
val r = TestInbox[Registered[_]]("r")
|
||||||
ctx.run(Register(ServiceKeyA, a1.ref)(r.ref))
|
testkit.run(Register(ServiceKeyA, a1.ref)(r.ref))
|
||||||
r.receiveMsg() should be(Registered(ServiceKeyA, a1.ref))
|
r.receiveMsg() should be(Registered(ServiceKeyA, a1.ref))
|
||||||
val a2 = TestInbox[ServiceA]("a2")
|
val a2 = TestInbox[ServiceA]("a2")
|
||||||
ctx.run(Register(ServiceKeyA, a2.ref)(r.ref))
|
testkit.run(Register(ServiceKeyA, a2.ref)(r.ref))
|
||||||
r.receiveMsg() should be(Registered(ServiceKeyA, a2.ref))
|
r.receiveMsg() should be(Registered(ServiceKeyA, a2.ref))
|
||||||
val q = TestInbox[Listing[_]]("q")
|
val q = TestInbox[Listing[_]]("q")
|
||||||
ctx.run(Find(ServiceKeyA)(q.ref))
|
testkit.run(Find(ServiceKeyA)(q.ref))
|
||||||
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a1.ref, a2.ref)))
|
q.receiveMsg() should be(Listing(ServiceKeyA, Set(a1.ref, a2.ref)))
|
||||||
ctx.run(Find(ServiceKeyB)(q.ref))
|
testkit.run(Find(ServiceKeyB)(q.ref))
|
||||||
q.receiveMsg() should be(Listing(ServiceKeyB, Set.empty[ActorRef[ServiceB]]))
|
q.receiveMsg() should be(Listing(ServiceKeyB, Set.empty[ActorRef[ServiceB]]))
|
||||||
assertEmpty(a1, a2, r, q)
|
assertEmpty(a1, a2, r, q)
|
||||||
}
|
}
|
||||||
|
|
||||||
"must unregister services when they terminate" in {
|
"unregister services when they terminate" in {
|
||||||
new TestSetup {
|
new TestSetup {
|
||||||
val regProbe = TestProbe[Any]("regProbe")
|
val regProbe = TestProbe[Any]("regProbe")
|
||||||
|
|
||||||
val serviceA = start(stoppableBehavior.narrow[ServiceA])
|
val serviceA = spawn(stoppableBehavior.narrow[ServiceA])
|
||||||
receptionist ! Register(ServiceKeyA, serviceA, regProbe.ref)
|
receptionist ! Register(ServiceKeyA, serviceA, regProbe.ref)
|
||||||
regProbe.expectMsg(Registered(ServiceKeyA, serviceA))
|
regProbe.expectMsg(Registered(ServiceKeyA, serviceA))
|
||||||
|
|
||||||
val serviceB = start(stoppableBehavior.narrow[ServiceB])
|
val serviceB = spawn(stoppableBehavior.narrow[ServiceB])
|
||||||
receptionist ! Register(ServiceKeyB, serviceB, regProbe.ref)
|
receptionist ! Register(ServiceKeyB, serviceB, regProbe.ref)
|
||||||
regProbe.expectMsg(Registered(ServiceKeyB, serviceB))
|
regProbe.expectMsg(Registered(ServiceKeyB, serviceB))
|
||||||
|
|
||||||
val serviceC = start(stoppableBehavior)
|
val serviceC = spawn(stoppableBehavior)
|
||||||
receptionist ! Register(ServiceKeyA, serviceC, regProbe.ref)
|
receptionist ! Register(ServiceKeyA, serviceC, regProbe.ref)
|
||||||
receptionist ! Register(ServiceKeyB, serviceC, regProbe.ref)
|
receptionist ! Register(ServiceKeyB, serviceC, regProbe.ref)
|
||||||
regProbe.expectMsg(Registered(ServiceKeyA, serviceC))
|
regProbe.expectMsg(Registered(ServiceKeyA, serviceC))
|
||||||
|
|
@ -125,7 +123,7 @@ class LocalReceptionistSpec extends TypedSpec with Eventually with StartSupport
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"must support subscribing to service changes" in {
|
"support subscribing to service changes" in {
|
||||||
new TestSetup {
|
new TestSetup {
|
||||||
val regProbe = TestProbe[Registered[_]]("regProbe")
|
val regProbe = TestProbe[Registered[_]]("regProbe")
|
||||||
|
|
||||||
|
|
@ -134,13 +132,13 @@ class LocalReceptionistSpec extends TypedSpec with Eventually with StartSupport
|
||||||
|
|
||||||
aSubscriber.expectMsg(Listing(ServiceKeyA, Set.empty[ActorRef[ServiceA]]))
|
aSubscriber.expectMsg(Listing(ServiceKeyA, Set.empty[ActorRef[ServiceA]]))
|
||||||
|
|
||||||
val serviceA: ActorRef[ServiceA] = start(stoppableBehavior)
|
val serviceA: ActorRef[ServiceA] = spawn(stoppableBehavior)
|
||||||
receptionist ! Register(ServiceKeyA, serviceA, regProbe.ref)
|
receptionist ! Register(ServiceKeyA, serviceA, regProbe.ref)
|
||||||
regProbe.expectMsg(Registered(ServiceKeyA, serviceA))
|
regProbe.expectMsg(Registered(ServiceKeyA, serviceA))
|
||||||
|
|
||||||
aSubscriber.expectMsg(Listing(ServiceKeyA, Set(serviceA)))
|
aSubscriber.expectMsg(Listing(ServiceKeyA, Set(serviceA)))
|
||||||
|
|
||||||
val serviceA2: ActorRef[ServiceA] = start(stoppableBehavior)
|
val serviceA2: ActorRef[ServiceA] = spawn(stoppableBehavior)
|
||||||
receptionist ! Register(ServiceKeyA, serviceA2, regProbe.ref)
|
receptionist ! Register(ServiceKeyA, serviceA2, regProbe.ref)
|
||||||
regProbe.expectMsg(Registered(ServiceKeyA, serviceA2))
|
regProbe.expectMsg(Registered(ServiceKeyA, serviceA2))
|
||||||
|
|
||||||
|
|
@ -153,40 +151,18 @@ class LocalReceptionistSpec extends TypedSpec with Eventually with StartSupport
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"must work with ask" in {
|
"work with ask" in {
|
||||||
sync(runTest("Receptionist") {
|
val receptionist = spawn(receptionistBehavior)
|
||||||
StepWise[Registered[ServiceA]] { (ctx, startWith) ⇒
|
val serviceA = spawn(behaviorA)
|
||||||
val self = ctx.self
|
val f: Future[Registered[ServiceA]] = receptionist ? Register(ServiceKeyA, serviceA)
|
||||||
startWith.withKeepTraces(true) {
|
f.futureValue should be(Registered(ServiceKeyA, serviceA))
|
||||||
val r = ctx.spawnAnonymous(behavior)
|
|
||||||
val s = ctx.spawnAnonymous(behaviorA)
|
|
||||||
val f = r ? Register(ServiceKeyA, s)
|
|
||||||
r ! Register(ServiceKeyA, s)(self)
|
|
||||||
(f, s)
|
|
||||||
}.expectMessage(1.second) {
|
|
||||||
case (msg, (f, s)) ⇒
|
|
||||||
msg should be(Registered(ServiceKeyA, s))
|
|
||||||
f.foreach(self ! _)(system.executionContext)
|
|
||||||
s
|
|
||||||
}.expectMessage(1.second) {
|
|
||||||
case (msg, s) ⇒
|
|
||||||
msg should be(Registered(ServiceKeyA, s))
|
|
||||||
}
|
|
||||||
}
|
|
||||||
})
|
|
||||||
}
|
}
|
||||||
|
|
||||||
"must be present in the system" in {
|
"be present in the system" in {
|
||||||
sync(runTest("systemReceptionist") {
|
val probe = TestProbe[Receptionist.Listing[_]]()
|
||||||
StepWise[Listing[ServiceA]] { (ctx, startWith) ⇒
|
system.receptionist ! Find(ServiceKeyA)(probe.ref)
|
||||||
val self = ctx.self
|
val listing: Listing[_] = probe.expectMsgType[Listing[_]]
|
||||||
startWith.withKeepTraces(true) {
|
listing.serviceInstances should be(Set())
|
||||||
system.receptionist ! Find(ServiceKeyA)(self)
|
|
||||||
}.expectMessage(1.second) { (msg, _) ⇒
|
|
||||||
msg.serviceInstances should ===(Set())
|
|
||||||
}
|
|
||||||
}
|
|
||||||
})
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -4,13 +4,12 @@
|
||||||
package akka.actor.typed
|
package akka.actor.typed
|
||||||
package scaladsl
|
package scaladsl
|
||||||
|
|
||||||
import akka.testkit.typed.{ BehaviorTestkit, TestKitSettings }
|
import akka.testkit.typed.{ BehaviorTestkit, TestKit, TestKitSettings }
|
||||||
import akka.testkit.typed.scaladsl.TestProbe
|
import akka.testkit.typed.scaladsl.TestProbe
|
||||||
|
|
||||||
import scala.concurrent.duration.DurationInt
|
import scala.concurrent.duration.DurationInt
|
||||||
|
|
||||||
class ImmutablePartialSpec extends TypedSpec with StartSupport {
|
class ImmutablePartialSpec extends TestKit with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
private implicit val testSettings = TestKitSettings(system)
|
|
||||||
|
|
||||||
"An immutable partial" must {
|
"An immutable partial" must {
|
||||||
|
|
||||||
|
|
@ -22,14 +21,14 @@ class ImmutablePartialSpec extends TypedSpec with StartSupport {
|
||||||
probe.ref ! Command2
|
probe.ref ! Command2
|
||||||
Actor.same
|
Actor.same
|
||||||
}
|
}
|
||||||
val context = new BehaviorTestkit("ctx", behavior)
|
val testkit = BehaviorTestkit(behavior)
|
||||||
|
|
||||||
context.run(Command1)
|
testkit.run(Command1)
|
||||||
context.currentBehavior shouldBe behavior
|
testkit.currentBehavior shouldBe behavior
|
||||||
probe.expectNoMsg(100.milliseconds)
|
probe.expectNoMsg(100.milliseconds)
|
||||||
|
|
||||||
context.run(Command2)
|
testkit.run(Command2)
|
||||||
context.currentBehavior shouldBe behavior
|
testkit.currentBehavior shouldBe behavior
|
||||||
probe.expectMsg(Command2)
|
probe.expectMsg(Command2)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -5,12 +5,10 @@ package akka.actor.typed
|
||||||
package scaladsl
|
package scaladsl
|
||||||
|
|
||||||
import akka.Done
|
import akka.Done
|
||||||
import akka.testkit.typed.TestKitSettings
|
import akka.testkit.typed.TestKit
|
||||||
import akka.testkit.typed.scaladsl.TestProbe
|
import akka.testkit.typed.scaladsl.TestProbe
|
||||||
|
|
||||||
final class OnSignalSpec extends TypedSpec with StartSupport {
|
final class OnSignalSpec extends TestKit with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
private implicit val testSettings = TestKitSettings(system)
|
|
||||||
|
|
||||||
"An Actor.OnSignal behavior" must {
|
"An Actor.OnSignal behavior" must {
|
||||||
"must correctly install the signal handler" in {
|
"must correctly install the signal handler" in {
|
||||||
|
|
@ -25,7 +23,7 @@ final class OnSignalSpec extends TypedSpec with StartSupport {
|
||||||
Actor.stopped
|
Actor.stopped
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
start[Nothing](behavior)
|
spawn[Nothing](behavior)
|
||||||
probe.expectMsg(Done)
|
probe.expectMsg(Done)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,8 @@ package docs.akka.typed
|
||||||
import akka.actor.typed._
|
import akka.actor.typed._
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.AskPattern._
|
import akka.actor.typed.scaladsl.AskPattern._
|
||||||
|
import akka.testkit.typed.TestKit
|
||||||
|
|
||||||
import scala.concurrent.Future
|
import scala.concurrent.Future
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
|
|
@ -73,7 +75,7 @@ object IntroSpec {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class IntroSpec extends TypedSpec {
|
class IntroSpec extends TestKit with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
import IntroSpec._
|
import IntroSpec._
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -7,6 +7,7 @@ package docs.akka.typed
|
||||||
import akka.actor.typed._
|
import akka.actor.typed._
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.ActorContext
|
import akka.actor.typed.scaladsl.ActorContext
|
||||||
|
import akka.testkit.typed.TestKit
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
|
|
@ -65,7 +66,7 @@ object MutableIntroSpec {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class MutableIntroSpec extends TypedSpec {
|
class MutableIntroSpec extends TestKit with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
import MutableIntroSpec._
|
import MutableIntroSpec._
|
||||||
|
|
||||||
|
|
@ -102,7 +103,7 @@ class MutableIntroSpec extends TypedSpec {
|
||||||
chatRoom ! GetSession("ol’ Gabbler", gabblerRef)
|
chatRoom ! GetSession("ol’ Gabbler", gabblerRef)
|
||||||
Actor.same
|
Actor.same
|
||||||
} onSignal {
|
} onSignal {
|
||||||
case (_, Terminated(ref)) ⇒
|
case (_, Terminated(_)) ⇒
|
||||||
println("Stopping guardian")
|
println("Stopping guardian")
|
||||||
Actor.stopped
|
Actor.stopped
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -22,7 +22,7 @@ object BasicAsyncTestingSpec {
|
||||||
}
|
}
|
||||||
|
|
||||||
//#test-header
|
//#test-header
|
||||||
class BasicAsyncTestingSpec extends TestKit(ActorSystem(Actor.empty, "BasicTestingSpec"))
|
class BasicAsyncTestingSpec extends TestKit("BasicTestingSpec")
|
||||||
with WordSpecLike with BeforeAndAfterAll {
|
with WordSpecLike with BeforeAndAfterAll {
|
||||||
//#test-header
|
//#test-header
|
||||||
|
|
||||||
|
|
@ -30,12 +30,21 @@ class BasicAsyncTestingSpec extends TestKit(ActorSystem(Actor.empty, "BasicTesti
|
||||||
|
|
||||||
"A testkit" must {
|
"A testkit" must {
|
||||||
"support verifying a response" in {
|
"support verifying a response" in {
|
||||||
//#test
|
//#test-spawn
|
||||||
val probe = TestProbe[Pong]()
|
val probe = TestProbe[Pong]()
|
||||||
val pinger = actorOf(echoActor, "ping")
|
val pinger = spawn(echoActor, "ping")
|
||||||
pinger ! Ping("hello", probe.ref)
|
pinger ! Ping("hello", probe.ref)
|
||||||
probe.expectMsg(Pong("hello"))
|
probe.expectMsg(Pong("hello"))
|
||||||
//#test
|
//#test-spawn
|
||||||
|
}
|
||||||
|
|
||||||
|
"support verifying a response - anonymous" in {
|
||||||
|
//#test-spawn-anonymous
|
||||||
|
val probe = TestProbe[Pong]()
|
||||||
|
val pinger = spawn(echoActor)
|
||||||
|
pinger ! Ping("hello", probe.ref)
|
||||||
|
probe.expectMsg(Pong("hello"))
|
||||||
|
//#test-spawn-anonymous
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -61,7 +61,7 @@ object SupervisorStrategy {
|
||||||
minBackoff: FiniteDuration,
|
minBackoff: FiniteDuration,
|
||||||
maxBackoff: FiniteDuration,
|
maxBackoff: FiniteDuration,
|
||||||
randomFactor: Double): BackoffSupervisorStrategy =
|
randomFactor: Double): BackoffSupervisorStrategy =
|
||||||
new Backoff(minBackoff, maxBackoff, randomFactor, resetBackoffAfter = minBackoff, loggingEnabled = true)
|
Backoff(minBackoff, maxBackoff, randomFactor, resetBackoffAfter = minBackoff, loggingEnabled = true)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* INTERNAL API
|
* INTERNAL API
|
||||||
|
|
|
||||||
|
|
@ -218,9 +218,9 @@ object Actor {
|
||||||
tap((_, msg) ⇒ monitor ! msg, unitFunction, behavior)
|
tap((_, msg) ⇒ monitor ! msg, unitFunction, behavior)
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Wrap the given behavior such that it is restarted (i.e. reset to its
|
* Wrap the given behavior with the given [[SupervisorStrategy]] for
|
||||||
* initial state) whenever it throws an exception of the given class or a
|
* the given exception.
|
||||||
* subclass thereof. Exceptions that are not subtypes of `Thr` will not be
|
* Exceptions that are not subtypes of `Thr` will not be
|
||||||
* caught and thus lead to the termination of the actor.
|
* caught and thus lead to the termination of the actor.
|
||||||
*
|
*
|
||||||
* It is possible to specify different supervisor strategies, such as restart,
|
* It is possible to specify different supervisor strategies, such as restart,
|
||||||
|
|
@ -247,7 +247,7 @@ object Actor {
|
||||||
private final val NothingClassTag = ClassTag(classOf[Nothing])
|
private final val NothingClassTag = ClassTag(classOf[Nothing])
|
||||||
private final val ThrowableClassTag = ClassTag(classOf[Throwable])
|
private final val ThrowableClassTag = ClassTag(classOf[Throwable])
|
||||||
final class Supervise[T] private[akka] (val wrapped: Behavior[T]) extends AnyVal {
|
final class Supervise[T] private[akka] (val wrapped: Behavior[T]) extends AnyVal {
|
||||||
/** Specify the [[SupervisorStrategy]] to be invoked when the wrapped behaior throws. */
|
/** Specify the [[SupervisorStrategy]] to be invoked when the wrapped behavior throws. */
|
||||||
def onFailure[Thr <: Throwable: ClassTag](strategy: SupervisorStrategy): Behavior[T] = {
|
def onFailure[Thr <: Throwable: ClassTag](strategy: SupervisorStrategy): Behavior[T] = {
|
||||||
val tag = implicitly[ClassTag[Thr]]
|
val tag = implicitly[ClassTag[Thr]]
|
||||||
val effectiveTag = if (tag == NothingClassTag) ThrowableClassTag else tag
|
val effectiveTag = if (tag == NothingClassTag) ThrowableClassTag else tag
|
||||||
|
|
|
||||||
|
|
@ -1,19 +1,13 @@
|
||||||
/*
|
/*
|
||||||
* Copyright (C) 2017 Lightbend Inc. <http://www.lightbend.com/>
|
* Copyright (C) 2017 Lightbend Inc. <http://www.lightbend.com/>
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package akka.cluster.sharding.typed
|
package akka.cluster.sharding.typed
|
||||||
|
|
||||||
import akka.actor.typed.{ ActorRef, Props, TypedSpec }
|
import akka.actor.typed.{ ActorRef, Behavior, Props, TypedAkkaSpecWithShutdown }
|
||||||
import com.typesafe.config.ConfigFactory
|
|
||||||
import org.scalatest.concurrent.ScalaFutures
|
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
|
||||||
import akka.actor.typed.Behavior
|
|
||||||
import akka.testkit.typed.TestKitSettings
|
|
||||||
import akka.testkit.typed.scaladsl.TestProbe
|
|
||||||
import akka.persistence.typed.scaladsl.PersistentActor
|
import akka.persistence.typed.scaladsl.PersistentActor
|
||||||
import akka.persistence.typed.scaladsl.PersistentActor.PersistNothing
|
import akka.testkit.typed.TestKit
|
||||||
|
import akka.testkit.typed.scaladsl.TestProbe
|
||||||
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
object ClusterShardingPersistenceSpec {
|
object ClusterShardingPersistenceSpec {
|
||||||
val config = ConfigFactory.parseString(
|
val config = ConfigFactory.parseString(
|
||||||
|
|
@ -48,7 +42,7 @@ object ClusterShardingPersistenceSpec {
|
||||||
PersistentActor.persistentEntity[Command, String, String](
|
PersistentActor.persistentEntity[Command, String, String](
|
||||||
persistenceIdFromActorName = name ⇒ "Test-" + name,
|
persistenceIdFromActorName = name ⇒ "Test-" + name,
|
||||||
initialState = "",
|
initialState = "",
|
||||||
commandHandler = CommandHandler((ctx, state, cmd) ⇒ cmd match {
|
commandHandler = CommandHandler((_, state, cmd) ⇒ cmd match {
|
||||||
case Add(s) ⇒ Effect.persist(s)
|
case Add(s) ⇒ Effect.persist(s)
|
||||||
case Get(replyTo) ⇒
|
case Get(replyTo) ⇒
|
||||||
replyTo ! state
|
replyTo ! state
|
||||||
|
|
@ -61,12 +55,12 @@ object ClusterShardingPersistenceSpec {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class ClusterShardingPersistenceSpec extends TypedSpec(ClusterShardingPersistenceSpec.config) with ScalaFutures {
|
class ClusterShardingPersistenceSpec extends TestKit("ClusterShardingPersistenceSPec", ClusterShardingPersistenceSpec.config)
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
with TypedAkkaSpecWithShutdown {
|
||||||
import ClusterShardingPersistenceSpec._
|
import ClusterShardingPersistenceSpec._
|
||||||
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
|
|
||||||
implicit val s = system
|
implicit val s = system
|
||||||
implicit val testkitSettings = TestKitSettings(system)
|
|
||||||
val sharding = ClusterSharding(system)
|
val sharding = ClusterSharding(system)
|
||||||
|
|
||||||
implicit val untypedSystem = system.toUntyped
|
implicit val untypedSystem = system.toUntyped
|
||||||
|
|
|
||||||
|
|
@ -4,25 +4,21 @@
|
||||||
|
|
||||||
package akka.cluster.sharding.typed
|
package akka.cluster.sharding.typed
|
||||||
|
|
||||||
import akka.cluster.sharding.ShardCoordinator.LeastShardAllocationStrategy
|
import java.nio.charset.StandardCharsets
|
||||||
import akka.actor.typed.{ ActorRef, ActorRefResolver, ActorSystem, Props, TypedSpec }
|
|
||||||
import akka.cluster.typed.Cluster
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
import akka.actor.typed.{ ActorRef, ActorRefResolver, Props, TypedAkkaSpecWithShutdown }
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
import akka.testkit.typed.TestKitSettings
|
import akka.cluster.MemberStatus
|
||||||
|
import akka.cluster.typed.{ Cluster, Join }
|
||||||
|
import akka.serialization.SerializerWithStringManifest
|
||||||
|
import akka.testkit.typed.TestKit
|
||||||
import akka.testkit.typed.scaladsl.TestProbe
|
import akka.testkit.typed.scaladsl.TestProbe
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.scalatest.concurrent.ScalaFutures
|
import org.scalatest.concurrent.{ Eventually, ScalaFutures }
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.Await
|
|
||||||
import akka.cluster.typed.Join
|
|
||||||
import org.scalatest.concurrent.Eventually
|
|
||||||
import akka.cluster.MemberStatus
|
|
||||||
import akka.actor.ExtendedActorSystem
|
|
||||||
import akka.serialization.SerializerWithStringManifest
|
|
||||||
import java.nio.charset.StandardCharsets
|
|
||||||
|
|
||||||
object ClusterShardingSpec {
|
object ClusterShardingSpec {
|
||||||
val config = ConfigFactory.parseString(
|
val config = ConfigFactory.parseString(
|
||||||
|
|
@ -118,13 +114,13 @@ object ClusterShardingSpec {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class ClusterShardingSpec extends TypedSpec(ClusterShardingSpec.config) with ScalaFutures with Eventually {
|
class ClusterShardingSpec extends TestKit("ClusterShardingSpec", ClusterShardingSpec.config)
|
||||||
|
with TypedAkkaSpecWithShutdown with ScalaFutures with Eventually {
|
||||||
|
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
|
||||||
import ClusterShardingSpec._
|
import ClusterShardingSpec._
|
||||||
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
|
|
||||||
implicit val s = system
|
implicit val s = system
|
||||||
implicit val testkitSettings = TestKitSettings(system)
|
|
||||||
val sharding = ClusterSharding(system)
|
val sharding = ClusterSharding(system)
|
||||||
|
|
||||||
implicit val untypedSystem = system.toUntyped
|
implicit val untypedSystem = system.toUntyped
|
||||||
|
|
@ -134,7 +130,7 @@ class ClusterShardingSpec extends TypedSpec(ClusterShardingSpec.config) with Sca
|
||||||
val sharding2 = ClusterSharding(system2)
|
val sharding2 = ClusterSharding(system2)
|
||||||
|
|
||||||
override def afterAll(): Unit = {
|
override def afterAll(): Unit = {
|
||||||
Await.result(system2.terminate, timeout.duration)
|
system2.terminate().futureValue
|
||||||
super.afterAll()
|
super.afterAll()
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
@ -173,17 +169,17 @@ class ClusterShardingSpec extends TypedSpec(ClusterShardingSpec.config) with Sca
|
||||||
Cluster(system2).manager ! Join(Cluster(system).selfMember.address)
|
Cluster(system2).manager ! Join(Cluster(system).selfMember.address)
|
||||||
|
|
||||||
eventually {
|
eventually {
|
||||||
Cluster(system).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(system).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
|
||||||
Cluster(system).state.members.size should ===(2)
|
Cluster(system).state.members.size should ===(2)
|
||||||
}
|
}
|
||||||
eventually {
|
eventually {
|
||||||
Cluster(system2).state.members.map(_.status) should ===(Set(MemberStatus.Up))
|
Cluster(system2).state.members.map(_.status) should ===(Set[MemberStatus](MemberStatus.Up))
|
||||||
Cluster(system2).state.members.size should ===(2)
|
Cluster(system2).state.members.size should ===(2)
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
"send messsages via cluster sharding, using envelopes" in {
|
"send messages via cluster sharding, using envelopes" in {
|
||||||
val ref = sharding.spawn(
|
val ref = sharding.spawn(
|
||||||
behavior,
|
behavior,
|
||||||
Props.empty,
|
Props.empty,
|
||||||
|
|
@ -207,7 +203,7 @@ class ClusterShardingSpec extends TypedSpec(ClusterShardingSpec.config) with Sca
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"send messsages via cluster sharding, without envelopes" in {
|
"send messages via cluster sharding, without envelopes" in {
|
||||||
val ref = sharding.spawn(
|
val ref = sharding.spawn(
|
||||||
behaviorWithId,
|
behaviorWithId,
|
||||||
Props.empty,
|
Props.empty,
|
||||||
|
|
@ -264,8 +260,6 @@ class ClusterShardingSpec extends TypedSpec(ClusterShardingSpec.config) with Sca
|
||||||
val bobRef = sharding.entityRefFor(typeKey, "bob")
|
val bobRef = sharding.entityRefFor(typeKey, "bob")
|
||||||
val charlieRef = sharding.entityRefFor(typeKey, "charlie")
|
val charlieRef = sharding.entityRefFor(typeKey, "charlie")
|
||||||
|
|
||||||
val p = TestProbe[String]()
|
|
||||||
|
|
||||||
val reply1 = bobRef ? WhoAreYou // TODO document that WhoAreYou(_) would not work
|
val reply1 = bobRef ? WhoAreYou // TODO document that WhoAreYou(_) would not work
|
||||||
reply1.futureValue should ===("I'm bob")
|
reply1.futureValue should ===("I'm bob")
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -3,14 +3,13 @@
|
||||||
*/
|
*/
|
||||||
package akka.cluster.sharding.typed
|
package akka.cluster.sharding.typed
|
||||||
|
|
||||||
|
import akka.actor.typed.TypedAkkaSpecWithShutdown
|
||||||
|
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
||||||
import akka.cluster.sharding.typed.internal.ShardingSerializer
|
import akka.cluster.sharding.typed.internal.ShardingSerializer
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import akka.actor.typed.TypedSpec
|
import akka.testkit.typed.TestKit
|
||||||
import akka.cluster.sharding.typed.internal.ShardingSerializer
|
|
||||||
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
|
||||||
import akka.actor.typed.scaladsl.AskPattern._
|
|
||||||
|
|
||||||
class ShardingSerializerSpec extends TypedSpec {
|
class ShardingSerializerSpec extends TestKit with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
"The typed ShardingSerializer" must {
|
"The typed ShardingSerializer" must {
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -12,13 +12,12 @@ class ConstantRateEntityRecoveryStrategySpec extends AkkaSpec {
|
||||||
import system.dispatcher
|
import system.dispatcher
|
||||||
|
|
||||||
val strategy = EntityRecoveryStrategy.constantStrategy(system, 1.second, 2)
|
val strategy = EntityRecoveryStrategy.constantStrategy(system, 1.second, 2)
|
||||||
|
|
||||||
"ConstantRateEntityRecoveryStrategy" must {
|
"ConstantRateEntityRecoveryStrategy" must {
|
||||||
"recover entities" taggedAs (TimingTest) in {
|
"recover entities" taggedAs TimingTest in {
|
||||||
val entities = Set[EntityId]("1", "2", "3", "4", "5")
|
val entities = Set[EntityId]("1", "2", "3", "4", "5")
|
||||||
val startTime = System.nanoTime()
|
val startTime = System.nanoTime()
|
||||||
val resultWithTimes = strategy.recoverEntities(entities).map(
|
val resultWithTimes = strategy.recoverEntities(entities).map(
|
||||||
_.map(entityIds ⇒ (entityIds → (System.nanoTime() - startTime).nanos)))
|
_.map(entityIds ⇒ entityIds → (System.nanoTime() - startTime).nanos))
|
||||||
|
|
||||||
val result = Await.result(Future.sequence(resultWithTimes), 6.seconds)
|
val result = Await.result(Future.sequence(resultWithTimes), 6.seconds)
|
||||||
.toVector.sortBy { case (_, duration) ⇒ duration }
|
.toVector.sortBy { case (_, duration) ⇒ duration }
|
||||||
|
|
|
||||||
|
|
@ -31,7 +31,7 @@ import akka.actor.typed.javadsl.ActorContext;
|
||||||
|
|
||||||
public class ReplicatorTest extends JUnitSuite {
|
public class ReplicatorTest extends JUnitSuite {
|
||||||
|
|
||||||
static interface ClientCommand {
|
interface ClientCommand {
|
||||||
}
|
}
|
||||||
|
|
||||||
static final class Increment implements ClientCommand {
|
static final class Increment implements ClientCommand {
|
||||||
|
|
@ -53,13 +53,13 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
static interface InternalMsg extends ClientCommand {
|
interface InternalMsg extends ClientCommand {
|
||||||
}
|
}
|
||||||
|
|
||||||
static final class InternalUpdateResponse<A extends ReplicatedData> implements InternalMsg {
|
static final class InternalUpdateResponse<A extends ReplicatedData> implements InternalMsg {
|
||||||
final Replicator.UpdateResponse<A> rsp;
|
final Replicator.UpdateResponse<A> rsp;
|
||||||
|
|
||||||
public InternalUpdateResponse(Replicator.UpdateResponse<A> rsp) {
|
InternalUpdateResponse(Replicator.UpdateResponse<A> rsp) {
|
||||||
this.rsp = rsp;
|
this.rsp = rsp;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -67,7 +67,7 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
static final class InternalGetResponse<A extends ReplicatedData> implements InternalMsg {
|
static final class InternalGetResponse<A extends ReplicatedData> implements InternalMsg {
|
||||||
final Replicator.GetResponse<A> rsp;
|
final Replicator.GetResponse<A> rsp;
|
||||||
|
|
||||||
public InternalGetResponse(Replicator.GetResponse<A> rsp) {
|
InternalGetResponse(Replicator.GetResponse<A> rsp) {
|
||||||
this.rsp = rsp;
|
this.rsp = rsp;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -75,7 +75,7 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
static final class InternalChanged<A extends ReplicatedData> implements InternalMsg {
|
static final class InternalChanged<A extends ReplicatedData> implements InternalMsg {
|
||||||
final Replicator.Changed<A> chg;
|
final Replicator.Changed<A> chg;
|
||||||
|
|
||||||
public InternalChanged(Replicator.Changed<A> chg) {
|
InternalChanged(Replicator.Changed<A> chg) {
|
||||||
this.chg = chg;
|
this.chg = chg;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
@ -91,15 +91,15 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
|
|
||||||
private int cachedValue = 0;
|
private int cachedValue = 0;
|
||||||
|
|
||||||
public Client(ActorRef<Command> replicator, Cluster node, ActorContext<ClientCommand> ctx) {
|
Client(ActorRef<Command> replicator, Cluster node, ActorContext<ClientCommand> ctx) {
|
||||||
this.replicator = replicator;
|
this.replicator = replicator;
|
||||||
this.node = node;
|
this.node = node;
|
||||||
|
|
||||||
updateResponseAdapter = ctx.spawnAdapter(m -> new InternalUpdateResponse<>(m));
|
updateResponseAdapter = ctx.spawnAdapter(InternalUpdateResponse::new);
|
||||||
|
|
||||||
getResponseAdapter = ctx.spawnAdapter(m -> new InternalGetResponse<>(m));
|
getResponseAdapter = ctx.spawnAdapter(InternalGetResponse::new);
|
||||||
|
|
||||||
changedAdapter = ctx.spawnAdapter(m -> new InternalChanged<>(m));
|
changedAdapter = ctx.spawnAdapter(InternalChanged::new);
|
||||||
|
|
||||||
replicator.tell(new Replicator.Subscribe<>(Key, changedAdapter));
|
replicator.tell(new Replicator.Subscribe<>(Key, changedAdapter));
|
||||||
}
|
}
|
||||||
|
|
@ -113,7 +113,7 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
return receiveBuilder()
|
return receiveBuilder()
|
||||||
.onMessage(Increment.class, cmd -> {
|
.onMessage(Increment.class, cmd -> {
|
||||||
replicator.tell(
|
replicator.tell(
|
||||||
new Replicator.Update<GCounter>(Key, GCounter.empty(), Replicator.writeLocal(), updateResponseAdapter,
|
new Replicator.Update<>(Key, GCounter.empty(), Replicator.writeLocal(), updateResponseAdapter,
|
||||||
curr -> curr.increment(node, 1)));
|
curr -> curr.increment(node, 1)));
|
||||||
return this;
|
return this;
|
||||||
})
|
})
|
||||||
|
|
@ -122,7 +122,7 @@ public class ReplicatorTest extends JUnitSuite {
|
||||||
})
|
})
|
||||||
.onMessage(GetValue.class, cmd -> {
|
.onMessage(GetValue.class, cmd -> {
|
||||||
replicator.tell(
|
replicator.tell(
|
||||||
new Replicator.Get<GCounter>(Key, Replicator.readLocal(), getResponseAdapter, Optional.of(cmd.replyTo)));
|
new Replicator.Get<>(Key, Replicator.readLocal(), getResponseAdapter, Optional.of(cmd.replyTo)));
|
||||||
return this;
|
return this;
|
||||||
})
|
})
|
||||||
.onMessage(GetCachedValue.class, cmd -> {
|
.onMessage(GetCachedValue.class, cmd -> {
|
||||||
|
|
|
||||||
|
|
@ -4,15 +4,15 @@
|
||||||
package akka.cluster.ddata.typed.scaladsl
|
package akka.cluster.ddata.typed.scaladsl
|
||||||
|
|
||||||
import akka.actor.Scheduler
|
import akka.actor.Scheduler
|
||||||
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, StartSupport, TypedSpec }
|
import akka.actor.typed.{ ActorRef, Behavior, TypedAkkaSpecWithShutdown }
|
||||||
import akka.actor.typed.scaladsl.Actor
|
|
||||||
import akka.actor.typed.scaladsl.AskPattern._
|
import akka.actor.typed.scaladsl.AskPattern._
|
||||||
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
import akka.testkit.typed.TestKitSettings
|
|
||||||
import akka.testkit.typed.scaladsl._
|
|
||||||
import akka.cluster.Cluster
|
import akka.cluster.Cluster
|
||||||
import akka.cluster.ddata.{ GCounter, GCounterKey, ReplicatedData }
|
|
||||||
import akka.cluster.ddata.typed.scaladsl.Replicator._
|
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.util.Timeout
|
import akka.util.Timeout
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.scalatest.concurrent.Eventually
|
import org.scalatest.concurrent.Eventually
|
||||||
|
|
@ -107,14 +107,14 @@ object ReplicatorSpec {
|
||||||
|
|
||||||
val reply4: Future[ReplicaCount] = replicator ? Replicator.GetReplicaCount()
|
val reply4: Future[ReplicaCount] = replicator ? Replicator.GetReplicaCount()
|
||||||
|
|
||||||
// supress unused compiler warnings
|
// suppress unused compiler warnings
|
||||||
println("" + reply1 + reply2 + reply3 + reply4)
|
println("" + reply1 + reply2 + reply3 + reply4)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class ReplicatorSpec extends TypedSpec(ReplicatorSpec.config) with Eventually with StartSupport {
|
class ReplicatorSpec extends TestKit(ReplicatorSpec.config) with TypedAkkaSpecWithShutdown with Eventually {
|
||||||
|
|
||||||
import ReplicatorSpec._
|
import ReplicatorSpec._
|
||||||
|
|
||||||
|
|
@ -125,8 +125,8 @@ class ReplicatorSpec extends TypedSpec(ReplicatorSpec.config) with Eventually wi
|
||||||
"Replicator" must {
|
"Replicator" must {
|
||||||
|
|
||||||
"have API for Update and Get" in {
|
"have API for Update and Get" in {
|
||||||
val replicator = start(Replicator.behavior(settings))
|
val replicator = spawn(Replicator.behavior(settings))
|
||||||
val c = start(client(replicator))
|
val c = spawn(client(replicator))
|
||||||
|
|
||||||
val probe = TestProbe[Int]
|
val probe = TestProbe[Int]
|
||||||
c ! Increment
|
c ! Increment
|
||||||
|
|
@ -135,8 +135,8 @@ class ReplicatorSpec extends TypedSpec(ReplicatorSpec.config) with Eventually wi
|
||||||
}
|
}
|
||||||
|
|
||||||
"have API for Subscribe" in {
|
"have API for Subscribe" in {
|
||||||
val replicator = start(Replicator.behavior(settings))
|
val replicator = spawn(Replicator.behavior(settings))
|
||||||
val c = start(client(replicator))
|
val c = spawn(client(replicator))
|
||||||
|
|
||||||
val probe = TestProbe[Int]
|
val probe = TestProbe[Int]
|
||||||
c ! Increment
|
c ! Increment
|
||||||
|
|
@ -154,7 +154,7 @@ class ReplicatorSpec extends TypedSpec(ReplicatorSpec.config) with Eventually wi
|
||||||
|
|
||||||
"have an extension" in {
|
"have an extension" in {
|
||||||
val replicator = DistributedData(system).replicator
|
val replicator = DistributedData(system).replicator
|
||||||
val c = start(client(replicator))
|
val c = spawn(client(replicator))
|
||||||
|
|
||||||
val probe = TestProbe[Int]
|
val probe = TestProbe[Int]
|
||||||
c ! Increment
|
c ! Increment
|
||||||
|
|
|
||||||
|
|
@ -3,13 +3,12 @@
|
||||||
*/
|
*/
|
||||||
package akka.cluster.typed
|
package akka.cluster.typed
|
||||||
|
|
||||||
|
import akka.actor.typed.TypedAkkaSpecWithShutdown
|
||||||
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
import akka.cluster.ClusterEvent._
|
import akka.cluster.ClusterEvent._
|
||||||
import akka.cluster.MemberStatus
|
import akka.cluster.MemberStatus
|
||||||
import akka.actor.typed.TypedSpec
|
|
||||||
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
|
||||||
import akka.testkit.typed.TestKitSettings
|
|
||||||
import akka.testkit.typed.scaladsl.TestProbe
|
import akka.testkit.typed.scaladsl.TestProbe
|
||||||
|
import akka.testkit.typed.{ TestKit, TestKitSettings }
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.scalatest.concurrent.ScalaFutures
|
import org.scalatest.concurrent.ScalaFutures
|
||||||
|
|
||||||
|
|
@ -33,7 +32,7 @@ object ClusterApiSpec {
|
||||||
""")
|
""")
|
||||||
}
|
}
|
||||||
|
|
||||||
class ClusterApiSpec extends TypedSpec(ClusterApiSpec.config) with ScalaFutures {
|
class ClusterApiSpec extends TestKit("ClusterApiSpec", ClusterApiSpec.config) with TypedAkkaSpecWithShutdown with ScalaFutures {
|
||||||
|
|
||||||
val testSettings = TestKitSettings(system)
|
val testSettings = TestKitSettings(system)
|
||||||
val clusterNode1 = Cluster(system)
|
val clusterNode1 = Cluster(system)
|
||||||
|
|
|
||||||
|
|
@ -6,14 +6,13 @@ package akka.cluster.typed
|
||||||
import java.nio.charset.StandardCharsets
|
import java.nio.charset.StandardCharsets
|
||||||
|
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.serialization.SerializerWithStringManifest
|
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
import akka.testkit.typed.TestKitSettings
|
import akka.testkit.typed.{ TestKit, TestKitSettings }
|
||||||
import akka.testkit.typed.scaladsl.TestProbe
|
import akka.testkit.typed.scaladsl.TestProbe
|
||||||
import akka.actor.typed.{ ActorRef, ActorRefResolver, Props, TypedSpec }
|
import akka.actor.typed.{ ActorRef, ActorRefResolver, Props, TypedAkkaSpecWithShutdown }
|
||||||
|
import akka.serialization.SerializerWithStringManifest
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.scalatest.concurrent.ScalaFutures
|
|
||||||
|
|
||||||
import scala.concurrent.Await
|
import scala.concurrent.Await
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
@ -49,7 +48,7 @@ object ClusterSingletonApiSpec {
|
||||||
|
|
||||||
case object Perish extends PingProtocol
|
case object Perish extends PingProtocol
|
||||||
|
|
||||||
val pingPong = Actor.immutable[PingProtocol] { (ctx, msg) ⇒
|
val pingPong = Actor.immutable[PingProtocol] { (_, msg) ⇒
|
||||||
|
|
||||||
msg match {
|
msg match {
|
||||||
case Ping(respondTo) ⇒
|
case Ping(respondTo) ⇒
|
||||||
|
|
@ -84,7 +83,7 @@ object ClusterSingletonApiSpec {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class ClusterSingletonApiSpec extends TypedSpec(ClusterSingletonApiSpec.config) with ScalaFutures {
|
class ClusterSingletonApiSpec extends TestKit("ClusterSingletonApiSpec", ClusterSingletonApiSpec.config) with TypedAkkaSpecWithShutdown {
|
||||||
import ClusterSingletonApiSpec._
|
import ClusterSingletonApiSpec._
|
||||||
|
|
||||||
implicit val testSettings = TestKitSettings(system)
|
implicit val testSettings = TestKitSettings(system)
|
||||||
|
|
|
||||||
|
|
@ -4,16 +4,12 @@
|
||||||
|
|
||||||
package akka.cluster.typed
|
package akka.cluster.typed
|
||||||
|
|
||||||
import akka.actor.typed.ActorRef
|
import akka.actor.typed.{ ActorRef, Behavior, Props, TypedAkkaSpecWithShutdown }
|
||||||
import akka.actor.typed.Behavior
|
|
||||||
import akka.actor.typed.Props
|
|
||||||
import akka.actor.typed.TypedSpec
|
|
||||||
import akka.persistence.typed.scaladsl.PersistentActor
|
import akka.persistence.typed.scaladsl.PersistentActor
|
||||||
import akka.persistence.typed.scaladsl.PersistentActor.{ CommandHandler, Effect }
|
import akka.persistence.typed.scaladsl.PersistentActor.{ CommandHandler, Effect }
|
||||||
import akka.testkit.typed.TestKitSettings
|
import akka.testkit.typed.TestKit
|
||||||
import akka.testkit.typed.scaladsl.TestProbe
|
import akka.testkit.typed.scaladsl.TestProbe
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.scalatest.concurrent.ScalaFutures
|
|
||||||
|
|
||||||
object ClusterSingletonPersistenceSpec {
|
object ClusterSingletonPersistenceSpec {
|
||||||
val config = ConfigFactory.parseString(
|
val config = ConfigFactory.parseString(
|
||||||
|
|
@ -44,7 +40,7 @@ object ClusterSingletonPersistenceSpec {
|
||||||
PersistentActor.immutable[Command, String, String](
|
PersistentActor.immutable[Command, String, String](
|
||||||
persistenceId = "TheSingleton",
|
persistenceId = "TheSingleton",
|
||||||
initialState = "",
|
initialState = "",
|
||||||
commandHandler = CommandHandler((ctx, state, cmd) ⇒ cmd match {
|
commandHandler = CommandHandler((_, state, cmd) ⇒ cmd match {
|
||||||
case Add(s) ⇒ Effect.persist(s)
|
case Add(s) ⇒ Effect.persist(s)
|
||||||
case Get(replyTo) ⇒
|
case Get(replyTo) ⇒
|
||||||
replyTo ! state
|
replyTo ! state
|
||||||
|
|
@ -55,12 +51,11 @@ object ClusterSingletonPersistenceSpec {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class ClusterSingletonPersistenceSpec extends TypedSpec(ClusterSingletonPersistenceSpec.config) with ScalaFutures {
|
class ClusterSingletonPersistenceSpec extends TestKit(ClusterSingletonPersistenceSpec.config) with TypedAkkaSpecWithShutdown {
|
||||||
import ClusterSingletonPersistenceSpec._
|
import ClusterSingletonPersistenceSpec._
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
|
|
||||||
implicit val s = system
|
implicit val s = system
|
||||||
implicit val testkitSettings = TestKitSettings(system)
|
|
||||||
|
|
||||||
implicit val untypedSystem = system.toUntyped
|
implicit val untypedSystem = system.toUntyped
|
||||||
private val untypedCluster = akka.cluster.Cluster(untypedSystem)
|
private val untypedCluster = akka.cluster.Cluster(untypedSystem)
|
||||||
|
|
|
||||||
|
|
@ -6,14 +6,14 @@ package akka.cluster.typed.internal.receptionist
|
||||||
import java.nio.charset.StandardCharsets
|
import java.nio.charset.StandardCharsets
|
||||||
|
|
||||||
import akka.actor.ExtendedActorSystem
|
import akka.actor.ExtendedActorSystem
|
||||||
import akka.cluster.Cluster
|
import akka.actor.typed.{ ActorRef, ActorRefResolver, TypedAkkaSpecWithShutdown }
|
||||||
import akka.serialization.SerializerWithStringManifest
|
|
||||||
import akka.actor.typed.{ ActorRef, ActorRefResolver, StartSupport, TypedSpec }
|
|
||||||
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
import akka.actor.typed.internal.adapter.ActorSystemAdapter
|
||||||
import akka.actor.typed.receptionist.Receptionist
|
import akka.actor.typed.receptionist.Receptionist
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
import akka.testkit.typed.TestKitSettings
|
import akka.cluster.Cluster
|
||||||
|
import akka.serialization.SerializerWithStringManifest
|
||||||
|
import akka.testkit.typed.{ TestKit, TestKitSettings }
|
||||||
import akka.testkit.typed.scaladsl.TestProbe
|
import akka.testkit.typed.scaladsl.TestProbe
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
||||||
|
|
@ -51,7 +51,7 @@ object ClusterReceptionistSpec {
|
||||||
|
|
||||||
case object Perish extends PingProtocol
|
case object Perish extends PingProtocol
|
||||||
|
|
||||||
val pingPong = Actor.immutable[PingProtocol] { (ctx, msg) ⇒
|
val pingPong = Actor.immutable[PingProtocol] { (_, msg) ⇒
|
||||||
|
|
||||||
msg match {
|
msg match {
|
||||||
case Ping(respondTo) ⇒
|
case Ping(respondTo) ⇒
|
||||||
|
|
@ -61,7 +61,6 @@ object ClusterReceptionistSpec {
|
||||||
case Perish ⇒
|
case Perish ⇒
|
||||||
Actor.stopped
|
Actor.stopped
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class PingSerializer(system: ExtendedActorSystem) extends SerializerWithStringManifest {
|
class PingSerializer(system: ExtendedActorSystem) extends SerializerWithStringManifest {
|
||||||
|
|
@ -88,7 +87,8 @@ object ClusterReceptionistSpec {
|
||||||
val PingKey = Receptionist.ServiceKey[PingProtocol]("pingy")
|
val PingKey = Receptionist.ServiceKey[PingProtocol]("pingy")
|
||||||
}
|
}
|
||||||
|
|
||||||
class ClusterReceptionistSpec extends TypedSpec(ClusterReceptionistSpec.config) with StartSupport {
|
class ClusterReceptionistSpec extends TestKit("ClusterReceptionistSpec", ClusterReceptionistSpec.config)
|
||||||
|
with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
import ClusterReceptionistSpec._
|
import ClusterReceptionistSpec._
|
||||||
|
|
||||||
|
|
@ -117,7 +117,7 @@ class ClusterReceptionistSpec extends TypedSpec(ClusterReceptionistSpec.config)
|
||||||
adaptedSystem2.receptionist ! Subscribe(PingKey, regProbe2.ref)
|
adaptedSystem2.receptionist ! Subscribe(PingKey, regProbe2.ref)
|
||||||
regProbe2.expectMsg(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
regProbe2.expectMsg(Listing(PingKey, Set.empty[ActorRef[PingProtocol]]))
|
||||||
|
|
||||||
val service = start(pingPong)
|
val service = spawn(pingPong)
|
||||||
system.receptionist ! Register(PingKey, service, regProbe.ref)
|
system.receptionist ! Register(PingKey, service, regProbe.ref)
|
||||||
regProbe.expectMsg(Registered(PingKey, service))
|
regProbe.expectMsg(Registered(PingKey, service))
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -15,13 +15,11 @@ This module is currently marked as @ref:[may change](common/may-change.md) in th
|
||||||
* `Inbox` has been renamed to `TestInbox` to allign with `TestProbe`
|
* `Inbox` has been renamed to `TestInbox` to allign with `TestProbe`
|
||||||
* Separated into modules e.g. `akka-actor-typed` `akka-persistence-typed` along with matching package names
|
* Separated into modules e.g. `akka-actor-typed` `akka-persistence-typed` along with matching package names
|
||||||
|
|
||||||
### Dependency
|
To use Akka Typed add the following dependency:
|
||||||
|
|
||||||
To use typed actors add the following dependency:
|
|
||||||
|
|
||||||
@@dependency [sbt,Maven,Gradle] {
|
@@dependency [sbt,Maven,Gradle] {
|
||||||
group=com.typesafe.akka
|
group=com.typesafe.akka
|
||||||
artifact=akka-actor-typed_2.11
|
artifact=akka-actor-typed_2.12
|
||||||
version=$version$
|
version=$version$
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -1,11 +1,9 @@
|
||||||
# Sharding
|
# Sharding
|
||||||
|
|
||||||
TODO
|
|
||||||
|
|
||||||
### Dependency
|
|
||||||
|
|
||||||
@@dependency [sbt,Maven,Gradle] {
|
@@dependency [sbt,Maven,Gradle] {
|
||||||
group=com.typesafe.akka
|
group=com.typesafe.akka
|
||||||
artifact=akka-cluster-sharding-typed_2.11
|
artifact=akka-cluster-sharding-typed_2.12
|
||||||
version=$version$
|
version=$version$
|
||||||
}
|
}
|
||||||
|
|
||||||
|
TODO
|
||||||
|
|
|
||||||
20
akka-docs/src/main/paradox/fault-tolerance-typed.md
Normal file
20
akka-docs/src/main/paradox/fault-tolerance-typed.md
Normal file
|
|
@ -0,0 +1,20 @@
|
||||||
|
# Fault Tolerance
|
||||||
|
|
||||||
|
As explained in @ref:[Actor Systems](general/actor-systems.md) each actor is the supervisor of its
|
||||||
|
children, and as such each actor defines fault handling supervisor strategy.
|
||||||
|
This strategy cannot be changed afterwards as it is an integral part of the
|
||||||
|
actor system’s structure.
|
||||||
|
|
||||||
|
## Creating a Supervisor Strategy
|
||||||
|
|
||||||
|
TODO
|
||||||
|
|
||||||
|
### Default Supervisor Strategy
|
||||||
|
|
||||||
|
### Restart Supervisor Strategy
|
||||||
|
|
||||||
|
### Stopping Supervisor Strategy
|
||||||
|
|
||||||
|
### Logging of Actor Failures
|
||||||
|
|
||||||
|
|
||||||
|
|
@ -5,6 +5,7 @@
|
||||||
@@@ index
|
@@@ index
|
||||||
|
|
||||||
* [actors](actors-typed.md)
|
* [actors](actors-typed.md)
|
||||||
|
* [fault-tolerance-typed.md](fault-tolerance-typed.md)
|
||||||
* [coexisting](coexisting.md)
|
* [coexisting](coexisting.md)
|
||||||
* [cluster](cluster-typed.md)
|
* [cluster](cluster-typed.md)
|
||||||
* [cluster-sharding](cluster-sharding-typed.md)
|
* [cluster-sharding](cluster-sharding-typed.md)
|
||||||
|
|
|
||||||
|
|
@ -10,12 +10,11 @@ This module is currently marked as @ref:[may change](common/may-change.md) in th
|
||||||
|
|
||||||
@@@
|
@@@
|
||||||
|
|
||||||
To use the testkit add the following dependency:
|
|
||||||
To use the testkit add the following dependency:
|
To use the testkit add the following dependency:
|
||||||
|
|
||||||
@@dependency [sbt,Maven,Gradle] {
|
@@dependency [sbt,Maven,Gradle] {
|
||||||
group=com.typesafe.akka
|
group=com.typesafe.akka
|
||||||
artifact=akka-testkit-typed_2.11
|
artifact=akka-testkit-typed_2.12
|
||||||
version=$version$
|
version=$version$
|
||||||
scope=test
|
scope=test
|
||||||
}
|
}
|
||||||
|
|
@ -63,7 +62,7 @@ Scala
|
||||||
Java
|
Java
|
||||||
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #child }
|
: @@snip [BasicSyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/sync/BasicSyncTestingTest.java) { #child }
|
||||||
|
|
||||||
All of the tests make use of the `BehaviourTestkit` to avoid the need for a real `ActorContext`. Some of the tests
|
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
|
make use of the `TestInbox` which allows the creation of an `ActorRef` that can be used for synchronous testing, similar to the
|
||||||
`TestProbe` used for asynchronous testing.
|
`TestProbe` used for asynchronous testing.
|
||||||
|
|
||||||
|
|
@ -124,11 +123,11 @@ The `BehaviorTestkit` keeps track other effects you can verify, look at the sub-
|
||||||
* Unwatched
|
* Unwatched
|
||||||
* Scheduled
|
* Scheduled
|
||||||
|
|
||||||
See the other public methods and API documentation on `BehaviourTestkit` for other types of verification.
|
See the other public methods and API documentation on `BehaviorTestkit` for other types of verification.
|
||||||
|
|
||||||
## Asynchronous testing
|
## Asynchronous testing
|
||||||
|
|
||||||
Asynchronous testing uses a real `ActorSystem` that allows you to test your Actors in a realistic environment.
|
Asynchronous testing uses a real `ActorSystem` that allows you to test your Actors in a more realistic environment.
|
||||||
|
|
||||||
The minimal setup consists of the test procedure, which provides the desired stimuli, the actor under test,
|
The minimal setup consists of the test procedure, which provides the desired stimuli, the actor under test,
|
||||||
and an actor receiving replies. Bigger systems replace the actor under test with a network of actors, apply stimuli
|
and an actor receiving replies. Bigger systems replace the actor under test with a network of actors, apply stimuli
|
||||||
|
|
@ -145,7 +144,10 @@ Scala
|
||||||
Java
|
Java
|
||||||
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.java) { #under-test }
|
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.java) { #under-test }
|
||||||
|
|
||||||
Tests can optionally extend `TestKit` or include the `TestKitBase`.
|
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
|
||||||
|
|
||||||
Scala
|
Scala
|
||||||
: @@snip [BasicTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/async/BasicAsyncTestingSpec.scala) { #test-header }
|
: @@snip [BasicTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/async/BasicAsyncTestingSpec.scala) { #test-header }
|
||||||
|
|
@ -163,13 +165,21 @@ Java
|
||||||
|
|
||||||
The following demonstrates:
|
The following demonstrates:
|
||||||
|
|
||||||
* Creating a typed actor from the `TestKit`'s system using `actorOf`
|
* Creating a typed actor from the `TestKit`'s system using `spawn`
|
||||||
* Creating a typed `TestProbe`
|
* Creating a typed `TestProbe`
|
||||||
* Verifying that the actor under test responds via the `TestProbe`
|
* Verifying that the actor under test responds via the `TestProbe`
|
||||||
|
|
||||||
Scala
|
Scala
|
||||||
: @@snip [BasicTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/async/BasicAsyncTestingSpec.scala) { #test }
|
: @@snip [BasicTestingSpec.scala]($akka$/akka-actor-typed-tests/src/test/scala/docs/akka/typed/testing/async/BasicAsyncTestingSpec.scala) { #test-spawn }
|
||||||
|
|
||||||
Java
|
Java
|
||||||
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.java) { #test }
|
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.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 }
|
||||||
|
|
||||||
|
Java
|
||||||
|
: @@snip [BasicAsyncTestingTest.java]($akka$/akka-actor-typed-tests/src/test/java/jdocs/akka/typed/testing/async/BasicAsyncTestingTest.java) { #test-spawn-anonymous }
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -4,9 +4,10 @@
|
||||||
package akka.persistence.typed.scaladsl
|
package akka.persistence.typed.scaladsl
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, StartSupport, SupervisorStrategy, Terminated, TypedSpec }
|
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior, SupervisorStrategy, Terminated, TypedAkkaSpecWithShutdown }
|
||||||
import akka.actor.typed.scaladsl.Actor
|
import akka.actor.typed.scaladsl.Actor
|
||||||
import akka.testkit.typed.TestKitSettings
|
import akka.testkit.typed.TestKitSettings
|
||||||
|
import akka.testkit.typed.TestKit
|
||||||
import akka.testkit.typed.scaladsl._
|
import akka.testkit.typed.scaladsl._
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
import org.scalatest.concurrent.Eventually
|
import org.scalatest.concurrent.Eventually
|
||||||
|
|
@ -39,7 +40,7 @@ object PersistentActorSpec {
|
||||||
val firstLogging = "first logging"
|
val firstLogging = "first logging"
|
||||||
val secondLogging = "second logging"
|
val secondLogging = "second logging"
|
||||||
|
|
||||||
def counter(persistenceId: String)(implicit actorSystem: ActorSystem[TypedSpec.Command], testSettings: TestKitSettings): Behavior[Command] =
|
def counter(persistenceId: String)(implicit actorSystem: ActorSystem[_], testSettings: TestKitSettings): Behavior[Command] =
|
||||||
counter(persistenceId, TestProbe[String].ref)
|
counter(persistenceId, TestProbe[String].ref)
|
||||||
|
|
||||||
def counter(persistenceId: String, loggingActor: ActorRef[String]): Behavior[Command] = {
|
def counter(persistenceId: String, loggingActor: ActorRef[String]): Behavior[Command] = {
|
||||||
|
|
@ -106,8 +107,7 @@ object PersistentActorSpec {
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eventually with StartSupport {
|
class PersistentActorSpec extends TestKit(PersistentActorSpec.config) with Eventually with TypedAkkaSpecWithShutdown {
|
||||||
|
|
||||||
import PersistentActorSpec._
|
import PersistentActorSpec._
|
||||||
|
|
||||||
implicit val testSettings = TestKitSettings(system)
|
implicit val testSettings = TestKitSettings(system)
|
||||||
|
|
@ -115,7 +115,7 @@ class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eve
|
||||||
"A typed persistent actor" must {
|
"A typed persistent actor" must {
|
||||||
|
|
||||||
"persist an event" in {
|
"persist an event" in {
|
||||||
val c = start(counter("c1"))
|
val c = spawn(counter("c1"))
|
||||||
|
|
||||||
val probe = TestProbe[State]
|
val probe = TestProbe[State]
|
||||||
c ! Increment
|
c ! Increment
|
||||||
|
|
@ -124,7 +124,7 @@ class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eve
|
||||||
}
|
}
|
||||||
|
|
||||||
"replay stored events" in {
|
"replay stored events" in {
|
||||||
val c = start(counter("c2"))
|
val c = spawn(counter("c2"))
|
||||||
|
|
||||||
val probe = TestProbe[State]
|
val probe = TestProbe[State]
|
||||||
c ! Increment
|
c ! Increment
|
||||||
|
|
@ -133,7 +133,7 @@ class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eve
|
||||||
c ! GetValue(probe.ref)
|
c ! GetValue(probe.ref)
|
||||||
probe.expectMsg(State(3, Vector(0, 1, 2)))
|
probe.expectMsg(State(3, Vector(0, 1, 2)))
|
||||||
|
|
||||||
val c2 = start(counter("c2"))
|
val c2 = spawn(counter("c2"))
|
||||||
c2 ! GetValue(probe.ref)
|
c2 ! GetValue(probe.ref)
|
||||||
probe.expectMsg(State(3, Vector(0, 1, 2)))
|
probe.expectMsg(State(3, Vector(0, 1, 2)))
|
||||||
c2 ! Increment
|
c2 ! Increment
|
||||||
|
|
@ -142,7 +142,7 @@ class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eve
|
||||||
}
|
}
|
||||||
|
|
||||||
"handle Terminated signal" in {
|
"handle Terminated signal" in {
|
||||||
val c = start(counter("c3"))
|
val c = spawn(counter("c3"))
|
||||||
|
|
||||||
val probe = TestProbe[State]
|
val probe = TestProbe[State]
|
||||||
c ! Increment
|
c ! Increment
|
||||||
|
|
@ -154,7 +154,7 @@ class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eve
|
||||||
}
|
}
|
||||||
|
|
||||||
"handle receive timeout" in {
|
"handle receive timeout" in {
|
||||||
val c = start(counter("c4"))
|
val c = spawn(counter("c4"))
|
||||||
|
|
||||||
val probe = TestProbe[State]
|
val probe = TestProbe[State]
|
||||||
c ! Increment
|
c ! Increment
|
||||||
|
|
@ -173,7 +173,7 @@ class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eve
|
||||||
*/
|
*/
|
||||||
"chainable side effects with events" in {
|
"chainable side effects with events" in {
|
||||||
val loggingProbe = TestProbe[String]
|
val loggingProbe = TestProbe[String]
|
||||||
val c = start(counter("c5", loggingProbe.ref))
|
val c = spawn(counter("c5", loggingProbe.ref))
|
||||||
|
|
||||||
val probe = TestProbe[State]
|
val probe = TestProbe[State]
|
||||||
|
|
||||||
|
|
@ -188,7 +188,7 @@ class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eve
|
||||||
/** Proves that side-effects are called when emitting an empty list of events */
|
/** Proves that side-effects are called when emitting an empty list of events */
|
||||||
"chainable side effects without events" in {
|
"chainable side effects without events" in {
|
||||||
val loggingProbe = TestProbe[String]
|
val loggingProbe = TestProbe[String]
|
||||||
val c = start(counter("c6", loggingProbe.ref))
|
val c = spawn(counter("c6", loggingProbe.ref))
|
||||||
|
|
||||||
val probe = TestProbe[State]
|
val probe = TestProbe[State]
|
||||||
c ! EmptyEventsListAndThenLog
|
c ! EmptyEventsListAndThenLog
|
||||||
|
|
@ -200,7 +200,7 @@ class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eve
|
||||||
/** Proves that side-effects are called when explicitly calling Effect.none */
|
/** Proves that side-effects are called when explicitly calling Effect.none */
|
||||||
"chainable side effects when doing nothing (Effect.none)" in {
|
"chainable side effects when doing nothing (Effect.none)" in {
|
||||||
val loggingProbe = TestProbe[String]
|
val loggingProbe = TestProbe[String]
|
||||||
val c = start(counter("c7", loggingProbe.ref))
|
val c = spawn(counter("c7", loggingProbe.ref))
|
||||||
|
|
||||||
val probe = TestProbe[State]
|
val probe = TestProbe[State]
|
||||||
c ! DoNothingAndThenLog
|
c ! DoNothingAndThenLog
|
||||||
|
|
@ -217,7 +217,7 @@ class PersistentActorSpec extends TypedSpec(PersistentActorSpec.config) with Eve
|
||||||
val probe = TestProbe[State]
|
val probe = TestProbe[State]
|
||||||
val behavior = Actor.supervise[Command](counter("c13"))
|
val behavior = Actor.supervise[Command](counter("c13"))
|
||||||
.onFailure(SupervisorStrategy.restartWithBackoff(1.second, 10.seconds, 0.1))
|
.onFailure(SupervisorStrategy.restartWithBackoff(1.second, 10.seconds, 0.1))
|
||||||
val c = start(behavior)
|
val c = spawn(behavior)
|
||||||
c ! Increment
|
c ! Increment
|
||||||
c ! GetValue(probe.ref)
|
c ! GetValue(probe.ref)
|
||||||
probe.expectMsg(State(1, Vector(0)))
|
probe.expectMsg(State(1, Vector(0)))
|
||||||
|
|
|
||||||
|
|
@ -5,7 +5,7 @@
|
||||||
package akka.remote.serialization
|
package akka.remote.serialization
|
||||||
|
|
||||||
import akka.actor._
|
import akka.actor._
|
||||||
import akka.remote.{ MessageSerializer, RemoteScope, RemoteWatcher }
|
import akka.remote.{ RemoteScope, RemoteWatcher }
|
||||||
import akka.serialization.SerializationExtension
|
import akka.serialization.SerializationExtension
|
||||||
import akka.testkit.AkkaSpec
|
import akka.testkit.AkkaSpec
|
||||||
import com.typesafe.config.ConfigFactory
|
import com.typesafe.config.ConfigFactory
|
||||||
|
|
|
||||||
|
|
@ -114,7 +114,7 @@ object BehaviorTestkit {
|
||||||
* JAVA API
|
* JAVA API
|
||||||
*/
|
*/
|
||||||
def create[T](initialBehavior: Behavior[T]): BehaviorTestkit[T] =
|
def create[T](initialBehavior: Behavior[T]): BehaviorTestkit[T] =
|
||||||
apply(initialBehavior, "ctx")
|
apply(initialBehavior, "testkit")
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
@ -122,7 +122,7 @@ object BehaviorTestkit {
|
||||||
* watching and offers access to what effects have taken place.
|
* watching and offers access to what effects have taken place.
|
||||||
*/
|
*/
|
||||||
@ApiMayChange
|
@ApiMayChange
|
||||||
class BehaviorTestkit[T](_name: String, _initialBehavior: Behavior[T]) {
|
class BehaviorTestkit[T] private (_name: String, _initialBehavior: Behavior[T]) {
|
||||||
|
|
||||||
import Effect._
|
import Effect._
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -65,6 +65,7 @@ private[typed] abstract class WatchableRef[-T](override val path: a.ActorPath) e
|
||||||
protected def terminate(): Unit
|
protected def terminate(): Unit
|
||||||
|
|
||||||
type S = Set[ActorRefImpl[Nothing]]
|
type S = Set[ActorRefImpl[Nothing]]
|
||||||
|
|
||||||
@volatile private[this] var _watchedBy: S = Set.empty
|
@volatile private[this] var _watchedBy: S = Set.empty
|
||||||
|
|
||||||
protected def isAlive: Boolean = _watchedBy != null
|
protected def isAlive: Boolean = _watchedBy != null
|
||||||
|
|
|
||||||
|
|
@ -1,51 +1,104 @@
|
||||||
package akka.testkit.typed
|
package akka.testkit.typed
|
||||||
|
|
||||||
|
import akka.actor.typed.scaladsl.Actor
|
||||||
|
import akka.actor.typed.scaladsl.AskPattern._
|
||||||
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior }
|
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior }
|
||||||
import akka.annotation.ApiMayChange
|
import akka.annotation.ApiMayChange
|
||||||
|
import akka.testkit.typed.TestKit._
|
||||||
import akka.util.Timeout
|
import akka.util.Timeout
|
||||||
|
import com.typesafe.config.Config
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
import scala.concurrent.{ Await, TimeoutException }
|
import scala.concurrent.{ Await, TimeoutException }
|
||||||
|
|
||||||
|
object TestKit {
|
||||||
|
|
||||||
|
private[akka] sealed trait TestKitCommand
|
||||||
|
private[akka] case class SpawnActor[T](name: String, behavior: Behavior[T], replyTo: ActorRef[ActorRef[T]]) extends TestKitCommand
|
||||||
|
private[akka] case class SpawnActorAnonymous[T](behavior: Behavior[T], replyTo: ActorRef[ActorRef[T]]) extends TestKitCommand
|
||||||
|
|
||||||
|
private val testKitGuardian = Actor.immutable[TestKitCommand] {
|
||||||
|
case (ctx, SpawnActor(name, behavior, reply)) ⇒
|
||||||
|
reply ! ctx.spawn(behavior, name)
|
||||||
|
Actor.same
|
||||||
|
case (ctx, SpawnActorAnonymous(behavior, reply)) ⇒
|
||||||
|
reply ! ctx.spawnAnonymous(behavior)
|
||||||
|
Actor.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
|
* Testkit for typed actors. Extending this removes some boiler plate when testing
|
||||||
* typed actors.
|
* typed actors.
|
||||||
*
|
*
|
||||||
* If a test can't extend then use the [[TestKitBase]] trait
|
* If a test can't extend then use the [[TestKitBase]] trait
|
||||||
*
|
|
||||||
* @param _system The [ActorSystem] for the test
|
|
||||||
*/
|
*/
|
||||||
@ApiMayChange
|
@ApiMayChange
|
||||||
class TestKit(_system: ActorSystem[_]) extends TestKitBase {
|
class TestKit(name: String, config: Option[Config]) extends TestKitBase {
|
||||||
implicit val system = _system
|
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(testKitGuardian, name, config = config)
|
||||||
}
|
}
|
||||||
|
|
||||||
@ApiMayChange
|
@ApiMayChange
|
||||||
trait TestKitBase {
|
trait TestKitBase {
|
||||||
def system: ActorSystem[_]
|
def system: ActorSystem[TestKitCommand]
|
||||||
implicit def testkitSettings = TestKitSettings(system)
|
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 = {
|
def shutdown(): Unit = {
|
||||||
shutdown(system, 5.seconds)
|
TestKit.shutdown(system, timeoutDuration)
|
||||||
}
|
}
|
||||||
|
|
||||||
def shutdown(
|
/**
|
||||||
actorSystem: ActorSystem[_],
|
* Spawn the given behavior. This is created as a child of the test kit
|
||||||
duration: Duration,
|
* guardian
|
||||||
verifySystemShutdown: Boolean = false): Unit = {
|
*/
|
||||||
system.terminate()
|
def spawn[T](behavior: Behavior[T]): ActorRef[T] =
|
||||||
try Await.ready(actorSystem.whenTerminated, duration) catch {
|
Await.result(system ? (SpawnActorAnonymous(behavior, _)), timeoutDuration)
|
||||||
case _: TimeoutException ⇒
|
|
||||||
val msg = "Failed to stop [%s] within [%s] \n%s".format(actorSystem.name, duration,
|
|
||||||
actorSystem.printTree)
|
|
||||||
if (verifySystemShutdown) throw new RuntimeException(msg)
|
|
||||||
else println(msg)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
// The only current impl of a typed actor system returns a Future.successful currently
|
/**
|
||||||
// hence the hardcoded timeouts
|
* Spawn the given behavior. This is created as a child of the test kit
|
||||||
def actorOf[T](behaviour: Behavior[T], name: String): ActorRef[T] =
|
* guardian
|
||||||
Await.result(system.systemActorOf(behaviour, name)(Timeout(20.seconds)), 21.seconds)
|
*/
|
||||||
|
def spawn[T](behavior: Behavior[T], name: String): ActorRef[T] =
|
||||||
|
Await.result(system ? (SpawnActor(name, behavior, _)), timeoutDuration)
|
||||||
|
|
||||||
|
def systemActor[T](behaviour: Behavior[T], name: String): ActorRef[T] =
|
||||||
|
Await.result(system.systemActorOf(behaviour, name), timeoutDuration)
|
||||||
|
|
||||||
|
def systemActor[T](behaviour: Behavior[T]): ActorRef[T] =
|
||||||
|
Await.result(system.systemActorOf(behaviour, childName.next()), timeoutDuration)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
||||||
|
|
@ -82,56 +82,51 @@ class BehaviorTestkitSpec extends WordSpec with Matchers {
|
||||||
|
|
||||||
private val props = Props.empty
|
private val props = Props.empty
|
||||||
|
|
||||||
"BehaviourTestkit's spawn" should {
|
"BehaviorTestkit's spawn" must {
|
||||||
"create children when no props specified" in {
|
"create children when no props specified" in {
|
||||||
val ctx = BehaviorTestkit[Father.Command](Father.init())
|
val testkit = BehaviorTestkit[Father.Command](Father.init())
|
||||||
|
testkit.run(SpawnChildren(2))
|
||||||
ctx.run(SpawnChildren(2))
|
val effects = testkit.retrieveAllEffects()
|
||||||
val effects = ctx.retrieveAllEffects()
|
|
||||||
effects should contain only (Spawned(Child.initial, "child0"), Spawned(Child.initial, "child1", Props.empty))
|
effects should contain only (Spawned(Child.initial, "child0"), Spawned(Child.initial, "child1", Props.empty))
|
||||||
}
|
}
|
||||||
|
|
||||||
"create children when props specified and record effects" in {
|
"create children when props specified and record effects" in {
|
||||||
val ctx = BehaviorTestkit[Father.Command](Father.init())
|
val testkit = BehaviorTestkit[Father.Command](Father.init())
|
||||||
|
testkit.run(SpawnChildrenWithProps(2, props))
|
||||||
ctx.run(SpawnChildrenWithProps(2, props))
|
val effects = testkit.retrieveAllEffects()
|
||||||
val effects = ctx.retrieveAllEffects()
|
|
||||||
effects should contain only (Spawned(Child.initial, "child0", props), Spawned(Child.initial, "child1", props))
|
effects should contain only (Spawned(Child.initial, "child0", props), Spawned(Child.initial, "child1", props))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"BehaviourTestkit's spawnAnonymous" should {
|
"BehaviorTestkit's spawnAnonymous" must {
|
||||||
"create children when no props specified and record effects" in {
|
"create children when no props specified and record effects" in {
|
||||||
val ctx = BehaviorTestkit[Father.Command](Father.init())
|
val testkit = BehaviorTestkit[Father.Command](Father.init())
|
||||||
|
testkit.run(SpawnAnonymous(2))
|
||||||
ctx.run(SpawnAnonymous(2))
|
val effects = testkit.retrieveAllEffects()
|
||||||
val effects = ctx.retrieveAllEffects()
|
|
||||||
effects shouldBe Seq(SpawnedAnonymous(Child.initial, Props.empty), SpawnedAnonymous(Child.initial, Props.empty))
|
effects shouldBe Seq(SpawnedAnonymous(Child.initial, Props.empty), SpawnedAnonymous(Child.initial, Props.empty))
|
||||||
}
|
}
|
||||||
|
|
||||||
"create children when props specified and record effects" in {
|
"create children when props specified and record effects" in {
|
||||||
val ctx = BehaviorTestkit[Father.Command](Father.init())
|
val testkit = BehaviorTestkit[Father.Command](Father.init())
|
||||||
|
|
||||||
ctx.run(SpawnAnonymousWithProps(2, props))
|
testkit.run(SpawnAnonymousWithProps(2, props))
|
||||||
val effects = ctx.retrieveAllEffects()
|
val effects = testkit.retrieveAllEffects()
|
||||||
effects shouldBe Seq(SpawnedAnonymous(Child.initial, props), SpawnedAnonymous(Child.initial, props))
|
effects shouldBe Seq(SpawnedAnonymous(Child.initial, props), SpawnedAnonymous(Child.initial, props))
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
"BehaviourTestkit's spawnAdapter" should {
|
"BehaviorTestkit's spawnAdapter" must {
|
||||||
"create adapters without name and record effects" in {
|
"create adapters without name and record effects" in {
|
||||||
val ctx = BehaviorTestkit[Father.Command](Father.init())
|
val testkit = BehaviorTestkit[Father.Command](Father.init())
|
||||||
|
testkit.run(SpawnAdapter)
|
||||||
ctx.run(SpawnAdapter)
|
val effects = testkit.retrieveAllEffects()
|
||||||
val effects = ctx.retrieveAllEffects()
|
|
||||||
effects shouldBe Seq(SpawnedAdapter)
|
effects shouldBe Seq(SpawnedAdapter)
|
||||||
}
|
}
|
||||||
|
|
||||||
"create adapters with name and record effects" in {
|
"create adapters with name and record effects" in {
|
||||||
val ctx = BehaviorTestkit[Father.Command](Father.init())
|
val testkit = BehaviorTestkit[Father.Command](Father.init())
|
||||||
|
testkit.run(SpawnAdapterWithName("adapter"))
|
||||||
ctx.run(SpawnAdapterWithName("adapter"))
|
val effects = testkit.retrieveAllEffects()
|
||||||
val effects = ctx.retrieveAllEffects()
|
|
||||||
effects shouldBe Seq(SpawnedAdapter)
|
effects shouldBe Seq(SpawnedAdapter)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue