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:
Christopher Batey 2018-01-04 15:09:38 +00:00 committed by GitHub
parent c7cbebb534
commit bd2a3de88a
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
44 changed files with 716 additions and 836 deletions

View file

@ -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
} }
} }

View file

@ -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))
} }

View file

@ -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 ===("")
} }
} }
} }

View file

@ -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 {

View file

@ -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)
} }

View file

@ -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

View file

@ -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")))
}
}
}

View file

@ -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"))

View file

@ -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)

View file

@ -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))
} }

View file

@ -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()
}

View file

@ -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")
})))
}
}
}
}

View file

@ -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
} }
} }
} }

View file

@ -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)
} }
} }

View file

@ -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())
}
}
})
} }
} }
} }

View file

@ -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)
} }
} }

View file

@ -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)
} }
} }

View file

@ -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._

View file

@ -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
} }

View file

@ -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
} }
} }

View file

@ -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

View file

@ -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

View file

@ -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

View file

@ -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")

View file

@ -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 {

View file

@ -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 }

View file

@ -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 -> {

View file

@ -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

View file

@ -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)

View file

@ -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)

View file

@ -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)

View file

@ -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))

View file

@ -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$
} }

View file

@ -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

View 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 systems structure.
## Creating a Supervisor Strategy
TODO
### Default Supervisor Strategy
### Restart Supervisor Strategy
### Stopping Supervisor Strategy
### Logging of Actor Failures

View file

@ -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)

View file

@ -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 }

View file

@ -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)))

View file

@ -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

View file

@ -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._

View file

@ -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

View file

@ -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)
} }

View file

@ -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)
} }
} }