[ttk] Add access to logged events in BehaviorTestKit #25905

* [ttk] Add access to logged events in BehaviorTestKit #25905

`BehaviorTestKit` now has methods to:
 - Access to logged events
 - Clear all log events

Added section in docs explaining how to check for logs.

Note about the implementation: `CapturedLogEvent` was effectively private to akka. It uses `OptionVal` internally, this makes it impossible to be used from outside akka. To overcome this, I added some methods to translate it to the corresponding `Option` and `Optional`.

* Apply feedback

Clean up `CapturedLogEvent` by using only `Option` in public methods
Use `immutable.Seq` instead of plain `Seq`
This commit is contained in:
Josep Prat 2018-11-30 13:01:37 +01:00 committed by Patrik Nordwall
parent 68dc288b08
commit de02d5d6bd
10 changed files with 235 additions and 24 deletions

View file

@ -0,0 +1,73 @@
/*
* Copyright (C) 2018 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed
import java.util.Optional
import akka.actor.typed.LogMarker
import akka.annotation.InternalApi
import akka.event.Logging.LogLevel
import akka.util.OptionVal
import scala.collection.JavaConverters._
import scala.compat.java8.OptionConverters._
/**
* Representation of a Log Event issued by a [[akka.actor.typed.Behavior]]
*/
final case class CapturedLogEvent(
logLevel: LogLevel,
message: String,
cause: Option[Throwable],
marker: Option[LogMarker],
mdc: Map[String, Any]) {
/**
* Constructor for Java API
*/
def this(logLevel: LogLevel, message: String, errorCause: Optional[Throwable], marker: Optional[LogMarker], mdc: java.util.Map[String, Any]) {
this(logLevel, message, errorCause.asScala, marker.asScala, mdc.asScala.toMap)
}
def getMdc: java.util.Map[String, Any] = mdc.asJava
def getErrorCause: Optional[Throwable] = cause.asJava
def getLogMarker: Optional[LogMarker] = marker.asJava
}
object CapturedLogEvent {
/**
* Helper method to convert [[OptionVal]] to [[Option]]
*/
private def toOption[A](optionVal: OptionVal[A]): Option[A] = optionVal match {
case OptionVal.Some(x) Some(x)
case _ None
}
def apply(
logLevel: LogLevel,
message: String,
cause: Option[Throwable] = None,
marker: Option[LogMarker] = None,
mdc: Map[String, Any] = Map.empty): CapturedLogEvent = {
new CapturedLogEvent(logLevel, message, cause, marker, mdc)
}
/**
* Auxiliary constructor that receives Akka's internal [[OptionVal]] as parameters and converts them to Scala's [[Option]].
* INTERNAL API
*/
@InternalApi
private[akka] def apply(
logLevel: LogLevel,
message: String,
errorCause: OptionVal[Throwable],
logMarker: OptionVal[LogMarker],
mdc: Map[String, Any]): CapturedLogEvent = {
new CapturedLogEvent(logLevel, message, toOption(errorCause), toOption(logMarker), mdc)
}
}

View file

@ -7,10 +7,9 @@ package akka.actor.testkit.typed.internal
import java.util import java.util
import akka.actor.ActorPath import akka.actor.ActorPath
import akka.actor.typed.{ ActorRef, Behavior, PostStop, Signal }
import akka.actor.typed.{ Behavior, PostStop, Signal, ActorRef }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.actor.testkit.typed.Effect import akka.actor.testkit.typed.{ CapturedLogEvent, Effect }
import akka.actor.testkit.typed.Effect._ import akka.actor.testkit.typed.Effect._
import scala.annotation.tailrec import scala.annotation.tailrec
@ -138,4 +137,10 @@ private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehav
} }
override def hasEffects(): Boolean = !context.effectQueue.isEmpty override def hasEffects(): Boolean = !context.effectQueue.isEmpty
override def getAllLogEntries(): util.List[CapturedLogEvent] = logEntries().asJava
override def logEntries(): immutable.Seq[CapturedLogEvent] = context.logEntries
override def clearLog(): Unit = context.clearLog()
} }

View file

@ -7,16 +7,15 @@ package akka.actor.testkit.typed.internal
import akka.actor.typed._ import akka.actor.typed._
import akka.actor.typed.internal._ import akka.actor.typed.internal._
import akka.actor.typed.internal.adapter.AbstractLogger import akka.actor.typed.internal.adapter.AbstractLogger
import akka.actor.testkit.typed.CapturedLogEvent
import akka.actor.testkit.typed.scaladsl.TestInbox import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.{ ActorPath, InvalidMessageException } import akka.actor.{ ActorPath, InvalidMessageException }
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.event.Logging import akka.event.Logging
import akka.event.Logging.LogLevel
import akka.util.{ Helpers, OptionVal } import akka.util.{ Helpers, OptionVal }
import akka.{ actor untyped } import akka.{ actor untyped }
import java.util.concurrent.ThreadLocalRandom.{ current rnd } import java.util.concurrent.ThreadLocalRandom.{ current rnd }
import scala.collection.JavaConverters._
import scala.collection.immutable.TreeMap import scala.collection.immutable.TreeMap
import scala.concurrent.ExecutionContextExecutor import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
@ -51,13 +50,6 @@ private[akka] final class FunctionRef[-T](
def isTerminated: Boolean = false def isTerminated: Boolean = false
} }
final case class CapturedLogEvent(logLevel: LogLevel, message: String,
cause: OptionVal[Throwable],
marker: OptionVal[LogMarker],
mdc: Map[String, Any]) {
def getMdc: java.util.Map[String, Any] = mdc.asJava
}
/** /**
* INTERNAL API * INTERNAL API
* *

View file

@ -4,10 +4,10 @@
package akka.actor.testkit.typed.javadsl package akka.actor.testkit.typed.javadsl
import akka.actor.typed.{ Behavior, Signal, ActorRef }
import akka.annotation.DoNotInherit
import akka.actor.testkit.typed.Effect
import akka.actor.testkit.typed.internal.BehaviorTestKitImpl import akka.actor.testkit.typed.internal.BehaviorTestKitImpl
import akka.actor.testkit.typed.{ CapturedLogEvent, Effect }
import akka.actor.typed.{ ActorRef, Behavior, Signal }
import akka.annotation.DoNotInherit
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
@ -127,4 +127,14 @@ abstract class BehaviorTestKit[T] {
* Send the signal to the beheavior and record any [[Effect]]s * Send the signal to the beheavior and record any [[Effect]]s
*/ */
def signal(signal: Signal): Unit def signal(signal: Signal): Unit
/**
* Returns all the [[CapturedLogEvent]] issued by this behavior(s)
*/
def getAllLogEntries(): java.util.List[CapturedLogEvent]
/**
* Clear the log entries
*/
def clearLog(): Unit
} }

View file

@ -4,10 +4,10 @@
package akka.actor.testkit.typed.scaladsl package akka.actor.testkit.typed.scaladsl
import akka.actor.typed.{ Behavior, Signal, ActorRef }
import akka.annotation.DoNotInherit
import akka.actor.testkit.typed.Effect
import akka.actor.testkit.typed.internal.BehaviorTestKitImpl import akka.actor.testkit.typed.internal.BehaviorTestKitImpl
import akka.actor.testkit.typed.{ CapturedLogEvent, Effect }
import akka.actor.typed.{ ActorRef, Behavior, Signal }
import akka.annotation.DoNotInherit
import java.util.concurrent.ThreadLocalRandom import java.util.concurrent.ThreadLocalRandom
@ -41,7 +41,7 @@ trait BehaviorTestKit[T] {
private[akka] def context: akka.actor.typed.ActorContext[T] private[akka] def context: akka.actor.typed.ActorContext[T]
/** /**
* Requests the oldest [[Effect]] or [[akka.actor.testkit.typed.scaladsl.Effects.NoEffects]] if no effects * Requests the oldest [[Effect]] or [[akka.actor.testkit.typed.Effect.NoEffects]] if no effects
* have taken place. The effect is consumed, subsequent calls won't * have taken place. The effect is consumed, subsequent calls won't
* will not include this effect. * will not include this effect.
*/ */
@ -129,7 +129,17 @@ trait BehaviorTestKit[T] {
def runOne(): Unit def runOne(): Unit
/** /**
* Send the signal to the beheavior and record any [[Effect]]s * Send the signal to the behavior and record any [[Effect]]s
*/ */
def signal(signal: Signal): Unit def signal(signal: Signal): Unit
/**
* Returns all the [[CapturedLogEvent]] issued by this behavior(s)
*/
def logEntries(): immutable.Seq[CapturedLogEvent]
/**
* Clear the log entries
*/
def clearLog(): Unit
} }

View file

@ -5,18 +5,22 @@
package akka.actor.testkit.typed.javadsl; package akka.actor.testkit.typed.javadsl;
import akka.Done; import akka.Done;
import akka.actor.testkit.typed.CapturedLogEvent;
import akka.actor.testkit.typed.Effect; import akka.actor.testkit.typed.Effect;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
import akka.actor.typed.Props; import akka.actor.typed.Props;
import akka.actor.typed.javadsl.Behaviors; import akka.actor.typed.javadsl.Behaviors;
import akka.event.Logging;
import org.junit.Ignore; import org.junit.Ignore;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Optional;
import java.util.function.Function; import java.util.function.Function;
import java.util.stream.IntStream; import java.util.stream.IntStream;
@ -111,6 +115,14 @@ public class BehaviorTestKitTest extends JUnitSuite {
} }
} }
public static class Log implements Command {
private final String what;
public Log(String what) {
this.what = what;
}
}
public interface Action { public interface Action {
} }
@ -171,6 +183,10 @@ public class BehaviorTestKitTest extends JUnitSuite {
message.replyTo.tell(Done.getInstance()); message.replyTo.tell(Done.getInstance());
return Behaviors.same(); return Behaviors.same();
}) })
.onMessage(Log.class, (context, message) -> {
context.getLog().info(message.what);
return Behaviors.same();
})
.build(); .build();
@ -194,6 +210,26 @@ public class BehaviorTestKitTest extends JUnitSuite {
test.expectEffectClass(Effect.NoEffects.class); test.expectEffectClass(Effect.NoEffects.class);
} }
@Test
public void allowRetrieveAllLogs() {
BehaviorTestKit<Command> test = BehaviorTestKit.create(behavior);
String what = "Hello!";
test.run(new Log(what));
final List<CapturedLogEvent> allLogEntries = test.getAllLogEntries();
assertEquals(1, allLogEntries.size());
assertEquals(new CapturedLogEvent(Logging.InfoLevel(), what, Optional.empty(), Optional.empty(), new HashMap<>()), allLogEntries.get(0));
}
@Test
public void allowClearLogs() {
BehaviorTestKit<Command> test = BehaviorTestKit.create(behavior);
String what = "Hello!";
test.run(new Log(what));
assertEquals(1, test.getAllLogEntries().size());
test.clearLog();
assertEquals(0, test.getAllLogEntries().size());
}
@Test @Test
public void returnEffectsThatHaveTakenPlace() { public void returnEffectsThatHaveTakenPlace() {
BehaviorTestKit<Command> test = BehaviorTestKit.create(behavior); BehaviorTestKit<Command> test = BehaviorTestKit.create(behavior);

View file

@ -5,15 +5,22 @@
package jdocs.akka.actor.testkit.typed.javadsl; package jdocs.akka.actor.testkit.typed.javadsl;
//#imports //#imports
import akka.actor.testkit.typed.CapturedLogEvent;
import akka.actor.testkit.typed.javadsl.BehaviorTestKit; import akka.actor.testkit.typed.javadsl.BehaviorTestKit;
import akka.actor.testkit.typed.javadsl.Effects; import akka.actor.testkit.typed.javadsl.Effects;
import akka.actor.testkit.typed.javadsl.TestInbox; import akka.actor.testkit.typed.javadsl.TestInbox;
import akka.actor.typed.*; import akka.actor.typed.*;
import akka.actor.typed.javadsl.*; import akka.actor.typed.javadsl.*;
import akka.event.Logging;
import java.util.HashMap;
import java.util.List;
import java.util.Optional;
//#imports //#imports
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.assertEquals;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
public class SyncTestingExampleTest extends JUnitSuite { public class SyncTestingExampleTest extends JUnitSuite {
//#child //#child
@ -43,6 +50,13 @@ public class SyncTestingExampleTest extends JUnitSuite {
this.who = who; this.who = who;
} }
} }
public static class LogAndSayHello implements Command {
private final ActorRef<String> who;
public LogAndSayHello(ActorRef<String> who) {
this.who = who;
}
}
public static Behavior<Command> myBehavior = Behaviors.receive(Command.class) public static Behavior<Command> myBehavior = Behaviors.receive(Command.class)
.onMessage(CreateAChild.class, (context, message) -> { .onMessage(CreateAChild.class, (context, message) -> {
@ -62,10 +76,17 @@ public class SyncTestingExampleTest extends JUnitSuite {
ActorRef<String> child = context.spawnAnonymous(childActor); ActorRef<String> child = context.spawnAnonymous(childActor);
child.tell("hello stranger"); child.tell("hello stranger");
return Behaviors.same(); return Behaviors.same();
}).onMessage(SayHello.class, (context, message) -> { })
.onMessage(SayHello.class, (context, message) -> {
message.who.tell("hello"); message.who.tell("hello");
return Behaviors.same(); return Behaviors.same();
}).build(); })
.onMessage(LogAndSayHello.class, (context, message) -> {
context.getLog().info("Saying hello to {}", message.who.path().name());
message.who.tell("hello");
return Behaviors.same();
})
.build();
//#under-test //#under-test
@ -117,4 +138,19 @@ public class SyncTestingExampleTest extends JUnitSuite {
childInbox.expectMessage("hello stranger"); childInbox.expectMessage("hello stranger");
//#test-child-message-anonymous //#test-child-message-anonymous
} }
@Test
public void testCheckLogging() {
//#test-check-logging
BehaviorTestKit<Command> test = BehaviorTestKit.create(myBehavior);
TestInbox<String> inbox = TestInbox.create("Inboxer");
test.run(new LogAndSayHello(inbox.getRef()));
List<CapturedLogEvent> allLogEntries = test.getAllLogEntries();
assertEquals(1, allLogEntries.size());
CapturedLogEvent expectedLogEvent = new CapturedLogEvent(Logging.InfoLevel(),"Saying hello to Inboxer",
Optional.empty(), Optional.empty(), new HashMap<>());
assertEquals(expectedLogEvent, allLogEntries.get(0));
//#test-check-logging
}
} }

View file

@ -7,10 +7,11 @@ package akka.actor.testkit.typed.scaladsl
import akka.Done import akka.Done
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, Behavior, Props } import akka.actor.typed.{ ActorRef, Behavior, Props }
import akka.actor.testkit.typed.Effect import akka.actor.testkit.typed.{ CapturedLogEvent, Effect }
import akka.actor.testkit.typed.Effect._ import akka.actor.testkit.typed.Effect._
import akka.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.{ Child, Father } import akka.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.{ Child, Father }
import akka.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.Father._ import akka.actor.testkit.typed.scaladsl.BehaviorTestKitSpec.Father._
import akka.event.Logging
import org.scalatest.{ Matchers, WordSpec } import org.scalatest.{ Matchers, WordSpec }
import scala.reflect.ClassTag import scala.reflect.ClassTag
@ -35,6 +36,7 @@ object BehaviorTestKitSpec {
case class SpawnAndWatchWith(name: String) extends Command case class SpawnAndWatchWith(name: String) extends Command
case class SpawnSession(replyTo: ActorRef[ActorRef[String]], sessionHandler: ActorRef[String]) extends Command case class SpawnSession(replyTo: ActorRef[ActorRef[String]], sessionHandler: ActorRef[String]) extends Command
case class KillSession(session: ActorRef[String], replyTo: ActorRef[Done]) extends Command case class KillSession(session: ActorRef[String], replyTo: ActorRef[Done]) extends Command
case class Log(what: String) extends Command
val init: Behavior[Command] = Behaviors.receive[Command] { (context, message) val init: Behavior[Command] = Behaviors.receive[Command] { (context, message)
message match { message match {
@ -97,7 +99,9 @@ object BehaviorTestKitSpec {
case CreateMessageAdapter(messageClass, f) case CreateMessageAdapter(messageClass, f)
context.messageAdapter(f)(ClassTag(messageClass)) context.messageAdapter(f)(ClassTag(messageClass))
Behaviors.same Behaviors.same
case Log(what)
context.log.info(what)
Behaviors.same
} }
} }
} }
@ -174,6 +178,22 @@ class BehaviorTestKitSpec extends WordSpec with Matchers {
} }
hasEffects should ===(false) hasEffects should ===(false)
} }
"allow retrieving log messages issued by behavior" in {
val what = "Hello!"
val testkit = BehaviorTestKit[Father.Command](Father.init)
testkit.run(Log(what))
testkit.logEntries() shouldBe Seq(CapturedLogEvent(Logging.InfoLevel, what))
}
"allow clearing log messages issued by behavior" in {
val what = "Hello!"
val testkit = BehaviorTestKit[Father.Command](Father.init)
testkit.run(Log(what))
testkit.logEntries() shouldBe Seq(CapturedLogEvent(Logging.InfoLevel, what))
testkit.clearLog()
testkit.logEntries() shouldBe Seq.empty
}
} }
"BehaviorTestkit's spawn" must { "BehaviorTestkit's spawn" must {

View file

@ -5,11 +5,13 @@
package docs.akka.actor.testkit.typed.scaladsl package docs.akka.actor.testkit.typed.scaladsl
//#imports //#imports
import akka.actor.testkit.typed.CapturedLogEvent
import akka.actor.testkit.typed.Effect._ import akka.actor.testkit.typed.Effect._
import akka.actor.testkit.typed.scaladsl.BehaviorTestKit import akka.actor.testkit.typed.scaladsl.BehaviorTestKit
import akka.actor.testkit.typed.scaladsl.TestInbox import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.typed._ import akka.actor.typed._
import akka.actor.typed.scaladsl._ import akka.actor.typed.scaladsl._
import akka.event.Logging
//#imports //#imports
import org.scalatest.Matchers import org.scalatest.Matchers
import org.scalatest.WordSpec import org.scalatest.WordSpec
@ -28,6 +30,7 @@ object SyncTestingExampleSpec {
case class SayHelloToChild(childName: String) extends Cmd case class SayHelloToChild(childName: String) extends Cmd
case object SayHelloToAnonymousChild extends Cmd case object SayHelloToAnonymousChild extends Cmd
case class SayHello(who: ActorRef[String]) extends Cmd case class SayHello(who: ActorRef[String]) extends Cmd
case class LogAndSayHello(who: ActorRef[String]) extends Cmd
val myBehavior = Behaviors.receivePartial[Cmd] { val myBehavior = Behaviors.receivePartial[Cmd] {
case (context, CreateChild(name)) case (context, CreateChild(name))
@ -47,6 +50,10 @@ object SyncTestingExampleSpec {
case (_, SayHello(who)) case (_, SayHello(who))
who ! "hello" who ! "hello"
Behaviors.same Behaviors.same
case (context, LogAndSayHello(who))
context.log.info("Saying hello to {}", who.path.name)
who ! "hello"
Behaviors.same
} }
//#under-test //#under-test
@ -102,5 +109,15 @@ class SyncTestingExampleSpec extends WordSpec with Matchers {
childInbox.expectMessage("hello stranger") childInbox.expectMessage("hello stranger")
//#test-child-message-anonymous //#test-child-message-anonymous
} }
"log a message to the logger" in {
//#test-check-logging
val testKit = BehaviorTestKit(myBehavior)
val inbox = TestInbox[String]("Inboxer")
testKit.run(LogAndSayHello(inbox.ref))
testKit.logEntries() shouldBe Seq(CapturedLogEvent(Logging.InfoLevel, "Saying hello to Inboxer"))
//#test-check-logging
}
} }
} }

View file

@ -126,6 +126,18 @@ The `BehaviorTestkit` keeps track other effects you can verify, look at the sub-
* Unwatched * Unwatched
* Scheduled * Scheduled
### Checking for Log Messages
The `BehaviorTestkit` also keeps track of everything that is being logged. Here, you can see an example on how to check
if the behavior logged certain messages:
Scala
: @@snip [SyncTestingExampleSpec.scala](/akka-actor-testkit-typed/src/test/scala/docs/akka/actor/testkit/typed/scaladsl/SyncTestingExampleSpec.scala) { #test-check-logging }
Java
: @@snip [SyncTestingExampleTest.java](/akka-actor-testkit-typed/src/test/java/jdocs/akka/actor/testkit/typed/javadsl/SyncTestingExampleTest.java) { #test-check-logging }
See the other public methods and API documentation on `BehaviorTestkit` for other types of verification. See the other public methods and API documentation on `BehaviorTestkit` for other types of verification.
## Asynchronous testing ## Asynchronous testing