Complete logging with SLF4J in Typed, #26537

* TestAppender interaction with LoggingEventFilter
  * TestAppender plays the same role as classic TestEventList
* replace Sl4jLoggingEvent.java
  * include MDC in testkit.LoggingEvent to allow for custom filter
    testing MDC
* fix tests
* fix log events in BehaviorSetKit, using SubstituteLogger
* MDC in EventSourcedBehavior
  * using raw MDC to set the persistenceId and persistencePhase fields
  * avoid access to MDC ThreadLocal if log isn't used

* Automatically enable eventStream Slf4jLogger for Typed (PR #27583)
  * amend the ActorSystem config on startup when config akka.use-slf4j=on
    and akka-slf4j in classpath
  * akka.use-slf4j is defined in akka-actor-typed reference.conf
  * also enable the Slf4jLoggingFilter automatically
  * remove config in tests

* Include actor path as akkaSource in MDC (PR #27587)
  * move logger init to ActorContextImpl since not specific to untyped (ActorContextAdapter)
  * careful to not access MDC ThreadLocal if logging isn't used (per message)
  * MDC is cleared (if used) from aroundReceive in ActorAdapter after processing each message
  * also changed MDC for EventSourcedBehavior to rely on context.log and the outer MDC.clear()
  * just removing the MDC values is not enough because the empty Map remains in the ThreadLocal

* Optimization of MDC interceptor (PR #27610)

* Extension methods to Logger as workaround for overload and varargs problem (PR #27605)
  * "ambiguous reference to overloaded definition" for 2 arg method
  * varargs not supported for primitive types
  * providing extension methods info2 and infoN (and friends) via implicit class LoggerOps
    as more convenient workaround for these problems

* New API for Typed LoggingEventFilter (PR #27611)
  * builder style
  * will make it easier to add more conditions
  * all conditions are AND:ed together which makes it easier to define custom filters
  * simplifies the implementation as a bonus
  * javadsl
  * use new api in all tests
  * leeway setting
  * verify that occurrences = 0 works
  * use javadsl in two tests
  * change interceptLogger, and some other feedback

* Testing utility to silence logging until failure (PR #27588)
  * Logback appender that captures all log events and can flush them
    later to other appenders
  * LogCapturing mixin for ScalaTest that flush the captured logging events
    when a test fails. Also clears the captured events after all tests.
  * Configuration in logback-test.xml
  * log capturing for JUnit tests too, using TestRule
  * Silence initial setup logging from Logback
  * make LogCapturing public
  * use LogCapturing in all tests
    * aside from multi-jvm, and a few extending AkkaSpec, so far
  * a few doc tests needed debug level

* mention in migration guide
* rename setLoggerClass to setLoggerName
* check for logback dependency to give nicer error message
* fix filterKeys, deprecated in 2.13
This commit is contained in:
Patrik Nordwall 2019-08-24 08:46:39 +02:00
parent dd96d21ffa
commit 68424b4f55
196 changed files with 3782 additions and 2236 deletions

View file

@ -6,26 +6,32 @@
# Make your edits/overrides in your application.conf. # Make your edits/overrides in your application.conf.
akka.actor.testkit.typed { akka.actor.testkit.typed {
# factor by which to scale timeouts during tests, e.g. to account for shared # Factor by which to scale timeouts during tests, e.g. to account for shared
# build system load # build system load.
timefactor = 1.0 timefactor = 1.0
# duration to wait in expectMsg and friends outside of within() block # Duration to wait in expectMsg and friends outside of within() block
# by default, will be dilated by the timefactor. # by default.
# Dilated by the timefactor.
single-expect-default = 3s single-expect-default = 3s
# duration to wait in expectNoMessage by default, # Duration to wait in expectNoMessage by default.
# will be dilated by the timefactor. # Dilated by the timefactor.
expect-no-message-default = 100ms expect-no-message-default = 100ms
# The timeout that is added as an implicit by DefaultTimeout trait, will be dilated by the timefactor. # The timeout that is used as an implicit Timeout.
# Dilated by the timefactor.
default-timeout = 5s default-timeout = 5s
# Default timeout for shutting down the actor system (used when no explicit timeout specified), # Default timeout for shutting down the actor system (used when no explicit timeout specified).
# will be dilated by the timefactor. # Dilated by the timefactor.
system-shutdown-default=10s system-shutdown-default=10s
# Throw an exception on shutdown if the timeout is hit, if false an error is printed to stdout instead. # Throw an exception on shutdown if the timeout is hit, if false an error is printed to stdout instead.
throw-on-shutdown-timeout=true throw-on-shutdown-timeout=true
# Duration to wait for all required logging events in LoggingEventFilter.intercept.
# Dilated by the timefactor.
filter-leeway = 3s
} }

View file

@ -6,69 +6,63 @@ package akka.actor.testkit.typed
import java.util.Optional import java.util.Optional
import akka.actor.typed.LogMarker
import akka.annotation.InternalApi import akka.annotation.InternalApi
import akka.event.Logging.LogLevel
import akka.util.OptionVal import akka.util.OptionVal
import akka.util.ccompat.JavaConverters._
import scala.compat.java8.OptionConverters._ import scala.compat.java8.OptionConverters._
import org.slf4j.Marker
import org.slf4j.event.Level
/** /**
* Representation of a Log Event issued by a [[akka.actor.typed.Behavior]] * Representation of a Log Event issued by a [[akka.actor.typed.Behavior]]
* when testing with [[akka.actor.testkit.typed.scaladsl.BehaviorTestKit`]]
* or [[akka.actor.testkit.typed.javadsl.BehaviorTestKit`]].
*/ */
final case class CapturedLogEvent( final case class CapturedLogEvent(level: Level, message: String, cause: Option[Throwable], marker: Option[Marker]) {
logLevel: LogLevel,
message: String,
cause: Option[Throwable],
marker: Option[LogMarker],
mdc: Map[String, Any]) {
/** /**
* Constructor for Java API * Constructor for Java API
*/ */
def this( def this(
logLevel: LogLevel, level: Level,
message: String, message: String,
errorCause: Optional[Throwable], errorCause: Optional[Throwable],
marker: Optional[LogMarker], marker: Optional[Marker],
mdc: java.util.Map[String, Any]) { mdc: java.util.Map[String, Any]) {
this(logLevel, message, errorCause.asScala, marker.asScala, mdc.asScala.toMap) this(level, message, errorCause.asScala, marker.asScala)
} }
/** /**
* Constructor for Java API * Constructor for Java API
*/ */
def this(logLevel: LogLevel, message: String) { def this(level: Level, message: String) {
this(logLevel, message, Option.empty, Option.empty, Map.empty[String, Any]) this(level, message, Option.empty, Option.empty)
} }
/** /**
* Constructor for Java API * Constructor for Java API
*/ */
def this(logLevel: LogLevel, message: String, errorCause: Throwable) { def this(level: Level, message: String, errorCause: Throwable) {
this(logLevel, message, Some(errorCause), Option.empty[LogMarker], Map.empty[String, Any]) this(level, message, Some(errorCause), Option.empty[Marker])
} }
/** /**
* Constructor for Java API * Constructor for Java API
*/ */
def this(logLevel: LogLevel, message: String, marker: LogMarker) { def this(level: Level, message: String, marker: Marker) {
this(logLevel, message, Option.empty[Throwable], Some(marker), Map.empty[String, Any]) this(level, message, Option.empty[Throwable], Some(marker))
} }
/** /**
* Constructor for Java API * Constructor for Java API
*/ */
def this(logLevel: LogLevel, message: String, errorCause: Throwable, marker: LogMarker) { def this(level: Level, message: String, errorCause: Throwable, marker: Marker) {
this(logLevel, message, Some(errorCause), Some(marker), Map.empty[String, Any]) this(level, message, Some(errorCause), Some(marker))
} }
def getMdc: java.util.Map[String, Any] = mdc.asJava
def getErrorCause: Optional[Throwable] = cause.asJava def getErrorCause: Optional[Throwable] = cause.asJava
def getLogMarker: Optional[LogMarker] = marker.asJava def getMarker: Optional[Marker] = marker.asJava
} }
object CapturedLogEvent { object CapturedLogEvent {
@ -81,8 +75,8 @@ object CapturedLogEvent {
case _ => None case _ => None
} }
def apply(logLevel: LogLevel, message: String): CapturedLogEvent = { def apply(level: Level, message: String): CapturedLogEvent = {
CapturedLogEvent(logLevel, message, None, None, Map.empty[String, Any]) CapturedLogEvent(level, message, None, None)
} }
/** /**
@ -91,11 +85,10 @@ object CapturedLogEvent {
*/ */
@InternalApi @InternalApi
private[akka] def apply( private[akka] def apply(
logLevel: LogLevel, level: Level,
message: String, message: String,
errorCause: OptionVal[Throwable], errorCause: OptionVal[Throwable],
logMarker: OptionVal[LogMarker], logMarker: OptionVal[Marker]): CapturedLogEvent = {
mdc: Map[String, Any]): CapturedLogEvent = { new CapturedLogEvent(level, message, toOption(errorCause), toOption(logMarker))
new CapturedLogEvent(logLevel, message, toOption(errorCause), toOption(logMarker), mdc)
} }
} }

View file

@ -0,0 +1,78 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed
import scala.compat.java8.OptionConverters._
import akka.util.ccompat.JavaConverters._
import java.util.Optional
import org.slf4j.Marker
import org.slf4j.event.Level
object LoggingEvent {
/**
* Scala API
*/
def apply(level: Level, loggerName: String, threadName: String, message: String, timeStamp: Long): LoggingEvent =
new LoggingEvent(level, loggerName, threadName, message, timeStamp, None, None, Map.empty)
/**
* Java API
*/
def create(level: Level, loggerName: String, threadName: String, message: String, timeStamp: Long): LoggingEvent =
apply(level, loggerName, threadName, message, timeStamp)
/**
* Java API
*/
def create(
level: Level,
loggerName: String,
threadName: String,
message: String,
timeStamp: Long,
marker: Optional[Marker],
throwable: Optional[Throwable],
mdc: java.util.Map[String, String]) =
apply(level, loggerName, threadName, message, timeStamp, marker.asScala, throwable.asScala, mdc.asScala.toMap)
}
/**
* Representation of logging event when testing with [[akka.actor.testkit.typed.scaladsl.LoggingEventFilter]]
* or [[akka.actor.testkit.typed.javadsl.LoggingEventFilter]].
*/
final case class LoggingEvent(
level: Level,
loggerName: String,
threadName: String,
message: String,
timeStamp: Long,
marker: Option[Marker],
throwable: Option[Throwable],
mdc: Map[String, String]) {
/**
* Java API
*/
def getMarker: Optional[Marker] =
marker.asJava
/**
* Java API
*/
def getThrowable: Optional[Throwable] =
throwable.asJava
/**
* Java API
*/
def getMdc: java.util.Map[String, String] = {
import akka.util.ccompat.JavaConverters._
mdc.asJava
}
}

View file

@ -5,11 +5,13 @@
package akka.actor.testkit.typed package akka.actor.testkit.typed
import com.typesafe.config.Config import com.typesafe.config.Config
import scala.concurrent.duration.{ Duration, FiniteDuration } import scala.concurrent.duration.{ Duration, FiniteDuration }
import akka.util.JavaDurationConverters._ import akka.util.JavaDurationConverters._
import akka.util.Timeout import akka.util.Timeout
import akka.actor.typed.ActorSystem import akka.actor.typed.ActorSystem
import akka.actor.typed.Extension
import akka.actor.typed.ExtensionId
object TestKitSettings { object TestKitSettings {
@ -17,7 +19,7 @@ object TestKitSettings {
* Reads configuration settings from `akka.actor.testkit.typed` section. * Reads configuration settings from `akka.actor.testkit.typed` section.
*/ */
def apply(system: ActorSystem[_]): TestKitSettings = def apply(system: ActorSystem[_]): TestKitSettings =
apply(system.settings.config.getConfig("akka.actor.testkit.typed")) Ext(system).settings
/** /**
* Reads configuration settings from given `Config` that * Reads configuration settings from given `Config` that
@ -38,30 +40,42 @@ object TestKitSettings {
*/ */
def create(config: Config): TestKitSettings = def create(config: Config): TestKitSettings =
new TestKitSettings(config) new TestKitSettings(config)
private object Ext extends ExtensionId[Ext] {
override def createExtension(system: ActorSystem[_]): Ext = new Ext(system)
def get(system: ActorSystem[_]): Ext = apply(system)
}
private class Ext(system: ActorSystem[_]) extends Extension {
val settings: TestKitSettings = TestKitSettings(system.settings.config.getConfig("akka.actor.testkit.typed"))
}
} }
final class TestKitSettings(val config: Config) { final class TestKitSettings(val config: Config) {
import akka.util.Helpers._ import akka.util.Helpers._
val TestTimeFactor = config val TestTimeFactor: Double = config
.getDouble("timefactor") .getDouble("timefactor")
.requiring(tf => !tf.isInfinite && tf > 0, "timefactor must be positive finite double") .requiring(tf => !tf.isInfinite && tf > 0, "timefactor must be positive finite double")
/** dilated with `TestTimeFactor` */ /** Dilated with `TestTimeFactor`. */
val SingleExpectDefaultTimeout: FiniteDuration = dilated(config.getMillisDuration("single-expect-default")) val SingleExpectDefaultTimeout: FiniteDuration = dilated(config.getMillisDuration("single-expect-default"))
/** dilated with `TestTimeFactor` */ /** Dilated with `TestTimeFactor`. */
val ExpectNoMessageDefaultTimeout: FiniteDuration = dilated(config.getMillisDuration("expect-no-message-default")) val ExpectNoMessageDefaultTimeout: FiniteDuration = dilated(config.getMillisDuration("expect-no-message-default"))
/** dilated with `TestTimeFactor` */ /** Dilated with `TestTimeFactor`. */
val DefaultTimeout: Timeout = Timeout(dilated(config.getMillisDuration("default-timeout"))) val DefaultTimeout: Timeout = Timeout(dilated(config.getMillisDuration("default-timeout")))
/** dilated with `TestTimeFactor` */ /** Dilated with `TestTimeFactor`. */
val DefaultActorSystemShutdownTimeout: FiniteDuration = dilated(config.getMillisDuration("system-shutdown-default")) val DefaultActorSystemShutdownTimeout: FiniteDuration = dilated(config.getMillisDuration("system-shutdown-default"))
val ThrowOnShutdownTimeout: Boolean = config.getBoolean("throw-on-shutdown-timeout") val ThrowOnShutdownTimeout: Boolean = config.getBoolean("throw-on-shutdown-timeout")
/** Dilated with `TestTimeFactor`. */
val FilterLeeway: FiniteDuration = dilated(config.getMillisDuration("filter-leeway"))
/** /**
* Scala API: Scale the `duration` with the configured `TestTimeFactor` * Scala API: Scale the `duration` with the configured `TestTimeFactor`
*/ */

View file

@ -21,14 +21,15 @@ import akka.annotation.InternalApi
import akka.{ actor => classic } import akka.{ actor => classic }
import akka.Done import akka.Done
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.compat.java8.FutureConverters import scala.compat.java8.FutureConverters
import scala.concurrent._ import scala.concurrent._
import akka.actor.ActorRefProvider import akka.actor.ActorRefProvider
import akka.actor.ReflectiveDynamicAccess
import akka.actor.typed.internal.InternalRecipientRef import akka.actor.typed.internal.InternalRecipientRef
import com.github.ghik.silencer.silent import com.github.ghik.silencer.silent
import org.slf4j.Logger import org.slf4j.Logger
import org.slf4j.helpers.SubstituteLoggerFactory import org.slf4j.LoggerFactory
/** /**
* INTERNAL API * INTERNAL API
@ -42,7 +43,14 @@ import org.slf4j.helpers.SubstituteLoggerFactory
override val path: classic.ActorPath = classic.RootActorPath(classic.Address("akka", name)) / "user" override val path: classic.ActorPath = classic.RootActorPath(classic.Address("akka", name)) / "user"
override val settings: Settings = new Settings(getClass.getClassLoader, ConfigFactory.empty, name) override val settings: Settings = {
val classLoader = getClass.getClassLoader
val dynamicAccess = new ReflectiveDynamicAccess(classLoader)
val config =
classic.ActorSystem.Settings.amendSlf4jConfig(ConfigFactory.defaultReference(classLoader), dynamicAccess)
val untypedSettings = new classic.ActorSystem.Settings(classLoader, config, name)
new Settings(untypedSettings)
}
override def tell(message: Nothing): Unit = override def tell(message: Nothing): Unit =
throw new UnsupportedOperationException("must not send message to ActorSystemStub") throw new UnsupportedOperationException("must not send message to ActorSystemStub")
@ -104,7 +112,5 @@ import org.slf4j.helpers.SubstituteLoggerFactory
override def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean = override def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean =
throw new UnsupportedOperationException("ActorSystemStub cannot register extensions") throw new UnsupportedOperationException("ActorSystemStub cannot register extensions")
val loggerFactory = new SubstituteLoggerFactory() override def log: Logger = LoggerFactory.getLogger(getClass)
override def log: Logger = loggerFactory.getLogger("StubbedLogger")
} }

View file

@ -141,7 +141,7 @@ private[akka] final class BehaviorTestKitImpl[T](_path: ActorPath, _initialBehav
override def getAllLogEntries(): util.List[CapturedLogEvent] = logEntries().asJava override def getAllLogEntries(): util.List[CapturedLogEvent] = logEntries().asJava
override def logEntries(): immutable.Seq[CapturedLogEvent] = Nil override def logEntries(): immutable.Seq[CapturedLogEvent] = context.logEntries
override def clearLog(): Unit = context.clearLog() override def clearLog(): Unit = context.clearLog()
} }

View file

@ -0,0 +1,89 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
import akka.annotation.InternalApi
import ch.qos.logback.classic.spi.ILoggingEvent
import ch.qos.logback.core.AppenderBase
/**
* INTERNAL API
*/
@InternalApi private[akka] object CapturingAppender {
import LogbackUtil._
private val CapturingAppenderName = "CapturingAppender"
def get(loggerName: String): CapturingAppender = {
val logbackLogger = getLogbackLogger(loggerName)
logbackLogger.getAppender(CapturingAppenderName) match {
case null =>
throw new IllegalStateException(
s"$CapturingAppenderName not defined for [${loggerNameOrRoot(loggerName)}] in logback-test.xml")
case appender: CapturingAppender => appender
case other =>
throw new IllegalStateException(s"Unexpected $CapturingAppender: $other")
}
}
}
/**
* INTERNAL API
*
* Logging from tests can be silenced by this appender. When there is a test failure
* the captured logging events are flushed to the appenders defined for the
* akka.actor.testkit.typed.internal.CapturingAppenderDelegate logger.
*
* The flushing on test failure is handled by [[akka.actor.testkit.typed.scaladsl.LogCapturing]]
* for ScalaTest and [[akka.actor.testkit.typed.javadsl.LogCapturing]] for JUnit.
*
* Use configuration like the following the logback-test.xml:
*
* {{{
* <appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
*
* <logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
* <appender-ref ref="STDOUT"/>
* </logger>
*
* <root level="DEBUG">
* <appender-ref ref="CapturingAppender"/>
* </root>
* }}}
*/
@InternalApi private[akka] class CapturingAppender extends AppenderBase[ILoggingEvent] {
import LogbackUtil._
private var buffer: Vector[ILoggingEvent] = Vector.empty
// invocations are synchronized via doAppend in AppenderBase
override def append(event: ILoggingEvent): Unit = {
event.prepareForDeferredProcessing()
buffer :+= event
}
/**
* Flush buffered logging events to the output appenders
* Also clears the buffer..
*/
def flush(): Unit = synchronized {
import akka.util.ccompat.JavaConverters._
val logbackLogger = getLogbackLogger(classOf[CapturingAppender].getName + "Delegate")
val appenders = logbackLogger.iteratorForAppenders().asScala.filterNot(_ == this).toList
for (event <- buffer; appender <- appenders) {
appender.doAppend(event)
}
clear()
}
/**
* Discards the buffered logging events without output.
*/
def clear(): Unit = synchronized {
buffer = Vector.empty
}
}

View file

@ -0,0 +1,40 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
import akka.annotation.InternalApi
import org.slf4j.LoggerFactory
import org.slf4j.event.Level
/**
* INTERNAL API
*/
@InternalApi private[akka] object LogbackUtil {
def loggerNameOrRoot(loggerName: String): String =
if (loggerName == "") org.slf4j.Logger.ROOT_LOGGER_NAME else loggerName
def getLogbackLogger(loggerName: String): ch.qos.logback.classic.Logger = {
LoggerFactory.getLogger(loggerNameOrRoot(loggerName)) match {
case logger: ch.qos.logback.classic.Logger => logger
case null =>
throw new IllegalArgumentException(s"Couldn't find logger for [$loggerName].")
case other =>
throw new IllegalArgumentException(
s"Requires Logback logger for [$loggerName], it was a [${other.getClass.getName}]")
}
}
def convertLevel(level: ch.qos.logback.classic.Level): Level = {
level.levelInt match {
case ch.qos.logback.classic.Level.TRACE_INT => Level.TRACE
case ch.qos.logback.classic.Level.DEBUG_INT => Level.DEBUG
case ch.qos.logback.classic.Level.INFO_INT => Level.INFO
case ch.qos.logback.classic.Level.WARN_INT => Level.WARN
case ch.qos.logback.classic.Level.ERROR_INT => Level.ERROR
case _ =>
throw new IllegalArgumentException("Level " + level.levelStr + ", " + level.levelInt + " is unknown.")
}
}
}

View file

@ -0,0 +1,144 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
import java.util.function.Supplier
import scala.concurrent.duration.Duration
import scala.reflect.ClassTag
import scala.util.matching.Regex
import akka.actor.testkit.typed.LoggingEvent
import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.javadsl
import akka.actor.testkit.typed.scaladsl
import akka.actor.typed.ActorSystem
import akka.annotation.InternalApi
import akka.testkit.TestKit
import org.slf4j.event.Level
/**
* INTERNAL API
*/
@InternalApi private[akka] object LoggingEventFilterImpl {
def empty: LoggingEventFilterImpl = new LoggingEventFilterImpl(1, None, None, None, None, None, None, Map.empty, None)
}
/**
* INTERNAL API
*/
@InternalApi private[akka] final case class LoggingEventFilterImpl(
occurrences: Int,
logLevel: Option[Level],
loggerName: Option[String],
source: Option[String],
messageContains: Option[String],
messageRegex: Option[Regex],
cause: Option[Class[_ <: Throwable]],
mdc: Map[String, String],
custom: Option[Function[LoggingEvent, Boolean]])
extends javadsl.LoggingEventFilter
with scaladsl.LoggingEventFilter {
@volatile // JMM does not guarantee visibility for non-final fields
private var todo = occurrences
def matches(event: LoggingEvent): Boolean = {
logLevel.forall(_ == event.level) &&
source.forall(_ == sourceOrEmpty(event)) &&
messageContains.forall(messageOrEmpty(event).contains) &&
messageRegex.forall(_.findFirstIn(messageOrEmpty(event)).isDefined) &&
cause.forall(c => event.throwable.isDefined && c.isInstance(event.throwable.get)) &&
mdc.forall { case (key, value) => event.mdc.contains(key) && event.mdc(key) == value } &&
custom.forall(f => f(event))
// loggerName is handled when installing the filter, in `intercept`
}
private def messageOrEmpty(event: LoggingEvent): String =
if (event.message == null) "" else event.message
private def sourceOrEmpty(event: LoggingEvent): String =
event.mdc.getOrElse("akkaSource", "")
def apply(event: LoggingEvent): Boolean = {
if (matches(event)) {
if (todo != Int.MaxValue) todo -= 1
true
} else false
}
private def awaitDone(max: Duration): Boolean = {
if (todo != Int.MaxValue && todo > 0) TestKit.awaitCond(todo <= 0, max, noThrow = true)
todo == Int.MaxValue || todo == 0
}
override def intercept[T](code: => T)(implicit system: ActorSystem[_]): T = {
val effectiveLoggerName = loggerName.getOrElse("")
checkLogback(system)
TestAppender.setupTestAppender(effectiveLoggerName)
TestAppender.addFilter(effectiveLoggerName, this)
val leeway = TestKitSettings(system).FilterLeeway
try {
val result = code
if (!awaitDone(leeway))
if (todo > 0)
throw new AssertionError(s"Timeout ($leeway) waiting for $todo messages on $this.")
else
throw new AssertionError(s"Received ${-todo} excess messages on $this.")
result
} finally {
todo = occurrences
TestAppender.removeFilter(effectiveLoggerName, this)
}
}
private def checkLogback(system: ActorSystem[_]): Unit = {
if (!system.dynamicAccess.classIsOnClasspath("ch.qos.logback.classic.spi.ILoggingEvent")) {
throw new IllegalStateException("LoggingEventFilter requires logback-classic dependency in classpath.")
}
}
override def withOccurrences(newOccurrences: Int): LoggingEventFilterImpl =
copy(occurrences = newOccurrences)
override def withLogLevel(newLogLevel: Level): LoggingEventFilterImpl =
copy(logLevel = Option(newLogLevel))
def withLoggerName(newLoggerName: String): LoggingEventFilterImpl =
copy(loggerName = Some(newLoggerName))
override def withSource(newSource: String): LoggingEventFilterImpl =
copy(source = Option(newSource))
override def withMessageContains(newMessageContains: String): LoggingEventFilterImpl =
copy(messageContains = Option(newMessageContains))
def withMessageRegex(newMessageRegex: String): LoggingEventFilterImpl =
copy(messageRegex = Option(new Regex(newMessageRegex)))
override def withCause[A <: Throwable: ClassTag]: LoggingEventFilterImpl = {
val causeClass = implicitly[ClassTag[A]].runtimeClass.asInstanceOf[Class[Throwable]]
copy(cause = Option(causeClass))
}
override def withMdc(newMdc: Map[String, String]): LoggingEventFilterImpl =
copy(mdc = newMdc)
override def withMdc(newMdc: java.util.Map[String, String]): javadsl.LoggingEventFilter = {
import akka.util.ccompat.JavaConverters._
withMdc(newMdc.asScala.toMap)
}
override def withCustom(newCustom: Function[LoggingEvent, Boolean]): LoggingEventFilterImpl =
copy(custom = Option(newCustom))
override def withCause(newCause: Class[_ <: Throwable]): javadsl.LoggingEventFilter =
copy(cause = Option(newCause))
override def intercept[T](system: ActorSystem[_], code: Supplier[T]): T =
intercept(code.get())(system)
}

View file

@ -17,9 +17,11 @@ import java.util.concurrent.ThreadLocalRandom.{ current => rnd }
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
import akka.actor.ActorRefProvider import akka.actor.ActorRefProvider
import org.slf4j.Logger import org.slf4j.Logger
import org.slf4j.helpers.{ SubstituteLogger, SubstituteLoggerFactory } import org.slf4j.helpers.MessageFormatter
import org.slf4j.helpers.SubstituteLoggerFactory
/** /**
* INTERNAL API * INTERNAL API
@ -76,8 +78,8 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
override val system = new ActorSystemStub("StubbedActorContext") override val system = new ActorSystemStub("StubbedActorContext")
private var _children = TreeMap.empty[String, BehaviorTestKitImpl[_]] private var _children = TreeMap.empty[String, BehaviorTestKitImpl[_]]
private val childName = Iterator.from(0).map(Helpers.base64(_)) private val childName = Iterator.from(0).map(Helpers.base64(_))
private val loggingAdapter: SubstituteLogger = private val substituteLoggerFactory = new SubstituteLoggerFactory
new SubstituteLoggerFactory().getLogger("StubbedLoggingAdapter").asInstanceOf[SubstituteLogger] private val logger: Logger = substituteLoggerFactory.getLogger("StubbedLogger")
private var unhandled: List[T] = Nil private var unhandled: List[T] = Nil
private[akka] def classicActorContext = private[akka] def classicActorContext =
@ -183,20 +185,36 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
override def toString: String = s"Inbox($self)" override def toString: String = s"Inbox($self)"
override def log: Logger = loggingAdapter override def log: Logger = logger
override def setLoggerClass(clazz: Class[_]): Unit = () // nop as we dont track logger class override def setLoggerName(name: String): Unit = () // nop as we don't track logger
override def setLoggerName(clazz: Class[_]): Unit = () // nop as we don't track logger
/** /**
* The log entries logged through context.log.{debug, info, warn, error} are captured and can be inspected through * The log entries logged through context.log.{debug, info, warn, error} are captured and can be inspected through
* this method. * this method.
*/ */
def logEntries: List[CapturedLogEvent] = ??? def logEntries: List[CapturedLogEvent] = {
import akka.util.ccompat.JavaConverters._
substituteLoggerFactory.getEventQueue
.iterator()
.asScala
.map { evt =>
CapturedLogEvent(
level = evt.getLevel,
message = MessageFormatter.arrayFormat(evt.getMessage, evt.getArgumentArray).getMessage,
cause = Option(evt.getThrowable),
marker = Option(evt.getMarker))
}
.toList
}
/** /**
* Clear the log entries. * Clear the log entries.
*/ */
def clearLog(): Unit = ??? def clearLog(): Unit =
substituteLoggerFactory.getEventQueue.clear()
override private[akka] def onUnhandled(msg: T): Unit = override private[akka] def onUnhandled(msg: T): Unit =
unhandled = msg :: unhandled unhandled = msg :: unhandled

View file

@ -0,0 +1,119 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.internal
import akka.actor.testkit.typed.LoggingEvent
import akka.annotation.InternalApi
import ch.qos.logback.classic.spi.ILoggingEvent
import ch.qos.logback.classic.spi.ThrowableProxy
import ch.qos.logback.core.AppenderBase
/**
* INTERNAL API
*
* The `TestAppender` emits the logging events to the registered [[LoggingEventFilterImpl]], which
* are added and removed to the appender dynamically from tests.
*
* `TestAppender` is currently requiring Logback as SLF4J implementation.
* Similar can probably be implemented with other backends, such as Log4j2.
*/
@InternalApi private[akka] object TestAppender {
import LogbackUtil._
private val TestAppenderName = "AkkaTestAppender"
def setupTestAppender(loggerName: String): Unit = {
val logbackLogger = getLogbackLogger(loggerName)
logbackLogger.getAppender(TestAppenderName) match {
case null =>
val testAppender = new TestAppender
testAppender.setName(TestAppenderName)
testAppender.setContext(logbackLogger.getLoggerContext)
testAppender.start()
logbackLogger.addAppender(testAppender)
case _: TestAppender =>
// ok, already setup
case other =>
throw new IllegalStateException(s"Unexpected $TestAppenderName already added: $other")
}
}
def addFilter(loggerName: String, filter: LoggingEventFilterImpl): Unit =
getTestAppender(loggerName).addTestFilter(filter)
def removeFilter(loggerName: String, filter: LoggingEventFilterImpl): Unit =
getTestAppender(loggerName).removeTestFilter(filter)
private def getTestAppender(loggerName: String): TestAppender = {
val logger = getLogbackLogger(loggerName)
logger.getAppender(TestAppenderName) match {
case testAppender: TestAppender => testAppender
case null =>
throw new IllegalStateException(s"No $TestAppenderName was setup for logger [${logger.getName}]")
case other =>
throw new IllegalStateException(
s"Unexpected $TestAppenderName already added for logger [${logger.getName}]: $other")
}
}
}
/**
* INTERNAL API
*/
@InternalApi private[akka] class TestAppender extends AppenderBase[ILoggingEvent] {
import LogbackUtil._
private var filters: List[LoggingEventFilterImpl] = Nil
// invocations are synchronized via doAppend in AppenderBase
override def append(event: ILoggingEvent): Unit = {
import akka.util.ccompat.JavaConverters._
val throwable = event.getThrowableProxy match {
case p: ThrowableProxy =>
Option(p.getThrowable)
case _ => None
}
val loggingEvent = LoggingEvent(
level = convertLevel(event.getLevel),
message = event.getFormattedMessage,
loggerName = event.getLoggerName,
threadName = event.getThreadName,
timeStamp = event.getTimeStamp,
marker = Option(event.getMarker),
throwable = throwable,
mdc = event.getMDCPropertyMap.asScala.toMap)
filter(loggingEvent)
}
private def filter(event: LoggingEvent): Boolean = {
filters.exists(f =>
try {
f.apply(event)
} catch {
case _: Exception => false
})
}
def addTestFilter(filter: LoggingEventFilterImpl): Unit = synchronized {
filters ::= filter
}
def removeTestFilter(filter: LoggingEventFilterImpl): Unit = synchronized {
@scala.annotation.tailrec
def removeFirst(
list: List[LoggingEventFilterImpl],
zipped: List[LoggingEventFilterImpl] = Nil): List[LoggingEventFilterImpl] =
list match {
case head :: tail if head == filter => tail.reverse_:::(zipped)
case head :: tail => removeFirst(tail, head :: zipped)
case Nil => filters // filter not found, just return original list
}
filters = removeFirst(filters)
}
}

View file

@ -0,0 +1,67 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
import scala.util.control.NonFatal
import akka.actor.testkit.typed.internal.CapturingAppender
import org.junit.rules.TestRule
import org.junit.runner.Description
import org.junit.runners.model.Statement
import org.slf4j.LoggerFactory
/**
* JUnit `TestRule` to make log lines appear only when the test failed.
*
* Use this in test by adding a public field annotated with `@TestRule`:
* {{{
* @Rule public final LogCapturing logCapturing = new LogCapturing();
* }}}
*
* Requires Logback and configuration like the following the logback-test.xml:
*
* {{{
* <appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
*
* <logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
* <appender-ref ref="STDOUT"/>
* </logger>
*
* <root level="DEBUG">
* <appender-ref ref="CapturingAppender"/>
* </root>
* }}}
*/
final class LogCapturing extends TestRule {
// eager access of CapturingAppender to fail fast if misconfigured
private val capturingAppender = CapturingAppender.get("")
private val myLogger = LoggerFactory.getLogger(classOf[LogCapturing])
override def apply(base: Statement, description: Description): Statement = {
new Statement {
override def evaluate(): Unit = {
try {
myLogger.info(s"Logging started for test [${description.getClassName}: ${description.getMethodName}]")
base.evaluate()
myLogger.info(
s"Logging finished for test [${description.getClassName}: ${description.getMethodName}] that was successful")
} catch {
case NonFatal(e) =>
println(
s"--> [${Console.BLUE}${description.getClassName}: ${description.getMethodName}${Console.RESET}] " +
s"Start of log messages of test that failed with ${e.getMessage}")
capturingAppender.flush()
println(
s"<-- [${Console.BLUE}${description.getClassName}: ${description.getMethodName}${Console.RESET}] " +
s"End of log messages of test that failed with ${e.getMessage}")
throw e
} finally {
capturingAppender.clear()
}
}
}
}
}

View file

@ -0,0 +1,194 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl
import java.util.function.Supplier
import akka.actor.testkit.typed.LoggingEvent
import akka.actor.testkit.typed.internal.LoggingEventFilterImpl
import akka.actor.typed.ActorSystem
import akka.annotation.DoNotInherit
import org.slf4j.event.Level
/**
* Facilities for selectively filtering out expected events from logging
* to verify that they were logged.
*
* Requires Logback.
*
* See the static factory methods as starting point for creating `LoggingEventFilter`.
*
* Not for user extension.
*/
@DoNotInherit abstract class LoggingEventFilter {
/**
* Number of events the filter is supposed to match. By default 1.
*/
def withOccurrences(newOccurrences: Int): LoggingEventFilter
/**
* Matching events with the given log level.
*/
def withLogLevel(newLogLevel: Level): LoggingEventFilter
/**
* Matching events with the given logger name or sub-names in the same way
* as configuration loggers are configured in logback.xml.
* By default the root logger is used.
*/
def withLoggerName(newLoggerName: String): LoggingEventFilter
/**
* Matching events that have "akkaSource" MDC value equal to the given value.
* "akkaSource" is typically the actor path.
*/
def withSource(newSource: String): LoggingEventFilter
/**
* Matching events with a message that contains the given value.
*/
def withMessageContains(newMessageContains: String): LoggingEventFilter
/**
* Matching events with a message that matches the given regular expression.
*/
def withMessageRegex(newMessageRegex: String): LoggingEventFilter
/**
* Matching events with an included `throwable` that is a class or subclass of the given
* `Throwable` class.
*/
def withCause(newCause: Class[_ <: Throwable]): LoggingEventFilter
/**
* Matching events with MDC containing all entries of the given `Map`.
* The event MDC may have more entries than the given `Map`.
*/
def withMdc(newMdc: java.util.Map[String, String]): LoggingEventFilter
/**
* Matching events for which the supplied function returns `true`.
*/
def withCustom(newCustom: Function[LoggingEvent, Boolean]): LoggingEventFilter
// this is a Scala Function, ^ but that can be used with lambda from Java
/**
* @return `true` if the event matches the conditions of the filter.
*/
def matches(event: LoggingEvent): Boolean
/**
* Apply this filter while executing the given code block.
* Assert that this filter has matched within the configured `akka.actor.testkit.typed.filter-leeway`
* as often as requested by its `occurrences` parameter specifies.
*
* Care is taken to remove the filter when the block is finished or aborted.
*/
def intercept[T](system: ActorSystem[_], code: Supplier[T]): T
}
/**
* Facilities for selectively matching expected events from logging.
*
* Requires Logback.
*/
object LoggingEventFilter {
/**
* An empty filter that doesn't match any events.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def empty: LoggingEventFilter = LoggingEventFilterImpl.empty
/**
* Create a filter for events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def messageContains(str: String): LoggingEventFilter =
empty.withMessageContains(str)
/**
* Create a filter for TRACE level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def trace(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.TRACE)
/**
* Create a filter for DEBUG level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def debug(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.DEBUG)
/**
* Create a filter for INFO level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def info(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.INFO)
/**
* Create a filter for WARN level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def warn(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.WARN)
/**
* Create a filter for WARN level events with a an included
* `throwable` that is a class or subclass of the given
* * `Throwable` class.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def warn(causeClass: Class[Throwable]): LoggingEventFilter =
empty.withLogLevel(Level.WARN).withCause(causeClass)
/**
* Create a filter for ERROR level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def error(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.ERROR)
/**
* Create a filter for WARN level events with a an included
* `throwable` that is a class or subclass of the given
* * `Throwable` class.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def error(causeClass: Class[_ <: Throwable]): LoggingEventFilter =
empty.withLogLevel(Level.ERROR).withCause(causeClass)
/**
* Create a custom event filter. The filter will match those events for
* which for which the supplied function returns `true`.
*/
def custom(test: Function[LoggingEvent, Boolean]): LoggingEventFilter =
empty.withCustom(test) // this is a Scala Function, but that can be used with lambda from Java
/**
* Filter for the logging of dead letters.
*/
def deadLetters(): LoggingEventFilter =
empty.withLogLevel(Level.INFO).withMessageRegex(".*was not delivered.*dead letters encountered.*")
}

View file

@ -0,0 +1,66 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
import scala.util.control.NonFatal
import akka.actor.testkit.typed.internal.CapturingAppender
import org.scalatest.BeforeAndAfterAll
import org.scalatest.Outcome
import org.scalatest.TestSuite
import org.slf4j.LoggerFactory
/**
* Mixin this trait to a ScalaTest test to make log lines appear only when the test failed.
*
* Requires Logback and configuration like the following the logback-test.xml:
*
* {{{
* <appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
*
* <logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
* <appender-ref ref="STDOUT"/>
* </logger>
*
* <root level="DEBUG">
* <appender-ref ref="CapturingAppender"/>
* </root>
* }}}
*/
trait LogCapturing extends BeforeAndAfterAll { self: TestSuite =>
// eager access of CapturingAppender to fail fast if misconfigured
private val capturingAppender = CapturingAppender.get("")
private val myLogger = LoggerFactory.getLogger(classOf[LogCapturing])
override protected def afterAll(): Unit = {
try {
super.afterAll()
} catch {
case NonFatal(e) =>
myLogger.error("Exception from afterAll", e)
capturingAppender.flush()
} finally {
capturingAppender.clear()
}
}
abstract override def withFixture(test: NoArgTest): Outcome = {
myLogger.info(s"Logging started for test [${self.getClass.getName}: ${test.name}]")
val res = test()
myLogger.info(s"Logging finished for test [${self.getClass.getName}: ${test.name}] that [$res]")
if (!(res.isSucceeded || res.isPending)) {
println(
s"--> [${Console.BLUE}${self.getClass.getName}: ${test.name}${Console.RESET}] Start of log messages of test that [$res]")
capturingAppender.flush()
println(
s"<-- [${Console.BLUE}${self.getClass.getName}: ${test.name}${Console.RESET}] End of log messages of test that [$res]")
}
res
}
}

View file

@ -0,0 +1,193 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
import scala.reflect.ClassTag
import akka.actor.testkit.typed.LoggingEvent
import akka.actor.testkit.typed.internal.LoggingEventFilterImpl
import akka.actor.typed.ActorSystem
import akka.annotation.DoNotInherit
import org.slf4j.event.Level
/**
* Facilities for selectively filtering out expected events from logging
* to verify that they were logged.
*
* Requires Logback.
*
* See the companion object for convenient factory methods.
*
* Not for user extension.
*/
@DoNotInherit trait LoggingEventFilter {
/**
* Number of events the filter is supposed to match. By default 1.
*/
def withOccurrences(newOccurrences: Int): LoggingEventFilter
/**
* Matching events with the given log level.
*/
def withLogLevel(newLogLevel: Level): LoggingEventFilter
/**
* Matching events with the given logger name or sub-names in the same way
* as configuration loggers are configured in logback.xml.
* By default the root logger is used.
*/
def withLoggerName(newLoggerName: String): LoggingEventFilter
/**
* Matching events that have "akkaSource" MDC value equal to the given value.
* "akkaSource" is typically the actor path.
*/
def withSource(newSource: String): LoggingEventFilter
/**
* Matching events with a message that contains the given value.
*/
def withMessageContains(newMessageContains: String): LoggingEventFilter
/**
* Matching events with a message that matches the given regular expression.
*/
def withMessageRegex(newMessageRegex: String): LoggingEventFilter
/**
* Matching events with an included `throwable` that is a class or subclass of the given
* `Throwable` `ClassTag`.
*/
def withCause[A <: Throwable: ClassTag]: LoggingEventFilter
/**
* Matching events with MDC containing all entries of the given `Map`.
* The event MDC may have more entries than the given `Map`.
*/
def withMdc(newMdc: Map[String, String]): LoggingEventFilter
/**
* Matching events for which the supplied function returns`true`.
*/
def withCustom(newCustom: Function[LoggingEvent, Boolean]): LoggingEventFilter
/**
* @return `true` if the event matches the conditions of the filter.
*/
def matches(event: LoggingEvent): Boolean
/**
* Apply this filter while executing the given code block.
* Assert that this filter has matched as often as requested by its
* `occurrences` parameter specifies.
*
* Care is taken to remove the filter when the block is finished or aborted.
*/
def intercept[T](code: => T)(implicit system: ActorSystem[_]): T
}
/**
* Facilities for selectively matching expected events from logging.
*
* Requires Logback.
*/
object LoggingEventFilter {
/**
* An empty filter that doesn't match any events.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def empty: LoggingEventFilter = LoggingEventFilterImpl.empty
/**
* Create a filter for events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def messageContains(str: String): LoggingEventFilter =
empty.withMessageContains(str)
/**
* Create a filter for TRACE level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def trace(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.TRACE)
/**
* Create a filter for DEBUG level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def debug(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.DEBUG)
/**
* Create a filter for INFO level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def info(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.INFO)
/**
* Create a filter for WARN level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def warn(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.WARN)
/**
* Create a filter for WARN level events with a an included
* `throwable` that is a class or subclass of the given
* `Throwable` `ClassTag`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def warn[A <: Throwable: ClassTag]: LoggingEventFilter =
empty.withLogLevel(Level.WARN).withCause[A]
/**
* Create a filter for ERROR level events with a log message
* that contains the given `messageIncludes`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def error(messageIncludes: String): LoggingEventFilter =
messageContains(messageIncludes).withLogLevel(Level.ERROR)
/**
* Create a filter for WARN level events with a an included
* `throwable` that is a class or subclass of the given
* `Throwable` `ClassTag`.
*
* More conditions can be added to the returned [LoggingEventFilter].
*/
def error[A <: Throwable: ClassTag]: LoggingEventFilter =
empty.withLogLevel(Level.ERROR).withCause[A]
/**
* Create a custom event filter. The filter will match those events for
* which the supplied function returns `true`.
*/
def custom(test: Function[LoggingEvent, Boolean]): LoggingEventFilter =
empty.withCustom(test)
/**
* Filter for the logging of dead letters.
*/
def deadLetters(): LoggingEventFilter =
empty.withLogLevel(Level.INFO).withMessageRegex(".*was not delivered.*dead letters encountered.*")
}

View file

@ -7,6 +7,7 @@ package akka.actor.testkit.typed.javadsl;
import akka.Done; import akka.Done;
import akka.actor.typed.javadsl.Behaviors; import akka.actor.typed.javadsl.Behaviors;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -21,6 +22,8 @@ public class ActorTestKitTest extends JUnitSuite {
@ClassRule public static TestKitJunitResource testKit = new TestKitJunitResource(); @ClassRule public static TestKitJunitResource testKit = new TestKitJunitResource();
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void systemNameShouldComeFromTestClassViaJunitResource() { public void systemNameShouldComeFromTestClassViaJunitResource() {
assertEquals("ActorTestKitTest", testKit.system().name()); assertEquals("ActorTestKitTest", testKit.system().name());

View file

@ -11,10 +11,10 @@ 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 org.slf4j.event.Level;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
@ -250,7 +250,7 @@ public class BehaviorTestKitTest extends JUnitSuite {
test.run(new Log(what)); test.run(new Log(what));
final List<CapturedLogEvent> allLogEntries = test.getAllLogEntries(); final List<CapturedLogEvent> allLogEntries = test.getAllLogEntries();
assertEquals(1, allLogEntries.size()); assertEquals(1, allLogEntries.size());
assertEquals(new CapturedLogEvent(Logging.InfoLevel(), what), allLogEntries.get(0)); assertEquals(new CapturedLogEvent(Level.INFO, what), allLogEntries.get(0));
} }
@Test @Test

View file

@ -0,0 +1,84 @@
/*
* Copyright (C) 2018-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.javadsl;
import akka.actor.testkit.typed.LoggingEvent;
import akka.actor.testkit.typed.TestException;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.scalatest.junit.JUnitSuite;
import org.slf4j.event.Level;
import java.util.Collections;
import java.util.Optional;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
public class LoggingEventFilterTest extends JUnitSuite {
@ClassRule public static TestKitJunitResource testKit = new TestKitJunitResource();
@Rule public final LogCapturing logCapturing = new LogCapturing();
private LoggingEvent errorNoCause() {
return LoggingEvent.create(
Level.ERROR,
getClass().getName(),
Thread.currentThread().getName(),
"this is an error",
System.currentTimeMillis(),
Optional.empty(),
Optional.empty(),
Collections.emptyMap());
}
private LoggingEvent errorWithCause(Throwable cause) {
return LoggingEvent.create(
Level.ERROR,
getClass().getName(),
Thread.currentThread().getName(),
"this is an error",
System.currentTimeMillis(),
Optional.empty(),
Optional.of(cause),
Collections.emptyMap());
}
@Test
public void filterErrorsWithMatchingMessage() {
assertTrue(
LoggingEventFilter.error("an error").matches(errorWithCause(new TestException("exc"))));
assertTrue(LoggingEventFilter.error("an error").matches(errorNoCause()));
assertFalse(LoggingEventFilter.error("another error").matches(errorNoCause()));
}
@Test
public void filterErrorsWithMatchingCause() {
assertTrue(
LoggingEventFilter.error(TestException.class)
.matches(errorWithCause(new TestException("exc"))));
assertFalse(
LoggingEventFilter.error(TestException.class)
.matches(errorWithCause(new RuntimeException("exc"))));
assertTrue(
LoggingEventFilter.error("an error")
.withCause(TestException.class)
.matches(errorWithCause(new TestException("exc"))));
assertFalse(
LoggingEventFilter.error("another error")
.withCause(TestException.class)
.matches(errorWithCause(new TestException("exc"))));
}
@Test
public void filterErrorsWithMatchingCustomFunction() {
assertTrue(LoggingEventFilter.custom(event -> true).matches(errorNoCause()));
assertFalse(
LoggingEventFilter.custom(event -> event.getMdc().containsKey("aKey"))
.matches(errorNoCause()));
}
}

View file

@ -11,6 +11,7 @@ import java.util.List;
import akka.actor.testkit.typed.scaladsl.TestProbeSpec; import akka.actor.testkit.typed.scaladsl.TestProbeSpec;
import akka.actor.testkit.typed.scaladsl.TestProbeSpec.*; import akka.actor.testkit.typed.scaladsl.TestProbeSpec.*;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -20,6 +21,8 @@ public class TestProbeTest extends JUnitSuite {
@ClassRule public static TestKitJunitResource testKit = new TestKitJunitResource(); @ClassRule public static TestKitJunitResource testKit = new TestKitJunitResource();
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void testReceiveMessage() { public void testReceiveMessage() {
TestProbe<EventT> probe = TestProbe.create(testKit.system()); TestProbe<EventT> probe = TestProbe.create(testKit.system());

View file

@ -9,16 +9,20 @@ import static jdocs.akka.actor.testkit.typed.javadsl.AsyncTestingExampleTest.Pon
import static jdocs.akka.actor.testkit.typed.javadsl.AsyncTestingExampleTest.echoActor; import static jdocs.akka.actor.testkit.typed.javadsl.AsyncTestingExampleTest.echoActor;
// #junit-integration // #junit-integration
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
public class JunitIntegrationExampleTest { public class JunitIntegrationExampleTest {
@ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource(); @ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource();
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void testSomething() { public void testSomething() {
ActorRef<Ping> pinger = testKit.spawn(echoActor(), "ping"); ActorRef<Ping> pinger = testKit.spawn(echoActor(), "ping");

View file

@ -6,10 +6,12 @@ package jdocs.akka.actor.testkit.typed.javadsl;
// #manual-scheduling-simple // #manual-scheduling-simple
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
import akka.actor.testkit.typed.javadsl.ManualTime; import akka.actor.testkit.typed.javadsl.ManualTime;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
import java.time.Duration; import java.time.Duration;
@ -24,6 +26,8 @@ public class ManualTimerExampleTest extends JUnitSuite {
@ClassRule @ClassRule
public static final TestKitJunitResource testKit = new TestKitJunitResource(ManualTime.config()); public static final TestKitJunitResource testKit = new TestKitJunitResource(ManualTime.config());
@Rule public final LogCapturing logCapturing = new LogCapturing();
private final ManualTime manualTime = ManualTime.get(testKit.system()); private final ManualTime manualTime = ManualTime.get(testKit.system());
static final class Tick {} static final class Tick {}

View file

@ -11,15 +11,14 @@ 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.HashMap;
import java.util.List; import java.util.List;
import java.util.Optional; import java.util.Optional;
import org.slf4j.event.Level;
// #imports // #imports
import org.junit.Test; import org.junit.Test;
import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertEquals;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
import org.slf4j.event.LoggingEvent;
public class SyncTestingExampleTest extends JUnitSuite { public class SyncTestingExampleTest extends JUnitSuite {
@ -173,7 +172,7 @@ public class SyncTestingExampleTest extends JUnitSuite {
assertEquals(1, allLogEntries.size()); assertEquals(1, allLogEntries.size());
CapturedLogEvent expectedLogEvent = CapturedLogEvent expectedLogEvent =
new CapturedLogEvent( new CapturedLogEvent(
Logging.InfoLevel(), Level.INFO,
"Saying hello to Inboxer", "Saying hello to Inboxer",
Optional.empty(), Optional.empty(),
Optional.empty(), Optional.empty(),

View file

@ -0,0 +1,35 @@
<?xml version="1.0" encoding="UTF-8"?>
<configuration>
<!-- Silence initial setup logging from Logback -->
<statusListener class="ch.qos.logback.core.status.NopStatusListener" />
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
<filter class="ch.qos.logback.classic.filter.LevelFilter">
<level>DEBUG</level>
<onMatch>DENY</onMatch>
</filter>
<encoder>
<pattern>%date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n</pattern>
</encoder>
</appender>
<!--
Logging from tests are silenced by this appender. When there is a test failure
the captured logging events are flushed to the appenders defined for the
akka.actor.testkit.typed.internal.CapturingAppenderDelegate logger.
-->
<appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
<!--
The appenders defined for this CapturingAppenderDelegate logger are used
when there is a test failure and all logging events from the test are
flushed to these appenders.
-->
<logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
<appender-ref ref="STDOUT"/>
</logger>
<root level="DEBUG">
<appender-ref ref="CapturingAppender"/>
</root>
</configuration>

View file

@ -14,7 +14,7 @@ import org.scalatest.Matchers
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class ActorTestKitSpec extends ScalaTestWithActorTestKit with WordSpecLike { class ActorTestKitSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
"the Scala testkit" should { "the Scala testkit" should {
@ -91,7 +91,7 @@ class ActorTestKitSpec extends ScalaTestWithActorTestKit with WordSpecLike {
} }
// derivative classes should also work fine (esp the naming part // derivative classes should also work fine (esp the naming part
abstract class MyBaseSpec extends ScalaTestWithActorTestKit with Matchers with WordSpecLike abstract class MyBaseSpec extends ScalaTestWithActorTestKit with Matchers with WordSpecLike with LogCapturing
class MyConcreteDerivateSpec extends MyBaseSpec { class MyConcreteDerivateSpec extends MyBaseSpec {
"A derivative test" should { "A derivative test" should {
@ -116,7 +116,7 @@ class MyConcreteDerivateSpec extends MyBaseSpec {
} }
class CompositionSpec extends WordSpec with Matchers with BeforeAndAfterAll { class CompositionSpec extends WordSpec with Matchers with BeforeAndAfterAll with LogCapturing {
val testKit = ActorTestKit() val testKit = ActorTestKit()
override def afterAll(): Unit = { override def afterAll(): Unit = {

View file

@ -11,11 +11,11 @@ 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
import org.slf4j.event.Level
object BehaviorTestKitSpec { object BehaviorTestKitSpec {
object Father { object Father {
@ -121,7 +121,7 @@ object BehaviorTestKitSpec {
} }
class BehaviorTestKitSpec extends WordSpec with Matchers { class BehaviorTestKitSpec extends WordSpec with Matchers with LogCapturing {
private val props = Props.empty.withDispatcherFromConfig("cat") private val props = Props.empty.withDispatcherFromConfig("cat")
@ -183,14 +183,14 @@ class BehaviorTestKitSpec extends WordSpec with Matchers {
val what = "Hello!" val what = "Hello!"
val testkit = BehaviorTestKit[Father.Command](Father.init) val testkit = BehaviorTestKit[Father.Command](Father.init)
testkit.run(Log(what)) testkit.run(Log(what))
testkit.logEntries() shouldBe Seq(CapturedLogEvent(Logging.InfoLevel, what)) testkit.logEntries() shouldBe Seq(CapturedLogEvent(Level.INFO, what))
} }
"allow clearing log messages issued by behavior" in { "allow clearing log messages issued by behavior" in {
val what = "Hello!" val what = "Hi!"
val testkit = BehaviorTestKit[Father.Command](Father.init) val testkit = BehaviorTestKit[Father.Command](Father.init)
testkit.run(Log(what)) testkit.run(Log(what))
testkit.logEntries() shouldBe Seq(CapturedLogEvent(Logging.InfoLevel, what)) testkit.logEntries() shouldBe Seq(CapturedLogEvent(Level.INFO, what))
testkit.clearLog() testkit.clearLog()
testkit.logEntries() shouldBe Seq.empty testkit.logEntries() shouldBe Seq.empty
} }

View file

@ -0,0 +1,141 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
import akka.actor.testkit.typed.LoggingEvent
import org.scalatest.WordSpecLike
import org.slf4j.event.Level
class LoggingEventFilterSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
private class AnError extends Exception
private def errorNoCause =
LoggingEvent(
level = Level.ERROR,
loggerName = getClass.getName,
message = "this is an error",
threadName = Thread.currentThread().getName,
timeStamp = System.currentTimeMillis())
private def errorWithCause(cause: Throwable) =
LoggingEvent(
level = Level.ERROR,
loggerName = getClass.getName,
message = "this is an error",
threadName = Thread.currentThread().getName,
timeStamp = System.currentTimeMillis(),
marker = None,
throwable = Option(cause),
mdc = Map.empty)
private def warningNoCause =
LoggingEvent(
level = Level.WARN,
loggerName = getClass.getName,
message = "this is a warning",
threadName = Thread.currentThread().getName,
timeStamp = System.currentTimeMillis())
private def warningWithCause(cause: Throwable) =
LoggingEvent(
level = Level.WARN,
loggerName = getClass.getName,
message = "this is a warning",
threadName = Thread.currentThread().getName,
timeStamp = System.currentTimeMillis(),
marker = None,
throwable = Option(cause),
mdc = Map.empty)
private def warningWithSource(source: String) =
LoggingEvent(
level = Level.WARN,
loggerName = getClass.getName,
message = "this is a warning",
threadName = Thread.currentThread().getName,
timeStamp = System.currentTimeMillis(),
marker = None,
throwable = None,
mdc = Map("akkaSource" -> source))
"The LoggingEventFilter.error" must {
"filter errors without cause" in {
val filter = LoggingEventFilter.empty.withLogLevel(Level.ERROR)
filter.matches(errorNoCause) should ===(true)
}
"filter errors with cause" in {
val filter = LoggingEventFilter.empty.withLogLevel(Level.ERROR)
filter.matches(errorWithCause(new AnError)) should ===(true)
}
"filter error with matching message" in {
LoggingEventFilter.error("an error").matches(errorWithCause(new AnError)) should ===(true)
LoggingEventFilter.error("an error").matches(errorNoCause) should ===(true)
LoggingEventFilter.error("another error").matches(errorNoCause) should ===(false)
}
"filter with matching MDC" in {
LoggingEventFilter.empty.withMdc(Map("a" -> "A")).matches(errorNoCause.copy(mdc = Map("a" -> "A"))) should ===(
true)
LoggingEventFilter.empty
.withMdc(Map("a" -> "A", "b" -> "B"))
.matches(errorNoCause.copy(mdc = Map("a" -> "A", "b" -> "B"))) should ===(true)
LoggingEventFilter.empty
.withMdc(Map("a" -> "A"))
.matches(errorNoCause.copy(mdc = Map("a" -> "A", "b" -> "B"))) should ===(true)
LoggingEventFilter.empty
.withMdc(Map("a" -> "A", "b" -> "B"))
.matches(errorNoCause.copy(mdc = Map("a" -> "A"))) should ===(false)
LoggingEventFilter.empty.withMdc(Map("a" -> "A", "b" -> "B")).matches(errorNoCause) should ===(false)
}
}
"The LoggingEventFilter with cause" must {
"not filter errors without cause" in {
val filter = LoggingEventFilter.error[AnError]
filter.matches(errorNoCause) should ===(false)
}
"not filter errors with an unrelated cause" in {
object AnotherError extends Exception
val filter = LoggingEventFilter.error[AnError]
filter.matches(errorWithCause(AnotherError)) should ===(false)
}
"filter errors with a matching cause" in {
val filter = LoggingEventFilter.error[AnError]
filter.matches(errorWithCause(new AnError)) should ===(true)
}
"filter errors with a matching cause and message" in {
val filter = LoggingEventFilter.error("this is an error").withCause[AnError]
filter.matches(errorWithCause(new AnError)) should ===(true)
}
}
"The LoggingEventFilter.warn" must {
"filter warnings without cause" in {
val filter = LoggingEventFilter.empty.withLogLevel(Level.WARN)
filter.matches(warningNoCause) should ===(true)
}
"filter warning with cause" in {
val filter = LoggingEventFilter.empty.withLogLevel(Level.WARN)
filter.matches(warningWithCause(new AnError)) should ===(true)
}
"filter warning with matching message" in {
LoggingEventFilter.warn("this is a warning").matches(warningWithCause(new AnError)) should ===(true)
LoggingEventFilter.warn("this is another warning").matches(warningWithCause(new AnError)) should ===(false)
}
"filter warning with matching source" in {
val source = "akka://Sys/user/foo"
LoggingEventFilter.empty
.withLogLevel(Level.WARN)
.withSource(source)
.matches(warningWithSource(source)) should ===(true)
LoggingEventFilter.empty
.withLogLevel(Level.WARN)
.withSource("akka://Sys/user/bar")
.matches(warningWithSource(source)) should ===(false)
}
}
}

View file

@ -0,0 +1,84 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.testkit.typed.scaladsl
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.testkit.typed.TestException
import org.scalatest.WordSpecLike
import org.slf4j.LoggerFactory
class TestAppenderSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
class AnotherLoggerClass
private val log = LoggerFactory.getLogger(getClass)
"TestAppender and LoggingEventFilter" must {
"filter errors without cause" in {
LoggingEventFilter.error("an error").withOccurrences(2).intercept {
log.error("an error")
log.error("an error")
}
}
"filter errors with cause" in {
LoggingEventFilter.error("err").withCause[TestException].intercept {
log.error("err", TestException("an error"))
}
}
"filter warnings" in {
LoggingEventFilter.warn("a warning").withOccurrences(2).intercept {
log.error("an error")
log.warn("a warning")
log.error("an error")
log.warn("a warning")
}
}
"only filter events for given logger name" in {
val count = new AtomicInteger
LoggingEventFilter
.custom({
case logEvent =>
count.incrementAndGet()
logEvent.message == "Hello from right logger" && logEvent.loggerName == classOf[AnotherLoggerClass].getName
})
.withOccurrences(2)
.withLoggerName(classOf[AnotherLoggerClass].getName)
.intercept {
LoggerFactory.getLogger(classOf[AnotherLoggerClass]).info("Hello from right logger")
log.info("Hello wrong logger")
LoggerFactory.getLogger(classOf[AnotherLoggerClass]).info("Hello from right logger")
}
count.get should ===(2)
}
"find unexpected events withOccurences(0)" in {
LoggingEventFilter.warn("a warning").withOccurrences(0).intercept {
log.error("an error")
log.warn("another warning")
}
intercept[AssertionError] {
LoggingEventFilter.warn("a warning").withOccurrences(0).intercept {
log.error("an error")
log.warn("a warning")
log.warn("another warning")
}
}.getMessage should include("Received 1 excess messages")
intercept[AssertionError] {
LoggingEventFilter.warn("a warning").withOccurrences(0).intercept {
log.warn("a warning")
log.warn("a warning")
}
}.getMessage should include("Received 2 excess messages")
}
}
}

View file

@ -10,7 +10,7 @@ import com.typesafe.config.ConfigFactory
import scala.concurrent.duration._ import scala.concurrent.duration._
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class TestProbeSpec extends ScalaTestWithActorTestKit with WordSpecLike { class TestProbeSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import TestProbeSpec._ import TestProbeSpec._
@ -181,7 +181,10 @@ object TestProbeSpec {
for (n <- 1 to expected) yield EventT(n) for (n <- 1 to expected) yield EventT(n)
} }
class TestProbeTimeoutSpec extends ScalaTestWithActorTestKit(TestProbeSpec.timeoutConfig) with WordSpecLike { class TestProbeTimeoutSpec
extends ScalaTestWithActorTestKit(TestProbeSpec.timeoutConfig)
with WordSpecLike
with LogCapturing {
import TestProbeSpec._ import TestProbeSpec._

View file

@ -4,6 +4,7 @@
package docs.akka.actor.testkit.typed.scaladsl package docs.akka.actor.testkit.typed.scaladsl
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.Scheduler import akka.actor.typed.Scheduler
//#test-header //#test-header
import akka.actor.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
@ -58,7 +59,7 @@ object AsyncTestingExampleSpec {
} }
//#test-header //#test-header
class AsyncTestingExampleSpec extends WordSpec with BeforeAndAfterAll with Matchers { class AsyncTestingExampleSpec extends WordSpec with BeforeAndAfterAll with Matchers with LogCapturing {
val testKit = ActorTestKit() val testKit = ActorTestKit()
//#test-header //#test-header

View file

@ -6,13 +6,15 @@ package docs.akka.actor.testkit.typed.scaladsl
//#manual-scheduling-simple //#manual-scheduling-simple
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.ManualTime import akka.actor.testkit.typed.scaladsl.ManualTime
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class ManualTimerExampleSpec extends ScalaTestWithActorTestKit(ManualTime.config) with WordSpecLike { class ManualTimerExampleSpec extends ScalaTestWithActorTestKit(ManualTime.config) with WordSpecLike with LogCapturing {
val manualTime: ManualTime = ManualTime() val manualTime: ManualTime = ManualTime()

View file

@ -11,7 +11,7 @@ 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 import org.slf4j.event.Level
//#imports //#imports
import org.scalatest.Matchers import org.scalatest.Matchers
import org.scalatest.WordSpec import org.scalatest.WordSpec
@ -115,7 +115,7 @@ class SyncTestingExampleSpec extends WordSpec with Matchers {
val testKit = BehaviorTestKit(myBehavior) val testKit = BehaviorTestKit(myBehavior)
val inbox = TestInbox[String]("Inboxer") val inbox = TestInbox[String]("Inboxer")
testKit.run(LogAndSayHello(inbox.ref)) testKit.run(LogAndSayHello(inbox.ref))
testKit.logEntries() shouldBe Seq(CapturedLogEvent(Logging.InfoLevel, "Saying hello to Inboxer")) testKit.logEntries() shouldBe Seq(CapturedLogEvent(Level.INFO, "Saying hello to Inboxer"))
//#test-check-logging //#test-check-logging
} }
} }

View file

@ -12,9 +12,10 @@ import akka.event.Logging.DefaultLogger
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.Assertions import org.scalatest.Assertions
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.ExtendedActorSystem
class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.findClassLoader())) with Assertions { class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.findClassLoader())) with Assertions {
"The default configuration file (i.e. reference.conf)" must { "The default configuration file (i.e. reference.conf)" must {
@ -164,4 +165,24 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin
} }
} }
} }
"SLF4J Settings" must {
"not be amended for default reference in akka-actor" in {
val dynamicAccess = system.asInstanceOf[ExtendedActorSystem].dynamicAccess
val config = ActorSystem.Settings.amendSlf4jConfig(ConfigFactory.defaultReference(), dynamicAccess)
config.getStringList("akka.loggers").size() should ===(1)
config.getStringList("akka.loggers").get(0) should ===(classOf[DefaultLogger].getName)
config.getString("akka.logging-filter") should ===(classOf[DefaultLoggingFilter].getName)
}
"not be amended when akka-slf4j is not in classpath" in {
val dynamicAccess = system.asInstanceOf[ExtendedActorSystem].dynamicAccess
val config = ActorSystem.Settings.amendSlf4jConfig(
ConfigFactory.parseString("akka.use-slf4j = on").withFallback(ConfigFactory.defaultReference()),
dynamicAccess)
config.getStringList("akka.loggers").size() should ===(1)
config.getStringList("akka.loggers").get(0) should ===(classOf[DefaultLogger].getName)
config.getString("akka.logging-filter") should ===(classOf[DefaultLoggingFilter].getName)
}
}
} }

View file

@ -4,6 +4,7 @@
package akka.actor.typed.javadsl; package akka.actor.typed.javadsl;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
@ -11,6 +12,7 @@ import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.util.Timeout; import akka.util.Timeout;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -22,6 +24,8 @@ public class ActorContextAskTest extends JUnitSuite {
@ClassRule @ClassRule
public static final TestKitJunitResource testKit = new TestKitJunitResource(AkkaSpec.testConf()); public static final TestKitJunitResource testKit = new TestKitJunitResource(AkkaSpec.testConf());
@Rule public final LogCapturing logCapturing = new LogCapturing();
static class Ping { static class Ping {
final ActorRef<Pong> respondTo; final ActorRef<Pong> respondTo;

View file

@ -4,12 +4,14 @@
package akka.actor.typed.javadsl; package akka.actor.typed.javadsl;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
import akka.actor.typed.Props; import akka.actor.typed.Props;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -28,6 +30,8 @@ public final class ActorContextPipeToSelfTest extends JUnitSuite {
"pipe-to-self-spec-dispatcher.executor = thread-pool-executor\n" "pipe-to-self-spec-dispatcher.executor = thread-pool-executor\n"
+ "pipe-to-self-spec-dispatcher.type = PinnedDispatcher\n")); + "pipe-to-self-spec-dispatcher.type = PinnedDispatcher\n"));
@Rule public final LogCapturing logCapturing = new LogCapturing();
static final class Msg { static final class Msg {
final String response; final String response;
final String selfName; final String selfName;

View file

@ -4,6 +4,7 @@
package akka.actor.typed.javadsl; package akka.actor.typed.javadsl;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior; import akka.actor.typed.Behavior;
@ -12,6 +13,7 @@ import akka.japi.pf.PFBuilder;
import akka.testkit.CustomEventFilter; import akka.testkit.CustomEventFilter;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
import scala.concurrent.duration.FiniteDuration; import scala.concurrent.duration.FiniteDuration;
@ -44,6 +46,8 @@ public class ActorLoggingTest extends JUnitSuite {
} }
} }
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void loggingProvidesClassWhereLogWasCalled() { public void loggingProvidesClassWhereLogWasCalled() {
CustomEventFilter eventFilter = CustomEventFilter eventFilter =
@ -72,7 +76,7 @@ public class ActorLoggingTest extends JUnitSuite {
Behaviors.setup( Behaviors.setup(
context -> context ->
Behaviors.withMdc( Behaviors.withMdc(
null, Protocol.class,
(message) -> { (message) -> {
Map<String, String> mdc = new HashMap<>(); Map<String, String> mdc = new HashMap<>();
mdc.put("txId", message.getTransactionId()); mdc.put("txId", message.getTransactionId());

View file

@ -4,7 +4,9 @@
package akka.actor.typed.javadsl; package akka.actor.typed.javadsl;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -240,6 +242,8 @@ public class AdapterTest extends JUnitSuite {
public static AkkaJUnitActorSystemResource actorSystemResource = public static AkkaJUnitActorSystemResource actorSystemResource =
new AkkaJUnitActorSystemResource("ActorSelectionTest", AkkaSpec.testConf()); new AkkaJUnitActorSystemResource("ActorSelectionTest", AkkaSpec.testConf());
@Rule public final LogCapturing logCapturing = new LogCapturing();
private final ActorSystem system = actorSystemResource.getSystem(); private final ActorSystem system = actorSystemResource.getSystem();
@Test @Test

View file

@ -4,9 +4,11 @@
package akka.actor.typed.javadsl; package akka.actor.typed.javadsl;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -25,6 +27,8 @@ public class BehaviorBuilderTest extends JUnitSuite {
@ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource(); @ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource();
@Rule public final LogCapturing logCapturing = new LogCapturing();
interface Message {} interface Message {}
static final class One implements Message { static final class One implements Message {

View file

@ -4,11 +4,13 @@
package akka.actor.typed.javadsl; package akka.actor.typed.javadsl;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.actor.typed.*; import akka.actor.typed.*;
import akka.testkit.AkkaSpec; import akka.testkit.AkkaSpec;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -17,6 +19,8 @@ public class InterceptTest extends JUnitSuite {
@ClassRule @ClassRule
public static final TestKitJunitResource testKit = new TestKitJunitResource(AkkaSpec.testConf()); public static final TestKitJunitResource testKit = new TestKitJunitResource(AkkaSpec.testConf());
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void interceptMessage() { public void interceptMessage() {
final TestProbe<String> interceptProbe = testKit.createTestProbe(); final TestProbe<String> interceptProbe = testKit.createTestProbe();

View file

@ -4,10 +4,12 @@
package akka.actor.typed.javadsl; package akka.actor.typed.javadsl;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -21,6 +23,8 @@ public class ReceiveBuilderTest extends JUnitSuite {
@ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource(); @ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource();
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void testMutableCounter() { public void testMutableCounter() {
Behavior<BehaviorBuilderTest.CounterMessage> mutable = Behavior<BehaviorBuilderTest.CounterMessage> mutable =

View file

@ -8,8 +8,10 @@ import java.util.concurrent.CompletionStage;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import akka.Done; import akka.Done;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
import org.junit.Test; import org.junit.Test;
@ -24,6 +26,8 @@ public class WatchTest extends JUnitSuite {
@ClassRule public static TestKitJunitResource testKit = new TestKitJunitResource(); @ClassRule public static TestKitJunitResource testKit = new TestKitJunitResource();
@Rule public final LogCapturing logCapturing = new LogCapturing();
interface Message {} interface Message {}
static final class RunTest implements Message { static final class RunTest implements Message {

View file

@ -4,10 +4,12 @@
package jdocs.akka.typed; package jdocs.akka.typed;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -22,6 +24,8 @@ public class BubblingSampleTest extends JUnitSuite {
public static final TestKitJunitResource testKit = public static final TestKitJunitResource testKit =
new TestKitJunitResource("akka.loglevel = off"); new TestKitJunitResource("akka.loglevel = off");
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void testBubblingSample() throws Exception { public void testBubblingSample() throws Exception {
ActorRef<Protocol.Command> boss = testKit.spawn(Boss.create(), "upper-management"); ActorRef<Protocol.Command> boss = testKit.spawn(Boss.create(), "upper-management");

View file

@ -4,6 +4,7 @@
package jdocs.akka.typed; package jdocs.akka.typed;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import akka.actor.typed.ActorSystem; import akka.actor.typed.ActorSystem;
@ -11,6 +12,7 @@ import akka.actor.typed.Behavior;
import akka.actor.typed.javadsl.*; import akka.actor.typed.javadsl.*;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -647,6 +649,8 @@ public class InteractionPatternsTest extends JUnitSuite {
@ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource(); @ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource();
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void fireAndForgetSample() throws Exception { public void fireAndForgetSample() throws Exception {
// #fire-and-forget-doit // #fire-and-forget-doit

View file

@ -5,6 +5,7 @@
package jdocs.akka.typed; package jdocs.akka.typed;
import akka.Done; import akka.Done;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
@ -13,6 +14,7 @@ import akka.actor.typed.MailboxSelector;
import akka.actor.typed.javadsl.Behaviors; import akka.actor.typed.javadsl.Behaviors;
import com.typesafe.config.ConfigFactory; import com.typesafe.config.ConfigFactory;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -22,6 +24,8 @@ public class MailboxDocTest extends JUnitSuite {
public static final TestKitJunitResource testKit = public static final TestKitJunitResource testKit =
new TestKitJunitResource(ConfigFactory.load("mailbox-config-sample.conf")); new TestKitJunitResource(ConfigFactory.load("mailbox-config-sample.conf"));
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void startSomeActorsWithDifferentMailboxes() { public void startSomeActorsWithDifferentMailboxes() {
TestProbe<Done> testProbe = testKit.createTestProbe(); TestProbe<Done> testProbe = testKit.createTestProbe();

View file

@ -5,10 +5,12 @@
package jdocs.akka.typed; package jdocs.akka.typed;
import akka.Done; import akka.Done;
import akka.actor.testkit.typed.javadsl.LogCapturing;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource; import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.testkit.typed.javadsl.TestProbe; import akka.actor.testkit.typed.javadsl.TestProbe;
import akka.actor.typed.ActorRef; import akka.actor.typed.ActorRef;
import org.junit.ClassRule; import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.scalatest.junit.JUnitSuite; import org.scalatest.junit.JUnitSuite;
@ -22,6 +24,8 @@ public class StashDocTest extends JUnitSuite {
@ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource(); @ClassRule public static final TestKitJunitResource testKit = new TestKitJunitResource();
@Rule public final LogCapturing logCapturing = new LogCapturing();
@Test @Test
public void stashingExample() throws Exception { public void stashingExample() throws Exception {
final DB db = final DB db =

View file

@ -1,27 +1,35 @@
<?xml version="1.0" encoding="UTF-8"?> <?xml version="1.0" encoding="UTF-8"?>
<configuration> <configuration>
<!-- Silence initial setup logging from Logback -->
<statusListener class="ch.qos.logback.core.status.NopStatusListener" />
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender"> <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
<filter class="ch.qos.logback.classic.filter.LevelFilter">
<level>DEBUG</level>
<onMatch>DENY</onMatch>
</filter>
<encoder> <encoder>
<pattern>%date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n</pattern> <pattern>%date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n</pattern>
</encoder> </encoder>
</appender> </appender>
<appender name="INTERCEPTOR" class="akka.actor.typed.testkit.TestAppender">
<encoder> <!--
<pattern>%date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n</pattern> Logging from tests are silenced by this appender. When there is a test failure
</encoder> the captured logging events are flushed to the appenders defined for the
</appender> akka.actor.testkit.typed.internal.CapturingAppenderDelegate logger.
<appender name="FILE" class="ch.qos.logback.core.FileAppender"> -->
<file>log-${byDay}.txt</file> <appender name="CapturingAppender" class="akka.actor.testkit.typed.internal.CapturingAppender" />
<append>true</append>
<encoder> <!--
<pattern>%-4relative [%thread] %-5level %logger{35} - %msg%n</pattern> The appenders defined for this CapturingAppenderDelegate logger are used
</encoder> when there is a test failure and all logging events from the test are
</appender> flushed to these appenders.
<logger name="akka.actor.typed.scaladsl.ActorLoggingSpec" level="DEBUG"> -->
<appender-ref ref="INTERCEPTOR"/> <logger name="akka.actor.testkit.typed.internal.CapturingAppenderDelegate" >
</logger>
<root level="DEBUG">
<appender-ref ref="FILE"/>
<appender-ref ref="STDOUT"/> <appender-ref ref="STDOUT"/>
</logger>
<root level="DEBUG">
<appender-ref ref="CapturingAppender"/>
</root> </root>
</configuration> </configuration>

View file

@ -7,12 +7,15 @@ package akka.actor.typed
import akka.actor.InvalidMessageException import akka.actor.InvalidMessageException
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.reflect.ClassTag import scala.reflect.ClassTag
import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.TestException import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.testkit.EventFilter import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.eventstream.EventStream
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
object ActorSpecMessages { object ActorSpecMessages {
@ -63,13 +66,7 @@ object ActorSpecMessages {
} }
abstract class ActorContextSpec extends ScalaTestWithActorTestKit(""" abstract class ActorContextSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
akka.loggers = [akka.testkit.TestEventListener]
""") with WordSpecLike {
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val classicSystem = system.toClassic
import ActorSpecMessages._ import ActorSpecMessages._
@ -89,6 +86,8 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit("""
} }
"canonicalize behaviors" in { "canonicalize behaviors" in {
val unhandledProbe = createTestProbe[UnhandledMessage]()
system.eventStream ! EventStream.Subscribe(unhandledProbe.ref)
val probe = TestProbe[Event]() val probe = TestProbe[Event]()
lazy val behavior: Behavior[Command] = Behaviors lazy val behavior: Behavior[Command] = Behaviors
@ -112,11 +111,11 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit("""
val actor = spawn(behavior) val actor = spawn(behavior)
actor ! Ping actor ! Ping
probe.expectMessage(Pong) probe.expectMessage(Pong)
// unhandled gives warning from EventFilter
EventFilter.warning(occurrences = 1).intercept {
actor ! Miss actor ! Miss
probe.expectMessage(Missed) probe.expectMessage(Missed)
} unhandledProbe.receiveMessage()
actor ! Renew(probe.ref) actor ! Renew(probe.ref)
probe.expectMessage(Renewed) probe.expectMessage(Renewed)
actor ! Ping actor ! Ping
@ -140,7 +139,7 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit("""
val behavior = Behaviors.supervise(internal).onFailure(SupervisorStrategy.restart) val behavior = Behaviors.supervise(internal).onFailure(SupervisorStrategy.restart)
val actor = spawn(behavior) val actor = spawn(behavior)
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
actor ! Fail actor ! Fail
} }
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
@ -204,7 +203,7 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit("""
val parentRef = spawn(parent) val parentRef = spawn(parent)
val childRef = probe.expectMessageType[ChildMade].ref val childRef = probe.expectMessageType[ChildMade].ref
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
childRef ! Fail childRef ! Fail
} }
probe.expectMessage(GotChildSignal(PreRestart)) probe.expectMessage(GotChildSignal(PreRestart))
@ -261,7 +260,7 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit("""
val actor = spawn(behavior) val actor = spawn(behavior)
actor ! Ping actor ! Ping
probe.expectMessage(1) probe.expectMessage(1)
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
actor ! Fail actor ! Fail
} }
actor ! Ping actor ! Ping
@ -287,7 +286,7 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit("""
val actor = spawn(behavior) val actor = spawn(behavior)
actor ! Ping actor ! Ping
probe.expectMessage(1) probe.expectMessage(1)
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
actor ! Fail actor ! Fail
} }
actor ! Ping actor ! Ping
@ -329,7 +328,7 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage(Pong) probe.expectMessage(Pong)
watcher ! Ping watcher ! Ping
probe.expectMessage(Pong) probe.expectMessage(Pong)
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
actorToWatch ! Fail actorToWatch ! Fail
} }
probe.expectMessage(ReceivedSignal(PostStop)) probe.expectMessage(ReceivedSignal(PostStop))
@ -514,7 +513,7 @@ abstract class ActorContextSpec extends ScalaTestWithActorTestKit("""
val childRef = probe.expectMessageType[ChildMade].ref val childRef = probe.expectMessageType[ChildMade].ref
actor ! Inert actor ! Inert
probe.expectMessage(InertEvent) probe.expectMessage(InertEvent)
EventFilter[DeathPactException](occurrences = 1).intercept { LoggingEventFilter.error[DeathPactException].intercept {
childRef ! Stop childRef ! Stop
probe.expectMessage(GotChildSignal(PostStop)) probe.expectMessage(GotChildSignal(PostStop))
probe.expectMessage(ReceivedSignal(PostStop)) probe.expectMessage(ReceivedSignal(PostStop))

View file

@ -8,36 +8,32 @@ import akka.actor.typed.internal.adapter.ActorSystemAdapter
import akka.actor.typed.scaladsl.AskPattern._ import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.Behaviors._ import akka.actor.typed.scaladsl.Behaviors._
import akka.testkit.EventFilter
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.util.Timeout import akka.util.Timeout
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.concurrent.{ ExecutionContext, TimeoutException } import scala.concurrent.{ ExecutionContext, TimeoutException }
import scala.util.Success import scala.util.Success
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import scala.concurrent.Future import scala.concurrent.Future
import akka.actor.DeadLetter
import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.eventstream.EventStream
object AskSpec { object AskSpec {
sealed trait Msg sealed trait Msg
final case class Foo(s: String, replyTo: ActorRef[String]) extends Msg final case class Foo(s: String, replyTo: ActorRef[String]) extends Msg
final case class Stop(replyTo: ActorRef[Unit]) extends Msg final case class Stop(replyTo: ActorRef[Unit]) extends Msg
} }
class AskSpec extends ScalaTestWithActorTestKit(""" class AskSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
akka.loglevel=warning
akka.loggers = [ akka.testkit.TestEventListener ]
""") with WordSpecLike {
// FIXME #24348: eventfilter support in typed testkit
import AskSpec._ import AskSpec._
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val classicSystem = system.toClassic
implicit def executor: ExecutionContext = implicit def executor: ExecutionContext =
system.executionContext system.executionContext
@ -58,10 +54,7 @@ class AskSpec extends ScalaTestWithActorTestKit("""
val probe = createTestProbe() val probe = createTestProbe()
probe.expectTerminated(ref, probe.remainingOrDefault) probe.expectTerminated(ref, probe.remainingOrDefault)
val answer: Future[String] = val answer: Future[String] = ref.ask(Foo("bar", _))
EventFilter.warning(pattern = ".*received dead letter.*", occurrences = 1).intercept {
ref.ask(Foo("bar", _))
}
val result = answer.failed.futureValue val result = answer.failed.futureValue
result shouldBe a[TimeoutException] result shouldBe a[TimeoutException]
result.getMessage should include("had already been terminated.") result.getMessage should include("had already been terminated.")
@ -80,17 +73,19 @@ class AskSpec extends ScalaTestWithActorTestKit("""
} }
"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 unhandledProbe = createTestProbe[UnhandledMessage]()
system.eventStream ! EventStream.Subscribe(unhandledProbe.ref)
val actor = spawn(Behaviors.empty[Foo]) val actor = spawn(Behaviors.empty[Foo])
implicit val timeout: Timeout = 10.millis implicit val timeout: Timeout = 10.millis
EventFilter.warning(pattern = ".*unhandled message.*", occurrences = 1).intercept {
val answer: Future[String] = actor.ask(Foo("bar", _)) val answer: Future[String] = actor.ask(Foo("bar", _))
unhandledProbe.receiveMessage()
val result = answer.failed.futureValue val result = answer.failed.futureValue
result shouldBe a[TimeoutException] result shouldBe a[TimeoutException]
result.getMessage should startWith("Ask timed out on") result.getMessage should startWith("Ask timed out on")
} }
}
/** See issue #19947 (MatchError with adapted ActorRef) */
"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 { val noSuchActor: ActorRef[Msg] = system match {
case adaptedSys: ActorSystemAdapter[_] => case adaptedSys: ActorSystemAdapter[_] =>
@ -100,13 +95,19 @@ class AskSpec extends ScalaTestWithActorTestKit("""
fail("this test must only run in an adapted actor system") fail("this test must only run in an adapted actor system")
} }
val answer: Future[String] = val deadLetterProbe = createTestProbe[DeadLetter]()
EventFilter.warning(pattern = ".*received dead letter.*", occurrences = 1).intercept { system.eventStream ! EventStream.Subscribe(deadLetterProbe.ref)
noSuchActor.ask(Foo("bar", _))
} val answer: Future[String] = noSuchActor.ask(Foo("bar", _))
val result = answer.failed.futureValue val result = answer.failed.futureValue
result shouldBe a[TimeoutException] result shouldBe a[TimeoutException]
result.getMessage should include("had already been terminated") result.getMessage should include("had already been terminated")
val deadLetter = deadLetterProbe.receiveMessage()
deadLetter.message match {
case Foo(s, _) => s should ===("bar")
case _ => fail(s"unexpected DeadLetter: $deadLetter")
}
} }
"transform a replied akka.actor.Status.Failure to a failed future" in { "transform a replied akka.actor.Status.Failure to a failed future" in {
@ -127,6 +128,7 @@ class AskSpec extends ScalaTestWithActorTestKit("""
val legacyActor = classicSystem.actorOf(akka.actor.Props(new LegacyActor)) val legacyActor = classicSystem.actorOf(akka.actor.Props(new LegacyActor))
import scaladsl.AskPattern._ import scaladsl.AskPattern._
import akka.actor.typed.scaladsl.adapter._
implicit val timeout: Timeout = 3.seconds implicit val timeout: Timeout = 3.seconds
implicit val scheduler = classicSystem.toTyped.scheduler implicit val scheduler = classicSystem.toTyped.scheduler
val typedLegacy: ActorRef[AnyRef] = legacyActor val typedLegacy: ActorRef[AnyRef] = legacyActor
@ -172,10 +174,11 @@ class AskSpec extends ScalaTestWithActorTestKit("""
ref ! "start-ask" ref ! "start-ask"
val Question(replyRef2) = probe.expectMessageType[Question] val Question(replyRef2) = probe.expectMessageType[Question]
EventFilter[RuntimeException](message = "Exception thrown out of adapter. Stopping myself.", occurrences = 1) LoggingEventFilter
.error("Exception thrown out of adapter. Stopping myself.")
.intercept { .intercept {
replyRef2 ! 42L replyRef2 ! 42L
}(system.toClassic) }(system)
probe.expectTerminated(ref, probe.remainingOrDefault) probe.expectTerminated(ref, probe.remainingOrDefault)
} }

View file

@ -12,6 +12,7 @@ import java.util.function.{ Function => F1 }
import akka.Done import akka.Done
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.{ BehaviorTestKit, TestInbox } import akka.actor.testkit.typed.scaladsl.{ BehaviorTestKit, TestInbox }
import org.scalactic.TypeCheckedTripleEquals import org.scalactic.TypeCheckedTripleEquals
import org.scalatest.Matchers import org.scalatest.Matchers
@ -68,7 +69,7 @@ object BehaviorSpec {
override def next = StateA override def next = StateA
} }
trait Common extends WordSpecLike with Matchers with TypeCheckedTripleEquals { trait Common extends WordSpecLike with Matchers with TypeCheckedTripleEquals with LogCapturing {
type Aux >: Null <: AnyRef type Aux >: Null <: AnyRef
def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux) def behavior(monitor: ActorRef[Event]): (Behavior[Command], Aux)
@silent("never used") @silent("never used")

View file

@ -4,13 +4,13 @@
package akka.actor.typed package akka.actor.typed
import akka.testkit.EventFilter
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.testkit.typed.TestKitSettings import akka.actor.testkit.typed.TestKitSettings
import akka.actor.testkit.typed.scaladsl._ import akka.actor.testkit.typed.scaladsl._
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import akka.actor.ActorInitializationException import akka.actor.ActorInitializationException
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import org.scalatest.{ Matchers, WordSpec, WordSpecLike } import org.scalatest.{ Matchers, WordSpec, WordSpecLike }
object DeferredSpec { object DeferredSpec {
@ -30,17 +30,11 @@ object DeferredSpec {
}) })
} }
class DeferredSpec extends ScalaTestWithActorTestKit(""" class DeferredSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
akka.loggers = [akka.testkit.TestEventListener]
""") with WordSpecLike {
import DeferredSpec._ import DeferredSpec._
implicit val testSettings = TestKitSettings(system) implicit val testSettings = TestKitSettings(system)
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val classicSystem = system.toClassic
"Deferred behavior" must { "Deferred behavior" must {
"must create underlying" in { "must create underlying" in {
val probe = TestProbe[Event]("evt") val probe = TestProbe[Event]("evt")
@ -68,7 +62,7 @@ class DeferredSpec extends ScalaTestWithActorTestKit("""
Behaviors.stopped Behaviors.stopped
} }
} }
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
spawn(behv) spawn(behv)
probe.expectMessage(Started) probe.expectMessage(Started)
probe.expectMessage(Pong) probe.expectMessage(Pong)
@ -144,7 +138,7 @@ class DeferredSpec extends ScalaTestWithActorTestKit("""
Behaviors.same Behaviors.same
} }
} }
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
val ref = spawn(behv) val ref = spawn(behv)
probe.expectTerminated(ref, probe.remainingOrDefault) probe.expectTerminated(ref, probe.remainingOrDefault)
} }
@ -152,7 +146,7 @@ class DeferredSpec extends ScalaTestWithActorTestKit("""
} }
} }
class DeferredStubbedSpec extends WordSpec with Matchers { class DeferredStubbedSpec extends WordSpec with Matchers with LogCapturing {
import DeferredSpec._ import DeferredSpec._

View file

@ -12,6 +12,7 @@ import scala.concurrent.Future
import akka.actor.BootstrapSetup import akka.actor.BootstrapSetup
import akka.actor.setup.ActorSystemSetup import akka.actor.setup.ActorSystemSetup
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.receptionist.Receptionist import akka.actor.typed.receptionist.Receptionist
import akka.actor.typed.receptionist.ServiceKey import akka.actor.typed.receptionist.ServiceKey
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
@ -72,7 +73,7 @@ akka.actor.typed {
""").resolve() """).resolve()
} }
class ExtensionsSpec extends ScalaTestWithActorTestKit with WordSpecLike { class ExtensionsSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
"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

@ -6,14 +6,15 @@ package akka.actor.typed
import java.util.concurrent.atomic.AtomicBoolean import java.util.concurrent.atomic.AtomicBoolean
import akka.testkit.EventFilter
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.ActorInitializationException import akka.actor.ActorInitializationException
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.internal.PoisonPill import akka.actor.typed.internal.PoisonPill
import akka.actor.typed.internal.PoisonPillInterceptor import akka.actor.typed.internal.PoisonPillInterceptor
@ -74,16 +75,10 @@ object InterceptSpec {
} }
} }
class InterceptSpec extends ScalaTestWithActorTestKit(""" class InterceptSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
akka.loggers = [akka.testkit.TestEventListener]
""") with WordSpecLike {
import BehaviorInterceptor._ import BehaviorInterceptor._
import InterceptSpec._ import InterceptSpec._
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val classicSystem = system.toClassic
private def snitchingInterceptor(probe: ActorRef[String]) = new BehaviorInterceptor[String, String] { private def snitchingInterceptor(probe: ActorRef[String]) = new BehaviorInterceptor[String, String] {
override def aroundReceive( override def aroundReceive(
context: TypedActorContext[String], context: TypedActorContext[String],
@ -287,7 +282,7 @@ class InterceptSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[String]() val probe = TestProbe[String]()
val interceptor = snitchingInterceptor(probe.ref) val interceptor = snitchingInterceptor(probe.ref)
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
val ref = spawn(Behaviors.intercept(() => interceptor)(Behaviors.setup[String] { _ => val ref = spawn(Behaviors.intercept(() => interceptor)(Behaviors.setup[String] { _ =>
Behaviors.same[String] Behaviors.same[String]
})) }))

View file

@ -5,176 +5,134 @@
package akka.actor.typed package akka.actor.typed
import akka.actor import akka.actor
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.testkit.LoggingEventFilter
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import org.slf4j.{ Logger, LoggerFactory, MDC }
import org.slf4j.event.Level import org.slf4j.event.Level
import org.slf4j.helpers.{ SubstituteLogger, SubstituteLoggerFactory }
//TODO review akka.testkit.TestEventListener as config. Has quite important implications
class LogMessagesSpec extends ScalaTestWithActorTestKit(""" class LogMessagesSpec extends ScalaTestWithActorTestKit("""
akka.loglevel = DEBUG # test verifies debug akka.loglevel = DEBUG # test verifies debug
akka.loggers = ["akka.actor.typed.testkit.TestEventListener"] """) with WordSpecLike with LogCapturing {
""") with WordSpecLike {
implicit val classic: actor.ActorSystem = system.toClassic implicit val classic: actor.ActorSystem = system.toClassic
"The log messages behavior" should { "The log messages behavior" should {
"log messages and signals" in { "log messages and signals" in {
val behavior: Behavior[String] = Behaviors.logMessages(Behaviors.ignore)
val factory = new SubstituteLoggerFactory()
val substituteLogger: SubstituteLogger = factory.getLogger("substitute").asInstanceOf[SubstituteLogger]
val opts = LogOptions().withLevel(Level.DEBUG).withLogger(substituteLogger)
val behavior: Behavior[String] = Behaviors.logMessages(opts, Behaviors.ignore)
val ref: ActorRef[String] = spawn(behavior) val ref: ActorRef[String] = spawn(behavior)
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref.path.toString}] received message: Hello").intercept {
.debug(s"actor ${ref.path.toString} received message Hello", source = ref.path.toString, occurrences = 1) ref ! "Hello"
.intercept(ref ! "Hello", factory.getEventQueue) }
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref.path}] received signal: PostStop").intercept {
.debug(s"actor ${ref.path.toString} received signal PostStop", source = ref.path.toString, occurrences = 1) testKit.stop(ref)
.intercept(testKit.stop(ref), factory.getEventQueue) }
} }
"log messages with provided log level" in { "log messages with provided log level" in {
val factory = new SubstituteLoggerFactory() val opts = LogOptions().withLevel(Level.INFO)
val substituteLogger: SubstituteLogger = factory.getLogger("substitute").asInstanceOf[SubstituteLogger]
val opts = LogOptions().withLevel(Level.INFO).withLogger(substituteLogger)
val behavior: Behavior[String] = Behaviors.logMessages(opts, Behaviors.ignore) val behavior: Behavior[String] = Behaviors.logMessages(opts, Behaviors.ignore)
val ref: ActorRef[String] = spawn(behavior) val ref: ActorRef[String] = spawn(behavior)
LoggingEventFilter LoggingEventFilter.info(s"actor [${ref.path}] received message: Hello").intercept {
.info(s"actor ${ref.path.toString} received message Hello", source = ref.path.toString, occurrences = 1) ref ! "Hello"
.intercept(ref ! "Hello", factory.getEventQueue) }
LoggingEventFilter LoggingEventFilter.info(s"actor [${ref.path}] received signal: PostStop").intercept {
.info(s"actor ${ref.path.toString} received signal PostStop", source = ref.path.toString, occurrences = 1) testKit.stop(ref)
.intercept(testKit.stop(ref), factory.getEventQueue) }
} }
"log messages with provided logger" in { "log messages with provided logger" in {
val logger = system.log
val factory = new SubstituteLoggerFactory() val opts = LogOptions().withLogger(logger)
val substituteLogger: SubstituteLogger = factory.getLogger("substitute").asInstanceOf[SubstituteLogger]
val opts = LogOptions().withLevel(Level.DEBUG).withLogger(substituteLogger)
val behavior: Behavior[String] = Behaviors.logMessages(opts, Behaviors.ignore) val behavior: Behavior[String] = Behaviors.logMessages(opts, Behaviors.ignore)
val ref: ActorRef[String] = spawn(behavior) val ref: ActorRef[String] = spawn(behavior)
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref.path}] received message: Hello").intercept {
.debug(s"actor ${ref.path.toString} received message Hello", source = ref.path.toString, occurrences = 1) ref ! "Hello"
.intercept(ref ! "Hello", factory.getEventQueue) }
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref.path}] received signal: PostStop").intercept {
.debug(s"actor ${ref.path.toString} received signal PostStop", source = ref.path.toString, occurrences = 1) testKit.stop(ref)
.intercept(testKit.stop(ref), factory.getEventQueue) }
} }
"not log messages when not enabled" in { "not log messages when not enabled" in {
val factory = new SubstituteLoggerFactory() val opts = LogOptions().withEnabled(false)
val substituteLogger: SubstituteLogger = factory.getLogger("substitute").asInstanceOf[SubstituteLogger]
substituteLogger.setDelegate(LoggerFactory.getLogger(Logger.ROOT_LOGGER_NAME))
val opts = LogOptions().withLevel(Level.DEBUG).withLogger(substituteLogger).withEnabled(false)
val behavior: Behavior[String] = Behaviors.logMessages(opts, Behaviors.ignore) val behavior: Behavior[String] = Behaviors.logMessages(opts, Behaviors.ignore)
val ref: ActorRef[String] = spawn(behavior) val ref: ActorRef[String] = spawn(behavior)
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref.path}] received message: Hello").withOccurrences(0).intercept {
.debug(s"actor ${ref.path.toString} received message Hello", source = ref.path.toString, occurrences = 0) ref ! "Hello"
.intercept(ref ! "Hello", factory.getEventQueue) }
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref.path}] received signal: PostStop").withOccurrences(0).intercept {
.debug(s"actor ${ref.path.toString} received signal PostStop", source = ref.path.toString, occurrences = 0) testKit.stop(ref)
.intercept(testKit.stop(ref), factory.getEventQueue) }
} }
"log messages with decorated MDC values" in { "log messages with decorated MDC values" in {
val factory = new SubstituteLoggerFactory() val opts = LogOptions().withLevel(Level.DEBUG)
val substituteLogger: SubstituteLogger = factory.getLogger("substitute").asInstanceOf[SubstituteLogger]
val opts = LogOptions().withLevel(Level.DEBUG).withLogger(substituteLogger)
val mdc = Map("mdc" -> "true") val mdc = Map("mdc" -> "true")
val behavior = Behaviors.withMdc[String](mdc)(Behaviors.logMessages(opts, Behaviors.ignore)) val behavior = Behaviors.withMdc[String](mdc)(Behaviors.logMessages(opts, Behaviors.ignore))
val ref = spawn(behavior) val ref = spawn(behavior)
LoggingEventFilter
.debug(
s"actor ${ref.path.toString} received message Hello MDC is $mdc",
source = ref.path.toString,
occurrences = 1)
.intercept(ref ! "Hello", factory.getEventQueue)
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref.path}] received message: Hello").withMdc(mdc).intercept {
.debug( ref ! "Hello"
s"actor ${ref.path.toString} received signal PostStop MDC is $mdc", }
source = ref.path.toString,
occurrences = 1) LoggingEventFilter.debug(s"actor [${ref.path}] received signal: PostStop").withMdc(mdc).intercept {
.intercept(testKit.stop(ref), factory.getEventQueue) testKit.stop(ref)
}
} }
"log messages with different decorated MDC values in different actors" in { "log messages with different decorated MDC values in different actors" in {
val factory = new SubstituteLoggerFactory() val opts = LogOptions().withLevel(Level.DEBUG)
val substituteLogger: SubstituteLogger = factory.getLogger("substitute").asInstanceOf[SubstituteLogger]
val opts = LogOptions().withLevel(Level.DEBUG).withLogger(substituteLogger)
val mdc1 = Map("mdc" -> "true") val mdc1 = Map("mdc" -> "true")
val behavior1 = Behaviors.withMdc[String](mdc1)(Behaviors.logMessages(opts, Behaviors.ignore)) val behavior1 = Behaviors.withMdc[String](mdc1)(Behaviors.logMessages(opts, Behaviors.ignore))
val mdc2 = Map("mdc" -> "false") val mdc2 = Map("mdc" -> "false")
val behavior2 = Behaviors.withMdc[String](mdc2)(Behaviors.logMessages(opts, Behaviors.ignore)) val behavior2 = Behaviors.withMdc[String](mdc2)(Behaviors.logMessages(opts, Behaviors.ignore))
val ref2 = spawn(behavior2) val ref2 = spawn(behavior2)
LoggingEventFilter
.debug( LoggingEventFilter.debug(s"actor [${ref2.path}] received message: Hello").withMdc(mdc2).intercept {
s"actor ${ref2.path.toString} received message Hello MDC is $mdc2", ref2 ! "Hello"
source = ref2.path.toString, }
occurrences = 1)
.intercept(ref2 ! "Hello", factory.getEventQueue)
val ref1 = spawn(behavior1) val ref1 = spawn(behavior1)
LoggingEventFilter
.debug(
s"actor ${ref1.path.toString} received message Hello MDC is $mdc1",
source = ref1.path.toString,
occurrences = 1)
.intercept(ref1 ! "Hello", factory.getEventQueue)
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref1.path}] received message: Hello").withMdc(mdc1).intercept {
.debug( ref1 ! "Hello"
s"actor ${ref2.path.toString} received signal PostStop MDC is $mdc2", }
source = ref2.path.toString,
occurrences = 1)
.intercept(testKit.stop(ref2), factory.getEventQueue)
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref2.path}] received signal: PostStop").withMdc(mdc2).intercept {
.debug( testKit.stop(ref2)
s"actor ${ref1.path.toString} received signal PostStop MDC is $mdc1", }
source = ref1.path.toString,
occurrences = 1)
.intercept(testKit.stop(ref1), factory.getEventQueue)
LoggingEventFilter.debug(s"actor [${ref1.path}] received signal: PostStop").withMdc(mdc1).intercept {
testKit.stop(ref1)
}
} }
"log messages of different type" in { "log messages of different type" in {
val factory = new SubstituteLoggerFactory() val behavior: Behavior[String] = Behaviors.logMessages(Behaviors.ignore[String])
val substituteLogger: Logger = factory.getLogger("substitute").asInstanceOf[SubstituteLogger]
val opts = LogOptions().withLevel(Level.DEBUG).withLogger(substituteLogger)
val behavior: Behavior[String] = Behaviors.logMessages(opts, Behaviors.ignore[String])
val ref = spawn(behavior) val ref = spawn(behavior)
LoggingEventFilter LoggingEventFilter.debug(s"actor [${ref.path}] received message: 13").intercept {
.debug(s"actor ${ref.path.toString} received message 13", source = ref.path.toString, occurrences = 1) ref.unsafeUpcast[Any] ! 13
.intercept(ref.unsafeUpcast[Any] ! 13, factory.getEventQueue) }
} }
} }

View file

@ -4,8 +4,13 @@
package akka.actor.typed package akka.actor.typed
import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit
import akka.actor.ActorCell import akka.actor.ActorCell
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.internal.adapter.ActorContextAdapter import akka.actor.typed.internal.adapter.ActorContextAdapter
import akka.actor.typed.scaladsl.AskPattern._ import akka.actor.typed.scaladsl.AskPattern._
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
@ -13,24 +18,14 @@ import akka.dispatch.BoundedMessageQueueSemantics
import akka.dispatch.BoundedNodeMessageQueue import akka.dispatch.BoundedNodeMessageQueue
import akka.dispatch.MessageQueue import akka.dispatch.MessageQueue
import akka.dispatch.UnboundedMessageQueueSemantics import akka.dispatch.UnboundedMessageQueueSemantics
import akka.testkit.EventFilter
import akka.testkit.TestLatch
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import scala.concurrent.Await
import scala.concurrent.duration._
class MailboxSelectorSpec extends ScalaTestWithActorTestKit(""" class MailboxSelectorSpec extends ScalaTestWithActorTestKit("""
specific-mailbox { specific-mailbox {
mailbox-type = "akka.dispatch.NonBlockingBoundedMailbox" mailbox-type = "akka.dispatch.NonBlockingBoundedMailbox"
mailbox-capacity = 4 mailbox-capacity = 4
} }
akka.loggers = [ akka.testkit.TestEventListener ] """) with WordSpecLike with LogCapturing {
""") with WordSpecLike {
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val classicSystem = system.toClassic
case class WhatsYourMailbox(replyTo: ActorRef[MessageQueue]) case class WhatsYourMailbox(replyTo: ActorRef[MessageQueue])
private def behavior: Behavior[WhatsYourMailbox] = private def behavior: Behavior[WhatsYourMailbox] =
@ -65,24 +60,24 @@ class MailboxSelectorSpec extends ScalaTestWithActorTestKit("""
} }
"set capacity on a bounded mailbox" in { "set capacity on a bounded mailbox" in {
val latch = TestLatch(1) val latch = new CountDownLatch(1)
val actor = spawn(Behaviors.receiveMessage[String] { val actor = spawn(Behaviors.receiveMessage[String] {
case "one" => case "one" =>
// block here so we can fill mailbox up // block here so we can fill mailbox up
Await.ready(latch, 10.seconds) latch.await(10, TimeUnit.SECONDS)
Behaviors.same Behaviors.same
case _ => case _ =>
Behaviors.same Behaviors.same
}, MailboxSelector.bounded(2)) }, MailboxSelector.bounded(2))
actor ! "one" // actor will block here actor ! "one" // actor will block here
actor ! "two" actor ! "two"
EventFilter.warning(start = "received dead letter:", occurrences = 1).intercept { LoggingEventFilter.deadLetters().intercept {
// one or both of these doesn't fit in mailbox // one or both of these doesn't fit in mailbox
// depending on race with how fast actor consumes // depending on race with how fast actor consumes
actor ! "three" actor ! "three"
actor ! "four" actor ! "four"
} }
latch.open() latch.countDown()
} }
"select an arbitrary mailbox from config" in { "select an arbitrary mailbox from config" in {

View file

@ -6,10 +6,11 @@ package akka.actor.typed
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class MonitorSpec extends ScalaTestWithActorTestKit with WordSpecLike { class MonitorSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
"The monitor behavior" should { "The monitor behavior" should {

View file

@ -183,7 +183,7 @@ object OrElseSpec {
} }
class OrElseSpec extends WordSpec with Matchers { class OrElseSpec extends WordSpec with Matchers with LogCapturing {
import OrElseSpec._ import OrElseSpec._

View file

@ -4,10 +4,11 @@
package akka.actor.typed package akka.actor.typed
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.Matchers import org.scalatest.Matchers
import org.scalatest.WordSpec import org.scalatest.WordSpec
class PropsSpec extends WordSpec with Matchers { class PropsSpec extends WordSpec with Matchers with LogCapturing {
val dispatcherFirst = Props.empty.withDispatcherFromConfig("pool").withDispatcherDefault val dispatcherFirst = Props.empty.withDispatcherFromConfig("pool").withDispatcherDefault

View file

@ -26,7 +26,7 @@ object SpawnProtocolSpec {
} }
} }
class SpawnProtocolSpec extends ScalaTestWithActorTestKit with WordSpecLike { class SpawnProtocolSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import SpawnProtocolSpec._ import SpawnProtocolSpec._
implicit val testSettings = TestKitSettings(system) implicit val testSettings = TestKitSettings(system)
@ -96,7 +96,7 @@ class SpawnProtocolSpec extends ScalaTestWithActorTestKit with WordSpecLike {
} }
} }
class StubbedSpawnProtocolSpec extends WordSpec with Matchers { class StubbedSpawnProtocolSpec extends WordSpec with Matchers with LogCapturing {
import SpawnProtocolSpec._ import SpawnProtocolSpec._

View file

@ -7,21 +7,25 @@ package akka.actor.typed
import java.io.IOException import java.io.IOException
import java.util.concurrent.CountDownLatch import java.util.concurrent.CountDownLatch
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
import java.util.concurrent.atomic.{ AtomicBoolean, AtomicInteger } import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.atomic.AtomicInteger
import scala.concurrent.duration._
import scala.util.control.NoStackTrace
import akka.actor.ActorInitializationException import akka.actor.ActorInitializationException
import akka.actor.typed.scaladsl.{ AbstractBehavior, Behaviors }
import akka.actor.typed.scaladsl.Behaviors._
import akka.testkit.EventFilter
import akka.actor.testkit.typed.scaladsl._
import akka.actor.testkit.typed._
import org.scalatest.{ Matchers, WordSpec, WordSpecLike }
import scala.util.control.NoStackTrace
import scala.concurrent.duration._
import akka.actor.Dropped import akka.actor.Dropped
import akka.actor.testkit.typed._
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.SupervisorStrategy.Resume import akka.actor.typed.SupervisorStrategy.Resume
import akka.event.Logging import akka.actor.typed.eventstream.EventStream
import akka.actor.typed.scaladsl.Behaviors._
import akka.actor.typed.scaladsl.AbstractBehavior
import akka.actor.typed.scaladsl.Behaviors
import org.scalatest.Matchers
import org.scalatest.WordSpec
import org.scalatest.WordSpecLike
import org.slf4j.event.Level import org.slf4j.event.Level
object SupervisionSpec { object SupervisionSpec {
@ -88,7 +92,7 @@ object SupervisionSpec {
} }
} }
class StubbedSupervisionSpec extends WordSpec with Matchers { class StubbedSupervisionSpec extends WordSpec with Matchers with LogCapturing {
import SupervisionSpec._ import SupervisionSpec._
@ -248,21 +252,15 @@ class StubbedSupervisionSpec extends WordSpec with Matchers {
} }
class SupervisionSpec extends ScalaTestWithActorTestKit(""" class SupervisionSpec extends ScalaTestWithActorTestKit("""
akka.loggers = [akka.testkit.TestEventListener]
akka.log-dead-letters = off akka.log-dead-letters = off
""") with WordSpecLike { """) with WordSpecLike with LogCapturing {
import SupervisionSpec._
import BehaviorInterceptor._ import BehaviorInterceptor._
import SupervisionSpec._
private val nameCounter = Iterator.from(0) private val nameCounter = Iterator.from(0)
private def nextName(prefix: String = "a"): String = s"$prefix-${nameCounter.next()}" private def nextName(prefix: String = "a"): String = s"$prefix-${nameCounter.next()}"
// FIXME #24348: eventfilter support in typed testkit
import akka.actor.typed.scaladsl.adapter._
implicit val classicSystem = system.toClassic
class FailingConstructorTestSetup(failCount: Int) { class FailingConstructorTestSetup(failCount: Int) {
val failCounter = new AtomicInteger(0) val failCounter = new AtomicInteger(0)
class FailingConstructor(monitor: ActorRef[Event]) extends AbstractBehavior[Command] { class FailingConstructor(monitor: ActorRef[Event]) extends AbstractBehavior[Command] {
@ -315,7 +313,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[Event]("evt") val probe = TestProbe[Event]("evt")
val behv = targetBehavior(probe.ref) val behv = targetBehavior(probe.ref)
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exc3](occurrences = 1).intercept { LoggingEventFilter.error[Exc3].intercept {
ref ! Throw(new Exc3) ref ! Throw(new Exc3)
probe.expectMessage(ReceivedSignal(PostStop)) probe.expectMessage(ReceivedSignal(PostStop))
probe.expectTerminated(ref) probe.expectTerminated(ref)
@ -325,7 +323,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[Event]("evt") val probe = TestProbe[Event]("evt")
val behv = Behaviors.supervise(targetBehavior(probe.ref)).onFailure[Throwable](SupervisorStrategy.stop) val behv = Behaviors.supervise(targetBehavior(probe.ref)).onFailure[Throwable](SupervisorStrategy.stop)
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exc3](occurrences = 1).intercept { LoggingEventFilter.error[Exc3].intercept {
ref ! Throw(new Exc3) ref ! Throw(new Exc3)
probe.expectMessage(ReceivedSignal(PostStop)) probe.expectMessage(ReceivedSignal(PostStop))
probe.expectTerminated(ref) probe.expectTerminated(ref)
@ -339,7 +337,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
targetBehavior(probe.ref) targetBehavior(probe.ref)
}) })
val behv = Behaviors.supervise(failedSetup).onFailure[Throwable](SupervisorStrategy.stop) val behv = Behaviors.supervise(failedSetup).onFailure[Throwable](SupervisorStrategy.stop)
EventFilter[Exc3](occurrences = 1).intercept { LoggingEventFilter.error[Exc3].intercept {
spawn(behv) spawn(behv)
} }
} }
@ -352,12 +350,12 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[IOException](occurrences = 1).intercept { LoggingEventFilter.error[IOException].intercept {
ref ! Throw(new IOException()) ref ! Throw(new IOException())
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
EventFilter[IllegalArgumentException](occurrences = 1).intercept { LoggingEventFilter.error[IllegalArgumentException].intercept {
ref ! Throw(new IllegalArgumentException("cat")) ref ! Throw(new IllegalArgumentException("cat"))
probe.expectMessage(ReceivedSignal(PostStop)) probe.expectMessage(ReceivedSignal(PostStop))
} }
@ -373,7 +371,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exception](occurrences = 1).intercept { LoggingEventFilter.error[Exception].intercept {
ref ! Throw(new IOException()) ref ! Throw(new IOException())
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
@ -381,7 +379,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! Ping(1) ref ! Ping(1)
probe.expectMessage(Pong(1)) probe.expectMessage(Pong(1))
EventFilter[IllegalArgumentException](occurrences = 1).intercept { LoggingEventFilter.error[IllegalArgumentException].intercept {
ref ! Throw(new IllegalArgumentException("cat")) ref ! Throw(new IllegalArgumentException("cat"))
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
@ -399,7 +397,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exception](occurrences = 1).intercept { LoggingEventFilter.error[Exception].intercept {
ref ! Throw(new IOException()) ref ! Throw(new IOException())
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
@ -407,7 +405,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! Ping(1) ref ! Ping(1)
probe.expectMessage(Pong(1)) probe.expectMessage(Pong(1))
EventFilter[IllegalArgumentException](occurrences = 1).intercept { LoggingEventFilter.error[IllegalArgumentException].intercept {
ref ! Throw(new IllegalArgumentException("cat")) ref ! Throw(new IllegalArgumentException("cat"))
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
@ -421,7 +419,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[Event]("evt") val probe = TestProbe[Event]("evt")
val behv = targetBehavior(probe.ref) val behv = targetBehavior(probe.ref)
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exc3](occurrences = 1).intercept { LoggingEventFilter.error[Exc3].intercept {
ref ! Throw(new Exc3) ref ! Throw(new Exc3)
probe.expectMessage(ReceivedSignal(PostStop)) probe.expectMessage(ReceivedSignal(PostStop))
} }
@ -431,7 +429,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[Event]("evt") val probe = TestProbe[Event]("evt")
val behv = Behaviors.supervise(targetBehavior(probe.ref)).onFailure[Exc1](SupervisorStrategy.restart) val behv = Behaviors.supervise(targetBehavior(probe.ref)).onFailure[Exc1](SupervisorStrategy.restart)
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exc3](occurrences = 1).intercept { LoggingEventFilter.error[Exc3].intercept {
ref ! Throw(new Exc3) ref ! Throw(new Exc3)
probe.expectMessage(ReceivedSignal(PostStop)) probe.expectMessage(ReceivedSignal(PostStop))
} }
@ -445,7 +443,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! GetState ref ! GetState
probe.expectMessage(State(1, Map.empty)) probe.expectMessage(State(1, Map.empty))
EventFilter[Exc2](occurrences = 1).intercept { LoggingEventFilter.error[Exc2].intercept {
ref ! Throw(new Exc2) ref ! Throw(new Exc2)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
@ -464,7 +462,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! GetState ref ! GetState
probe.expectMessage(State(1, Map.empty)) probe.expectMessage(State(1, Map.empty))
EventFilter[Exc2](occurrences = 3).intercept { LoggingEventFilter.error[Exc2].withOccurrences(3).intercept {
ref ! Throw(new Exc2) ref ! Throw(new Exc2)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
ref ! Throw(new Exc2) ref ! Throw(new Exc2)
@ -472,10 +470,8 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! Throw(new Exc2) ref ! Throw(new Exc2)
probe.expectMessage(ReceivedSignal(PostStop)) probe.expectMessage(ReceivedSignal(PostStop))
} }
EventFilter.warning(start = "received dead letter", occurrences = 1).intercept {
ref ! GetState probe.expectTerminated(ref)
probe.expectNoMessage()
}
} }
"reset fixed limit after timeout" in { "reset fixed limit after timeout" in {
@ -489,7 +485,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! GetState ref ! GetState
probe.expectMessage(State(1, Map.empty)) probe.expectMessage(State(1, Map.empty))
EventFilter[Exc2](occurrences = 3).intercept { LoggingEventFilter.error[Exc2].withOccurrences(3).intercept {
ref ! Throw(new Exc2) ref ! Throw(new Exc2)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
ref ! Throw(new Exc2) ref ! Throw(new Exc2)
@ -529,7 +525,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! GetState ref ! GetState
parentProbe.expectMessageType[State].children.keySet should ===(Set(child1Name, child2Name)) parentProbe.expectMessageType[State].children.keySet should ===(Set(child1Name, child2Name))
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
parentProbe.expectMessage(ReceivedSignal(PreRestart)) parentProbe.expectMessage(ReceivedSignal(PreRestart))
ref ! GetState ref ! GetState
@ -567,7 +563,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! GetState ref ! GetState
parentProbe.expectMessageType[State].children.keySet should contain(childName) parentProbe.expectMessageType[State].children.keySet should contain(childName)
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
parentProbe.expectMessage(ReceivedSignal(PreRestart)) parentProbe.expectMessage(ReceivedSignal(PreRestart))
ref ! GetState ref ! GetState
@ -598,7 +594,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val child2Name = nextName() val child2Name = nextName()
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
parentProbe.expectMessage(ReceivedSignal(PreRestart)) parentProbe.expectMessage(ReceivedSignal(PreRestart))
ref ! GetState ref ! GetState
@ -607,7 +603,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
} }
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
slowStop.countDown() slowStop.countDown()
childProbe.expectMessage(ReceivedSignal(PostStop)) // child1 childProbe.expectMessage(ReceivedSignal(PostStop)) // child1
parentProbe.expectMessageType[State].children.keySet should ===(Set.empty) parentProbe.expectMessageType[State].children.keySet should ===(Set.empty)
@ -652,7 +648,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
} }
.onFailure[RuntimeException](strategy) .onFailure[RuntimeException](strategy)
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
val ref = spawn(behv) val ref = spawn(behv)
slowStop1.countDown() slowStop1.countDown()
child1Probe.expectMessage(ReceivedSignal(PostStop)) child1Probe.expectMessage(ReceivedSignal(PostStop))
@ -703,12 +699,12 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
throwFromSetup.set(true) throwFromSetup.set(true)
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
parentProbe.expectMessage(ReceivedSignal(PreRestart)) parentProbe.expectMessage(ReceivedSignal(PreRestart))
} }
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
slowStop1.countDown() slowStop1.countDown()
child1Probe.expectMessage(ReceivedSignal(PostStop)) child1Probe.expectMessage(ReceivedSignal(PostStop))
child1Probe.expectMessage(ReceivedSignal(PostStop)) child1Probe.expectMessage(ReceivedSignal(PostStop))
@ -729,7 +725,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! GetState ref ! GetState
probe.expectMessage(State(1, Map.empty)) probe.expectMessage(State(1, Map.empty))
EventFilter[Exc2](occurrences = 1).intercept { LoggingEventFilter.error[Exc2].intercept {
ref ! Throw(new Exc2) ref ! Throw(new Exc2)
ref ! GetState ref ! GetState
probe.expectMessage(State(1, Map.empty)) probe.expectMessage(State(1, Map.empty))
@ -747,7 +743,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage(State(1, Map.empty)) probe.expectMessage(State(1, Map.empty))
// resume // resume
EventFilter[Exc2](occurrences = 1).intercept { LoggingEventFilter.error[Exc2].intercept {
ref ! Throw(new Exc2) ref ! Throw(new Exc2)
probe.expectNoMessage() probe.expectNoMessage()
ref ! GetState ref ! GetState
@ -755,7 +751,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
} }
// restart // restart
EventFilter[Exc3](occurrences = 1).intercept { LoggingEventFilter.error[Exc3].intercept {
ref ! Throw(new Exc3) ref ! Throw(new Exc3)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
ref ! GetState ref ! GetState
@ -763,13 +759,16 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
} }
// stop // stop
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
probe.expectMessage(ReceivedSignal(PostStop)) probe.expectMessage(ReceivedSignal(PostStop))
} }
} }
"publish dropped messages while backing off and stash is full" in { "publish dropped messages while backing off and stash is full" in {
import akka.actor.typed.scaladsl.adapter._
val droppedMessagesProbe = TestProbe[Dropped]()
system.eventStream ! EventStream.Subscribe(droppedMessagesProbe.ref)
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
@ -781,15 +780,13 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
}) })
.onFailure[Exception](strategy) .onFailure[Exception](strategy)
val droppedMessagesProbe = TestProbe[Dropped]()
system.toClassic.eventStream.subscribe(droppedMessagesProbe.ref.toClassic, classOf[Dropped])
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
startedProbe.expectMessage(Started) startedProbe.expectMessage(Started)
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
EventFilter.warning(start = "dropped message", occurrences = 2).intercept {
ref ! Ping(1) ref ! Ping(1)
ref ! Ping(2) ref ! Ping(2)
ref ! Ping(3) ref ! Ping(3)
@ -799,9 +796,11 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
droppedMessagesProbe.expectMessage(Dropped(Ping(3), "Stash is full in [RestartSupervisor]", ref.toClassic)) droppedMessagesProbe.expectMessage(Dropped(Ping(3), "Stash is full in [RestartSupervisor]", ref.toClassic))
droppedMessagesProbe.expectMessage(Dropped(Ping(4), "Stash is full in [RestartSupervisor]", ref.toClassic)) droppedMessagesProbe.expectMessage(Dropped(Ping(4), "Stash is full in [RestartSupervisor]", ref.toClassic))
} }
}
"restart after exponential backoff" in { "restart after exponential backoff" in {
import akka.actor.typed.scaladsl.adapter._
val droppedMessagesProbe = TestProbe[Dropped]()
system.eventStream ! EventStream.Subscribe(droppedMessagesProbe.ref)
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
@ -817,14 +816,13 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
.onFailure[Exception](strategy) .onFailure[Exception](strategy)
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
startedProbe.expectMessage(Started) startedProbe.expectMessage(Started)
ref ! IncrementState ref ! IncrementState
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
EventFilter.warning(start = "dropped message", occurrences = 1).intercept {
ref ! Ping(1) // dropped due to backoff, no stashing ref ! Ping(1) // dropped due to backoff, no stashing
} droppedMessagesProbe.expectMessage(Dropped(Ping(1), "Stash is full in [RestartSupervisor]", ref.toClassic))
} }
startedProbe.expectNoMessage(minBackoff - 100.millis) startedProbe.expectNoMessage(minBackoff - 100.millis)
@ -834,13 +832,12 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage(State(0, Map.empty)) probe.expectMessage(State(0, Map.empty))
// one more time // one more time
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
ref ! IncrementState ref ! IncrementState
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
EventFilter.warning(start = "dropped message", occurrences = 1).intercept {
ref ! Ping(2) // dropped due to backoff, no stashing ref ! Ping(2) // dropped due to backoff, no stashing
} droppedMessagesProbe.expectMessage(Dropped(Ping(2), "Stash is full in [RestartSupervisor]", ref.toClassic))
} }
startedProbe.expectNoMessage((minBackoff * 2) - 100.millis) startedProbe.expectNoMessage((minBackoff * 2) - 100.millis)
@ -873,8 +870,8 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
.onFailure[Exception](strategy) .onFailure[Exception](strategy)
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
EventFilter[TestException](occurrences = 2).intercept { LoggingEventFilter.error[TestException].withOccurrences(2).intercept {
startedProbe.expectMessage(Started) startedProbe.expectMessage(Started)
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
probe.expectTerminated(ref, 3.seconds) probe.expectTerminated(ref, 3.seconds)
@ -883,6 +880,9 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
} }
"reset exponential backoff count after reset timeout" in { "reset exponential backoff count after reset timeout" in {
import akka.actor.typed.scaladsl.adapter._
val droppedMessagesProbe = TestProbe[Dropped]()
system.eventStream ! EventStream.Subscribe(droppedMessagesProbe.ref)
val probe = TestProbe[Event]("evt") val probe = TestProbe[Event]("evt")
val minBackoff = 1.seconds val minBackoff = 1.seconds
val strategy = SupervisorStrategy val strategy = SupervisorStrategy
@ -892,13 +892,12 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val behv = supervise(targetBehavior(probe.ref)).onFailure[Exc1](strategy) val behv = supervise(targetBehavior(probe.ref)).onFailure[Exc1](strategy)
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
ref ! IncrementState ref ! IncrementState
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
EventFilter.warning(start = "dropped message", occurrences = 1).intercept {
ref ! Ping(1) // dropped due to backoff, no stash ref ! Ping(1) // dropped due to backoff, no stash
} droppedMessagesProbe.expectMessage(Dropped(Ping(1), "Stash is full in [RestartSupervisor]", ref.toClassic))
} }
probe.expectNoMessage(minBackoff + 100.millis.dilated) probe.expectNoMessage(minBackoff + 100.millis.dilated)
@ -906,14 +905,13 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage(State(0, Map.empty)) probe.expectMessage(State(0, Map.empty))
// one more time after the reset timeout // one more time after the reset timeout
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
probe.expectNoMessage(strategy.resetBackoffAfter + 100.millis.dilated) probe.expectNoMessage(strategy.resetBackoffAfter + 100.millis.dilated)
ref ! IncrementState ref ! IncrementState
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
EventFilter.warning(start = "dropped message", occurrences = 1).intercept {
ref ! Ping(2) // dropped due to backoff ref ! Ping(2) // dropped due to backoff
} droppedMessagesProbe.expectMessage(Dropped(Ping(2), "Stash is full in [RestartSupervisor]", ref.toClassic))
} }
// backoff was reset, so restarted after the minBackoff // backoff was reset, so restarted after the minBackoff
@ -938,7 +936,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
failCount = 1, failCount = 1,
strategy = SupervisorStrategy.restart) { strategy = SupervisorStrategy.restart) {
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
spawn(behv) spawn(behv)
} }
} }
@ -946,7 +944,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
"fail to restart when deferred factory throws unhandled" in new FailingUnhandledTestSetup( "fail to restart when deferred factory throws unhandled" in new FailingUnhandledTestSetup(
strategy = SupervisorStrategy.restart) { strategy = SupervisorStrategy.restart) {
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
spawn(behv) spawn(behv)
} }
} }
@ -954,8 +952,8 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
"fail to resume when deferred factory throws" in new FailingDeferredTestSetup( "fail to resume when deferred factory throws" in new FailingDeferredTestSetup(
failCount = 1, failCount = 1,
strategy = SupervisorStrategy.resume) { strategy = SupervisorStrategy.resume) {
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
spawn(behv) spawn(behv)
} }
} }
@ -965,7 +963,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
failCount = 1, failCount = 1,
strategy = SupervisorStrategy.restartWithBackoff(minBackoff = 100.millis.dilated, maxBackoff = 1.second, 0)) { strategy = SupervisorStrategy.restartWithBackoff(minBackoff = 100.millis.dilated, maxBackoff = 1.second, 0)) {
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
spawn(behv) spawn(behv)
probe.expectMessage(StartFailed) probe.expectMessage(StartFailed)
@ -978,7 +976,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
"fail instead of restart with exponential backoff when deferred factory throws unhandled" in new FailingUnhandledTestSetup( "fail instead of restart with exponential backoff when deferred factory throws unhandled" in new FailingUnhandledTestSetup(
strategy = SupervisorStrategy.restartWithBackoff(minBackoff = 100.millis.dilated, maxBackoff = 1.second, 0)) { strategy = SupervisorStrategy.restartWithBackoff(minBackoff = 100.millis.dilated, maxBackoff = 1.second, 0)) {
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
spawn(behv) spawn(behv)
probe.expectMessage(StartFailed) probe.expectMessage(StartFailed)
} }
@ -988,7 +986,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
failCount = 1, failCount = 1,
strategy = SupervisorStrategy.restart.withLimit(3, 1.second)) { strategy = SupervisorStrategy.restart.withLimit(3, 1.second)) {
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
spawn(behv) spawn(behv)
probe.expectMessage(StartFailed) probe.expectMessage(StartFailed)
@ -1000,8 +998,8 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
failCount = 20, failCount = 20,
strategy = SupervisorStrategy.restart.withLimit(2, 1.second)) { strategy = SupervisorStrategy.restart.withLimit(2, 1.second)) {
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
EventFilter[TestException](occurrences = 2).intercept { LoggingEventFilter.error[TestException].withOccurrences(2).intercept {
spawn(behv) spawn(behv)
// first one from initial setup // first one from initial setup
@ -1017,7 +1015,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
"fail instead of restart with limit when deferred factory throws unhandled" in new FailingUnhandledTestSetup( "fail instead of restart with limit when deferred factory throws unhandled" in new FailingUnhandledTestSetup(
strategy = SupervisorStrategy.restart.withLimit(3, 1.second)) { strategy = SupervisorStrategy.restart.withLimit(3, 1.second)) {
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
spawn(behv) spawn(behv)
probe.expectMessage(StartFailed) probe.expectMessage(StartFailed)
} }
@ -1028,7 +1026,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val behv = supervise(setup[Command](_ => new FailingConstructor(probe.ref))) val behv = supervise(setup[Command](_ => new FailingConstructor(probe.ref)))
.onFailure[Exception](SupervisorStrategy.restart) .onFailure[Exception](SupervisorStrategy.restart)
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
spawn(behv) spawn(behv)
probe.expectMessage(Started) // first one before failure probe.expectMessage(Started) // first one before failure
} }
@ -1073,7 +1071,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
actor ! "ping" actor ! "ping"
probe.expectMessage("pong") probe.expectMessage("pong")
EventFilter[RuntimeException](occurrences = 1).intercept { LoggingEventFilter.error[RuntimeException].intercept {
// Should be supervised as resume // Should be supervised as resume
actor ! "boom" actor ! "boom"
} }
@ -1121,7 +1119,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
actor ! "ping" actor ! "ping"
probe.expectMessage("pong") probe.expectMessage("pong")
EventFilter[RuntimeException](occurrences = 1).intercept { LoggingEventFilter.error[RuntimeException].intercept {
// Should be supervised as resume // Should be supervised as resume
actor ! "boom" actor ! "boom"
} }
@ -1165,7 +1163,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage("started 1") probe.expectMessage("started 1")
ref ! "ping" ref ! "ping"
probe.expectMessage("pong 1") probe.expectMessage("pong 1")
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
ref ! "boom" ref ! "boom"
probe.expectMessage("crashing 1") probe.expectMessage("crashing 1")
ref ! "ping" ref ! "ping"
@ -1175,10 +1173,12 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage("pong 2") // from "ping" that was stashed probe.expectMessage("pong 2") // from "ping" that was stashed
ref ! "ping" ref ! "ping"
probe.expectMessage("pong 2") probe.expectMessage("pong 2")
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter
.error[TestException]
.intercept {
ref ! "boom" // now we should have replaced supervision with the resuming one ref ! "boom" // now we should have replaced supervision with the resuming one
probe.expectMessage("crashing 2") probe.expectMessage("crashing 2")
} }(system)
ref ! "ping" ref ! "ping"
probe.expectMessage("pong 2") probe.expectMessage("pong 2")
} }
@ -1208,7 +1208,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
}) })
.onFailure[DeathPactException](SupervisorStrategy.restart)) .onFailure[DeathPactException](SupervisorStrategy.restart))
EventFilter[DeathPactException](occurrences = 1).intercept { LoggingEventFilter.error[DeathPactException].intercept {
actor ! "boom" actor ! "boom"
val child = probe.expectMessageType[ActorRef[_]] val child = probe.expectMessageType[ActorRef[_]]
probe.expectTerminated(child, 3.seconds) probe.expectTerminated(child, 3.seconds)
@ -1223,7 +1223,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
.supervise(targetBehavior(probe.ref)) .supervise(targetBehavior(probe.ref))
.onFailure[Exc1](SupervisorStrategy.restart.withLoggingEnabled(true).withLogLevel(Level.INFO)) .onFailure[Exc1](SupervisorStrategy.restart.withLoggingEnabled(true).withLogLevel(Level.INFO))
val ref = spawn(behv) val ref = spawn(behv)
EventFilter.info(pattern = "exc-1", source = ref.path.toString, occurrences = 1).intercept { LoggingEventFilter.info("exc-1").intercept {
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
@ -1235,7 +1235,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
.supervise(targetBehavior(probe.ref)) .supervise(targetBehavior(probe.ref))
.onFailure[Exc1](SupervisorStrategy.restart.withLoggingEnabled(true).withLogLevel(Level.DEBUG)) .onFailure[Exc1](SupervisorStrategy.restart.withLoggingEnabled(true).withLogLevel(Level.DEBUG))
val ref = spawn(behv) val ref = spawn(behv)
EventFilter.info(pattern = "exc-1", source = ref.path.toString, occurrences = 0).intercept { LoggingEventFilter.info("exc-1").withSource(ref.path.toString).withOccurrences(0).intercept {
ref ! Throw(new Exc1) ref ! Throw(new Exc1)
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
@ -1263,7 +1263,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
ref ! Ping(1) ref ! Ping(1)
probe.expectMessage(Pong(1)) probe.expectMessage(Pong(1))
EventFilter[Exc1](occurrences = 1).intercept { LoggingEventFilter.error[Exc1].intercept {
ref.unsafeUpcast ! "boom" ref.unsafeUpcast ! "boom"
probe.expectMessage(ReceivedSignal(PreRestart)) probe.expectMessage(ReceivedSignal(PreRestart))
} }
@ -1311,7 +1311,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
}) })
.onFailure[TestException](strategy)) .onFailure[TestException](strategy))
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
actor ! "boom" actor ! "boom"
} }
createTestProbe().expectTerminated(actor, 3.second) createTestProbe().expectTerminated(actor, 3.second)

View file

@ -5,9 +5,10 @@
package akka.actor.typed package akka.actor.typed
import akka.actor.testkit.typed.scaladsl.TestInbox import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.{ Matchers, WordSpec } import org.scalatest.{ Matchers, WordSpec }
class TerminatedSpec extends WordSpec with Matchers { class TerminatedSpec extends WordSpec with Matchers with LogCapturing {
"Child Failed" must { "Child Failed" must {
"should be pattern matchable" in { "should be pattern matchable" in {

View file

@ -13,19 +13,15 @@ import scala.util.control.NoStackTrace
import akka.actor.DeadLetter import akka.actor.DeadLetter
import akka.actor.testkit.typed.TestException import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl._ import akka.actor.testkit.typed.scaladsl._
import akka.actor.typed.eventstream.EventStream
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.TimerScheduler import akka.actor.typed.scaladsl.TimerScheduler
import akka.testkit.{ EventFilter, TimingTest } import akka.testkit.TimingTest
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class TimerSpec extends ScalaTestWithActorTestKit(""" class TimerSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
akka.loggers = [ akka.testkit.TestEventListener ]
""") with WordSpecLike {
// FIXME #24348: eventfilter support in typed testkit
import scaladsl.adapter._
implicit val classicSystem = system.toClassic
sealed trait Command sealed trait Command
case class Tick(n: Int) extends Command case class Tick(n: Int) extends Command
@ -179,7 +175,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage(Tock(1)) probe.expectMessage(Tock(1))
val latch = new CountDownLatch(1) val latch = new CountDownLatch(1)
EventFilter[Exc](occurrences = 1).intercept { LoggingEventFilter.error[Exc].intercept {
// next Tock(1) is enqueued in mailbox, but should be discarded by new incarnation // next Tock(1) is enqueued in mailbox, but should be discarded by new incarnation
ref ! SlowThenThrow(latch, new Exc) ref ! SlowThenThrow(latch, new Exc)
@ -209,7 +205,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage(Tock(2)) probe.expectMessage(Tock(2))
EventFilter[Exc](occurrences = 1).intercept { LoggingEventFilter.error[Exc].intercept {
val latch = new CountDownLatch(1) val latch = new CountDownLatch(1)
// next Tock(2) is enqueued in mailbox, but should be discarded by new incarnation // next Tock(2) is enqueued in mailbox, but should be discarded by new incarnation
ref ! SlowThenThrow(latch, new Exc) ref ! SlowThenThrow(latch, new Exc)
@ -230,7 +226,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
target(probe.ref, timer, 1) target(probe.ref, timer, 1)
} }
val ref = spawn(behv) val ref = spawn(behv)
EventFilter[Exc](occurrences = 1).intercept { LoggingEventFilter.error[Exc].intercept {
ref ! Throw(new Exc) ref ! Throw(new Exc)
probe.expectMessage(GotPostStop(false)) probe.expectMessage(GotPostStop(false))
} }
@ -300,7 +296,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
} }
"not leak timers when PostStop is used" in { "not leak timers when PostStop is used" in {
val probe = TestProbe[Any]() val probe = TestProbe[DeadLetter]()
val ref = spawn(Behaviors.withTimers[String] { timers => val ref = spawn(Behaviors.withTimers[String] { timers =>
Behaviors.setup { _ => Behaviors.setup { _ =>
timers.startTimerWithFixedDelay("test", "test", 250.millis) timers.startTimerWithFixedDelay("test", "test", 250.millis)
@ -309,11 +305,11 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
} }
} }
}) })
EventFilter.info("stopping").intercept { LoggingEventFilter.info("stopping").intercept {
ref ! "stop" ref ! "stop"
} }
probe.expectTerminated(ref) probe.expectTerminated(ref)
system.toClassic.eventStream.subscribe(probe.ref.toClassic, classOf[DeadLetter]) system.eventStream ! EventStream.Subscribe(probe.ref)
probe.expectNoMessage(1.second) probe.expectNoMessage(1.second)
} }
} }
@ -348,7 +344,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
Behaviors.unhandled Behaviors.unhandled
} }
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
val ref = spawn(Behaviors.supervise(behv).onFailure[TestException](SupervisorStrategy.restart)) val ref = spawn(Behaviors.supervise(behv).onFailure[TestException](SupervisorStrategy.restart))
ref ! Tick(-1) ref ! Tick(-1)
probe.expectMessage(Tock(-1)) probe.expectMessage(Tock(-1))
@ -383,7 +379,7 @@ class TimerSpec extends ScalaTestWithActorTestKit("""
Behaviors.unhandled Behaviors.unhandled
} }
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
val ref = spawn(Behaviors.supervise(behv).onFailure[TestException](SupervisorStrategy.restart)) val ref = spawn(Behaviors.supervise(behv).onFailure[TestException](SupervisorStrategy.restart))
ref ! Tick(-1) ref ! Tick(-1)
probe.expectMessage(Tock(-1)) probe.expectMessage(Tock(-1))

View file

@ -11,10 +11,12 @@ import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.LogCapturing
object TransformMessagesSpec { object TransformMessagesSpec {
// this is the sample from the Scaladoc // this is the sample from the Scaladoc
@ -31,9 +33,7 @@ object TransformMessagesSpec {
} }
} }
class TransformMessagesSpec extends ScalaTestWithActorTestKit(""" class TransformMessagesSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
akka.loggers = [akka.testkit.TestEventListener]
""") with WordSpecLike {
implicit val classicSystem = system.toClassic implicit val classicSystem = system.toClassic
@ -62,15 +62,12 @@ class TransformMessagesSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[String]() val probe = TestProbe[String]()
val ref = spawn(intToString(probe.ref)) val ref = spawn(intToString(probe.ref))
// TestEventListener logs unhandled as warnings, silence that
EventFilter.warning(occurrences = 1).intercept {
ref ! 42 ref ! 42
ref ! 13 ref ! 13
ref ! 43 ref ! 43
probe.expectMessage("42") probe.expectMessage("42")
probe.expectMessage("43") probe.expectMessage("43")
} }
}
"not build up when the same transformMessages is used many times (initially)" in { "not build up when the same transformMessages is used many times (initially)" in {
val probe = TestProbe[String]() val probe = TestProbe[String]()
@ -137,7 +134,7 @@ class TransformMessagesSpec extends ScalaTestWithActorTestKit("""
case s => s.toLowerCase case s => s.toLowerCase
} }
EventFilter[ActorInitializationException](occurrences = 1).intercept { LoggingEventFilter.error[ActorInitializationException].intercept {
val ref = spawn(transform(transform(Behaviors.receiveMessage[String] { _ => val ref = spawn(transform(transform(Behaviors.receiveMessage[String] { _ =>
Behaviors.same Behaviors.same
}))) })))

View file

@ -8,20 +8,17 @@ import akka.Done
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.AbstractBehavior import akka.actor.typed.scaladsl.AbstractBehavior
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import scala.concurrent._ import scala.concurrent._
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.testkit.typed.TestException import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import com.typesafe.config.ConfigFactory import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
object WatchSpec { object WatchSpec {
val config = ConfigFactory.parseString("""
akka.loggers = ["akka.testkit.TestEventListener"]
""".stripMargin)
case object Stop case object Stop
@ -44,7 +41,7 @@ object WatchSpec {
case class StartWatchingWith(watchee: ActorRef[Stop.type], message: CustomTerminationMessage) extends Message case class StartWatchingWith(watchee: ActorRef[Stop.type], message: CustomTerminationMessage) extends Message
} }
class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpecLike { class WatchSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
implicit def classicSystem = system.toClassic implicit def classicSystem = system.toClassic
@ -110,7 +107,7 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
}, },
"supervised-child-parent") "supervised-child-parent")
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
parent ! "boom" parent ! "boom"
} }
probe.expectMessageType[ChildHasFailed].t.cause shouldEqual ex probe.expectMessageType[ChildHasFailed].t.cause shouldEqual ex
@ -145,7 +142,7 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
} }
val parent = spawn(behavior, "parent") val parent = spawn(behavior, "parent")
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
parent ! "boom" parent ! "boom"
} }
probe.expectMessageType[ChildHasFailed].t.cause shouldEqual ex probe.expectMessageType[ChildHasFailed].t.cause shouldEqual ex
@ -184,8 +181,8 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
}, },
"grosso-bosso") "grosso-bosso")
EventFilter[TestException](occurrences = 1).intercept { LoggingEventFilter.error[TestException].intercept {
EventFilter[DeathPactException](occurrences = 1).intercept { LoggingEventFilter.error[DeathPactException].intercept {
grossoBosso ! "boom" grossoBosso ! "boom"
} }
} }
@ -325,7 +322,9 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
"fail when watch is used after watchWith on same subject" in new ErrorTestSetup { "fail when watch is used after watchWith on same subject" in new ErrorTestSetup {
watcher ! StartWatchingWith(terminator, CustomTerminationMessage) watcher ! StartWatchingWith(terminator, CustomTerminationMessage)
EventFilter[IllegalStateException](pattern = ".*termination message was not overwritten.*", occurrences = 1) LoggingEventFilter
.error[IllegalStateException]
.withMessageContains("termination message was not overwritten")
.intercept { .intercept {
watcher ! StartWatching(terminator) watcher ! StartWatching(terminator)
} }
@ -336,7 +335,9 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
"fail when watchWitch is used after watchWith with different termination message" in new ErrorTestSetup { "fail when watchWitch is used after watchWith with different termination message" in new ErrorTestSetup {
watcher ! StartWatchingWith(terminator, CustomTerminationMessage) watcher ! StartWatchingWith(terminator, CustomTerminationMessage)
EventFilter[IllegalStateException](pattern = ".*termination message was not overwritten.*", occurrences = 1) LoggingEventFilter
.error[IllegalStateException]
.withMessageContains("termination message was not overwritten")
.intercept { .intercept {
watcher ! StartWatchingWith(terminator, CustomTerminationMessage2) watcher ! StartWatchingWith(terminator, CustomTerminationMessage2)
} }
@ -346,7 +347,9 @@ class WatchSpec extends ScalaTestWithActorTestKit(WatchSpec.config) with WordSpe
"fail when watchWith is used after watch on same subject" in new ErrorTestSetup { "fail when watchWith is used after watch on same subject" in new ErrorTestSetup {
watcher ! StartWatching(terminator) watcher ! StartWatching(terminator)
EventFilter[IllegalStateException](pattern = ".*termination message was not overwritten.*", occurrences = 1) LoggingEventFilter
.error[IllegalStateException]
.withMessageContains("termination message was not overwritten")
.intercept { .intercept {
watcher ! StartWatchingWith(terminator, CustomTerminationMessage) watcher ! StartWatchingWith(terminator, CustomTerminationMessage)
} }

View file

@ -5,6 +5,7 @@
package akka.actor.typed.coexistence package akka.actor.typed.coexistence
import akka.actor.Actor import akka.actor.Actor
import akka.actor.testkit.typed.TestException import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe }
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
@ -42,7 +43,7 @@ object TypedSupervisingClassicSpec {
class TypedSupervisingClassicSpec extends ScalaTestWithActorTestKit(""" class TypedSupervisingClassicSpec extends ScalaTestWithActorTestKit("""
akka.loglevel = INFO akka.loglevel = INFO
""".stripMargin) with WordSpecLike { """.stripMargin) with WordSpecLike with LogCapturing {
import TypedSupervisingClassicSpec._ import TypedSupervisingClassicSpec._
"Typed supervising classic" should { "Typed supervising classic" should {

View file

@ -6,10 +6,11 @@ package akka.actor.typed.eventstream
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe }
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class EventStreamSpec extends ScalaTestWithActorTestKit with WordSpecLike { class EventStreamSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import EventStreamSpec._ import EventStreamSpec._
import EventStream._ import EventStream._

View file

@ -9,6 +9,7 @@ import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.serialization.{ JavaSerializer, SerializationExtension } import akka.serialization.{ JavaSerializer, SerializationExtension }
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
@ -27,7 +28,10 @@ object ActorRefSerializationSpec {
case class MessageWrappingActorRef(s: String, ref: ActorRef[Unit]) extends java.io.Serializable case class MessageWrappingActorRef(s: String, ref: ActorRef[Unit]) extends java.io.Serializable
} }
class ActorRefSerializationSpec extends ScalaTestWithActorTestKit(ActorRefSerializationSpec.config) with WordSpecLike { class ActorRefSerializationSpec
extends ScalaTestWithActorTestKit(ActorRefSerializationSpec.config)
with WordSpecLike
with LogCapturing {
val serialization = SerializationExtension(system.toClassic) val serialization = SerializationExtension(system.toClassic)

View file

@ -14,13 +14,20 @@ import akka.Done
import akka.actor.CoordinatedShutdown import akka.actor.CoordinatedShutdown
import akka.actor.InvalidMessageException import akka.actor.InvalidMessageException
import akka.actor.testkit.typed.scaladsl.TestInbox import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import org.scalatest._ import org.scalatest._
import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually
import org.scalatest.concurrent.ScalaFutures import org.scalatest.concurrent.ScalaFutures
class ActorSystemSpec extends WordSpec with Matchers with BeforeAndAfterAll with ScalaFutures with Eventually { class ActorSystemSpec
extends WordSpec
with Matchers
with BeforeAndAfterAll
with ScalaFutures
with Eventually
with LogCapturing {
override implicit val patienceConfig = PatienceConfig(1.second) override implicit val patienceConfig = PatienceConfig(1.second)
def system[T](behavior: Behavior[T], name: String) = ActorSystem(behavior, name) def system[T](behavior: Behavior[T], name: String) = ActorSystem(behavior, name)

View file

@ -5,10 +5,12 @@
package akka.actor.typed.internal.receptionist package akka.actor.typed.internal.receptionist
import scala.concurrent.Future import scala.concurrent.Future
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
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.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed._ import akka.actor.typed._
import akka.actor.typed.receptionist.Receptionist import akka.actor.typed.receptionist.Receptionist
import akka.actor.typed.receptionist.Receptionist._ import akka.actor.typed.receptionist.Receptionist._
@ -36,7 +38,7 @@ object LocalReceptionistSpec {
} }
class LocalReceptionistSpec extends ScalaTestWithActorTestKit with WordSpecLike { class LocalReceptionistSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import LocalReceptionistSpec._ import LocalReceptionistSpec._
abstract class TestSetup { abstract class TestSetup {
@ -128,7 +130,7 @@ class LocalReceptionistSpec extends ScalaTestWithActorTestKit with WordSpecLike
} }
} }
class LocalReceptionistBehaviorSpec extends WordSpec with Matchers { class LocalReceptionistBehaviorSpec extends WordSpec with Matchers with LogCapturing {
import LocalReceptionistSpec._ import LocalReceptionistSpec._
def assertEmpty(inboxes: TestInbox[_]*): Unit = { def assertEmpty(inboxes: TestInbox[_]*): Unit = {

View file

@ -9,11 +9,13 @@ import akka.actor.typed.receptionist.ServiceKey
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.serialization.SerializationExtension import akka.serialization.SerializationExtension
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class ServiceKeySerializationSpec class ServiceKeySerializationSpec
extends ScalaTestWithActorTestKit(ActorRefSerializationSpec.config) extends ScalaTestWithActorTestKit(ActorRefSerializationSpec.config)
with WordSpecLike { with WordSpecLike
with LogCapturing {
val serialization = SerializationExtension(system.toClassic) val serialization = SerializationExtension(system.toClassic)

View file

@ -5,10 +5,11 @@
package akka.actor.typed.internal.routing package akka.actor.typed.internal.routing
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.Matchers import org.scalatest.Matchers
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class RoutingLogicSpec extends ScalaTestWithActorTestKit with WordSpecLike with Matchers { class RoutingLogicSpec extends ScalaTestWithActorTestKit with WordSpecLike with Matchers with LogCapturing {
"The round robin routing logic" must { "The round robin routing logic" must {

View file

@ -4,22 +4,21 @@
package akka.actor.typed.scaladsl package akka.actor.typed.scaladsl
import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.{ ActorRef, PostStop, Props } import akka.actor.typed.{ ActorRef, PostStop, Props }
import akka.testkit.EventFilter
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.concurrent.TimeoutException import scala.concurrent.TimeoutException
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.reflect.ClassTag import scala.reflect.ClassTag
import scala.util.{ Failure, Success } import scala.util.{ Failure, Success }
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
object ActorContextAskSpec { object ActorContextAskSpec {
val config = ConfigFactory.parseString(""" val config = ConfigFactory.parseString("""
akka.loggers = ["akka.testkit.TestEventListener"]
ping-pong-dispatcher { ping-pong-dispatcher {
executor = thread-pool-executor executor = thread-pool-executor
type = PinnedDispatcher type = PinnedDispatcher
@ -31,9 +30,10 @@ object ActorContextAskSpec {
""") """)
} }
class ActorContextAskSpec extends ScalaTestWithActorTestKit(ActorContextAskSpec.config) with WordSpecLike { class ActorContextAskSpec
extends ScalaTestWithActorTestKit(ActorContextAskSpec.config)
implicit val classic = system.toClassic // FIXME #24348: eventfilter support in testkit with WordSpecLike
with LogCapturing {
"The Scala DSL ActorContext" must { "The Scala DSL ActorContext" must {
@ -104,7 +104,7 @@ class ActorContextAskSpec extends ScalaTestWithActorTestKit(ActorContextAskSpec.
} }
} }
EventFilter[NotImplementedError](occurrences = 1, start = "Pong").intercept { LoggingEventFilter.error[NotImplementedError].withMessageContains("Pong").intercept {
spawn(snitch) spawn(snitch)
} }
@ -126,11 +126,7 @@ class ActorContextAskSpec extends ScalaTestWithActorTestKit(ActorContextAskSpec.
} }
} }
EventFilter.warning(occurrences = 1, message = "received dead letter: boo").intercept {
EventFilter.info(occurrences = 1, start = "Message [java.lang.String]").intercept {
spawn(snitch) spawn(snitch)
}
}
val exc = probe.expectMessageType[TimeoutException] val exc = probe.expectMessageType[TimeoutException]
exc.getMessage should include("had already been terminated") exc.getMessage should include("had already been terminated")

View file

@ -8,6 +8,7 @@ import scala.concurrent.Future
import scala.util.control.NoStackTrace import scala.util.control.NoStackTrace
import scala.util.{ Failure, Success } import scala.util.{ Failure, Success }
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe } import akka.actor.testkit.typed.scaladsl.{ ScalaTestWithActorTestKit, TestProbe }
import akka.actor.typed.Props import akka.actor.typed.Props
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
@ -24,7 +25,8 @@ object ActorContextPipeToSelfSpec {
final class ActorContextPipeToSelfSpec final class ActorContextPipeToSelfSpec
extends ScalaTestWithActorTestKit(ActorContextPipeToSelfSpec.config) extends ScalaTestWithActorTestKit(ActorContextPipeToSelfSpec.config)
with WordSpecLike { with WordSpecLike
with LogCapturing {
"The Scala DSL ActorContext pipeToSelf" must { "The Scala DSL ActorContext pipeToSelf" must {
"handle success" in { responseFrom(Future.successful("hi")) should ===("ok: hi") } "handle success" in { responseFrom(Future.successful("hi")) should ===("ok: hi") }

View file

@ -5,20 +5,24 @@
package akka.actor.typed.scaladsl package akka.actor.typed.scaladsl
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import java.util.concurrent.atomic.AtomicReference
import akka.actor.testkit.typed.TestException import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.ActorTestKit
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.testkit.LoggingEventFilter._ import akka.event.DefaultLoggingFilter
import akka.actor.typed.testkit.{ AppenderInterceptor, LoggingEventFilter } import akka.event.Logging.DefaultLogger
import akka.actor.typed.{ ActorRef, Behavior, LogOptions } import akka.event.slf4j.Slf4jLogger
import akka.event.Logging import akka.event.slf4j.Slf4jLoggingFilter
import akka.event.Logging.{ LogEvent, LogEventWithCause, LogEventWithMarker } import com.typesafe.config.ConfigFactory
import akka.testkit.EventFilter
import ch.qos.logback.classic.spi.ILoggingEvent
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import org.slf4j.event.{ Level, LoggingEvent } import org.slf4j.LoggerFactory
import org.slf4j.helpers.{ BasicMarkerFactory, SubstituteLogger, SubstituteLoggerFactory } import org.slf4j.MDC
import org.slf4j.helpers.BasicMarkerFactory
class SomeClass class SomeClass
@ -43,128 +47,120 @@ class BehaviorWhereTheLoggerIsUsed(context: ActorContext[String]) extends Abstra
class ActorLoggingSpec extends ScalaTestWithActorTestKit(""" class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
akka.loglevel = DEBUG # test verifies debug akka.loglevel = DEBUG # test verifies debug
akka.loggers = ["akka.actor.typed.testkit.TestEventListener"] """) with WordSpecLike with LogCapturing {
""") with WordSpecLike {
val marker = new BasicMarkerFactory().getMarker("marker") val marker = new BasicMarkerFactory().getMarker("marker")
val cause = new TestException("böö") val cause = TestException("böö")
implicit val classic = system.toClassic implicit val classic = system.toClassic
class AnotherLoggerClass
"Logging in an actor" must { "Logging in an actor" must {
"log messages and signals" in {
val factory = new SubstituteLoggerFactory()
val substituteLogger: SubstituteLogger = factory.getLogger("substitute").asInstanceOf[SubstituteLogger]
val opts = LogOptions().withLevel(Level.DEBUG).withLogger(substituteLogger)
val behavior: Behavior[String] = Behaviors.logMessages(opts, Behaviors.ignore)
val ref: ActorRef[String] = spawn(behavior)
debug(s"actor ${ref.path.toString} received message Hello", source = ref.path.toString, occurrences = 1)
.intercept(ref ! "Hello", factory.getEventQueue)
debug(s"actor ${ref.path.toString} received signal PostStop", source = ref.path.toString, occurrences = 1)
.intercept(testKit.stop(ref), factory.getEventQueue)
}
//TODO be aware of context.log.xyz logging are very different as they aren't created by the InterceptorImpl! @see LogMessagesInterceptor.aroundReceive
// That's why AppenderInterceptor.events approach has been taken
"be conveniently available from the context" in { "be conveniently available from the context" in {
val behavior: Behavior[String] = Behaviors.setup[String] { context => val behavior: Behavior[String] = Behaviors.setup[String] { context =>
println(s"context out ${context.executionContext.hashCode()}")
context.log.info("Started") context.log.info("Started")
Behaviors.receive { (context, message) => Behaviors.receive { (context, message) =>
println(s"context in ${context.executionContext.hashCode()}")
context.log.info("got message {}", message) context.log.info("got message {}", message)
Behaviors.same Behaviors.same
} }
} }
val actor = LoggingEventFilter val actor = LoggingEventFilter.info("Started").intercept(spawn(behavior, "the-actor"))
.info("Started", occurrences = 1)
.interceptIt(spawn(behavior, "the-actor"), AppenderInterceptor.events)
LoggingEventFilter.info("got message Hello").intercept(actor ! "Hello")
}
"log with custom Logger class" in {
val behavior: Behavior[String] = Behaviors.setup[String] { context =>
context.setLoggerName(classOf[AnotherLoggerClass])
context.log.info("Started")
Behaviors.receive { (context, message) =>
context.log.info("got message {}", message)
Behaviors.same
}
}
val actor =
LoggingEventFilter.info("Started").withLoggerName(classOf[AnotherLoggerClass].getName).intercept {
spawn(behavior, "the-other-actor")
}
// verify that it's logged with `AnotherLoggerClass`
// verify that it's only capturing log events for that logger and not any other logger when interceptLogger
// is used
val count = new AtomicInteger
LoggingEventFilter LoggingEventFilter
.info("got message Hello", occurrences = 1) .custom { logEvent =>
.interceptIt(actor ! "Hello", AppenderInterceptor.events) count.incrementAndGet()
logEvent.message == "got message Hello" && logEvent.loggerName == classOf[AnotherLoggerClass].getName
}
.withLoggerName(classOf[AnotherLoggerClass].getName)
.withOccurrences(2)
.intercept {
actor ! "Hello"
LoggerFactory.getLogger(classOf[ActorLoggingSpec]).debug("Hello from other logger")
actor ! "Hello"
}
count.get should ===(2)
} }
"contain the class name where the first log was called" in { "contain the class name where the first log was called" in {
val eventFilter = custom({ val eventFilter = LoggingEventFilter.custom({
case l: LoggingEvent if l.getLoggerName == classOf[ActorLoggingSpec].getName => case event if event.loggerName == classOf[ActorLoggingSpec].getName =>
true true
case l: LoggingEvent => case event =>
println(l.getLoggerName) println(event.loggerName)
false false
}, occurrences = 1) })
eventFilter.interceptIt(spawn(Behaviors.setup[String] { context => eventFilter.intercept(spawn(Behaviors.setup[String] { context =>
context.log.info("Started") context.log.info("Started")
Behaviors.receive { (context, message) => Behaviors.receive { (context, message) =>
context.log.info("got message {}", message) context.log.info("got message {}", message)
Behaviors.same Behaviors.same
} }
}, "the-actor-with-class"), AppenderInterceptor.events) }, "the-actor-with-class"))
} }
//TODO all below
"contain the object class name where the first log was called" in { "contain the object class name where the first log was called" in {
val eventFilter = EventFilter.custom({ val eventFilter = LoggingEventFilter.custom({
case l: LogEvent if l.logClass == WhereTheBehaviorIsDefined.getClass => true case event if event.loggerName == WhereTheBehaviorIsDefined.getClass.getName => true
case l: LogEvent => case other =>
println(l.logClass) println(other.loggerName)
false false
}, occurrences = 1) })
eventFilter.intercept { eventFilter.intercept(spawn(WhereTheBehaviorIsDefined.behavior, "the-actor-with-object"))
spawn(WhereTheBehaviorIsDefined.behavior, "the-actor-with-object")
}
} }
"contain the abstract behavior class name where the first log was called" in { "contain the abstract behavior class name where the first log was called" in {
val eventFilter = EventFilter.custom({ val eventFilter = LoggingEventFilter.custom({
case l: LogEvent if l.logClass == classOf[BehaviorWhereTheLoggerIsUsed] => true case event if event.loggerName == classOf[BehaviorWhereTheLoggerIsUsed].getName => true
case l: LogEvent => case other =>
println(l.logClass) println(other.loggerName)
false false
}, occurrences = 1) })
eventFilter.intercept { eventFilter.intercept {
spawn(BehaviorWhereTheLoggerIsUsed.behavior, "the-actor-with-behavior") spawn(Behaviors.setup[String](context => new BehaviorWhereTheLoggerIsUsed(context)), "the-actor-with-behavior")
} }
} }
"allow for adapting log source and class" in {
//TODO WIP...
val eventFilter = custom({
case l: ILoggingEvent =>
l.getLoggerName == classOf[SomeClass] &&
l.getCallerData == "who-knows-where-it-came-from" &&
l.getMDCPropertyMap == Map("mdc" -> true) // mdc should be kept
}, occurrences = 1)
spawn(Behaviors.setup[String] { context =>
context.log.info("Started")
Behaviors.empty
}, "the-actor-with-custom-class")
Thread.sleep(1)
eventFilter.interceptIt(println(""), AppenderInterceptor.events)
}
"pass markers to the log" in { "pass markers to the log" in {
EventFilter LoggingEventFilter
.custom({ .custom { event =>
case event: LogEventWithMarker if event.marker.name == marker.getName => true event.marker.map(_.getName) == Option(marker.getName)
}, occurrences = 9) }
.withOccurrences(5)
.intercept(spawn(Behaviors.setup[Any] { context => .intercept(spawn(Behaviors.setup[Any] { context =>
context.log.debug(marker, "whatever") context.log.debug(marker, "whatever")
context.log.info(marker, "whatever") context.log.info(marker, "whatever")
@ -176,10 +172,11 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
} }
"pass cause with warn" in { "pass cause with warn" in {
EventFilter LoggingEventFilter
.custom({ .custom { event =>
case event: LogEventWithCause if event.cause == cause => true event.throwable == Option(cause)
}, occurrences = 2) }
.withOccurrences(2)
.intercept(spawn(Behaviors.setup[Any] { context => .intercept(spawn(Behaviors.setup[Any] { context =>
context.log.warn("whatever", cause) context.log.warn("whatever", cause)
context.log.warn(marker, "whatever", cause) context.log.warn(marker, "whatever", cause)
@ -191,20 +188,22 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
// Not the best test but at least it exercises every log overload ;) // Not the best test but at least it exercises every log overload ;)
EventFilter LoggingEventFilter
.custom({ .custom { _ =>
case _ => true // any is fine, we're just after the right count of statements reaching the listener true // any is fine, we're just after the right count of statements reaching the listener
}, occurrences = 120) }
.intercept { .withOccurrences(36)
spawn(Behaviors.setup[String] { context => .intercept({
spawn(Behaviors.setup[String] {
context =>
context.log.debug("message") context.log.debug("message")
context.log.debug("{}", "arg1") context.log.debug("{}", "arg1")
context.log // using `: Any` to avoid "ambiguous reference to overloaded definition", see also LoggerOpsSpec
.debug("{} {}", "arg1", "arg2": Any) //using Int to avoid ambiguous reference to overloaded definition context.log.debug("{} {}", "arg1", "arg2": Any)
context.log.debug("{} {} {}", "arg1", "arg2", "arg3") context.log.debug("{} {} {}", "arg1", "arg2", "arg3")
context.log.debug(marker, "message") context.log.debug(marker, "message")
context.log.debug(marker, "{}", "arg1") context.log.debug(marker, "{}", "arg1")
context.log.debug(marker, "{} {}", "arg1", "arg2": Any) //using Int to avoid ambiguous reference to overloaded definition context.log.debug(marker, "{} {}", "arg1", "arg2": Any)
context.log.debug(marker, "{} {} {}", "arg1", "arg2", "arg3") context.log.debug(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.info("message") context.log.info("message")
@ -230,19 +229,69 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
context.log.error("{}", "arg1") context.log.error("{}", "arg1")
context.log.error("{} {}", "arg1", "arg2": Any) context.log.error("{} {}", "arg1", "arg2": Any)
context.log.error("{} {} {}", "arg1", "arg2", "arg3") context.log.error("{} {} {}", "arg1", "arg2", "arg3")
// using to avoid vararg problem for primitive type, see also LoggerOpsSpec
context.log.error("{} {} {}", "arg1", "arg2", 3.asInstanceOf[AnyRef])
context.log.error(marker, "message") context.log.error(marker, "message")
context.log.error(marker, "{}", "arg1") context.log.error(marker, "{}", "arg1")
context.log.error(marker, "{} {}", "arg1", "arg2": Any) context.log.error(marker, "{} {}", "arg1", "arg2": Any)
context.log.error(marker, "{} {} {}", "arg1", "arg2", "arg3") context.log.error(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.error(marker, "{} {} {}", "arg1", "arg2", 3.asInstanceOf[AnyRef])
context.log.error("message", cause) context.log.error("message", cause)
Behaviors.stopped Behaviors.stopped
}) })
})
}
"use Slf4jLogger from akka-slf4j automatically" in {
LoggingEventFilter.info("via Slf4jLogger").intercept {
// this will log via classic eventStream
system.toClassic.log.info("via Slf4jLogger")
} }
} }
} }
"SLF4J Settings" must {
import akka.actor.typed.scaladsl.adapter._
import akka.actor.ExtendedActorSystem
import akka.actor.{ ActorSystem => ClassicActorSystem }
"by default be amended to use Slf4jLogger" in {
system.settings.config.getStringList("akka.loggers").size() should ===(1)
system.settings.config.getStringList("akka.loggers").get(0) should ===(classOf[Slf4jLogger].getName)
system.settings.config.getString("akka.logging-filter") should ===(classOf[Slf4jLoggingFilter].getName)
system.toClassic.settings.Loggers should ===(List(classOf[Slf4jLogger].getName))
system.toClassic.settings.LoggingFilter should ===(classOf[Slf4jLoggingFilter].getName)
}
"by default be amended to use Slf4jLogger when starting classic ActorSystem" in {
val classicSys = akka.actor.ActorSystem(system.name)
try {
classicSys.settings.config.getStringList("akka.loggers").size() should ===(1)
classicSys.settings.config.getStringList("akka.loggers").get(0) should ===(classOf[Slf4jLogger].getName)
classicSys.settings.config.getString("akka.logging-filter") should ===(classOf[Slf4jLoggingFilter].getName)
classicSys.settings.Loggers should ===(List(classOf[Slf4jLogger].getName))
classicSys.settings.LoggingFilter should ===(classOf[Slf4jLoggingFilter].getName)
} finally {
ActorTestKit.shutdown(classicSys.toTyped)
}
}
"not be amended when use-slf4j=off" in {
val dynamicAccess = system.toClassic.asInstanceOf[ExtendedActorSystem].dynamicAccess
val config = ClassicActorSystem.Settings.amendSlf4jConfig(
ConfigFactory.parseString("akka.use-slf4j = off").withFallback(ConfigFactory.defaultReference()),
dynamicAccess)
config.getStringList("akka.loggers").size() should ===(1)
config.getStringList("akka.loggers").get(0) should ===(classOf[DefaultLogger].getName)
config.getString("akka.logging-filter") should ===(classOf[DefaultLoggingFilter].getName)
}
}
trait Protocol { trait Protocol {
def transactionId: Long def transactionId: Long
} }
@ -254,7 +303,6 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
val behaviors = Behaviors.withMdc[Protocol]( val behaviors = Behaviors.withMdc[Protocol](
Map("static" -> "1"), Map("static" -> "1"),
// FIXME why u no infer the type here Scala?? // FIXME why u no infer the type here Scala??
//TODO review that change from Map[String,Any] to Map[String,String] is viable
(message: Protocol) => (message: Protocol) =>
if (message.transactionId == 1) if (message.transactionId == 1)
Map("txId" -> message.transactionId.toString, "first" -> "true") Map("txId" -> message.transactionId.toString, "first" -> "true")
@ -268,47 +316,28 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
} }
} }
// mdc on defer is empty (thread and timestamp MDC is added by logger backend) // mdc on defer is empty
val ref = EventFilter val ref = LoggingEventFilter
.custom( .info("Starting")
{ // not counting for example "akkaSource", but it shouldn't have any other entries
case logEvent if logEvent.level == Logging.InfoLevel => .withCustom(logEvent => logEvent.mdc.keysIterator.forall(_.startsWith("akka")))
logEvent.message should ===("Starting")
logEvent.mdc shouldBe empty
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept { .intercept {
spawn(behaviors) spawn(behaviors)
} }
// mdc on message // mdc on message
EventFilter LoggingEventFilter
.custom( .info("Got message!")
{ .withMdc(Map("static" -> "1", "txId" -> "1", "first" -> "true"))
case logEvent if logEvent.level == Logging.InfoLevel =>
logEvent.message should ===("Got message!")
logEvent.mdc should ===(Map("static" -> 1, "txId" -> 1L, "first" -> true))
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept { .intercept {
ref ! Message(1, "first") ref ! Message(1, "first")
} }
// mdc does not leak between messages // mdc does not leak between messages
EventFilter LoggingEventFilter
.custom( .info("Got message!")
{ .withMdc(Map("static" -> "1", "txId" -> "2"))
case logEvent if logEvent.level == Logging.InfoLevel => .withCustom(event => !event.mdc.contains("first"))
logEvent.message should ===("Got message!")
logEvent.mdc should ===(Map("static" -> 1, "txId" -> 2L))
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept { .intercept {
ref ! Message(2, "second") ref ! Message(2, "second")
} }
@ -327,16 +356,10 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
} }
val ref = spawn(behavior) val ref = spawn(behavior)
EventFilter LoggingEventFilter
.custom( .info("message")
{ .withMdc(Map("outermost" -> "true"))
case logEvent if logEvent.level == Logging.InfoLevel => .withCustom(event => !event.mdc.contains("innermost"))
logEvent.message should ===("message")
logEvent.mdc should ===(Map("outermost" -> true))
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept { .intercept {
ref ! "message" ref ! "message"
} }
@ -355,32 +378,15 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
} }
val ref = spawn(Behaviors.withMdc(Map("hasMdc" -> "true"))(behavior)) val ref = spawn(Behaviors.withMdc(Map("hasMdc" -> "true"))(behavior))
EventFilter LoggingEventFilter.info("message").withMdc(Map("hasMdc" -> "true")).intercept {
.custom(
{
case logEvent if logEvent.level == Logging.InfoLevel =>
logEvent.message should ===("message")
logEvent.mdc should ===(Map("hasMdc" -> true))
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept {
ref ! "message" ref ! "message"
} }
ref ! "new-behavior" ref ! "new-behavior"
EventFilter LoggingEventFilter
.custom( .info("message")
{ .withMdc(Map("hasMdc" -> "true")) // original mdc should stay
case logEvent if logEvent.level == Logging.InfoLevel =>
logEvent.message should ===("message")
logEvent.mdc should ===(Map("hasMdc" -> true)) // original mdc should stay
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept { .intercept {
ref ! "message" ref ! "message"
} }
@ -404,30 +410,13 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
} }
val ref = spawn(behavior) val ref = spawn(behavior)
EventFilter LoggingEventFilter.info("message").withMdc(Map("mdc-version" -> "1")).intercept {
.custom(
{
case logEvent if logEvent.level == Logging.InfoLevel =>
logEvent.message should ===("message")
logEvent.mdc should ===(Map("mdc-version" -> 1))
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept {
ref ! "message" ref ! "message"
} }
ref ! "new-mdc" ref ! "new-mdc"
EventFilter LoggingEventFilter
.custom( .info("message")
{ .withMdc(Map("mdc-version" -> "2")) // mdc should have been replaced
case logEvent if logEvent.level == Logging.InfoLevel =>
logEvent.message should ===("message")
logEvent.mdc should ===(Map("mdc-version" -> 2)) // mdc should have been replaced
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept { .intercept {
ref ! "message" ref ! "message"
} }
@ -437,43 +426,57 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
"provide a withMdc decorator" in { "provide a withMdc decorator" in {
val behavior = Behaviors.withMdc[Protocol](Map("mdc" -> "outer"))(Behaviors.setup { context => val behavior = Behaviors.withMdc[Protocol](Map("mdc" -> "outer"))(Behaviors.setup { context =>
Behaviors.receiveMessage { _ => Behaviors.receiveMessage { _ =>
org.slf4j.MDC.put("mdc", "inner") context.log.info("first")
context.log.info("Got message log.withMDC!") org.slf4j.MDC.put("mdc", "inner-" + org.slf4j.MDC.get("mdc"))
// after log.withMdc so we know it didn't change the outer mdc context.log.info("second")
context.log.info("Got message behavior.withMdc!")
Behaviors.same Behaviors.same
} }
}) })
// mdc on message // mdc on message
val ref = spawn(behavior) val ref = spawn(behavior)
EventFilter LoggingEventFilter.info("first").withMdc(Map("mdc" -> "outer")).intercept {
.custom( LoggingEventFilter.info("second").withMdc(Map("mdc" -> "inner-outer")).intercept {
{
case logEvent if logEvent.level == Logging.InfoLevel =>
logEvent.message should ===("Got message behavior.withMdc!")
logEvent.mdc should ===(Map("mdc" -> "outer"))
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept {
EventFilter
.custom(
{
case logEvent if logEvent.level == Logging.InfoLevel =>
logEvent.message should ===("Got message log.withMDC!")
logEvent.mdc should ===(Map("mdc" -> "inner"))
true
case other => system.log.error(s"Unexpected log event: {}", other); false
},
occurrences = 1)
.intercept {
ref ! Message(1, "first") ref ! Message(1, "first")
} }
} }
} }
"always include some MDC values in the log" in {
// need AtomicReference because LoggingFilter defined before actor is created and ActorTestKit names are dynamic
val actorPathStr = new AtomicReference[String]
val behavior =
Behaviors.setup[Message] { context =>
actorPathStr.set(context.self.path.toString)
context.log.info("Starting")
Behaviors.receiveMessage { _ =>
if (MDC.get("logSource") != null)
throw new IllegalStateException("MDC wasn't cleared. logSource has value before context.log is accessed.")
context.log.info("Got message!")
Behaviors.same
}
}
// log from setup
// can't use LoggingEventFilter.withMdc here because the actorPathStr isn't know yet
val ref =
LoggingEventFilter.info("Starting").withCustom(event => event.mdc("akkaSource") == actorPathStr.get).intercept {
spawn(behavior)
}
// on message
LoggingEventFilter
.info("Got message!")
.withMdc(Map("akkaSource" -> actorPathStr.get))
.withOccurrences(10)
.intercept {
(1 to 10).foreach { n =>
ref ! Message(n, s"msg-$n")
}
}
}
} }
} }

View file

@ -9,6 +9,7 @@ import akka.actor.setup.ActorSystemSetup
import akka.actor.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
@ -39,7 +40,10 @@ object DispatcherSelectorSpec {
} }
class DispatcherSelectorSpec extends ScalaTestWithActorTestKit(DispatcherSelectorSpec.config) with WordSpecLike { class DispatcherSelectorSpec
extends ScalaTestWithActorTestKit(DispatcherSelectorSpec.config)
with WordSpecLike
with LogCapturing {
import DispatcherSelectorSpec.PingPong import DispatcherSelectorSpec.PingPong
import DispatcherSelectorSpec.PingPong._ import DispatcherSelectorSpec.PingPong._

View file

@ -9,9 +9,10 @@ import akka.Done
import akka.NotUsed import akka.NotUsed
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
final class GracefulStopSpec extends ScalaTestWithActorTestKit with WordSpecLike { final class GracefulStopSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
"Graceful stop" must { "Graceful stop" must {

View file

@ -0,0 +1,77 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.typed.scaladsl
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.WordSpecLike
import org.slf4j.LoggerFactory
object LoggerOpsSpec {
case class Value1(i: Int)
case class Value2(i: Int)
case class Value3(i: Int)
}
class LoggerOpsSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import LoggerOpsSpec._
val log = LoggerFactory.getLogger(getClass)
"LoggerOps" must {
"provide extension method for 2 arguments" in {
LoggingEventFilter.info("[template a b]").intercept {
log.info2("[template {} {}]", "a", "b")
}
LoggingEventFilter.info("[template a 2]").intercept {
log.info2("[template {} {}]", "a", 2)
}
LoggingEventFilter.info("[template 1 2]").intercept {
log.info2("[template {} {}]", 1, 2)
}
LoggingEventFilter.info("[template 1 b]").intercept {
log.info2("[template {} {}]", 1, "b")
}
LoggingEventFilter.info("[template a Value2(2)]").intercept {
log.info2("[template {} {}]", "a", Value2(2))
}
LoggingEventFilter.info("[template Value1(1) Value1(1)]").intercept {
log.info2("[template {} {}]", Value1(1), Value1(1))
}
LoggingEventFilter.info("[template Value1(1) Value2(2)]").intercept {
log.info2("[template {} {}]", Value1(1), Value2(2))
}
}
"provide extension method for vararg arguments" in {
LoggingEventFilter.info("[template a b c]").intercept {
log.infoN("[template {} {} {}]", "a", "b", "c")
}
LoggingEventFilter.info("[template a b 3]").intercept {
log.infoN("[template {} {} {}]", "a", "b", 3)
}
LoggingEventFilter.info("[template a 2 c]").intercept {
log.infoN("[template {} {} {}]", "a", 2, "c")
}
LoggingEventFilter.info("[template 1 2 3]").intercept {
log.infoN("[template {} {} {}]", 1, 2, 3)
}
LoggingEventFilter.info("[template 1 b c]").intercept {
log.infoN("[template {} {} {}]", 1, "b", "c")
}
LoggingEventFilter.info("[template a Value2(2) Value3(3)]").intercept {
log.infoN("[template {} {} {}]", "a", Value2(2), Value3(3))
}
LoggingEventFilter.info("[template Value1(1) Value1(1) Value1(1)]").intercept {
log.infoN("[template {} {} {}]", Value1(1), Value1(1), Value1(1))
}
LoggingEventFilter.info("[template Value1(1) Value2(2) Value3(3)]").intercept {
log.infoN("[template {} {} {}]", Value1(1), Value2(2), Value3(3))
}
}
}
}

View file

@ -4,21 +4,22 @@
package akka.actor.typed.scaladsl package akka.actor.typed.scaladsl
import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.TestException import akka.actor.testkit.typed.TestException
import akka.actor.typed.scaladsl.adapter._ import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
import akka.actor.typed.PostStop import akka.actor.typed.PostStop
import akka.actor.typed.Props import akka.actor.typed.Props
import akka.testkit.EventFilter
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.eventstream.EventStream
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
object MessageAdapterSpec { object MessageAdapterSpec {
val config = ConfigFactory.parseString(""" val config = ConfigFactory.parseString("""
akka.loggers = ["akka.testkit.TestEventListener"]
akka.log-dead-letters = off akka.log-dead-letters = off
ping-pong-dispatcher { ping-pong-dispatcher {
executor = thread-pool-executor executor = thread-pool-executor
@ -31,9 +32,10 @@ object MessageAdapterSpec {
""") """)
} }
class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.config) with WordSpecLike { class MessageAdapterSpec
extends ScalaTestWithActorTestKit(MessageAdapterSpec.config)
implicit val classic = system.toClassic // FIXME #24348: eventfilter support in testkit with WordSpecLike
with LogCapturing {
"Message adapters" must { "Message adapters" must {
@ -139,6 +141,8 @@ class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.co
Behaviors.same Behaviors.same
}) })
val unhandledProbe = createTestProbe[UnhandledMessage]()
system.eventStream ! EventStream.Subscribe(unhandledProbe.ref)
val probe = TestProbe[Wrapped]() val probe = TestProbe[Wrapped]()
val snitch = Behaviors.setup[Wrapped] { context => val snitch = Behaviors.setup[Wrapped] { context =>
@ -155,9 +159,8 @@ class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.co
} }
} }
EventFilter.warning(start = "unhandled message", occurrences = 1).intercept {
spawn(snitch) spawn(snitch)
} unhandledProbe.receiveMessage()
probe.expectMessage(Wrapped("1", Pong1("hello-1"))) probe.expectMessage(Wrapped("1", Pong1("hello-1")))
// hello-2 discarded because it was wrong type // hello-2 discarded because it was wrong type
@ -199,11 +202,7 @@ class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.co
} }
} }
EventFilter.warning(pattern = ".*received dead letter.*", occurrences = 1).intercept {
EventFilter[TestException](occurrences = 1).intercept {
spawn(snitch) spawn(snitch)
}
}
probe.expectMessage(Wrapped(1, Pong("hello"))) probe.expectMessage(Wrapped(1, Pong("hello")))
probe.expectMessage(Wrapped(2, Pong("hello"))) probe.expectMessage(Wrapped(2, Pong("hello")))
@ -250,12 +249,10 @@ class MessageAdapterSpec extends ScalaTestWithActorTestKit(MessageAdapterSpec.co
behv(count = 1) behv(count = 1)
} }
EventFilter.warning(pattern = ".*received dead letter.*", occurrences = 2).intercept {
// Not expecting "Exception thrown out of adapter. Stopping myself" // Not expecting "Exception thrown out of adapter. Stopping myself"
EventFilter[TestException](message = "boom", occurrences = 1).intercept { LoggingEventFilter.error[TestException].withMessageContains("boom").intercept {
spawn(snitch) spawn(snitch)
} }
}
probe.expectMessage(1) probe.expectMessage(1)
probe.expectMessage(2) probe.expectMessage(2)

View file

@ -8,9 +8,10 @@ package scaladsl
import akka.Done import akka.Done
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
final class OnSignalSpec extends ScalaTestWithActorTestKit with WordSpecLike { final class OnSignalSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
"An Actor.OnSignal behavior" must { "An Actor.OnSignal behavior" must {
"must correctly install the signal handler" in { "must correctly install the signal handler" in {

View file

@ -7,9 +7,10 @@ package scaladsl
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class ReceivePartialSpec extends ScalaTestWithActorTestKit with WordSpecLike { class ReceivePartialSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
implicit val ec = system.executionContext implicit val ec = system.executionContext

View file

@ -6,23 +6,24 @@ package akka.actor.typed.scaladsl
import java.util.concurrent.atomic.AtomicInteger import java.util.concurrent.atomic.AtomicInteger
import akka.actor.Dropped import akka.actor.Dropped
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
import akka.actor.typed.eventstream.EventStream
import akka.actor.typed.internal.routing.GroupRouterImpl import akka.actor.typed.internal.routing.GroupRouterImpl
import akka.actor.typed.internal.routing.RoutingLogics import akka.actor.typed.internal.routing.RoutingLogics
import akka.actor.typed.receptionist.Receptionist import akka.actor.typed.receptionist.Receptionist
import akka.actor.typed.receptionist.ServiceKey import akka.actor.typed.receptionist.ServiceKey
import akka.actor.typed.scaladsl.adapter._ import akka.actor.typed.scaladsl.adapter._
import akka.testkit.EventFilter
import org.scalatest.Matchers import org.scalatest.Matchers
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class RoutersSpec extends ScalaTestWithActorTestKit(""" class RoutersSpec extends ScalaTestWithActorTestKit("""
akka.loggers = ["akka.testkit.TestEventListener"]
akka.loglevel=debug akka.loglevel=debug
""") with WordSpecLike with Matchers { """) with WordSpecLike with Matchers with LogCapturing {
// needed for the event filter // needed for the event filter
implicit val classicSystem = system.toClassic implicit val classicSystem = system.toClassic
@ -84,7 +85,7 @@ class RoutersSpec extends ScalaTestWithActorTestKit("""
Behaviors.same Behaviors.same
})) }))
EventFilter.debug(start = "Pool child stopped", occurrences = 2).intercept { LoggingEventFilter.debug("Pool child stopped").withOccurrences(2).intercept {
pool ! "stop" pool ! "stop"
pool ! "stop" pool ! "stop"
} }
@ -109,7 +110,7 @@ class RoutersSpec extends ScalaTestWithActorTestKit("""
Behaviors.stopped Behaviors.stopped
})) }))
EventFilter.info(start = "Last pool child stopped, stopping pool", occurrences = 1).intercept { LoggingEventFilter.info("Last pool child stopped, stopping pool").intercept {
(0 to 3).foreach { _ => (0 to 3).foreach { _ =>
pool ! "stop" pool ! "stop"
} }
@ -152,19 +153,13 @@ class RoutersSpec extends ScalaTestWithActorTestKit("""
val serviceKey = ServiceKey[String]("group-routing-2") val serviceKey = ServiceKey[String]("group-routing-2")
val group = spawn(Routers.group(serviceKey), "group-router-2") val group = spawn(Routers.group(serviceKey), "group-router-2")
val probe = TestProbe[Dropped]() val probe = TestProbe[Dropped]()
system.toClassic.eventStream.subscribe(probe.ref.toClassic, classOf[Dropped]) system.eventStream ! EventStream.Subscribe(probe.ref)
(0 to 3).foreach { n => (0 to 3).foreach { n =>
val msg = s"message-$n" val msg = s"message-$n"
// EventFilter.info(start = "Message [java.lang.String] ... was not delivered.", occurrences = 1).intercept { */
EventFilter.warning(start = "dropped message", occurrences = 1).intercept {
EventFilter.info(pattern = ".*was dropped. No routees in group router", occurrences = 1).intercept {
group ! msg group ! msg
probe.expectMessageType[Dropped] probe.expectMessageType[Dropped]
} }
}
/* } */
}
testKit.stop(group) testKit.stop(group)
} }

View file

@ -7,9 +7,10 @@ package akka.actor.typed.scaladsl
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
import akka.actor.testkit.typed.internal.StubbedActorContext import akka.actor.testkit.typed.internal.StubbedActorContext
import akka.actor.testkit.typed.scaladsl.TestInbox import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.{ Matchers, WordSpec } import org.scalatest.{ Matchers, WordSpec }
class StashBufferSpec extends WordSpec with Matchers { class StashBufferSpec extends WordSpec with Matchers with LogCapturing {
val context = new StubbedActorContext[String]( val context = new StubbedActorContext[String](
"StashBufferSpec", "StashBufferSpec",

View file

@ -10,10 +10,14 @@ import java.util.concurrent.TimeUnit
import akka.actor.DeadLetter import akka.actor.DeadLetter
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.UnhandledMessage
import akka.actor.testkit.typed.TestException import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.testkit.EventFilter import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.eventstream.EventStream
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
object AbstractStashSpec { object AbstractStashSpec {
@ -197,7 +201,7 @@ class MutableStashSpec extends AbstractStashSpec {
} }
} }
abstract class AbstractStashSpec extends ScalaTestWithActorTestKit with WordSpecLike { abstract class AbstractStashSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import AbstractStashSpec._ import AbstractStashSpec._
def testQualifier: String def testQualifier: String
@ -252,15 +256,7 @@ abstract class AbstractStashSpec extends ScalaTestWithActorTestKit with WordSpec
} }
class UnstashingSpec extends ScalaTestWithActorTestKit(""" class UnstashingSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
akka.loggers = ["akka.testkit.TestEventListener"]
""") with WordSpecLike {
// needed for EventFilter
private implicit val classicSys: akka.actor.ActorSystem = {
import akka.actor.typed.scaladsl.adapter._
system.toClassic
}
private def slowStoppingChild(latch: CountDownLatch): Behavior[String] = private def slowStoppingChild(latch: CountDownLatch): Behavior[String] =
Behaviors.receiveSignal { Behaviors.receiveSignal {
@ -422,13 +418,16 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
ref ! "stash" ref ! "stash"
ref ! "stash-fail" ref ! "stash-fail"
ref ! "stash" ref ! "stash"
EventFilter[TestException](start = "unstash-fail", occurrences = 1).intercept { LoggingEventFilter
.error[TestException]
.withMessageContains("unstash-fail")
.intercept {
ref ! "unstash" ref ! "unstash"
probe.expectMessage("unstashing-0") probe.expectMessage("unstashing-0")
probe.expectMessage("unstashing-1") probe.expectMessage("unstashing-1")
probe.expectMessage("stash-fail-2") probe.expectMessage("stash-fail-2")
probe.expectMessage("post-stop-2") probe.expectMessage("post-stop-2")
} }(system)
} }
"signal PostStop to the latest unstashed behavior on failure" in { "signal PostStop to the latest unstashed behavior on failure" in {
@ -449,7 +448,9 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
ref ! "stash" ref ! "stash"
ref ! "stash-fail" ref ! "stash-fail"
ref ! "stash" ref ! "stash"
EventFilter[TestException](start = "Supervisor RestartSupervisor saw failure: unstash-fail", occurrences = 1) LoggingEventFilter
.error[TestException]
.withMessageContains("Supervisor RestartSupervisor saw failure: unstash-fail")
.intercept { .intercept {
ref ! "unstash" ref ! "unstash"
// when childLatch is defined this be stashed in the internal stash of the RestartSupervisor // when childLatch is defined this be stashed in the internal stash of the RestartSupervisor
@ -466,7 +467,7 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
ref ! "get-stash-size" ref ! "get-stash-size"
probe.expectMessage("stash-size-0") probe.expectMessage("stash-size-0")
} }(system)
} }
"signal PreRestart to the latest unstashed behavior on failure with restart supervision" in { "signal PreRestart to the latest unstashed behavior on failure with restart supervision" in {
@ -530,7 +531,9 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
ref ! "stash" ref ! "stash"
ref ! "stash-fail" ref ! "stash-fail"
ref ! "stash" ref ! "stash"
EventFilter[TestException](start = "Supervisor ResumeSupervisor saw failure: unstash-fail", occurrences = 1) LoggingEventFilter
.error[TestException]
.withMessageContains("Supervisor ResumeSupervisor saw failure: unstash-fail")
.intercept { .intercept {
ref ! "unstash" ref ! "unstash"
ref ! "get-current" ref ! "get-current"
@ -541,7 +544,7 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
probe.expectMessage("current-2") probe.expectMessage("current-2")
ref ! "get-stash-size" ref ! "get-stash-size"
probe.expectMessage("stash-size-5") probe.expectMessage("stash-size-5")
} }(system)
ref ! "unstash" ref ! "unstash"
ref ! "get-current" ref ! "get-current"
@ -588,6 +591,9 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
"deal with unhandled the same way as normal unhandled" in { "deal with unhandled the same way as normal unhandled" in {
val probe = TestProbe[String]() val probe = TestProbe[String]()
val unhandledProbe = createTestProbe[UnhandledMessage]()
system.eventStream ! EventStream.Subscribe(unhandledProbe.ref)
val ref = spawn(Behaviors.withStash[String](10) { stash => val ref = spawn(Behaviors.withStash[String](10) { stash =>
stash.stash("unhandled") stash.stash("unhandled")
stash.stash("handled") stash.stash("handled")
@ -609,11 +615,12 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
} }
}) })
EventFilter.warning(start = "unhandled message from", occurrences = 2).intercept {
ref ! "unstash" ref ! "unstash"
}
unhandledProbe.receiveMessage()
probe.expectMessage("handled 1") probe.expectMessage("handled 1")
probe.expectMessage("handled 2") probe.expectMessage("handled 2")
unhandledProbe.receiveMessage()
probe.expectMessage("handled 3") probe.expectMessage("handled 3")
ref ! "handled" ref ! "handled"
@ -637,16 +644,15 @@ class UnstashingSpec extends ScalaTestWithActorTestKit("""
"deal with stop" in { "deal with stop" in {
val probe = TestProbe[Any] val probe = TestProbe[Any]
import akka.actor.typed.scaladsl.adapter._ system.eventStream ! EventStream.Subscribe(probe.ref.narrow[DeadLetter])
classicSys.eventStream.subscribe(probe.ref.toClassic, classOf[DeadLetter])
val ref = spawn(Behaviors.withStash[String](10) { stash => val ref = spawn(Behaviors.withStash[String](10) { stash =>
stash.stash("one") stash.stash("one")
stash.stash("two") stash.stash("two")
Behaviors.receiveMessage { Behaviors.receiveMessage {
case "unstash" => case "unstash" =>
stash.unstashAll(Behaviors.receiveMessage { stash.unstashAll(Behaviors.receiveMessage { unstashed =>
case unstashed =>
probe.ref ! unstashed probe.ref ! unstashed
Behaviors.stopped Behaviors.stopped
}) })

View file

@ -7,13 +7,14 @@ package akka.actor.typed.scaladsl
import akka.Done import akka.Done
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed import akka.actor.typed
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
import akka.actor.typed.BehaviorInterceptor import akka.actor.typed.BehaviorInterceptor
import akka.actor.typed.PostStop import akka.actor.typed.PostStop
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class StopSpec extends ScalaTestWithActorTestKit with WordSpecLike { class StopSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import BehaviorInterceptor._ import BehaviorInterceptor._
"Stopping an actor" should { "Stopping an actor" should {

View file

@ -16,6 +16,7 @@ import akka.actor.typed.Terminated
import akka.testkit._ import akka.testkit._
import akka.Done import akka.Done
import akka.NotUsed import akka.NotUsed
import akka.actor.testkit.typed.scaladsl.LoggingEventFilter
import akka.{ actor => classic } import akka.{ actor => classic }
object AdapterSpec { object AdapterSpec {
@ -161,9 +162,7 @@ object AdapterSpec {
} }
class AdapterSpec extends AkkaSpec(""" class AdapterSpec extends AkkaSpec {
akka.loggers = [akka.testkit.TestEventListener]
""") {
import AdapterSpec._ import AdapterSpec._
"ActorSystem adaption" must { "ActorSystem adaption" must {
@ -278,10 +277,12 @@ class AdapterSpec extends AkkaSpec("""
val typedRef = system.spawnAnonymous(typed1(ignore, probe.ref)) val typedRef = system.spawnAnonymous(typed1(ignore, probe.ref))
// only stop supervisorStrategy // only stop supervisorStrategy
EventFilter[AdapterSpec.ThrowIt3.type](occurrences = 1).intercept { LoggingEventFilter
.error[AdapterSpec.ThrowIt3.type]
.intercept {
typedRef ! "supervise-restart" typedRef ! "supervise-restart"
probe.expectMsg("ok") probe.expectMsg("ok")
} }(system.toTyped)
} }
"stop typed child from classic parent" in { "stop typed child from classic parent" in {
@ -302,10 +303,12 @@ class AdapterSpec extends AkkaSpec("""
"log exception if not by handled typed supervisor" in { "log exception if not by handled typed supervisor" in {
val throwMsg = "sad panda" val throwMsg = "sad panda"
EventFilter.warning(pattern = ".*sad panda.*").intercept { LoggingEventFilter
.error("sad panda")
.intercept {
system.spawnAnonymous(unhappyTyped(throwMsg)) system.spawnAnonymous(unhappyTyped(throwMsg))
Thread.sleep(1000) Thread.sleep(1000)
} }(system.toTyped)
} }
} }
} }

View file

@ -9,13 +9,14 @@ import java.util.concurrent.TimeUnit
import akka.actor.ActorSystemImpl import akka.actor.ActorSystemImpl
import akka.actor.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorSystem import akka.actor.typed.ActorSystem
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import org.scalatest.Matchers import org.scalatest.Matchers
import org.scalatest.WordSpec import org.scalatest.WordSpec
import org.scalatest.concurrent.ScalaFutures import org.scalatest.concurrent.ScalaFutures
class GuardianStartupSpec extends WordSpec with Matchers with ScalaFutures { class GuardianStartupSpec extends WordSpec with Matchers with ScalaFutures with LogCapturing {
"The user guardian" must { "The user guardian" must {

View file

@ -1,32 +0,0 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.typed.testkit
import java.util.concurrent.locks.ReentrantLock
import ch.qos.logback.classic.spi.ILoggingEvent
import org.slf4j.LoggerFactory
import org.slf4j.event.LoggingEvent
object AppenderInterceptor {
val logger = LoggerFactory.getLogger("akka.actor.typed.scaladsl.ActorLoggingSpec")
val root = logger.asInstanceOf[ch.qos.logback.classic.Logger]
val myAppender = root.getAppender("INTERCEPTOR").asInstanceOf[TestAppender]
def events:() => Seq[LoggingEvent] = () => myAppender.events.map { each =>
each match {
//TODO implement also for log4j, ILoggingEvent is logback specific
case event: ILoggingEvent => {
new Sl4jLoggingEvent(
event.getLevel(),
event.getMessage(),
event.getLoggerName(),
event.getTimeStamp,
event.getArgumentArray,
)
}
}
}
}

View file

@ -1,560 +0,0 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.typed.testkit
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
import java.util
import java.util.concurrent.LinkedBlockingQueue
import akka.actor.{ActorSystem, DeadLetter, Dropped, NoSerializationVerificationNeeded, UnhandledMessage}
import akka.dispatch.sysmsg.{SystemMessage, Terminate}
import akka.event.Logging
import akka.event.Logging._
import akka.event.slf4j.Slf4jLogger
import akka.japi.Util.immutableSeq
import akka.testkit.TestEvent.{Mute, UnMute}
import akka.testkit.{EventFilter, TestEvent, TestKit, TestKitExtension}
import akka.util.BoxedType
import akka.util.ccompat.ccompatUsedUntil213
import ch.qos.logback.classic.spi.ILoggingEvent
import org.slf4j.Marker
import org.slf4j.event.{Level, LoggingEvent}
import scala.collection.mutable.ArrayBuffer
import scala.concurrent.duration.Duration
import scala.reflect.ClassTag
import scala.util.matching.Regex
/**
* Facilities for selectively filtering out expected org.slf4j.event.LoggingEvent from logging so
* that you can keep your test runs console output clean and do not miss real
* error messages.
*
* See the companion object for convenient factory methods.
*
* If the `occurrences` is set to Int.MaxValue, no tracking is done.
*/
abstract class LoggingEventFilter[LE <: LoggingEvent](occurrences: Int) {
/*
* these default values are just there for easier subclassing
*/
protected val source: Option[String] = None
protected val message: Either[String, Regex] = Left("")
protected val complete: Boolean = false
@volatile // JMM does not guarantee visibility for non-final fields
private var todo = occurrences
import scala.collection.JavaConverters._
/**
* Apply this filter while executing the given code block. Care is taken to
* remove the filter when the block is finished or aborted.
*/
def intercept[T, LE <: LoggingEvent](code: => T, loggingEvents: util.Collection[LE])(implicit system: ActorSystem): T = {
//TODO @see original TestEventListener#116: system.eventStream.publish(TestEvent.Mute(this))
def leftToDo: Int = todo - loggingEvents.asScala.count(matches)
val leeway = TestKitExtension(system).TestEventFilterLeeway
val result = code
if (!awaitDone(leeway, leftToDo))
if (leftToDo > 0)
throw new AssertionError(s"timeout ($leeway) waiting for $leftToDo messages on $this")
else
throw new AssertionError(s"received ${-leftToDo} excess messages on $this")
result
}
import scala.concurrent.duration._
/**
* Apply this filter while executing the given code block. Care is taken to
* remove the filter when the block is finished or aborted.
*/
def interceptIt[T] (code: => T, loggingEvents: () => Seq[LoggingEvent])(implicit system: ActorSystem): T = {
//TODO @see original TestEventListener#116: system.eventStream.publish(TestEvent.Mute(this))
val result = code
def leftToDo: Int = todo - loggingEvents.apply().count(matches)
val leeway = TestKitExtension(system).TestEventFilterLeeway
if (!awaitDone(leeway, leftToDo))
if (leftToDo > 0)
throw new AssertionError(s"timeout ($leeway) waiting for $leftToDo messages on $this")
else
throw new AssertionError(s"received ${-leftToDo} excess messages on $this")
result
}
def awaitDone(max: Duration, leftToDo: => Int): Boolean = {
if (leftToDo != Int.MaxValue && leftToDo > 0) TestKit.awaitCond(leftToDo <= 0, max, noThrow = true)
leftToDo == Int.MaxValue || leftToDo == 0
}
/**
* This method decides whether to filter the event (<code>true</code>) or not
* (<code>false</code>).
*/
protected def matches(event: LoggingEvent): Boolean
/**
* internal implementation helper, no guaranteed API
*/
protected def doMatch(msg: Any) = {
val msgstr = if (msg != null) msg.toString else "null"
(message match {
case Left(s) =>
if (complete) msgstr == s else msgstr.startsWith(s)
case Right(p) => p.findFirstIn(msgstr).isDefined
})
}
}
/**
* Facilities for selectively filtering out expected events from logging so
* that you can keep your test runs console output clean and do not miss real
* error messages.
*
* '''Also have a look at the `akka.testkit` package objects `filterEvents` and
* `filterException` methods.'''
*
* The source filters do accept `Class[_]` arguments, matching any
* object which is an instance of the given class, e.g.
*
* {{{
* EventFilter.info(source = classOf[MyActor]) // will match Info events from any MyActor instance
* }}}
*
* The message object will be converted to a string before matching (`"null"` if it is `null`).
*/
object LoggingEventFilter {
/**
* Create a filter for Error events. Give up to one of <code>start</code> and <code>pattern</code>:
*
* {{{
* EventFilter[MyException]() // filter only on exception type
* EventFilter[MyException]("message") // filter on exactly matching message
* EventFilter[MyException](source = obj) // filter on event source
* EventFilter[MyException](start = "Expected") // filter on start of message
* EventFilter[MyException](source = obj, pattern = "weird.*message") // filter on pattern and message
* }}}
*
* ''Please note that filtering on the `source` being
* `null` does NOT work (passing `null` disables the
* source filter).''
*/
private def apply[A <: Throwable: ClassTag](
message: String = null,
source: String = null,
start: String = "",
pattern: String = null,
occurrences: Int = Int.MaxValue,
marker: Marker)(implicit ev: A = Logging.Error.NoCause.getClass): LoggingEventFilter[_] =
ErrorFilterLogging(
ev.getClass,
Option(source),
if (message ne null) Left(message) else Option(pattern).map(new Regex(_)).toRight(start),
message ne null)(occurrences)
/**
* Create a filter for Error events. See apply() for more details.
*/
def error(
message: String = null,
source: String = null,
start: String = "",
pattern: String = null,
occurrences: Int = Int.MaxValue): LoggingEventFilter[_] =
ErrorFilterLogging(
Logging.Error.NoCause.getClass,
Option(source),
if (message ne null) Left(message) else Option(pattern).map(new Regex(_)).toRight(start),
message ne null)(occurrences)
/**
* Create a filter for Warning events. Give up to one of <code>start</code> and <code>pattern</code>:
*
* {{{
* EventFilter.warning() // filter only on warning event
* EventFilter.warning(source = obj) // filter on event source
* EventFilter.warning(start = "Expected") // filter on start of message
* EventFilter.warning(source = obj, pattern = "weird.*message") // filter on pattern and message
* }}}
*
* ''Please note that filtering on the `source` being
* `null` does NOT work (passing `null` disables the
* source filter).''
*/
def warning(
message: String = null,
source: String = null,
start: String = "",
pattern: String = null,
occurrences: Int = Int.MaxValue): LoggingEventFilter[_] =
WarningFilterLogging(
Option(source),
if (message ne null) Left(message) else Option(pattern).map(new Regex(_)).toRight(start),
message ne null)(occurrences)
/**
* Create a filter for Info events. Give up to one of <code>start</code> and <code>pattern</code>:
*
* {{{
* EventFilter.info() // filter only on info event
* EventFilter.info(source = obj) // filter on event source
* EventFilter.info(start = "Expected") // filter on start of message
* EventFilter.info(source = obj, pattern = "weird.*message") // filter on pattern and message
* }}}
*
* ''Please note that filtering on the `source` being
* `null` does NOT work (passing `null` disables the
* source filter).''
*/
def info(
message: String = null,
source: String = null,
start: String = "",
pattern: String = null,
occurrences: Int = Int.MaxValue): LoggingEventFilter[_] =
InfoFilterLogging(
Option(source),
if (message ne null) Left(message) else Option(pattern).map(new Regex(_)).toRight(start),
message ne null)(occurrences)
/**
* Create a filter for Debug events. Give up to one of <code>start</code> and <code>pattern</code>:
*
* {{{
* EventFilter.debug() // filter only on debug type
* EventFilter.debug(source = obj) // filter on event source
* EventFilter.debug(start = "Expected") // filter on start of message
* EventFilter.debug(source = obj, pattern = "weird.*message") // filter on pattern and message
* }}}
*
* ''Please note that filtering on the `source` being
* `null` does NOT work (passing `null` disables the
* source filter).''
*/
def debug(
message: String = null,
source: String = null,
start: String = "",
pattern: String = null,
occurrences: Int = Int.MaxValue): LoggingEventFilter[_] =
DebugFilterLogging(
Option(source),
if (message ne null) Left(message) else Option(pattern).map(new Regex(_)).toRight(start),
message ne null)(occurrences)
/**
* Create a custom event filter. The filter will affect those events for
* which the supplied partial function is defined and returns
* `true`.
*
* {{{
* EventFilter.custom {
* case Warning(ref, "my warning") if ref == actor || ref == null => true
* }
* }}}
*/
def custom(
test: PartialFunction[LoggingEvent, Boolean],
occurrences: Int = Int.MaxValue): LoggingEventFilter[LoggingEvent] =
CustomLoggingEventFilter(test)(occurrences)
}
/**
* Filter which matches Error events, if they satisfy the given criteria:
* <ul>
* <li><code>throwable</code> applies an upper bound on the type of exception contained in the Error event</li>
* <li><code>source</code>, if given, applies a filter on the events origin</li>
* <li><code>message</code> applies a filter on the events message (either
* with String.startsWith or Regex.findFirstIn().isDefined); if the message
* itself does not match, the match is retried with the contained Exceptions
* message; if both are <code>null</code>, the filter always matches if at
* the same time the Exceptions stack trace is empty (this catches
* JVM-omitted fast-throw exceptions)</li>
* </ul>
* If you want to match all Error events, the most efficient is to use <code>Left("")</code>.
*/
final case class ErrorFilterLogging(
throwable: Class[_],
override val source: Option[String],
override val message: Either[String, Regex],
override val complete: Boolean)(occurrences: Int)
extends LoggingEventFilter[LoggingEvent](occurrences) {
def matches(event: LoggingEvent) = {
event.getLevel match {
case Level.ERROR => true
case _ => false
}
}
/**
* Java API: create an ErrorFilter
*
* @param source
* apply this filter only to events from the given source; do not filter on source if this is given as <code>null</code>
* @param message
* apply this filter only to events whose message matches; do not filter on message if this is given as <code>null</code>
* @param pattern
* if <code>false</code>, the message string must start with the given
* string, otherwise the <code>message</code> argument is treated as
* regular expression which is matched against the message (may match only
* a substring to filter)
* @param complete
* whether the events message must match the given message string or pattern completely
*/
def this(
throwable: Class[_],
source: String,
message: String,
pattern: Boolean,
complete: Boolean,
occurrences: Int) =
this(
throwable,
Option(source),
if (message eq null) Left("")
else if (pattern) Right(new Regex(message))
else Left(message),
complete)(occurrences)
/**
* Java API: filter only on the given type of exception
*/
def this(throwable: Class[_]) = this(throwable, null, null, false, false, Int.MaxValue)
}
/**
* Filter which matches Warning events, if they satisfy the given criteria:
* <ul>
* <li><code>source</code>, if given, applies a filter on the events origin</li>
* <li><code>message</code> applies a filter on the events message (either with String.startsWith or Regex.findFirstIn().isDefined)</li>
* </ul>
* If you want to match all Warning events, the most efficient is to use <code>Left("")</code>.
*/
final case class WarningFilterLogging(
override val source: Option[String],
override val message: Either[String, Regex],
override val complete: Boolean)(occurrences: Int)
extends LoggingEventFilter[LoggingEvent](occurrences) {
def matches(event: LoggingEvent) = {
event.getLevel match {
case Level.WARN => doMatch(event.getLoggerName, event.getMessage)
case _ => false
}
}
/**
* Java API: create a WarningFilter
*
* @param source
* apply this filter only to events from the given source; do not filter on source if this is given as <code>null</code>
* @param message
* apply this filter only to events whose message matches; do not filter on message if this is given as <code>null</code>
* @param pattern
* if <code>false</code>, the message string must start with the given
* string, otherwise the <code>message</code> argument is treated as
* regular expression which is matched against the message (may match only
* a substring to filter)
* @param complete
* whether the events message must match the given message string or pattern completely
*/
def this(source: String, message: String, pattern: Boolean, complete: Boolean, occurrences: Int) =
this(
Option(source),
if (message eq null) Left("")
else if (pattern) Right(new Regex(message))
else Left(message),
complete)(occurrences)
}
/**
* Filter which matches Info events, if they satisfy the given criteria:
* <ul>
* <li><code>source</code>, if given, applies a filter on the events origin</li>
* <li><code>message</code> applies a filter on the events message (either with String.startsWith or Regex.findFirstIn().isDefined)</li>
* </ul>
* If you want to match all Info events, the most efficient is to use <code>Left("")</code>.
*/
final case class InfoFilterLogging(
override val source: Option[String],
override val message: Either[String, Regex],
override val complete: Boolean)(occurrences: Int)
extends LoggingEventFilter[LoggingEvent](occurrences) {
def matches(event: LoggingEvent) = {
event.getLevel match {
case Level.INFO => doMatch(event.getMessage)
case _ => false
}
}
/**
* Java API: create an InfoFilter
*
* @param source
* apply this filter only to events from the given source; do not filter on source if this is given as <code>null</code>
* @param message
* apply this filter only to events whose message matches; do not filter on message if this is given as <code>null</code>
* @param pattern
* if <code>false</code>, the message string must start with the given
* string, otherwise the <code>message</code> argument is treated as
* regular expression which is matched against the message (may match only
* a substring to filter)
* @param complete
* whether the events message must match the given message string or pattern completely
*/
def this(source: String, message: String, pattern: Boolean, complete: Boolean, occurrences: Int) =
this(
Option(source),
if (message eq null) Left("")
else if (pattern) Right(new Regex(message))
else Left(message),
complete)(occurrences)
}
/**
* Filter which matches Debug events, if they satisfy the given criteria:
* <ul>
* <li><code>source</code>, if given, applies a filter on the events origin</li>
* <li><code>message</code> applies a filter on the events message (either with String.startsWith or Regex.findFirstIn().isDefined)</li>
* </ul>
* If you want to match all Debug events, the most efficient is to use <code>Left("")</code>.
*/
final case class DebugFilterLogging(
override val source: Option[String],
override val message: Either[String, Regex],
override val complete: Boolean)(occurrences: Int)
extends LoggingEventFilter[LoggingEvent](occurrences) {
def matches(event: LoggingEvent) = {
event.getLevel match {
case Level.DEBUG => doMatch(event.getMessage)
case _ => false
}
}
/**
* Java API: create a DebugFilter
*
* @param source
* apply this filter only to events from the given source; do not filter on source if this is given as <code>null</code>
* @param message
* apply this filter only to events whose message matches; do not filter on message if this is given as <code>null</code>
* @param pattern
* if <code>false</code>, the message string must start with the given
* string, otherwise the <code>message</code> argument is treated as
* regular expression which is matched against the message (may match only
* a substring to filter)
* @param complete
* whether the events message must match the given message string or pattern completely
*/
def this(source: String, message: String, pattern: Boolean, complete: Boolean, occurrences: Int) =
this(
Option(source),
if (message eq null) Left("")
else if (pattern) Right(new Regex(message))
else Left(message),
complete)(occurrences)
}
/**
* Custom event filter when the others do not fit the bill.
*
* If the partial function is defined and returns true, filter the event.
*/
final case class CustomLoggingEventFilter(test: PartialFunction[LoggingEvent, Boolean])(occurrences: Int)
extends LoggingEventFilter[LoggingEvent](occurrences) {
def matches(event: LoggingEvent) = {
test.isDefinedAt(event) && test(event)
}
}
object DeadLettersFilterLogging {
def apply[T](implicit t: ClassTag[T]): DeadLettersFilterLogging =
new DeadLettersFilterLogging(t.runtimeClass.asInstanceOf[Class[T]])(Int.MaxValue)
}
/**
* Filter which matches DeadLetter events, if the wrapped message conforms to the
* given type.
*/
final case class DeadLettersFilterLogging(val messageClass: Class[_])(occurrences: Int)
extends LoggingEventFilter[LoggingEvent](occurrences) {
def matches(event: LoggingEvent) = {
event.getLevel match {
case Level.WARN => BoxedType(messageClass).isInstance(event.getMessage)
case _ => false
}
}
}
//TODO this still has EventFilter!! Slf4jLogger has also to be changed
@ccompatUsedUntil213
class TestEventListener extends Slf4jLogger {
import TestEvent._
var filters: List[EventFilter] = Nil
override def receive = {
case InitializeLogger(bus) =>
Seq(classOf[Mute], classOf[UnMute], classOf[DeadLetter], classOf[UnhandledMessage], classOf[Dropped])
.foreach(bus.subscribe(context.self, _))
sender() ! LoggerInitialized
case Mute(filters) => filters.foreach(addFilter)
case UnMute(filters) => filters.foreach(removeFilter)
case event: LogEvent => if (!filter(event)) print(event)
case DeadLetter(msg, snd, rcp) =>
if (!msg.isInstanceOf[Terminate]) {
val event = Warning(rcp.path.toString, rcp.getClass, msg)
if (!filter(event)) {
val msgPrefix =
if (msg.isInstanceOf[SystemMessage]) "received dead system message"
else if (snd eq context.system.deadLetters) "received dead letter"
else "received dead letter from " + snd
val event2 = Warning(rcp.path.toString, rcp.getClass, msgPrefix + ": " + msg)
if (!filter(event2)) print(event2)
}
}
case UnhandledMessage(msg, sender, rcp) =>
val event = Warning(rcp.path.toString, rcp.getClass, s"unhandled message from $sender: $msg")
if (!filter(event)) print(event)
case Dropped(msg, reason, sender, rcp) =>
val event =
Warning(rcp.path.toString, rcp.getClass, s"dropped message from $sender. $reason: $msg")
if (!filter(event)) print(event)
case m => print(Debug(context.system.name, this.getClass, m))
}
def filter(event: LogEvent): Boolean =
filters.exists(f =>
try {
f(event)
} catch { case _: Exception => false })
def addFilter(filter: EventFilter): Unit = filters ::= filter
def removeFilter(filter: EventFilter): Unit = {
@scala.annotation.tailrec
def removeFirst(list: List[EventFilter], zipped: List[EventFilter] = Nil): List[EventFilter] = list match {
case head :: tail if head == filter => tail.reverse_:::(zipped)
case head :: tail => removeFirst(tail, head :: zipped)
case Nil => filters // filter not found, just return original list
}
filters = removeFirst(filters)
}
}

View file

@ -1,134 +0,0 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.typed.testkit;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.slf4j.Marker;
import org.slf4j.event.Level;
import org.slf4j.event.LoggingEvent;
public class Sl4jLoggingEvent implements LoggingEvent {
Level level;
Marker marker;
String loggerName;
Logger logger;
String threadName;
String message;
Object[] argArray;
long timeStamp;
Throwable throwable;
public Sl4jLoggingEvent(Level level, String msg, String loggerName) {
setLevel(level);
setMessage(msg);
setLoggerName(loggerName);
}
public Sl4jLoggingEvent(ch.qos.logback.classic.Level level, String msg, String loggerName, long timeStamp, Object[] argArray) {
setLevel(toLogbackLevel(level));
setArgumentArray(argArray);
setMessage(msg);
setLoggerName(loggerName);
setTimeStamp(timeStamp);
}
public Level toLogbackLevel(ch.qos.logback.classic.Level level) {
switch (level.levelInt) {
case ch.qos.logback.classic.Level.TRACE_INT:
return Level.TRACE;
case ch.qos.logback.classic.Level.DEBUG_INT:
return Level.DEBUG;
case ch.qos.logback.classic.Level.INFO_INT:
return Level.INFO;
case ch.qos.logback.classic.Level.WARN_INT:
return Level.WARN;
case ch.qos.logback.classic.Level.ERROR_INT:
return Level.ERROR;
default:
throw new IllegalStateException("Level " + level.levelStr + ", " + level.levelInt + " is unknown.");
}
}
public Level getLevel() {
return level;
}
public void setLevel(Level level) {
this.level = level;
}
public Marker getMarker() {
return marker;
}
public void setMarker(Marker marker) {
this.marker = marker;
}
public String getLoggerName() {
return loggerName;
}
public void setLoggerName(String loggerName) {
this.loggerName = loggerName;
setLogger(LoggerFactory.getLogger(loggerName));
}
public Logger getLogger() {
return logger;
}
public void setLogger(Logger logger) {
this.logger = logger;
}
public String getMessage() {
return message;
}
public void setMessage(String message) {
if(getArgumentArray() != null) {
for (int i = 0; i < getArgumentArray().length; i++) {
message = message.replaceFirst("\\{\\}", getArgumentArray()[i].toString());
}
}
this.message = message;
}
public Object[] getArgumentArray() {
return argArray;
}
public void setArgumentArray(Object[] argArray) {
this.argArray = argArray;
}
public long getTimeStamp() {
return timeStamp;
}
public void setTimeStamp(long timeStamp) {
this.timeStamp = timeStamp;
}
public String getThreadName() {
return threadName;
}
public void setThreadName(String threadName) {
this.threadName = threadName;
}
public Throwable getThrowable() {
return throwable;
}
public void setThrowable(Throwable throwable) {
this.throwable = throwable;
}
}

View file

@ -1,18 +0,0 @@
/*
* Copyright (C) 2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.typed.testkit
import ch.qos.logback.classic.spi.ILoggingEvent
import ch.qos.logback.core.AppenderBase
import scala.collection.mutable.ArrayBuffer
class TestAppender extends AppenderBase[ILoggingEvent] {
var events = new ArrayBuffer[ILoggingEvent]()
override def append(event: ILoggingEvent) =
events += event
}

View file

@ -14,9 +14,10 @@ import DispatchersDocSpec._
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import scala.concurrent.Future import scala.concurrent.Future
import akka.actor.testkit.typed.scaladsl.LogCapturing
object DispatchersDocSpec { object DispatchersDocSpec {
val config = ConfigFactory.parseString(""" val config = ConfigFactory.parseString("""
@ -57,7 +58,10 @@ object DispatchersDocSpec {
} }
class DispatchersDocSpec extends ScalaTestWithActorTestKit(DispatchersDocSpec.config) with WordSpecLike { class DispatchersDocSpec
extends ScalaTestWithActorTestKit(DispatchersDocSpec.config)
with WordSpecLike
with LogCapturing {
"Actor Dispatchers" should { "Actor Dispatchers" should {
"support default and blocking dispatcher" in { "support default and blocking dispatcher" in {

View file

@ -7,10 +7,11 @@ package docs.akka.typed
import akka.actor.testkit.typed.scaladsl.TestProbe import akka.actor.testkit.typed.scaladsl.TestProbe
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorRef, Behavior } import akka.actor.typed.{ ActorRef, Behavior }
import scala.collection.immutable import scala.collection.immutable
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
object FSMDocSpec { object FSMDocSpec {
@ -67,7 +68,7 @@ object FSMDocSpec {
//#test-code //#test-code
} }
class FSMDocSpec extends ScalaTestWithActorTestKit with WordSpecLike { class FSMDocSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import FSMDocSpec._ import FSMDocSpec._

View file

@ -5,6 +5,8 @@
package docs.akka.typed package docs.akka.typed
//#imports //#imports
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorSystem, PostStop } import akka.actor.typed.{ ActorSystem, PostStop }
@ -72,7 +74,7 @@ object GracefulStopDocSpec {
} }
class GracefulStopDocSpec extends ScalaTestWithActorTestKit with WordSpecLike { class GracefulStopDocSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import GracefulStopDocSpec._ import GracefulStopDocSpec._

View file

@ -6,17 +6,23 @@ package docs.akka.typed
import java.net.URI import java.net.URI
import akka.NotUsed
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior }
import akka.actor.typed.scaladsl.{ Behaviors, TimerScheduler }
import scala.concurrent.Future import scala.concurrent.Future
import scala.concurrent.duration._ import scala.concurrent.duration._
import scala.util.{ Failure, Success } import scala.util.Failure
import scala.util.Success
import akka.NotUsed
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.ActorRef
import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.LoggerOps
import akka.actor.typed.scaladsl.TimerScheduler
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLike { class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
"The interaction patterns docs" must { "The interaction patterns docs" must {
@ -120,10 +126,10 @@ class InteractionPatternsSpec extends ScalaTestWithActorTestKit with WordSpecLik
context.log.info("Started {}", taskId) context.log.info("Started {}", taskId)
Behaviors.same Behaviors.same
case Backend.JobProgress(taskId, progress) => case Backend.JobProgress(taskId, progress) =>
context.log.info("Progress {}: {}", taskId, progress) context.log.info2("Progress {}: {}", taskId, progress)
Behaviors.same Behaviors.same
case Backend.JobCompleted(taskId, result) => case Backend.JobCompleted(taskId, result) =>
context.log.info("Completed {}: {}", taskId, result) context.log.info2("Completed {}: {}", taskId, result)
inProgress(taskId) ! result inProgress(taskId) ! result
active(inProgress - taskId, count) active(inProgress - taskId, count)
} }

View file

@ -6,7 +6,9 @@ package docs.akka.typed
//#fiddle_code //#fiddle_code
//#imports //#imports
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.LoggerOps
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior }
//#imports //#imports
//#fiddle_code //#fiddle_code
@ -54,7 +56,7 @@ object IntroSpec {
Behaviors.receive { (context, message) => Behaviors.receive { (context, message) =>
val n = greetingCounter + 1 val n = greetingCounter + 1
//#fiddle_code //#fiddle_code
context.log.info("Greeting {} for {}", n, message.whom) context.log.info2("Greeting {} for {}", n, message.whom)
//#fiddle_code //#fiddle_code
//#hello-world-bot //#hello-world-bot
println(s"Greeting $n for ${message.whom}") println(s"Greeting $n for ${message.whom}")
@ -190,7 +192,7 @@ object IntroSpec {
handle ! PostMessage("Hello World!") handle ! PostMessage("Hello World!")
Behaviors.same Behaviors.same
case MessagePosted(screenName, message) => case MessagePosted(screenName, message) =>
context.log.info("message has been posted by '{}': {}", screenName, message: Any) context.log.info2("message has been posted by '{}': {}", screenName, message)
Behaviors.stopped Behaviors.stopped
} }
} }
@ -221,7 +223,7 @@ object IntroSpec {
} }
class IntroSpec extends ScalaTestWithActorTestKit with WordSpecLike { class IntroSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import IntroSpec._ import IntroSpec._

View file

@ -6,6 +6,7 @@ package docs.akka.typed
import akka.Done import akka.Done
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
import akka.actor.typed.MailboxSelector import akka.actor.typed.MailboxSelector
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
@ -14,7 +15,8 @@ import org.scalatest.WordSpecLike
class MailboxDocSpec class MailboxDocSpec
extends ScalaTestWithActorTestKit(ConfigFactory.load("mailbox-config-sample.conf")) extends ScalaTestWithActorTestKit(ConfigFactory.load("mailbox-config-sample.conf"))
with WordSpecLike { with WordSpecLike
with LogCapturing {
"Specifying mailbox through props" must { "Specifying mailbox through props" must {
"work" in { "work" in {

View file

@ -6,8 +6,9 @@ package docs.akka.typed
//#imports //#imports
import akka.Done import akka.Done
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.{ ActorRef, ActorSystem, Behavior } import akka.actor.typed.{ ActorRef, ActorSystem, Behavior }
import akka.actor.typed.scaladsl.{ AbstractBehavior, ActorContext, Behaviors } import akka.actor.typed.scaladsl.{ AbstractBehavior, ActorContext, Behaviors, LoggerOps }
//#imports //#imports
import akka.NotUsed import akka.NotUsed
@ -104,7 +105,7 @@ object OOIntroSpec {
handle ! PostMessage("Hello World!") handle ! PostMessage("Hello World!")
Behaviors.same Behaviors.same
case MessagePosted(screenName, message) => case MessagePosted(screenName, message) =>
context.log.info("message has been posted by '{}': {}", Array(screenName, message): _*) context.log.info2("message has been posted by '{}': {}", screenName, message)
Behaviors.stopped Behaviors.stopped
} }
} }
@ -135,7 +136,7 @@ object OOIntroSpec {
} }
class OOIntroSpec extends ScalaTestWithActorTestKit with WordSpecLike { class OOIntroSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import OOIntroSpec._ import OOIntroSpec._

View file

@ -6,6 +6,7 @@ package docs.akka.typed
// #pool // #pool
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
import akka.actor.typed.SupervisorStrategy import akka.actor.typed.SupervisorStrategy
import akka.actor.typed.receptionist.Receptionist import akka.actor.typed.receptionist.Receptionist
@ -39,7 +40,7 @@ object RouterSpec {
val serviceKey = ServiceKey[Worker.Command]("log-worker") val serviceKey = ServiceKey[Worker.Command]("log-worker")
} }
class RouterSpec extends ScalaTestWithActorTestKit("akka.loglevel=warning") with WordSpecLike { class RouterSpec extends ScalaTestWithActorTestKit("akka.loglevel=warning") with WordSpecLike with LogCapturing {
import RouterSpec._ import RouterSpec._
"The routing sample" must { "The routing sample" must {

View file

@ -7,8 +7,10 @@ package docs.akka.typed
import scala.concurrent.ExecutionContext import scala.concurrent.ExecutionContext
import scala.concurrent.Future import scala.concurrent.Future
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.testkit.typed.scaladsl.ActorTestKit import akka.actor.testkit.typed.scaladsl.ActorTestKit
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import docs.akka.typed.IntroSpec.HelloWorld import docs.akka.typed.IntroSpec.HelloWorld
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
import com.github.ghik.silencer.silent import com.github.ghik.silencer.silent
@ -17,6 +19,7 @@ import com.github.ghik.silencer.silent
import akka.actor.typed.Behavior import akka.actor.typed.Behavior
import akka.actor.typed.SpawnProtocol import akka.actor.typed.SpawnProtocol
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.LoggerOps
//#imports1 //#imports1
@ -46,7 +49,7 @@ object SpawnProtocolDocSpec {
//#main //#main
} }
class SpawnProtocolDocSpec extends ScalaTestWithActorTestKit with WordSpecLike { class SpawnProtocolDocSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import SpawnProtocolDocSpec._ import SpawnProtocolDocSpec._
@ -67,7 +70,7 @@ class SpawnProtocolDocSpec extends ScalaTestWithActorTestKit with WordSpecLike {
system.ask(SpawnProtocol.Spawn(behavior = HelloWorld(), name = "greeter", props = Props.empty, _)) system.ask(SpawnProtocol.Spawn(behavior = HelloWorld(), name = "greeter", props = Props.empty, _))
val greetedBehavior = Behaviors.receive[HelloWorld.Greeted] { (context, message) => val greetedBehavior = Behaviors.receive[HelloWorld.Greeted] { (context, message) =>
context.log.info("Greeting for {} from {}", Array(message.whom, message.from): _*) context.log.info2("Greeting for {} from {}", message.whom, message.from)
Behaviors.stopped Behaviors.stopped
} }

View file

@ -5,7 +5,7 @@
package docs.akka.typed package docs.akka.typed
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.scaladsl.LogCapturing
import org.scalatest.WordSpecLike import org.scalatest.WordSpecLike
object StashDocSpec { object StashDocSpec {
@ -100,7 +100,7 @@ object StashDocSpec {
// #stashing // #stashing
} }
class StashDocSpec extends ScalaTestWithActorTestKit with WordSpecLike { class StashDocSpec extends ScalaTestWithActorTestKit with WordSpecLike with LogCapturing {
import StashDocSpec.DB import StashDocSpec.DB
import StashDocSpec.DataAccess import StashDocSpec.DataAccess
import scala.concurrent.Future import scala.concurrent.Future

View file

@ -9,6 +9,7 @@ import scala.concurrent.Future
import akka.actor.typed.ActorSystem import akka.actor.typed.ActorSystem
import akka.actor.typed.ActorRef import akka.actor.typed.ActorRef
import akka.actor.typed.scaladsl.LoggerOps
import akka.actor.typed.scaladsl.TimerScheduler import akka.actor.typed.scaladsl.TimerScheduler
import scala.concurrent.duration.FiniteDuration import scala.concurrent.duration.FiniteDuration
@ -126,7 +127,7 @@ object StyleGuideDocExamples {
Behaviors.same Behaviors.same
case Increment => case Increment =>
val newValue = n + 1 val newValue = n + 1
context.log.debug("[{}] Incremented counter to [{}]", name, newValue) context.log.debug2("[{}] Incremented counter to [{}]", name, newValue)
counter(name, timers, newValue) counter(name, timers, newValue)
case GetValue(replyTo) => case GetValue(replyTo) =>
replyTo ! Value(n) replyTo ! Value(n)
@ -160,16 +161,16 @@ object StyleGuideDocExamples {
private def counter(setup: Setup, n: Int): Behavior[Command] = private def counter(setup: Setup, n: Int): Behavior[Command] =
Behaviors.receiveMessage { Behaviors.receiveMessage {
case IncrementRepeatedly(interval) => case IncrementRepeatedly(interval) =>
setup.context.log.debug( setup.context.log.debugN(
"[{}] Starting repeated increments with interval [{}], current count is [{}]", "[{}] Starting repeated increments with interval [{}], current count is [{}]",
setup.name, setup.name,
interval.toString, interval,
n.toString) n)
setup.timers.startTimerWithFixedDelay("repeat", Increment, interval) setup.timers.startTimerWithFixedDelay("repeat", Increment, interval)
Behaviors.same Behaviors.same
case Increment => case Increment =>
val newValue = n + 1 val newValue = n + 1
setup.context.log.debug("[{}] Incremented counter to [{}]", setup.name, newValue) setup.context.log.debug2("[{}] Incremented counter to [{}]", setup.name, newValue)
counter(setup, newValue) counter(setup, newValue)
case GetValue(replyTo) => case GetValue(replyTo) =>
replyTo ! Value(n) replyTo ! Value(n)
@ -207,16 +208,16 @@ object StyleGuideDocExamples {
private def counter(n: Int): Behavior[Command] = private def counter(n: Int): Behavior[Command] =
Behaviors.receiveMessage { Behaviors.receiveMessage {
case IncrementRepeatedly(interval) => case IncrementRepeatedly(interval) =>
context.log.debug( context.log.debugN(
"[{}] Starting repeated increments with interval [{}], current count is [{}]", "[{}] Starting repeated increments with interval [{}], current count is [{}]",
name, name,
interval.toString, interval,
n.toString) n)
timers.startTimerWithFixedDelay("repeat", Increment, interval) timers.startTimerWithFixedDelay("repeat", Increment, interval)
Behaviors.same Behaviors.same
case Increment => case Increment =>
val newValue = n + 1 val newValue = n + 1
context.log.debug("[{}] Incremented counter to [{}]", name, newValue) context.log.debug2("[{}] Incremented counter to [{}]", name, newValue)
counter(newValue) counter(newValue)
case GetValue(replyTo) => case GetValue(replyTo) =>
replyTo ! Value(n) replyTo ! Value(n)
@ -243,16 +244,16 @@ object StyleGuideDocExamples {
def counter(n: Int): Behavior[Command] = def counter(n: Int): Behavior[Command] =
Behaviors.receiveMessage { Behaviors.receiveMessage {
case IncrementRepeatedly(interval) => case IncrementRepeatedly(interval) =>
context.log.debug( context.log.debugN(
"[{}] Starting repeated increments with interval [{}], current count is [{}]", "[{}] Starting repeated increments with interval [{}], current count is [{}]",
name, name,
interval.toString, interval,
n.toString) n)
timers.startTimerWithFixedDelay("repeat", Increment, interval) timers.startTimerWithFixedDelay("repeat", Increment, interval)
Behaviors.same Behaviors.same
case Increment => case Increment =>
val newValue = n + 1 val newValue = n + 1
context.log.debug("[{}] Incremented counter to [{}]", name, newValue) context.log.debug2("[{}] Incremented counter to [{}]", name, newValue)
counter(newValue) counter(newValue)
case GetValue(replyTo) => case GetValue(replyTo) =>
replyTo ! Value(n) replyTo ! Value(n)
@ -353,11 +354,11 @@ object StyleGuideDocExamples {
Behaviors.receiveMessage { Behaviors.receiveMessage {
case Increment => case Increment =>
val newValue = n + 1 val newValue = n + 1
context.log.debug("[{}] Incremented counter to [{}]", name, newValue) context.log.debug2("[{}] Incremented counter to [{}]", name, newValue)
counter(newValue) counter(newValue)
case Tick => case Tick =>
val newValue = n + 1 val newValue = n + 1
context.log.debug("[{}] Incremented counter by background tick to [{}]", name, newValue) context.log.debug2("[{}] Incremented counter by background tick to [{}]", name, newValue)
counter(newValue) counter(newValue)
case GetValue(replyTo) => case GetValue(replyTo) =>
replyTo ! Value(n) replyTo ! Value(n)
@ -404,11 +405,11 @@ object StyleGuideDocExamples {
Behaviors.receiveMessage { Behaviors.receiveMessage {
case Increment => case Increment =>
val newValue = n + 1 val newValue = n + 1
context.log.debug("[{}] Incremented counter to [{}]", name, newValue) context.log.debug2("[{}] Incremented counter to [{}]", name, newValue)
counter(newValue) counter(newValue)
case Tick => case Tick =>
val newValue = n + 1 val newValue = n + 1
context.log.debug("[{}] Incremented counter by background tick to [{}]", name, newValue) context.log.debug2("[{}] Incremented counter by background tick to [{}]", name, newValue)
counter(newValue) counter(newValue)
case GetValue(replyTo) => case GetValue(replyTo) =>
replyTo ! Value(n) replyTo ! Value(n)

View file

@ -5,6 +5,7 @@
package docs.akka.typed.coexistence package docs.akka.typed.coexistence
import akka.actor.ActorLogging import akka.actor.ActorLogging
import akka.actor.testkit.typed.scaladsl.LogCapturing
import akka.actor.typed._ import akka.actor.typed._
import akka.actor.typed.scaladsl.Behaviors import akka.actor.typed.scaladsl.Behaviors
import akka.testkit.TestKit import akka.testkit.TestKit
@ -71,7 +72,7 @@ object ClassicWatchingTypedSpec {
//#typed //#typed
} }
class ClassicWatchingTypedSpec extends WordSpec { class ClassicWatchingTypedSpec extends WordSpec with LogCapturing {
import ClassicWatchingTypedSpec._ import ClassicWatchingTypedSpec._

Some files were not shown because too many files have changed in this diff Show more