Cleanup typed test output (#25669)

* Cleanup ActorContextSpec output

* Default to not log dead letters and hide up to info level from tests

* Use log instead of println to benefit from config to silence

* The silence of the AdapterSpec

* Hushed the AskSpec

* Subdue the TimerSpec

* Muffled the FaultToleranceDocTest

* Remove the application.conf from the test in waiting for #25708
This commit is contained in:
Johan Andrén 2018-10-11 13:34:28 +02:00 committed by Patrik Nordwall
parent 7b264e03d6
commit ab545430e0
7 changed files with 165 additions and 83 deletions

View file

@ -7,7 +7,10 @@ package jdocs.akka.typed;
import akka.actor.typed.ActorRef;
import akka.actor.typed.ActorSystem;
import akka.actor.typed.Behavior;
import akka.actor.typed.internal.adapter.ActorSystemAdapter;
import akka.actor.typed.javadsl.Behaviors;
import akka.testkit.javadsl.EventFilter;
import com.typesafe.config.ConfigFactory;
import org.junit.Test;
import org.scalatest.junit.JUnitSuite;
@ -27,10 +30,10 @@ public class FaultToleranceDocTest extends JUnitSuite {
public void bubblingSample() {
// #bubbling-example
final Behavior<Message> failingChildBehavior = Behaviors.receive(Message.class)
.onMessage(Fail.class, (ctx, message) -> {
throw new RuntimeException(message.text);
})
.build();
.onMessage(Fail.class, (ctx, message) -> {
throw new RuntimeException(message.text);
})
.build();
Behavior<Message> middleManagementBehavior = Behaviors.setup((ctx) -> {
ctx.getLog().info("Middle management starting up");
@ -43,11 +46,11 @@ public class FaultToleranceDocTest extends JUnitSuite {
// when the child fails or stops gracefully this actor will
// fail with a DeathWatchException
return Behaviors.receive(Message.class)
.onMessage(Message.class, (innerCtx, msg) -> {
// just pass messages on to the child
child.tell(msg);
return Behaviors.same();
}).build();
.onMessage(Message.class, (innerCtx, msg) -> {
// just pass messages on to the child
child.tell(msg);
return Behaviors.same();
}).build();
});
Behavior<Message> bossBehavior = Behaviors.setup((ctx) -> {
@ -59,17 +62,33 @@ public class FaultToleranceDocTest extends JUnitSuite {
// when middle management fails with a DeathWatchException
// this actor will also fail
return Behaviors.receive(Message.class)
.onMessage(Message.class, (innerCtx, msg) -> {
// just pass messages on to the child
middleManagement.tell(msg);
return Behaviors.same();
}).build();
.onMessage(Message.class, (innerCtx, msg) -> {
// just pass messages on to the child
middleManagement.tell(msg);
return Behaviors.same();
}).build();
});
{
// #bubbling-example
final ActorSystem<Message> system =
ActorSystem.create(bossBehavior, "boss");
ActorSystem.create(bossBehavior, "boss");
// #bubbling-example
}
final ActorSystem<Message> system =
ActorSystem.create(bossBehavior, "boss", ConfigFactory.parseString(
"akka.loggers = [ akka.testkit.TestEventListener ]\n" +
"akka.loglevel=warning"));
// #bubbling-example
// actual exception and thent the deathpacts
new EventFilter(Exception.class, ActorSystemAdapter.toUntyped(system)).occurrences(4).intercept(() -> {
// #bubbling-example
system.tell(new Fail("boom"));
// #bubbling-example
return null;
});
// #bubbling-example
// this will now bubble up all the way to the boss and as that is the user guardian it means
// the entire actor system will stop

View file

@ -12,6 +12,7 @@ import scala.concurrent.duration._
import scala.reflect.ClassTag
import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.testkit.EventFilter
import org.scalatest.WordSpecLike
object ActorSpecMessages {
@ -62,7 +63,14 @@ object ActorSpecMessages {
}
abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecLike {
abstract class ActorContextSpec extends ScalaTestWithActorTestKit(
"""
akka.loggers = [akka.testkit.TestEventListener]
""") with WordSpecLike {
// FIXME eventfilter support in typed testkit
import scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
import ActorSpecMessages._
@ -103,8 +111,11 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecL
val actor = spawn(behavior)
actor ! Ping
probe.expectMessage(Pong)
actor ! Miss
probe.expectMessage(Missed)
// unhandled gives warning from EventFilter
EventFilter.warning(occurrences = 1).intercept {
actor ! Miss
probe.expectMessage(Missed)
}
actor ! Renew(probe.ref)
probe.expectMessage(Renewed)
actor ! Ping
@ -125,7 +136,9 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecL
val behavior = Behaviors.supervise(internal).onFailure(SupervisorStrategy.restart)
val actor = spawn(behavior)
actor ! Fail
EventFilter[TestException](occurrences = 1).intercept {
actor ! Fail
}
probe.expectMessage(GotSignal(PreRestart))
}
@ -180,7 +193,9 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecL
val parentRef = spawn(parent)
val childRef = probe.expectMessageType[ChildMade].ref
childRef ! Fail
EventFilter[TestException](occurrences = 1).intercept {
childRef ! Fail
}
probe.expectMessage(GotChildSignal(PreRestart))
childRef ! Ping
probe.expectMessage(Pong)
@ -229,7 +244,9 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecL
val actor = spawn(behavior)
actor ! Ping
probe.expectMessage(1)
actor ! Fail
EventFilter[TestException](occurrences = 1).intercept {
actor ! Fail
}
actor ! Ping
probe.expectMessage(1)
}
@ -251,7 +268,9 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecL
val actor = spawn(behavior)
actor ! Ping
probe.expectMessage(1)
actor ! Fail
EventFilter[TestException](occurrences = 1).intercept {
actor ! Fail
}
actor ! Ping
probe.expectMessage(2)
}
@ -286,7 +305,9 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecL
probe.expectMessage(Pong)
watcher ! Ping
probe.expectMessage(Pong)
actorToWatch ! Fail
EventFilter[TestException](occurrences = 1).intercept {
actorToWatch ! Fail
}
probe.expectMessage(GotSignal(PostStop))
probe.expectTerminated(actorToWatch, timeout.duration)
}
@ -320,7 +341,7 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecL
case (_, Stop)
Behaviors.stopped
}.decorate
val actor: ActorRef[Command] = spawn(
spawn(
Behaviors.setup[Command](ctx {
val childRef = ctx.spawn(child, "A")
ctx.watch(childRef)
@ -443,10 +464,12 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecL
val childRef = probe.expectMessageType[ChildMade].ref
actor ! Inert
probe.expectMessage(InertEvent)
childRef ! Stop
probe.expectMessage(GotChildSignal(PostStop))
probe.expectMessage(GotSignal(PostStop))
probe.expectTerminated(actor, timeout.duration)
EventFilter[DeathPactException](occurrences = 1).intercept {
childRef ! Stop
probe.expectMessage(GotChildSignal(PostStop))
probe.expectMessage(GotSignal(PostStop))
probe.expectTerminated(actor, timeout.duration)
}
}
"return the right context info" in {

View file

@ -8,11 +8,9 @@ import akka.actor.typed.internal.adapter.ActorSystemAdapter
import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.Behaviors._
import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter
import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.util.Timeout
import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._
import scala.concurrent.{ ExecutionContext, TimeoutException }
@ -26,11 +24,18 @@ object AskSpec {
final case class Stop(replyTo: ActorRef[Unit]) extends Msg
}
class AskSpec extends ScalaTestWithActorTestKit(
"akka.loggers = [ akka.testkit.TestEventListener ]") with WordSpecLike {
class AskSpec extends ScalaTestWithActorTestKit("""
akka.loglevel=warning
akka.loggers = [ akka.testkit.TestEventListener ]
""") with WordSpecLike {
// FIXME eventfilter support in typed testkit
import AskSpec._
// FIXME eventfilter support in typed testkit
import scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
implicit def executor: ExecutionContext =
system.executionContext
@ -44,34 +49,39 @@ class AskSpec extends ScalaTestWithActorTestKit(
}
"Ask pattern" must {
"must fail the future if the actor is already terminated" in {
"fail the future if the actor is already terminated" in {
val ref = spawn(behavior)
(ref ? Stop).futureValue
val probe = createTestProbe()
probe.expectTerminated(ref, probe.remainingOrDefault)
val answer = ref ? Foo("bar")
val answer =
EventFilter.warning(pattern = ".*received dead letter.*", occurrences = 1).intercept {
ref ? Foo("bar")
}
val result = answer.failed.futureValue
result shouldBe a[TimeoutException]
result.getMessage should include("had already been terminated.")
}
"must succeed when the actor is alive" in {
"succeed when the actor is alive" in {
val ref = spawn(behavior)
val response = ref ? Foo("bar")
response.futureValue should ===("foo")
}
"must fail the future if the actor doesn't reply in time" in {
"fail the future if the actor doesn't reply in time" in {
val actor = spawn(Behaviors.empty[Foo])
implicit val timeout: Timeout = 10.millis
val answer = actor ? Foo("bar")
val result = answer.failed.futureValue
result shouldBe a[TimeoutException]
result.getMessage should startWith("Ask timed out on")
EventFilter.warning(pattern = ".*unhandled message.*", occurrences = 1).intercept {
val answer = actor ? Foo("bar")
val result = answer.failed.futureValue
result shouldBe a[TimeoutException]
result.getMessage should startWith("Ask timed out on")
}
}
/** See issue #19947 (MatchError with adapted ActorRef) */
"must fail the future if the actor doesn't exist" in {
"fail the future if the actor doesn't exist" in {
val noSuchActor: ActorRef[Msg] = system match {
case adaptedSys: ActorSystemAdapter[_]
import akka.actor.typed.scaladsl.adapter._
@ -80,13 +90,16 @@ class AskSpec extends ScalaTestWithActorTestKit(
fail("this test must only run in an adapted actor system")
}
val answer = noSuchActor ? Foo("bar")
val answer =
EventFilter.warning(pattern = ".*received dead letter.*", occurrences = 1).intercept {
noSuchActor ? Foo("bar")
}
val result = answer.failed.futureValue
result shouldBe a[TimeoutException]
result.getMessage should include("had already been terminated")
}
"must transform a replied akka.actor.Status.Failure to a failed future" in {
"transform a replied akka.actor.Status.Failure to a failed future" in {
// It's unlikely but possible that this happens, since the receiving actor would
// have to accept a message with an actoref that accepts AnyRef or be doing crazy casting
// For completeness sake though
@ -126,12 +139,14 @@ class AskSpec extends ScalaTestWithActorTestKit(
case _ "test"
}
Behavior.same
case (ctx, "test")
case (_, "test")
probe.ref ! "got-test"
Behavior.same
case (ctx, "get-state")
case (_, "get-state")
probe.ref ! "running"
Behavior.same
case (_, _)
Behavior.unhandled
}
val ref = spawn(behv)

View file

@ -79,9 +79,6 @@ class StubbedSupervisionSpec extends WordSpec with Matchers {
import SupervisionSpec._
def mkTestkit(behv: Behavior[Command]): BehaviorTestKit[Command] =
BehaviorTestKit(behv)
"A restarter (stubbed)" must {
"receive message" in {
val inbox = TestInbox[Event]("evt")
@ -235,7 +232,7 @@ class StubbedSupervisionSpec extends WordSpec with Matchers {
inbox.ref ! Started
targetBehavior(inbox.ref)
}).onFailure[Exc1](SupervisorStrategy.restart)
mkTestkit(behv)
BehaviorTestKit(behv)
// it's supposed to be created immediately (not waiting for first message)
inbox.receiveMessage() should ===(Started)
}
@ -711,7 +708,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit(
.onFailure[Exception](SupervisorStrategy.restart)
EventFilter[ActorInitializationException](occurrences = 1).intercept {
val ref = spawn(behv)
spawn(behv)
probe.expectMessage(Started) // first one before failure
}
}

View file

@ -13,10 +13,17 @@ import scala.util.control.NoStackTrace
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.TimerScheduler
import akka.testkit.TimingTest
import akka.testkit.{ EventFilter, TimingTest }
import org.scalatest.WordSpecLike
class TimerSpec extends ScalaTestWithActorTestKit with WordSpecLike {
class TimerSpec extends ScalaTestWithActorTestKit(
"""
akka.loggers = [ akka.testkit.TestEventListener ]
""") with WordSpecLike {
// FIXME eventfilter support in typed testkit
import scaladsl.adapter._
implicit val untypedSystem = system.toUntyped
sealed trait Command
case class Tick(n: Int) extends Command
@ -165,14 +172,16 @@ class TimerSpec extends ScalaTestWithActorTestKit with WordSpecLike {
probe.expectMessage(Tock(1))
val latch = new CountDownLatch(1)
// next Tock(1) is enqueued in mailbox, but should be discarded by new incarnation
ref ! SlowThenThrow(latch, new Exc)
probe.expectNoMessage(interval + 100.millis.dilated)
latch.countDown()
probe.expectMessage(GotPreRestart(false))
probe.expectNoMessage(interval / 2)
probe.expectMessage(Tock(2))
EventFilter[Exc](occurrences = 1).intercept {
// next Tock(1) is enqueued in mailbox, but should be discarded by new incarnation
ref ! SlowThenThrow(latch, new Exc)
probe.expectNoMessage(interval + 100.millis.dilated)
latch.countDown()
probe.expectMessage(GotPreRestart(false))
probe.expectNoMessage(interval / 2)
probe.expectMessage(Tock(2))
}
ref ! End
probe.expectMessage(GotPostStop(false))
}
@ -191,13 +200,15 @@ class TimerSpec extends ScalaTestWithActorTestKit with WordSpecLike {
probe.expectMessage(Tock(2))
val latch = new CountDownLatch(1)
// next Tock(2) is enqueued in mailbox, but should be discarded by new incarnation
ref ! SlowThenThrow(latch, new Exc)
probe.expectNoMessage(interval + 100.millis.dilated)
latch.countDown()
probe.expectMessage(GotPreRestart(false))
probe.expectMessage(Tock(1))
EventFilter[Exc](occurrences = 1).intercept {
val latch = new CountDownLatch(1)
// next Tock(2) is enqueued in mailbox, but should be discarded by new incarnation
ref ! SlowThenThrow(latch, new Exc)
probe.expectNoMessage(interval + 100.millis.dilated)
latch.countDown()
probe.expectMessage(GotPreRestart(false))
probe.expectMessage(Tock(1))
}
ref ! End
probe.expectMessage(GotPostStop(false))
@ -210,8 +221,10 @@ class TimerSpec extends ScalaTestWithActorTestKit with WordSpecLike {
target(probe.ref, timer, 1)
}
val ref = spawn(behv)
ref ! Throw(new Exc)
probe.expectMessage(GotPostStop(false))
EventFilter[Exc](occurrences = 1).intercept {
ref ! Throw(new Exc)
probe.expectMessage(GotPostStop(false))
}
}
"cancel timers when stopped voluntarily" taggedAs TimingTest in {

View file

@ -145,7 +145,10 @@ object AdapterSpec {
}
class AdapterSpec extends AkkaSpec {
class AdapterSpec extends AkkaSpec(
"""
akka.loggers = [akka.testkit.TestEventListener]
""") {
import AdapterSpec._
"ActorSystem adaption" must {
@ -256,29 +259,40 @@ class AdapterSpec extends AkkaSpec {
val ign = system.spawnAnonymous(Behaviors.ignore[Ping])
val untypedRef = system.actorOf(untyped2(ign, probe.ref))
untypedRef ! "supervise-stop"
probe.expectMsg("thrown-stop")
// ping => ok should not get through here
probe.expectMsg("terminated")
EventFilter[AdapterSpec.ThrowIt1.type](occurrences = 1).intercept {
EventFilter.warning(pattern = """.*received dead letter.*""", occurrences = 1).intercept {
untypedRef ! "supervise-stop"
probe.expectMsg("thrown-stop")
// ping => ok should not get through here
probe.expectMsg("terminated")
}
}
untypedRef ! "supervise-resume"
probe.expectMsg("thrown-resume")
probe.expectMsg("ok")
untypedRef ! "supervise-restart"
probe.expectMsg("thrown-restart")
probe.expectMsg("ok")
EventFilter[AdapterSpec.ThrowIt3.type](occurrences = 1).intercept {
untypedRef ! "supervise-restart"
probe.expectMsg("thrown-restart")
probe.expectMsg("ok")
}
}
"supervise untyped child from typed parent" in {
// FIXME there's a warning with null logged from the untyped empty child here, where does that come from?
val probe = TestProbe()
val ignore = system.actorOf(untyped.Props.empty)
val typedRef = system.spawnAnonymous(typed1(ignore, probe.ref))
// only stop supervisorStrategy
typedRef ! "supervise-stop"
probe.expectMsg("terminated")
probe.expectNoMsg(100.millis) // no pong
EventFilter[AdapterSpec.ThrowIt3.type](occurrences = 1).intercept {
EventFilter.warning(pattern = """.*received dead letter.*""", occurrences = 1).intercept {
typedRef ! "supervise-stop"
probe.expectMsg("terminated")
probe.expectNoMessage(100.millis) // no pong
}
}
}
"stop typed child from untyped parent" in {

View file

@ -4,6 +4,7 @@
package docs.akka.typed.coexistence
import akka.actor.ActorLogging
import akka.actor.typed._
import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.TestKit
@ -25,7 +26,7 @@ object UntypedWatchingTypedSpec {
}
//#untyped-watch
class Untyped extends untyped.Actor {
class Untyped extends untyped.Actor with ActorLogging {
// context.spawn is an implicit extension method
val second: ActorRef[Typed.Command] =
context.spawn(Typed.behavior, "second")
@ -40,11 +41,11 @@ object UntypedWatchingTypedSpec {
override def receive = {
case Typed.Pong
println(s"$self got Pong from ${sender()}")
log.info(s"$self got Pong from ${sender()}")
// context.stop is an implicit extension method
context.stop(second)
case untyped.Terminated(ref)
println(s"$self observed termination of $ref")
log.info(s"$self observed termination of $ref")
context.stop(self)
}
}
@ -60,7 +61,7 @@ object UntypedWatchingTypedSpec {
Behaviors.receive { (ctx, msg)
msg match {
case Ping(replyTo)
println(s"${ctx.self} got Ping from $replyTo")
ctx.log.info(s"${ctx.self} got Ping from $replyTo")
// replyTo is an untyped actor that has been converted for coexistence
replyTo ! Pong
Behaviors.same