First step of changing to SL4J in Typed logging, #26537

This commit is contained in:
franciscolopezsancho 2019-08-23 13:00:00 +01:00 committed by Patrik Nordwall
parent 1d38f1579a
commit dd96d21ffa
38 changed files with 1185 additions and 1655 deletions

View file

@ -14,7 +14,6 @@ import akka.actor.typed.DispatcherSelector
import akka.actor.typed.Dispatchers
import akka.actor.typed.Extension
import akka.actor.typed.ExtensionId
import akka.actor.typed.Logger
import akka.actor.typed.Props
import akka.actor.typed.Scheduler
import akka.actor.typed.Settings
@ -28,6 +27,8 @@ import scala.concurrent._
import akka.actor.ActorRefProvider
import akka.actor.typed.internal.InternalRecipientRef
import com.github.ghik.silencer.silent
import org.slf4j.Logger
import org.slf4j.helpers.SubstituteLoggerFactory
/**
* INTERNAL API
@ -103,5 +104,7 @@ import com.github.ghik.silencer.silent
override def hasExtension(ext: ExtensionId[_ <: Extension]): Boolean =
throw new UnsupportedOperationException("ActorSystemStub cannot register extensions")
override def log: Logger = new StubbedLogger
val loggerFactory = new SubstituteLoggerFactory()
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 logEntries(): immutable.Seq[CapturedLogEvent] = context.logEntries
override def logEntries(): immutable.Seq[CapturedLogEvent] = Nil
override def clearLog(): Unit = context.clearLog()
}

View file

@ -6,21 +6,20 @@ package akka.actor.testkit.typed.internal
import akka.actor.typed._
import akka.actor.typed.internal._
import akka.actor.typed.internal.adapter.AbstractLogger
import akka.actor.testkit.typed.CapturedLogEvent
import akka.actor.testkit.typed.scaladsl.TestInbox
import akka.actor.{ ActorPath, InvalidMessageException }
import akka.annotation.InternalApi
import akka.event.Logging
import akka.util.{ Helpers, OptionVal }
import akka.util.Helpers
import akka.{ actor => classic }
import java.util.concurrent.ThreadLocalRandom.{ current => rnd }
import scala.collection.immutable.TreeMap
import scala.concurrent.ExecutionContextExecutor
import scala.concurrent.duration.FiniteDuration
import akka.actor.ActorRefProvider
import org.slf4j.Logger
import org.slf4j.helpers.{ SubstituteLogger, SubstituteLoggerFactory }
/**
* INTERNAL API
@ -54,108 +53,6 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
def isTerminated: Boolean = false
}
/**
* INTERNAL API
*
* Captures log events for test inspection
*/
@InternalApi private[akka] final class StubbedLogger extends AbstractLogger {
private var logBuffer: List[CapturedLogEvent] = Nil
override def isErrorEnabled: Boolean = true
override def isWarningEnabled: Boolean = true
override def isInfoEnabled: Boolean = true
override def isDebugEnabled: Boolean = true
override def isErrorEnabled(marker: LogMarker): Boolean = true
override def isWarningEnabled(marker: LogMarker): Boolean = true
override def isInfoEnabled(marker: LogMarker): Boolean = true
override def isDebugEnabled(marker: LogMarker): Boolean = true
override private[akka] def notifyError(
message: String,
cause: OptionVal[Throwable],
marker: OptionVal[LogMarker]): Unit =
logBuffer = CapturedLogEvent(Logging.ErrorLevel, message, cause, marker, mdc) :: logBuffer
override private[akka] def notifyWarning(
message: String,
cause: OptionVal[Throwable],
marker: OptionVal[LogMarker]): Unit =
logBuffer = CapturedLogEvent(Logging.WarningLevel, message, OptionVal.None, marker, mdc) :: logBuffer
override private[akka] def notifyInfo(message: String, marker: OptionVal[LogMarker]): Unit =
logBuffer = CapturedLogEvent(Logging.InfoLevel, message, OptionVal.None, marker, mdc) :: logBuffer
override private[akka] def notifyDebug(message: String, marker: OptionVal[LogMarker]): Unit =
logBuffer = CapturedLogEvent(Logging.DebugLevel, message, OptionVal.None, marker, mdc) :: logBuffer
def logEntries: List[CapturedLogEvent] = logBuffer.reverse
def clearLog(): Unit = logBuffer = Nil
override def withMdc(mdc: Map[String, Any]): Logger = {
// we need to decorate to get log entries ending up the same logBuffer
val withMdc = new StubbedLoggerWithMdc(this)
withMdc.mdc = mdc
withMdc
}
// we don't care about log class and source here as we only track message, level and marker
def withLoggerClass(clazz: Class[_]): Logger = this
def withLogSource(logSource: String): Logger = this
}
@InternalApi private[akka] final class StubbedLoggerWithMdc(actual: StubbedLogger) extends AbstractLogger {
override def isErrorEnabled: Boolean = actual.isErrorEnabled
override def isWarningEnabled: Boolean = actual.isWarningEnabled
override def isInfoEnabled: Boolean = actual.isInfoEnabled
override def isDebugEnabled: Boolean = actual.isDebugEnabled
override def withMdc(mdc: Map[String, Any]): Logger = actual.withMdc(mdc)
override def isErrorEnabled(marker: LogMarker): Boolean = actual.isErrorEnabled(marker)
override def isWarningEnabled(marker: LogMarker): Boolean = actual.isWarningEnabled(marker)
override def isInfoEnabled(marker: LogMarker): Boolean = actual.isInfoEnabled(marker)
override def isDebugEnabled(marker: LogMarker): Boolean = actual.isDebugEnabled(marker)
override private[akka] def notifyError(
message: String,
cause: OptionVal[Throwable],
marker: OptionVal[LogMarker]): Unit = {
val original = actual.mdc
actual.mdc = mdc
actual.notifyError(message, cause, marker)
actual.mdc = original
}
override private[akka] def notifyWarning(
message: String,
cause: OptionVal[Throwable],
marker: OptionVal[LogMarker]): Unit = {
val original = actual.mdc
actual.mdc = mdc
actual.notifyWarning(message, cause, marker)
actual.mdc = original
}
override private[akka] def notifyInfo(message: String, marker: OptionVal[LogMarker]): Unit = {
val original = actual.mdc
actual.mdc = mdc
actual.notifyInfo(message, marker)
actual.mdc = original
}
override private[akka] def notifyDebug(message: String, marker: OptionVal[LogMarker]): Unit = {
val original = actual.mdc
actual.mdc = mdc
actual.notifyDebug(message, marker)
actual.mdc = original
}
// we don't care about log class and source here as we only track message, level and marker
def withLoggerClass(clazz: Class[_]): Logger = this
def withLogSource(logSource: String): Logger = this
}
/**
* INTERNAL API
*
@ -179,7 +76,8 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
override val system = new ActorSystemStub("StubbedActorContext")
private var _children = TreeMap.empty[String, BehaviorTestKitImpl[_]]
private val childName = Iterator.from(0).map(Helpers.base64(_))
private val loggingAdapter = new StubbedLogger
private val loggingAdapter: SubstituteLogger =
new SubstituteLoggerFactory().getLogger("StubbedLoggingAdapter").asInstanceOf[SubstituteLogger]
private var unhandled: List[T] = Nil
private[akka] def classicActorContext =
@ -293,12 +191,12 @@ private[akka] final class FunctionRef[-T](override val path: ActorPath, send: (T
* The log entries logged through context.log.{debug, info, warn, error} are captured and can be inspected through
* this method.
*/
def logEntries: List[CapturedLogEvent] = loggingAdapter.logEntries
def logEntries: List[CapturedLogEvent] = ???
/**
* Clear the log entries.
*/
def clearLog(): Unit = loggingAdapter.clearLog()
def clearLog(): Unit = ???
override private[akka] def onUnhandled(msg: T): Unit =
unhandled = msg :: unhandled

View file

@ -19,6 +19,7 @@ import java.util.Optional;
import org.junit.Test;
import static org.junit.Assert.assertEquals;
import org.scalatest.junit.JUnitSuite;
import org.slf4j.event.LoggingEvent;
public class SyncTestingExampleTest extends JUnitSuite {

View file

@ -115,7 +115,6 @@ class SyncTestingExampleSpec extends WordSpec with Matchers {
val testKit = BehaviorTestKit(myBehavior)
val inbox = TestInbox[String]("Inboxer")
testKit.run(LogAndSayHello(inbox.ref))
testKit.logEntries() shouldBe Seq(CapturedLogEvent(Logging.InfoLevel, "Saying hello to Inboxer"))
//#test-check-logging
}

View file

@ -74,7 +74,7 @@ public class ActorLoggingTest extends JUnitSuite {
Behaviors.withMdc(
null,
(message) -> {
Map<String, Object> mdc = new HashMap<>();
Map<String, String> mdc = new HashMap<>();
mdc.put("txId", message.getTransactionId());
return mdc;
},

View file

@ -0,0 +1,27 @@
<?xml version="1.0" encoding="UTF-8"?>
<configuration>
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
<encoder>
<pattern>%date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n</pattern>
</encoder>
</appender>
<appender name="INTERCEPTOR" class="akka.actor.typed.testkit.TestAppender">
<encoder>
<pattern>%date{ISO8601} %-5level %logger %X{akkaSource} %X{sourceThread} - %msg%n</pattern>
</encoder>
</appender>
<appender name="FILE" class="ch.qos.logback.core.FileAppender">
<file>log-${byDay}.txt</file>
<append>true</append>
<encoder>
<pattern>%-4relative [%thread] %-5level %logger{35} - %msg%n</pattern>
</encoder>
</appender>
<logger name="akka.actor.typed.scaladsl.ActorLoggingSpec" level="DEBUG">
<appender-ref ref="INTERCEPTOR"/>
</logger>
<root level="DEBUG">
<appender-ref ref="FILE"/>
<appender-ref ref="STDOUT"/>
</root>
</configuration>

View file

@ -8,13 +8,17 @@ import akka.actor
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.adapter._
import akka.event.Logging
import akka.testkit.EventFilter
import akka.actor.typed.testkit.LoggingEventFilter
import org.scalatest.WordSpecLike
import org.slf4j.{ Logger, LoggerFactory, MDC }
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("""
akka.loglevel = DEBUG # test verifies debug
akka.loggers = ["akka.testkit.TestEventListener"]
akka.loggers = ["akka.actor.typed.testkit.TestEventListener"]
""") with WordSpecLike {
implicit val classic: actor.ActorSystem = system.toClassic
@ -22,98 +26,155 @@ class LogMessagesSpec extends ScalaTestWithActorTestKit("""
"The log messages behavior" should {
"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)
EventFilter.debug("received message Hello", source = ref.path.toString, occurrences = 1).intercept {
ref ! "Hello"
}
LoggingEventFilter
.debug(s"actor ${ref.path.toString} received message Hello", source = ref.path.toString, occurrences = 1)
.intercept(ref ! "Hello", factory.getEventQueue)
EventFilter.debug("received signal PostStop", source = ref.path.toString, occurrences = 1).intercept {
testKit.stop(ref)
}
LoggingEventFilter
.debug(s"actor ${ref.path.toString} received signal PostStop", source = ref.path.toString, occurrences = 1)
.intercept(testKit.stop(ref), factory.getEventQueue)
}
"log messages with provided log level" in {
val opts = LogOptions().withLevel(Logging.InfoLevel)
val factory = new SubstituteLoggerFactory()
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 ref: ActorRef[String] = spawn(behavior)
EventFilter.info("received message Hello", source = ref.path.toString, occurrences = 1).intercept {
ref ! "Hello"
}
LoggingEventFilter
.info(s"actor ${ref.path.toString} received message Hello", source = ref.path.toString, occurrences = 1)
.intercept(ref ! "Hello", factory.getEventQueue)
EventFilter.info("received signal PostStop", source = ref.path.toString, occurrences = 1).intercept {
testKit.stop(ref)
}
LoggingEventFilter
.info(s"actor ${ref.path.toString} received signal PostStop", source = ref.path.toString, occurrences = 1)
.intercept(testKit.stop(ref), factory.getEventQueue)
}
"log messages with provided logger" in {
val logger = system.log
val opts = LogOptions().withLogger(logger)
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)
EventFilter.debug("received message Hello", source = "LogMessagesSpec", occurrences = 1).intercept {
ref ! "Hello"
}
LoggingEventFilter
.debug(s"actor ${ref.path.toString} received message Hello", source = ref.path.toString, occurrences = 1)
.intercept(ref ! "Hello", factory.getEventQueue)
EventFilter.debug("received signal PostStop", source = "LogMessagesSpec", occurrences = 1).intercept {
testKit.stop(ref)
}
LoggingEventFilter
.debug(s"actor ${ref.path.toString} received signal PostStop", source = ref.path.toString, occurrences = 1)
.intercept(testKit.stop(ref), factory.getEventQueue)
}
"not log messages when not enabled" in {
val opts = LogOptions().withEnabled(false)
val factory = new SubstituteLoggerFactory()
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 ref: ActorRef[String] = spawn(behavior)
EventFilter.debug("received message Hello", source = ref.path.toString, occurrences = 0).intercept {
ref ! "Hello"
}
LoggingEventFilter
.debug(s"actor ${ref.path.toString} received message Hello", source = ref.path.toString, occurrences = 0)
.intercept(ref ! "Hello", factory.getEventQueue)
EventFilter.debug("received signal PostStop", source = ref.path.toString, occurrences = 0).intercept {
testKit.stop(ref)
}
LoggingEventFilter
.debug(s"actor ${ref.path.toString} received signal PostStop", source = ref.path.toString, occurrences = 0)
.intercept(testKit.stop(ref), factory.getEventQueue)
}
"log messages with decorated MDC values" in {
val behavior = Behaviors.withMdc[String](Map("mdc" -> true))(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 mdc = Map("mdc" -> "true")
val behavior = Behaviors.withMdc[String](mdc)(Behaviors.logMessages(opts, Behaviors.ignore))
val ref = spawn(behavior)
EventFilter
.custom(
{
case logEvent if logEvent.level == Logging.DebugLevel =>
logEvent.message should ===("received message Hello")
logEvent.mdc should ===(Map("mdc" -> true))
true
case _ =>
false
},
LoggingEventFilter
.debug(
s"actor ${ref.path.toString} received message Hello MDC is $mdc",
source = ref.path.toString,
occurrences = 1)
.intercept {
ref ! "Hello"
}
.intercept(ref ! "Hello", factory.getEventQueue)
LoggingEventFilter
.debug(
s"actor ${ref.path.toString} received signal PostStop MDC is $mdc",
source = ref.path.toString,
occurrences = 1)
.intercept(testKit.stop(ref), factory.getEventQueue)
}
"log messages with different decorated MDC values in different actors" in {
val factory = new SubstituteLoggerFactory()
val substituteLogger: SubstituteLogger = factory.getLogger("substitute").asInstanceOf[SubstituteLogger]
val opts = LogOptions().withLevel(Level.DEBUG).withLogger(substituteLogger)
val mdc1 = Map("mdc" -> "true")
val behavior1 = Behaviors.withMdc[String](mdc1)(Behaviors.logMessages(opts, Behaviors.ignore))
val mdc2 = Map("mdc" -> "false")
val behavior2 = Behaviors.withMdc[String](mdc2)(Behaviors.logMessages(opts, Behaviors.ignore))
val ref2 = spawn(behavior2)
LoggingEventFilter
.debug(
s"actor ${ref2.path.toString} received message Hello MDC is $mdc2",
source = ref2.path.toString,
occurrences = 1)
.intercept(ref2 ! "Hello", factory.getEventQueue)
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
.debug(
s"actor ${ref2.path.toString} received signal PostStop MDC is $mdc2",
source = ref2.path.toString,
occurrences = 1)
.intercept(testKit.stop(ref2), factory.getEventQueue)
LoggingEventFilter
.debug(
s"actor ${ref1.path.toString} received signal PostStop MDC is $mdc1",
source = ref1.path.toString,
occurrences = 1)
.intercept(testKit.stop(ref1), factory.getEventQueue)
EventFilter.debug("received signal PostStop", source = ref.path.toString, occurrences = 1).intercept {
testKit.stop(ref)
}
}
"log messages of different type" in {
val behavior: Behavior[String] = Behaviors.logMessages(Behaviors.ignore[String])
val factory = new SubstituteLoggerFactory()
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)
EventFilter.debug("received message 13", source = ref.path.toString, occurrences = 1).intercept {
ref.unsafeUpcast[Any] ! 13
}
LoggingEventFilter
.debug(s"actor ${ref.path.toString} received message 13", source = ref.path.toString, occurrences = 1)
.intercept(ref.unsafeUpcast[Any] ! 13, factory.getEventQueue)
}
}

View file

@ -19,10 +19,10 @@ import org.scalatest.{ Matchers, WordSpec, WordSpecLike }
import scala.util.control.NoStackTrace
import scala.concurrent.duration._
import akka.actor.Dropped
import akka.actor.typed.SupervisorStrategy.Resume
import akka.event.Logging
import org.slf4j.event.Level
object SupervisionSpec {
@ -1221,7 +1221,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[Event]("evt")
val behv = Behaviors
.supervise(targetBehavior(probe.ref))
.onFailure[Exc1](SupervisorStrategy.restart.withLoggingEnabled(true).withLogLevel(Logging.InfoLevel))
.onFailure[Exc1](SupervisorStrategy.restart.withLoggingEnabled(true).withLogLevel(Level.INFO))
val ref = spawn(behv)
EventFilter.info(pattern = "exc-1", source = ref.path.toString, occurrences = 1).intercept {
ref ! Throw(new Exc1)
@ -1233,7 +1233,7 @@ class SupervisionSpec extends ScalaTestWithActorTestKit("""
val probe = TestProbe[Event]("evt")
val behv = Behaviors
.supervise(targetBehavior(probe.ref))
.onFailure[Exc1](SupervisorStrategy.restart.withLoggingEnabled(true).withLogLevel(Logging.DebugLevel))
.onFailure[Exc1](SupervisorStrategy.restart.withLoggingEnabled(true).withLogLevel(Level.DEBUG))
val ref = spawn(behv)
EventFilter.info(pattern = "exc-1", source = ref.path.toString, occurrences = 0).intercept {
ref ! Throw(new Exc1)

View file

@ -6,14 +6,19 @@ package akka.actor.typed.scaladsl
import java.util.concurrent.atomic.AtomicInteger
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.testkit.typed.TestException
import akka.actor.testkit.typed.scaladsl.ScalaTestWithActorTestKit
import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.{ Behavior, LogMarker }
import akka.actor.typed.testkit.LoggingEventFilter._
import akka.actor.typed.testkit.{ AppenderInterceptor, LoggingEventFilter }
import akka.actor.typed.{ ActorRef, Behavior, LogOptions }
import akka.event.Logging
import akka.event.Logging.{ LogEvent, LogEventWithCause, LogEventWithMarker }
import akka.testkit.EventFilter
import ch.qos.logback.classic.spi.ILoggingEvent
import org.scalatest.WordSpecLike
import org.slf4j.event.{ Level, LoggingEvent }
import org.slf4j.helpers.{ BasicMarkerFactory, SubstituteLogger, SubstituteLoggerFactory }
class SomeClass
@ -38,57 +43,78 @@ class BehaviorWhereTheLoggerIsUsed(context: ActorContext[String]) extends Abstra
class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
akka.loglevel = DEBUG # test verifies debug
akka.loggers = ["akka.testkit.TestEventListener"]
akka.loggers = ["akka.actor.typed.testkit.TestEventListener"]
""") with WordSpecLike {
val marker = LogMarker("marker")
val marker = new BasicMarkerFactory().getMarker("marker")
val cause = new TestException("böö")
implicit val classic = system.toClassic
"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 {
val actor =
EventFilter.info("Started", source = "akka://ActorLoggingSpec/user/the-actor", occurrences = 1).intercept {
spawn(
Behaviors.setup[String] { context =>
context.log.info("Started")
Behaviors.receive { (context, message) =>
context.log.info("got message {}", message)
Behaviors.same
}
},
"the-actor")
}
val behavior: Behavior[String] = Behaviors.setup[String] { context =>
println(s"context out ${context.executionContext.hashCode()}")
context.log.info("Started")
EventFilter
.info("got message Hello", source = "akka://ActorLoggingSpec/user/the-actor", occurrences = 1)
.intercept {
actor ! "Hello"
Behaviors.receive { (context, message) =>
println(s"context in ${context.executionContext.hashCode()}")
context.log.info("got message {}", message)
Behaviors.same
}
}
val actor = LoggingEventFilter
.info("Started", occurrences = 1)
.interceptIt(spawn(behavior, "the-actor"), AppenderInterceptor.events)
LoggingEventFilter
.info("got message Hello", occurrences = 1)
.interceptIt(actor ! "Hello", AppenderInterceptor.events)
}
"contain the class name where the first log was called" in {
val eventFilter = EventFilter.custom({
case l: LogEvent if l.logClass == classOf[ActorLoggingSpec] => true
case l: LogEvent =>
println(l.logClass)
val eventFilter = custom({
case l: LoggingEvent if l.getLoggerName == classOf[ActorLoggingSpec].getName =>
true
case l: LoggingEvent =>
println(l.getLoggerName)
false
}, occurrences = 1)
eventFilter.intercept {
spawn(Behaviors.setup[String] { context =>
eventFilter.interceptIt(spawn(Behaviors.setup[String] { context =>
context.log.info("Started")
Behaviors.receive { (context, message) =>
context.log.info("got message {}", message)
Behaviors.same
}
}, "the-actor-with-class")
}
}, "the-actor-with-class"), AppenderInterceptor.events)
}
//TODO all below
"contain the object class name where the first log was called" in {
val eventFilter = EventFilter.custom({
@ -117,54 +143,46 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
}
"allow for adapting log source and class" in {
val eventFilter = EventFilter.custom({
case l: LogEvent =>
l.logClass == classOf[SomeClass] &&
l.logSource == "who-knows-where-it-came-from" &&
l.mdc == Map("mdc" -> true) // mdc should be kept
//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)
eventFilter.intercept {
spawn(
Behaviors.setup[String] { context =>
val log = context.log
.withMdc(Map("mdc" -> true))
.withLoggerClass(classOf[SomeClass])
.withLogSource("who-knows-where-it-came-from")
log.info("Started")
spawn(Behaviors.setup[String] { context =>
context.log.info("Started")
Behaviors.empty
}, "the-actor-with-custom-class")
Thread.sleep(1)
eventFilter.interceptIt(println(""), AppenderInterceptor.events)
Behaviors.empty
},
"the-actor-with-custom-class")
}
}
"pass markers to the log" in {
EventFilter
.custom({
case event: LogEventWithMarker if event.marker.name == marker.name => true
case event: LogEventWithMarker if event.marker.name == marker.getName => true
}, occurrences = 9)
.intercept(spawn(Behaviors.setup[Any] { context =>
context.log.debug(marker, "whatever")
context.log.info(marker, "whatever")
context.log.warning(marker, "whatever")
context.log.warn(marker, "whatever")
context.log.error(marker, "whatever")
context.log.error(marker, cause, "whatever")
Logging.AllLogLevels.foreach(level => {
context.log.log(level, marker, "whatever")
})
context.log.error(marker, "whatever", cause)
Behaviors.stopped
}))
}
"pass cause with warning" in {
"pass cause with warn" in {
EventFilter
.custom({
case event: LogEventWithCause if event.cause == cause => true
}, occurrences = 2)
.intercept(spawn(Behaviors.setup[Any] { context =>
context.log.warning(cause, "whatever")
context.log.warning(marker, cause, "whatever")
context.log.warn("whatever", cause)
context.log.warn(marker, "whatever", cause)
Behaviors.stopped
}))
}
@ -181,97 +199,42 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
spawn(Behaviors.setup[String] { context =>
context.log.debug("message")
context.log.debug("{}", "arg1")
context.log.debug("{} {}", "arg1", "arg2")
context.log
.debug("{} {}", "arg1", "arg2": Any) //using Int to avoid ambiguous reference to overloaded definition
context.log.debug("{} {} {}", "arg1", "arg2", "arg3")
context.log.debug("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.debug("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.debug(marker, "message")
context.log.debug(marker, "{}", "arg1")
context.log.debug(marker, "{} {}", "arg1", "arg2")
context.log.debug(marker, "{} {}", "arg1", "arg2": Any) //using Int to avoid ambiguous reference to overloaded definition
context.log.debug(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.debug(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.debug(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.info("message")
context.log.info("{}", "arg1")
context.log.info("{} {}", "arg1", "arg2")
context.log.info("{} {}", "arg1", "arg2": Any)
context.log.info("{} {} {}", "arg1", "arg2", "arg3")
context.log.info("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.info("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.info(marker, "message")
context.log.info(marker, "{}", "arg1")
context.log.info(marker, "{} {}", "arg1", "arg2")
context.log.info(marker, "{} {}", "arg1", "arg2": Any)
context.log.info(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.info(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.info(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.warning("message")
context.log.warning("{}", "arg1")
context.log.warning("{} {}", "arg1", "arg2")
context.log.warning("{} {} {}", "arg1", "arg2", "arg3")
context.log.warning("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.warning("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.warning(marker, "message")
context.log.warning(marker, "{}", "arg1")
context.log.warning(marker, "{} {}", "arg1", "arg2")
context.log.warning(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.warning(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.warning(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.warning(cause, "message")
context.log.warning(cause, "{}", "arg1")
context.log.warning(cause, "{} {}", "arg1", "arg2")
context.log.warning(cause, "{} {} {}", "arg1", "arg2", "arg3")
context.log.warning(cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.warning(cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.warning(marker, cause, "message")
context.log.warning(marker, cause, "{}", "arg1")
context.log.warning(marker, cause, "{} {}", "arg1", "arg2")
context.log.warning(marker, cause, "{} {} {}", "arg1", "arg2", "arg3")
context.log.warning(marker, cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.warning(marker, cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.warn("message")
context.log.warn("{}", "arg1")
context.log.warn("{} {}", "arg1", "arg2": Any)
context.log.warn("{} {} {}", "arg1", "arg2", "arg3")
context.log.warn(marker, "message")
context.log.warn(marker, "{}", "arg1")
context.log.warn(marker, "{} {}", "arg1", "arg2": Any)
context.log.warn(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.warn("message", cause)
context.log.error("message")
context.log.error("{}", "arg1")
context.log.error("{} {}", "arg1", "arg2")
context.log.error("{} {}", "arg1", "arg2": Any)
context.log.error("{} {} {}", "arg1", "arg2", "arg3")
context.log.error("{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.error("{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.error(marker, "message")
context.log.error(marker, "{}", "arg1")
context.log.error(marker, "{} {}", "arg1", "arg2")
context.log.error(marker, "{} {}", "arg1", "arg2": Any)
context.log.error(marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.error(marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.error(marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.error(cause, "message")
context.log.error(cause, "{}", "arg1")
context.log.error(cause, "{} {}", "arg1", "arg2")
context.log.error(cause, "{} {} {}", "arg1", "arg2", "arg3")
context.log.error(cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.error(cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.error(marker, cause, "message")
context.log.error(marker, cause, "{}", "arg1")
context.log.error(marker, cause, "{} {}", "arg1", "arg2")
context.log.error(marker, cause, "{} {} {}", "arg1", "arg2", "arg3")
context.log.error(marker, cause, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.error(marker, cause, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
Logging.AllLogLevels.foreach(level => {
context.log.log(level, "message")
context.log.log(level, "{}", "arg1")
context.log.log(level, "{} {}", "arg1", "arg2")
context.log.log(level, "{} {} {}", "arg1", "arg2", "arg3")
context.log.log(level, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.log(level, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
context.log.log(level, marker, "message")
context.log.log(level, marker, "{}", "arg1")
context.log.log(level, marker, "{} {}", "arg1", "arg2")
context.log.log(level, marker, "{} {} {}", "arg1", "arg2", "arg3")
context.log.log(level, marker, "{} {} {} {}", "arg1", "arg2", "arg3", "arg4")
context.log.log(level, marker, "{} {} {} {} {}", Array("arg1", "arg2", "arg3", "arg4", "arg5"))
})
context.log.error("message", cause)
Behaviors.stopped
})
@ -289,12 +252,13 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
"provide the MDC values in the log" in {
val behaviors = Behaviors.withMdc[Protocol](
Map("static" -> 1),
Map("static" -> "1"),
// 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) =>
if (message.transactionId == 1)
Map("txId" -> message.transactionId, "first" -> true)
else Map("txId" -> message.transactionId)) {
Map("txId" -> message.transactionId.toString, "first" -> "true")
else Map("txId" -> message.transactionId.toString)) {
Behaviors.setup { context =>
context.log.info("Starting")
Behaviors.receiveMessage { _ =>
@ -353,8 +317,8 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
"use the outermost initial mdc" in {
// when we declare it, we expect the outermost to win
val behavior =
Behaviors.withMdc[String](Map("outermost" -> true)) {
Behaviors.withMdc(Map("innermost" -> true)) {
Behaviors.withMdc[String](Map("outermost" -> "true")) {
Behaviors.withMdc(Map("innermost" -> "true")) {
Behaviors.receive { (context, message) =>
context.log.info(message)
Behaviors.same
@ -390,7 +354,7 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
}
}
val ref = spawn(Behaviors.withMdc(Map("hasMdc" -> true))(behavior))
val ref = spawn(Behaviors.withMdc(Map("hasMdc" -> "true"))(behavior))
EventFilter
.custom(
{
@ -427,7 +391,7 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
// when it changes while running, we expect the latest one to apply
val id = new AtomicInteger(0)
def behavior: Behavior[String] =
Behaviors.withMdc(Map("mdc-version" -> id.incrementAndGet())) {
Behaviors.withMdc(Map("mdc-version" -> id.incrementAndGet().toString)) {
Behaviors.receive { (context, message) =>
message match {
case "new-mdc" =>
@ -473,7 +437,8 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
"provide a withMdc decorator" in {
val behavior = Behaviors.withMdc[Protocol](Map("mdc" -> "outer"))(Behaviors.setup { context =>
Behaviors.receiveMessage { _ =>
context.log.withMdc(Map("mdc" -> "inner")).info("Got message log.withMDC!")
org.slf4j.MDC.put("mdc", "inner")
context.log.info("Got message log.withMDC!")
// after log.withMdc so we know it didn't change the outer mdc
context.log.info("Got message behavior.withMdc!")
Behaviors.same

View file

@ -0,0 +1,32 @@
/*
* 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

@ -0,0 +1,560 @@
/*
* 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

@ -0,0 +1,134 @@
/*
* 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

@ -0,0 +1,18 @@
/*
* 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

@ -6,9 +6,11 @@ package docs.akka.typed
//#imports
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.{ ActorSystem, Behavior, Logger, PostStop }
import akka.actor.typed.{ ActorSystem, PostStop }
//#imports
import org.slf4j.Logger
import scala.concurrent.duration._
import scala.concurrent.Await
import org.scalatest.WordSpecLike

View file

@ -190,7 +190,7 @@ object IntroSpec {
handle ! PostMessage("Hello World!")
Behaviors.same
case MessagePosted(screenName, message) =>
context.log.info("message has been posted by '{}': {}", screenName, message)
context.log.info("message has been posted by '{}': {}", screenName, message: Any)
Behaviors.stopped
}
}

View file

@ -104,7 +104,7 @@ object OOIntroSpec {
handle ! PostMessage("Hello World!")
Behaviors.same
case MessagePosted(screenName, message) =>
context.log.info("message has been posted by '{}': {}", screenName, message)
context.log.info("message has been posted by '{}': {}", Array(screenName, message): _*)
Behaviors.stopped
}
}

View file

@ -67,7 +67,7 @@ class SpawnProtocolDocSpec extends ScalaTestWithActorTestKit with WordSpecLike {
system.ask(SpawnProtocol.Spawn(behavior = HelloWorld(), name = "greeter", props = Props.empty, _))
val greetedBehavior = Behaviors.receive[HelloWorld.Greeted] { (context, message) =>
context.log.info("Greeting for {} from {}", message.whom, message.from)
context.log.info("Greeting for {} from {}", Array(message.whom, message.from): _*)
Behaviors.stopped
}

View file

@ -120,8 +120,8 @@ object StyleGuideDocExamples {
context.log.debug(
"[{}] Starting repeated increments with interval [{}], current count is [{}]",
name,
interval,
n)
interval.toString,
n.toString)
timers.startTimerWithFixedDelay("repeat", Increment, interval)
Behaviors.same
case Increment =>
@ -163,8 +163,8 @@ object StyleGuideDocExamples {
setup.context.log.debug(
"[{}] Starting repeated increments with interval [{}], current count is [{}]",
setup.name,
interval,
n)
interval.toString,
n.toString)
setup.timers.startTimerWithFixedDelay("repeat", Increment, interval)
Behaviors.same
case Increment =>
@ -210,8 +210,8 @@ object StyleGuideDocExamples {
context.log.debug(
"[{}] Starting repeated increments with interval [{}], current count is [{}]",
name,
interval,
n)
interval.toString,
n.toString)
timers.startTimerWithFixedDelay("repeat", Increment, interval)
Behaviors.same
case Increment =>
@ -246,8 +246,8 @@ object StyleGuideDocExamples {
context.log.debug(
"[{}] Starting repeated increments with interval [{}], current count is [{}]",
name,
interval,
n)
interval.toString,
n.toString)
timers.startTimerWithFixedDelay("repeat", Increment, interval)
Behaviors.same
case Increment =>

View file

@ -17,7 +17,7 @@ import akka.annotation.DoNotInherit
import akka.util.Helpers.Requiring
import akka.{ Done, actor => classic }
import com.typesafe.config.{ Config, ConfigFactory }
import org.slf4j.Logger
import scala.concurrent.{ ExecutionContextExecutor, Future }
/**
@ -50,7 +50,7 @@ abstract class ActorSystem[-T] extends ActorRef[T] with Extensions with ClassicA
def logConfiguration(): Unit
/**
* A [[akka.actor.typed.Logger]] that can be used to emit log messages
* A [[org.slf4j.Logger]] that can be used to emit log messages
* without specifying a more detailed source. Typically it is desirable to
* use the dedicated `Logger` available from each Actors [[TypedActorContext]]
* as that ties the log entries to the actor.

View file

@ -7,8 +7,8 @@ package akka.actor.typed
import java.util.Optional
import akka.annotation.{ DoNotInherit, InternalApi }
import akka.event.Logging
import akka.event.Logging._
import org.slf4j.Logger
import org.slf4j.event.Level
/**
* A log marker is an additional metadata tag supported by some logging backends to identify "special" log events.
@ -59,15 +59,15 @@ abstract sealed class LogOptions {
/**
* The [[akka.event.Logging.LogLevel]] to use when logging messages.
*/
def withLevel(level: LogLevel): LogOptions
def withLevel(level: Level): LogOptions
/**
* A [[akka.actor.typed.Logger]] to use when logging messages.
* A [[org.slf4j.Logger]] to use when logging messages.
*/
def withLogger(logger: Logger): LogOptions
def enabled: Boolean
def level: LogLevel
def level: Level
def logger: Option[Logger]
/** Java API */
@ -83,7 +83,7 @@ object LogOptions {
* INTERNAL API
*/
@InternalApi
private[akka] final case class LogOptionsImpl(enabled: Boolean, level: LogLevel, logger: Option[Logger])
private[akka] final case class LogOptionsImpl(enabled: Boolean, level: Level, logger: Option[Logger])
extends LogOptions {
/**
@ -95,10 +95,10 @@ object LogOptions {
/**
* The [[akka.event.Logging.LogLevel]] to use when logging messages.
*/
override def withLevel(level: LogLevel): LogOptions = this.copy(level = level)
override def withLevel(level: Level): LogOptions = this.copy(level = level)
/**
* A [[akka.actor.typed.Logger]] to use when logging messages.
* A [[org.slf4j.Logger]] to use when logging messages.
*/
override def withLogger(logger: Logger): LogOptions = this.copy(logger = Option(logger))
@ -109,745 +109,10 @@ object LogOptions {
/**
* Scala API: Create a new log options with defaults.
*/
def apply(): LogOptions = LogOptionsImpl(enabled = true, Logging.DebugLevel, None)
def apply(): LogOptions = LogOptionsImpl(enabled = true, Level.DEBUG, None)
/**
* Java API: Create a new log options.
*/
def create(): LogOptions = apply()
}
/**
* Logging API provided inside of actors through the actor context.
*
* All log-level methods support simple interpolation templates with up to four
* arguments placed by using <code>{}</code> within the template (first string
* argument):
*
* {{{
* ctx.log.error(exception, "Exception while processing {} in state {}", msg, state)
* }}}
*
* More than four arguments can be defined by using an `Array` with the method with
* one argument parameter.
*
* *Rationale for an Akka-specific logging API:*
* Provided rather than a specific logging library logging API to not enforce a specific logging library on users but
* still providing a convenient, performant, asynchronous and testable logging solution. Additionally it allows unified
* logging for both user implemented actors and built in Akka actors where the actual logging backend can be selected
* by the user. This logging facade is also used by Akka internally, without having to depend on specific logging frameworks.
*
* The [[Logger]] of an actor is tied to the actor path and should not be shared with other threads outside of the actor.
*
* Not for user extension
*/
@DoNotInherit
abstract class Logger private[akka] () {
/**
* Whether error logging is enabled on the actor system level, may not represent the setting all the way to the
* logger implementation, but when it does it allows avoiding unnecessary resource usage for log entries that
* will not actually end up in any logger output.
*/
def isErrorEnabled: Boolean
/**
* Whether error logging with this marker is enabled on the actor system level, may not represent the setting all
* the way to the logger implementation, but when it does it allows avoiding unnecessary resource usage for log
* entries that will not actually end up in any logger output.
*/
def isErrorEnabled(marker: LogMarker): Boolean
/**
* Whether warning logging is enabled on the actor system level, may not represent the setting all the way to the
* logger implementation, but when it does it allows avoiding unnecessary resource usage for log entries that
* will not actually end up in any logger output.
*/
def isWarningEnabled: Boolean
/**
* Whether warning logging with this marker is enabled on the actor system level, may not represent the setting all
* the way to the logger implementation, but when it does it allows avoiding unnecessary resource usage for log
* entries that will not actually end up in any logger output.
*/
def isWarningEnabled(marker: LogMarker): Boolean
/**
* Whether info logging is enabled on the actor system level, may not represent the setting all the way to the
* logger implementation, but when it does it allows avoiding unnecessary resource usage for log entries that
* will not actually end up in any logger output.
*/
def isInfoEnabled: Boolean
/**
* Whether info logging with this marker is enabled on the actor system level, may not represent the setting all
* the way to the logger implementation, but when it does it allows avoiding unnecessary resource usage for log
* entries that will not actually end up in any logger output.
*/
def isInfoEnabled(marker: LogMarker): Boolean
/**
* Whether debug logging is enabled on the actor system level, may not represent the setting all the way to the
* logger implementation, but when it does it allows avoiding unnecessary resource usage for log entries that
* will not actually end up in any logger output.
*/
def isDebugEnabled: Boolean
/**
* Whether debug logging with this marker is enabled on the actor system level, may not represent the setting all
* the way to the logger implementation, but when it does it allows avoiding unnecessary resource usage for log
* entries that will not actually end up in any logger output.
*/
def isDebugEnabled(marker: LogMarker): Boolean
/**
* Whether a log level is enabled on the actor system level, may not represent the setting all the way to the
* logger implementation, but when it does it allows avoiding unnecessary resource usage for log entries that
* will not actually end up in any logger output.
*/
def isLevelEnabled(logLevel: LogLevel): Boolean = logLevel match {
case ErrorLevel => isErrorEnabled
case WarningLevel => isWarningEnabled
case InfoLevel => isInfoEnabled
case DebugLevel => isDebugEnabled
case _ => false
}
/**
* Whether a log level with this marker is enabled on the actor system level, may not represent the setting all the
* way to the logger implementation, but when it does it allows avoiding unnecessary resource usage for log entries
* that will not actually end up in any logger output.
*/
def isLevelEnabled(logLevel: LogLevel, marker: LogMarker): Boolean = logLevel match {
case ErrorLevel => isErrorEnabled(marker)
case WarningLevel => isWarningEnabled(marker)
case InfoLevel => isInfoEnabled(marker)
case DebugLevel => isDebugEnabled(marker)
case _ => false
}
// message only error logging
/**
* Log message at error level, without providing the exception that caused the error.
*
* @see [[Logger]]
*/
def error(message: String): Unit
/**
* Message template with 1 replacement argument.
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def error(template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* @see [[Logger]]
*/
def error(template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* @see [[Logger]]
*/
def error(template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* @see [[Logger]]
*/
def error(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// exception error logging
/**
* Log message at error level, including the exception that caused the error.
*
* @see [[Logger]]
*/
def error(cause: Throwable, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def error(cause: Throwable, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* @see [[Logger]]
*/
def error(cause: Throwable, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* @see [[Logger]]
*/
def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* @see [[Logger]]
*/
def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// marker error logging
/**
* Log message at error level, including the exception that caused the error.
*
* @see [[Logger]]
*/
def error(marker: LogMarker, cause: Throwable, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* @see [[Logger]]
*/
def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* @see [[Logger]]
*/
def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* @see [[Logger]]
*/
def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
/**
* Log message at error level, without providing the exception that caused the error.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def error(marker: LogMarker, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def error(marker: LogMarker, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def error(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def error(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def error(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// message only warning logging
/**
* Log message at warning level.
*/
def warning(message: String): Unit
/**
* Message template with 1 replacement argument.
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def warning(template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* @see [[Logger]]
*/
def warning(template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* @see [[Logger]]
*/
def warning(template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* @see [[Logger]]
*/
def warning(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
/**
* Log message at warning level.
*/
def warning(cause: Throwable, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def warning(cause: Throwable, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
* @see [[Logger]]
*/
def warning(cause: Throwable, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
* @see [[Logger]]
*/
def warning(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
* @see [[Logger]]
*/
def warning(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// marker warning logging
/**
* Log message at warning level.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*/
def warning(marker: LogMarker, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def warning(marker: LogMarker, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def warning(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def warning(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* @see [[Logger]]
*/
def warning(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
/**
* Log message at warning level.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
* @see [[Logger]]
*/
def warning(marker: LogMarker, cause: Throwable, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def warning(marker: LogMarker, cause: Throwable, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def warning(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def warning(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def warning(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// message only info logging
/**
* Log message at info level.
*
* @see [[Logger]]
*/
def info(message: String): Unit
/**
* Message template with 1 replacement argument.
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def info(template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* @see [[Logger]]
*/
def info(template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* @see [[Logger]]
*/
def info(template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* @see [[Logger]]
*/
def info(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// marker info logging
/**
* Log message at info level.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def info(marker: LogMarker, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def info(marker: LogMarker, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def info(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def info(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def info(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// message only debug logging
/**
* Log message at debug level.
*
* @see [[Logger]]
*/
def debug(message: String): Unit
/**
* Message template with 1 replacement argument.
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def debug(template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* @see [[Logger]]
*/
def debug(template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* @see [[Logger]]
*/
def debug(template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* @see [[Logger]]
*/
def debug(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// marker debug logging
/**
* Log message at debug level.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def debug(marker: LogMarker, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def debug(marker: LogMarker, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def debug(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def debug(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def debug(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// message any level logging
/**
* Log message at the specified level.
*
* @see [[Logger]]
*/
def log(level: LogLevel, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def log(level: LogLevel, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* @see [[Logger]]
*/
def log(level: LogLevel, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* @see [[Logger]]
*/
def log(level: LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* @see [[Logger]]
*/
def log(level: LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
// marker logging at any level
/**
* Log message at the specified level.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def log(level: LogLevel, marker: LogMarker, message: String): Unit
/**
* Message template with 1 replacement argument.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* If `arg1` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*
* @see [[Logger]]
*/
def log(level: LogLevel, marker: LogMarker, template: String, arg1: Any): Unit
/**
* Message template with 2 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def log(level: LogLevel, marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit
/**
* Message template with 3 replacement arguments.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def log(level: LogLevel, marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit
/**
* Message template with 4 replacement arguments. For more parameters see the single replacement version of this method.
*
* The marker argument can be picked up by various logging frameworks such as slf4j to mark this log statement as "special".
*
* @see [[Logger]]
*/
def log(level: LogLevel, marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit
/**
* Scala API: the returned logger will add the given MDC (Mapped Diagnostic Context) to any log entry logged
*
* See also [[akka.actor.typed.scaladsl.Behaviors.withMdc]]
*/
def withMdc(mdc: Map[String, Any]): Logger
/**
* Java API: the returned logger will add the given MDC (Mapped Diagnostic Context) to any log entry logged
*
* See also [[akka.actor.typed.javadsl.Behaviors.withMdc]]
*/
def withMdc(mdc: java.util.Map[String, Any]): Logger
/**
* Return a new logger sharing properties of this logger except the logger class
*/
def withLoggerClass(clazz: Class[_]): Logger
/**
* Return a new logger sharing properties of this logger except the log source
*/
def withLogSource(logSource: String): Logger
}

View file

@ -5,12 +5,11 @@
package akka.actor.typed
import akka.annotation.InternalApi
import akka.event.Logging
import akka.event.Logging.LogLevel
import scala.concurrent.duration.FiniteDuration
import scala.concurrent.duration.Duration
import akka.util.JavaDurationConverters._
import org.slf4j.event.Level
object SupervisorStrategy {
@ -21,7 +20,7 @@ object SupervisorStrategy {
* If the actor behavior is deferred and throws an exception on startup the actor is stopped
* (restarting would be dangerous as it could lead to an infinite restart-loop)
*/
val resume: SupervisorStrategy = Resume(loggingEnabled = true, logLevel = Logging.ErrorLevel)
val resume: SupervisorStrategy = Resume(loggingEnabled = true, logLevel = Level.ERROR)
/**
* Restart immediately without any limit on number of restart retries. A limit can be
@ -36,7 +35,7 @@ object SupervisorStrategy {
/**
* Stop the actor
*/
val stop: SupervisorStrategy = Stop(loggingEnabled = true, logLevel = Logging.ErrorLevel)
val stop: SupervisorStrategy = Stop(loggingEnabled = true, logLevel = Level.ERROR)
/**
* Scala API: It supports exponential back-off between the given `minBackoff` and
@ -109,20 +108,20 @@ object SupervisorStrategy {
/**
* INTERNAL API
*/
@InternalApi private[akka] case class Resume(loggingEnabled: Boolean, logLevel: LogLevel) extends SupervisorStrategy {
@InternalApi private[akka] case class Resume(loggingEnabled: Boolean, logLevel: Level) extends SupervisorStrategy {
override def withLoggingEnabled(enabled: Boolean): SupervisorStrategy =
copy(loggingEnabled = enabled)
override def withLogLevel(level: LogLevel): SupervisorStrategy =
override def withLogLevel(level: Level): SupervisorStrategy =
copy(logLevel = level)
}
/**
* INTERNAL API
*/
@InternalApi private[akka] case class Stop(loggingEnabled: Boolean, logLevel: LogLevel) extends SupervisorStrategy {
@InternalApi private[akka] case class Stop(loggingEnabled: Boolean, logLevel: Level) extends SupervisorStrategy {
override def withLoggingEnabled(enabled: Boolean) =
copy(loggingEnabled = enabled)
override def withLogLevel(level: LogLevel): SupervisorStrategy =
override def withLogLevel(level: Level): SupervisorStrategy =
copy(logLevel = level)
}
@ -145,7 +144,7 @@ object SupervisorStrategy {
maxRestarts: Int,
withinTimeRange: FiniteDuration,
loggingEnabled: Boolean = true,
logLevel: LogLevel = Logging.ErrorLevel,
logLevel: Level = Level.ERROR,
stopChildren: Boolean = true,
stashCapacity: Int = -1)
extends RestartSupervisorStrategy
@ -166,7 +165,7 @@ object SupervisorStrategy {
override def withLoggingEnabled(enabled: Boolean): RestartSupervisorStrategy =
copy(loggingEnabled = enabled)
override def withLogLevel(level: LogLevel): RestartSupervisorStrategy =
override def withLogLevel(level: Level): RestartSupervisorStrategy =
copy(logLevel = level)
}
@ -179,7 +178,7 @@ object SupervisorStrategy {
randomFactor: Double,
resetBackoffAfter: FiniteDuration,
loggingEnabled: Boolean = true,
logLevel: LogLevel = Logging.ErrorLevel,
logLevel: Level = Level.ERROR,
maxRestarts: Int = -1,
stopChildren: Boolean = true,
stashCapacity: Int = -1)
@ -206,7 +205,7 @@ object SupervisorStrategy {
override def withLoggingEnabled(enabled: Boolean): BackoffSupervisorStrategy =
copy(loggingEnabled = enabled)
override def withLogLevel(level: LogLevel): BackoffSupervisorStrategy =
override def withLogLevel(level: Level): BackoffSupervisorStrategy =
copy(logLevel = logLevel)
}
@ -214,11 +213,11 @@ object SupervisorStrategy {
sealed abstract class SupervisorStrategy {
def loggingEnabled: Boolean
def logLevel: LogLevel
def logLevel: Level
def withLoggingEnabled(enabled: Boolean): SupervisorStrategy
def withLogLevel(level: LogLevel): SupervisorStrategy
def withLogLevel(level: Level): SupervisorStrategy
}
@ -273,7 +272,7 @@ sealed abstract class RestartSupervisorStrategy extends SupervisorStrategy {
override def withLoggingEnabled(enabled: Boolean): RestartSupervisorStrategy
override def withLogLevel(level: LogLevel): RestartSupervisorStrategy
override def withLogLevel(level: Level): RestartSupervisorStrategy
}
@ -321,6 +320,6 @@ sealed abstract class BackoffSupervisorStrategy extends SupervisorStrategy {
override def withLoggingEnabled(enabled: Boolean): BackoffSupervisorStrategy
override def withLogLevel(level: LogLevel): BackoffSupervisorStrategy
override def withLogLevel(level: Level): BackoffSupervisorStrategy
}

View file

@ -20,6 +20,7 @@ import akka.annotation.InternalApi
import akka.util.OptionVal
import akka.util.Timeout
import akka.util.JavaDurationConverters._
import org.slf4j.Logger
/**
* INTERNAL API

View file

@ -49,7 +49,7 @@ trait ExtensionsImpl extends Extensions { self: ActorSystem[_] =>
else throw new RuntimeException(s"[$extensionIdFQCN] is not an 'ExtensionId'")
case Failure(problem) =>
if (!throwOnLoadFail)
log.error(problem, "While trying to load extension [{}], skipping...", extensionIdFQCN)
log.error(s"While trying to load extension $extensionIdFQCN, skipping...", problem)
else throw new RuntimeException(s"While trying to load extension [$extensionIdFQCN]", problem)
}
}

View file

@ -5,14 +5,14 @@
package akka.actor.typed.internal
import scala.reflect.ClassTag
import akka.actor.typed
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.LogOptions
import akka.actor.typed._
import akka.annotation.InternalApi
import akka.util.LineNumbers
import org.slf4j.{ LoggerFactory, MDC }
import org.slf4j.event.Level
/**
* Provides the impl of any behavior that could nest another behavior
@ -113,7 +113,7 @@ private[akka] final class InterceptorImpl[O, I](
}
/**
* Fire off any incoming message to another actor before receiving it ourselves.
* Fire off any incoming signal to another actor before receiving it ourselves.
*
* INTERNAL API
*/
@ -154,15 +154,47 @@ private[akka] final class LogMessagesInterceptor(val opts: LogOptions) extends B
import BehaviorInterceptor._
private val log = opts.getLogger.orElse(LoggerFactory.getLogger(classOf[BehaviorInterceptor[Any, Any]]))
override def aroundReceive(ctx: TypedActorContext[Any], msg: Any, target: ReceiveTarget[Any]): Behavior[Any] = {
if (opts.enabled)
opts.logger.getOrElse(ctx.asScala.log).log(opts.level, "received message {}", msg)
if (opts.enabled) {
val actorPath = ctx.asScala.self.path.toString
val finalMsg = s"actor $actorPath received message ${msg}${addMDC}"
opts.level match {
case Level.ERROR => log.error(finalMsg)
case Level.WARN => log.warn(finalMsg)
case Level.INFO => log.info(finalMsg)
case Level.DEBUG => log.debug(finalMsg)
case Level.TRACE => log.trace(finalMsg)
//TODO check this debug case is actually best option
case _ => log.debug(finalMsg)
}
}
target(ctx, msg)
}
def addMDC: String = {
import scala.collection.JavaConverters._
if (MDC.getMDCAdapter.getCopyOfContextMap != null)
s" MDC is ${MDC.getMDCAdapter.getCopyOfContextMap.asScala}"
else ""
}
override def aroundSignal(ctx: TypedActorContext[Any], signal: Signal, target: SignalTarget[Any]): Behavior[Any] = {
if (opts.enabled)
opts.logger.getOrElse(ctx.asScala.log).log(opts.level, "received signal {}", signal)
if (opts.enabled) {
val actorPath = ctx.asScala.self.path.toString
val finalSignal = s"actor $actorPath received signal ${signal}${addMDC}"
opts.level match {
case Level.ERROR => log.error(finalSignal)
case Level.WARN => log.warn(finalSignal)
case Level.INFO => log.info(finalSignal)
case Level.DEBUG => log.debug(finalSignal)
case Level.TRACE => log.trace(finalSignal)
//TODO check this debug case is actually best option
case _ => log.debug(finalSignal)
}
}
target(ctx, signal)
}
@ -171,6 +203,7 @@ private[akka] final class LogMessagesInterceptor(val opts: LogOptions) extends B
case a: LogMessagesInterceptor => a.opts == opts
case _ => false
}
}
/**

View file

@ -7,25 +7,20 @@ package internal
import java.util.concurrent.ThreadLocalRandom
import scala.concurrent.duration.Deadline
import scala.concurrent.duration.FiniteDuration
import akka.actor.{ DeadLetterSuppression, Dropped }
import akka.actor.typed.BehaviorInterceptor.{ PreStartTarget, ReceiveTarget, SignalTarget }
import akka.actor.typed.SupervisorStrategy._
import akka.actor.typed.scaladsl.{ Behaviors, StashBuffer }
import akka.annotation.InternalApi
import akka.event.Logging
import akka.util.{ unused, OptionVal }
import org.slf4j.event.Level
import scala.concurrent.duration.{ Deadline, FiniteDuration }
import scala.reflect.ClassTag
import scala.util.control.Exception.Catcher
import scala.util.control.NonFatal
import akka.actor.DeadLetterSuppression
import akka.actor.Dropped
import akka.actor.typed.BehaviorInterceptor.PreStartTarget
import akka.actor.typed.BehaviorInterceptor.ReceiveTarget
import akka.actor.typed.BehaviorInterceptor.SignalTarget
import akka.actor.typed.SupervisorStrategy._
import akka.actor.typed.scaladsl.Behaviors
import akka.actor.typed.scaladsl.StashBuffer
import akka.annotation.InternalApi
import akka.event.Logging
import akka.util.OptionVal
import akka.util.unused
/**
* INTERNAL API
*/
@ -54,9 +49,6 @@ private abstract class AbstractSupervisor[I, Thr <: Throwable](strategy: Supervi
private val throwableClass = implicitly[ClassTag[Thr]].runtimeClass
protected def isInstanceOfTheThrowableClass(t: Throwable): Boolean =
throwableClass.isAssignableFrom(UnstashException.unwrap(t).getClass)
override def isSame(other: BehaviorInterceptor[Any, Any]): Boolean = {
other match {
case as: AbstractSupervisor[_, Thr] if throwableClass == as.throwableClass => true
@ -77,14 +69,22 @@ private abstract class AbstractSupervisor[I, Thr <: Throwable](strategy: Supervi
}
def log(ctx: TypedActorContext[_], t: Throwable): Unit = {
val msg = s"Supervisor $this saw failure:"
if (strategy.loggingEnabled) {
val unwrapped = UnstashException.unwrap(t)
strategy.logLevel match {
case Logging.ErrorLevel =>
ctx.asScala.log.error(unwrapped, "Supervisor {} saw failure: {}", this, unwrapped.getMessage)
case Logging.WarningLevel =>
ctx.asScala.log.warning(unwrapped, "Supervisor {} saw failure: {}", this, unwrapped.getMessage)
case level => ctx.asScala.log.log(level, "Supervisor {} saw failure: {}", this, unwrapped.getMessage)
case Level.ERROR =>
ctx.asScala.log.error(msg, unwrapped)
case Level.WARN =>
ctx.asScala.log.warn(msg, unwrapped)
case Level.INFO =>
ctx.asScala.log.info(msg, unwrapped)
case Level.DEBUG =>
ctx.asScala.log.debug(msg, unwrapped)
case Level.TRACE =>
ctx.asScala.log.trace(msg, unwrapped)
//TODO check this debug case is actually best option when other level is found
case _ => ctx.asScala.log.debug(msg, unwrapped)
}
}
}
@ -95,11 +95,16 @@ private abstract class AbstractSupervisor[I, Thr <: Throwable](strategy: Supervi
.publish(Dropped(signalOrMessage, s"Stash is full in [${getClass.getSimpleName}]", ctx.asScala.self.toClassic))
}
protected def handleExceptionOnStart(ctx: TypedActorContext[Any], target: PreStartTarget[I]): Catcher[Behavior[I]]
protected def handleSignalException(ctx: TypedActorContext[Any], target: SignalTarget[I]): Catcher[Behavior[I]]
protected def handleReceiveException(ctx: TypedActorContext[Any], target: ReceiveTarget[I]): Catcher[Behavior[I]]
override def toString: String = Logging.simpleName(getClass)
protected def isInstanceOfTheThrowableClass(t: Throwable): Boolean =
throwableClass.isAssignableFrom(UnstashException.unwrap(t).getClass)
protected def handleExceptionOnStart(ctx: TypedActorContext[Any], target: PreStartTarget[I]): Catcher[Behavior[I]]
protected def handleSignalException(ctx: TypedActorContext[Any], target: SignalTarget[I]): Catcher[Behavior[I]]
protected def handleReceiveException(ctx: TypedActorContext[Any], target: ReceiveTarget[I]): Catcher[Behavior[I]]
}
/**
@ -114,6 +119,9 @@ private abstract class SimpleSupervisor[T, Thr <: Throwable: ClassTag](ss: Super
} catch handleReceiveException(ctx, target)
}
protected def handleReceiveException(ctx: TypedActorContext[Any], target: ReceiveTarget[T]): Catcher[Behavior[T]] =
handleException(ctx)
protected def handleException(@unused ctx: TypedActorContext[Any]): Catcher[Behavior[T]] = {
case NonFatal(t) if isInstanceOfTheThrowableClass(t) =>
BehaviorImpl.failed(t)
@ -122,10 +130,9 @@ private abstract class SimpleSupervisor[T, Thr <: Throwable: ClassTag](ss: Super
// convenience if target not required to handle exception
protected def handleExceptionOnStart(ctx: TypedActorContext[Any], target: PreStartTarget[T]): Catcher[Behavior[T]] =
handleException(ctx)
protected def handleSignalException(ctx: TypedActorContext[Any], target: SignalTarget[T]): Catcher[Behavior[T]] =
handleException(ctx)
protected def handleReceiveException(ctx: TypedActorContext[Any], target: ReceiveTarget[T]): Catcher[Behavior[T]] =
handleException(ctx)
}
private class StopSupervisor[T, Thr <: Throwable: ClassTag](@unused initial: Behavior[T], strategy: Stop)
@ -183,11 +190,6 @@ private class RestartSupervisor[T, Thr <: Throwable: ClassTag](initial: Behavior
private var gotScheduledRestart = true
private var deadline: OptionVal[Deadline] = OptionVal.None
private def deadlineHasTimeLeft: Boolean = deadline match {
case OptionVal.None => true
case OptionVal.Some(d) => d.hasTimeLeft
}
override def aroundSignal(ctx: TypedActorContext[Any], signal: Signal, target: SignalTarget[T]): Behavior[T] = {
restartingInProgress match {
case OptionVal.None =>
@ -288,6 +290,7 @@ private class RestartSupervisor[T, Thr <: Throwable: ClassTag](initial: Behavior
case _ => target(ctx, PreRestart)
})
}
override protected def handleReceiveException(
ctx: TypedActorContext[Any],
target: ReceiveTarget[T]): Catcher[Behavior[T]] = {
@ -297,6 +300,11 @@ private class RestartSupervisor[T, Thr <: Throwable: ClassTag](initial: Behavior
})
}
private def deadlineHasTimeLeft: Boolean = deadline match {
case OptionVal.None => true
case OptionVal.Some(d) => d.hasTimeLeft
}
private def handleException(ctx: TypedActorContext[Any], signalRestart: Throwable => Unit): Catcher[Behavior[T]] = {
case NonFatal(t) if isInstanceOfTheThrowableClass(t) =>
ctx.asScala.cancelAllTimers()
@ -311,7 +319,7 @@ private class RestartSupervisor[T, Thr <: Throwable: ClassTag](initial: Behavior
} else {
try signalRestart(t)
catch {
case NonFatal(ex) => ctx.asScala.log.error(ex, "failure during PreRestart")
case NonFatal(ex) => ctx.asScala.log.error("failure during PreRestart", ex)
}
prepareRestart(ctx, t)

View file

@ -8,7 +8,6 @@ package internal
import java.time.Duration
import scala.concurrent.duration.FiniteDuration
import akka.actor.Cancellable
import akka.actor.NotInfluenceReceiveTimeout
import akka.actor.typed.scaladsl.ActorContext
@ -16,6 +15,7 @@ import akka.annotation.InternalApi
import akka.dispatch.ExecutionContexts
import akka.util.JavaDurationConverters._
import akka.util.OptionVal
import org.slf4j.Logger
/**
* INTERNAL API
@ -160,9 +160,7 @@ import akka.util.OptionVal
// it was from an old timer that was enqueued in mailbox before canceled
log.debug(
"Received timer [{}] from old generation [{}], expected generation [{}], discarding",
timerMsg.key,
timerMsg.generation,
t.generation)
Array(timerMsg.key, timerMsg.generation, t.generation))
OptionVal.none // message should be ignored
}
}

View file

@ -4,9 +4,11 @@
package akka.actor.typed.internal
import akka.actor.typed.internal.adapter.AbstractLogger
import akka.actor.typed.{ Behavior, BehaviorInterceptor, Signal, TypedActorContext }
import akka.annotation.InternalApi
import org.slf4j.MDC
import scala.collection.JavaConverters._
import scala.collection.immutable.HashMap
import scala.reflect.ClassTag
@ -14,13 +16,12 @@ import scala.reflect.ClassTag
* INTERNAL API
*/
@InternalApi private[akka] object WithMdcBehaviorInterceptor {
val noMdcPerMessage = (_: Any) => Map.empty[String, Any]
val noMdcPerMessage = (_: Any) => Map.empty[String, String]
def apply[T: ClassTag](
staticMdc: Map[String, Any],
mdcForMessage: T => Map[String, Any],
staticMdc: Map[String, String],
mdcForMessage: T => Map[String, String],
behavior: Behavior[T]): Behavior[T] = {
BehaviorImpl.intercept(() => new WithMdcBehaviorInterceptor[T](staticMdc, mdcForMessage))(behavior)
}
@ -32,8 +33,8 @@ import scala.reflect.ClassTag
* INTERNAL API
*/
@InternalApi private[akka] final class WithMdcBehaviorInterceptor[T: ClassTag] private (
staticMdc: Map[String, Any],
mdcForMessage: T => Map[String, Any])
staticMdc: Map[String, String],
mdcForMessage: T => Map[String, String])
extends BehaviorInterceptor[T, T] {
import BehaviorInterceptor._
@ -74,31 +75,31 @@ import scala.reflect.ClassTag
override def aroundReceive(ctx: TypedActorContext[T], msg: T, target: ReceiveTarget[T]): Behavior[T] = {
val mdc = merge(staticMdc, mdcForMessage(msg))
ctx.asScala.log.asInstanceOf[AbstractLogger].mdc = mdc
MDC.getMDCAdapter.setContextMap(mdc.asJava)
val next =
try {
target(ctx, msg)
} finally {
ctx.asScala.log.asInstanceOf[AbstractLogger].mdc = Map.empty
MDC.clear()
}
next
}
override def aroundSignal(ctx: TypedActorContext[T], signal: Signal, target: SignalTarget[T]): Behavior[T] = {
ctx.asScala.log.asInstanceOf[AbstractLogger].mdc = staticMdc
MDC.getMDCAdapter.setContextMap(staticMdc.asJava)
try {
target(ctx, signal)
} finally {
ctx.asScala.log.asInstanceOf[AbstractLogger].mdc = Map.empty
MDC.clear()
}
}
private def merge(staticMdc: Map[String, Any], mdcForMessage: Map[String, Any]): Map[String, Any] = {
private def merge(staticMdc: Map[String, String], mdcForMessage: Map[String, String]): Map[String, String] = {
if (staticMdc.isEmpty) mdcForMessage
else if (mdcForMessage.isEmpty) staticMdc
else if (staticMdc.isInstanceOf[HashMap[String, Any]] && mdcForMessage.isInstanceOf[HashMap[String, Any]]) {
else if (staticMdc.isInstanceOf[HashMap[String, String]] && mdcForMessage.isInstanceOf[HashMap[String, String]]) {
// merged is more efficient than ++
mdcForMessage.asInstanceOf[HashMap[String, Any]].merged(staticMdc.asInstanceOf[HashMap[String, Any]])(null)
mdcForMessage.asInstanceOf[HashMap[String, String]].merged(staticMdc.asInstanceOf[HashMap[String, String]])(null)
} else {
staticMdc ++ mdcForMessage
}

View file

@ -6,10 +6,9 @@ package akka.actor.typed
package internal
package adapter
import akka.actor.ExtendedActorSystem
import akka.annotation.InternalApi
import akka.event.LoggingFilterWithMarker
import akka.util.OptionVal
import org.slf4j.{ Logger, LoggerFactory }
import akka.{ actor => classic }
import scala.concurrent.ExecutionContextExecutor
@ -121,11 +120,10 @@ private[akka] object ActorContextAdapter {
ActorRefAdapter[U](ref)
}
private def initLoggerWithClass(logClass: Class[_]): LoggerAdapterImpl = {
val logSource = self.path.toString
val system = classicContext.system.asInstanceOf[ExtendedActorSystem]
val logger =
new LoggerAdapterImpl(system.eventStream, logClass, logSource, LoggingFilterWithMarker.wrap(system.logFilter))
private def initLoggerWithClass(logClass: Class[_]): Logger = {
val logger = LoggerFactory.getLogger(logClass)
//TODO remove it from MDC. Under discussion
org.slf4j.MDC.put("actorPath", self.path.name)
actorLogger = OptionVal.Some(logger)
logger
}

View file

@ -19,7 +19,6 @@ import akka.actor.typed.ActorSystem
import akka.actor.typed.Behavior
import akka.actor.typed.DispatcherSelector
import akka.actor.typed.Dispatchers
import akka.actor.typed.Logger
import akka.actor.typed.Props
import akka.actor.typed.Scheduler
import akka.actor.typed.Settings
@ -31,8 +30,8 @@ import akka.actor.typed.internal.PropsImpl.DispatcherFromConfig
import akka.actor.typed.internal.PropsImpl.DispatcherSameAsParent
import akka.actor.typed.internal.SystemMessage
import akka.annotation.InternalApi
import akka.event.LoggingFilterWithMarker
import akka.{ actor => classic }
import org.slf4j.{ Logger, LoggerFactory }
/**
* INTERNAL API. Lightweight wrapper for presenting a classic ActorSystem to a Behavior (via the context).
@ -88,19 +87,15 @@ import akka.{ actor => classic }
override def shutdown(): Unit = () // there was no shutdown in classic Akka
}
override def dynamicAccess: classic.DynamicAccess = system.dynamicAccess
implicit override def executionContext: scala.concurrent.ExecutionContextExecutor = system.dispatcher
override val log: Logger = new LoggerAdapterImpl(
system.eventStream,
classOf[ActorSystem[_]],
name,
LoggingFilterWithMarker.wrap(system.logFilter))
override def logConfiguration(): Unit = system.logConfiguration()
override def name: String = system.name
override val scheduler: Scheduler = new SchedulerAdapter(system.scheduler)
override def settings: Settings = new Settings(system.settings)
override def startTime: Long = system.startTime
implicit override def executionContext: scala.concurrent.ExecutionContextExecutor = classicSystem.dispatcher
override val log: Logger = LoggerFactory.getLogger(classOf[ActorSystem[_]])
override def logConfiguration(): Unit = classicSystem.logConfiguration()
override def name: String = classicSystem.name
override val scheduler: Scheduler = new SchedulerAdapter(classicSystem.scheduler)
override def settings: Settings = new Settings(classicSystem.settings)
override def startTime: Long = classicSystem.startTime
override def threadFactory: java.util.concurrent.ThreadFactory = system.threadFactory
override def uptime: Long = system.uptime
override def uptime: Long = classicSystem.uptime
override def printTree: String = system.printTree
import akka.dispatch.ExecutionContexts.sameThreadExecutionContext

View file

@ -1,469 +0,0 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.typed.internal.adapter
import akka.actor.typed.{ LogMarker, Logger }
import akka.annotation.InternalApi
import akka.event.Logging._
import akka.event.{ LoggingBus, LoggingFilterWithMarker, LogMarker => ClassicLM }
import akka.util.OptionVal
import akka.util.ccompat.JavaConverters._
/**
* INTERNAL API
*/
private[akka] abstract class AbstractLogger extends Logger {
// actual log entry emitting methods
private[akka] def notifyError(message: String, cause: OptionVal[Throwable], marker: OptionVal[LogMarker]): Unit
private[akka] def notifyWarning(message: String, cause: OptionVal[Throwable], marker: OptionVal[LogMarker]): Unit
private[akka] def notifyInfo(message: String, marker: OptionVal[LogMarker]): Unit
private[akka] def notifyDebug(message: String, marker: OptionVal[LogMarker]): Unit
// is set directly by Behaviors.withMdc
private[akka] var mdc: Map[String, Any] = Map.empty
// user api implementations
override def withMdc(mdc: java.util.Map[String, Any]): Logger =
withMdc(mdc.asScala.toMap)
override def error(message: String): Unit = {
if (isErrorEnabled) notifyError(message, OptionVal.None, OptionVal.None)
}
override def error(template: String, arg1: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1), OptionVal.None, OptionVal.None)
}
override def error(template: String, arg1: Any, arg2: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2), OptionVal.None, OptionVal.None)
}
override def error(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3), OptionVal.None, OptionVal.None)
}
override def error(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3, arg4), OptionVal.None, OptionVal.None)
}
override def error(cause: Throwable, message: String): Unit = {
if (isErrorEnabled) notifyError(message, OptionVal.Some(cause), OptionVal.None)
}
override def error(cause: Throwable, template: String, arg1: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1), OptionVal.Some(cause), OptionVal.None)
}
override def error(cause: Throwable, template: String, arg1: Any, arg2: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2), OptionVal.Some(cause), OptionVal.None)
}
override def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3), OptionVal.Some(cause), OptionVal.None)
}
override def error(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3, arg4), OptionVal.Some(cause), OptionVal.None)
}
override def error(marker: LogMarker, cause: Throwable, message: String): Unit = {
if (isErrorEnabled) notifyError(message, OptionVal.Some(cause), OptionVal.Some(marker))
}
override def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1), OptionVal.Some(cause), OptionVal.Some(marker))
}
override def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2), OptionVal.Some(cause), OptionVal.Some(marker))
}
override def error(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3), OptionVal.Some(cause), OptionVal.Some(marker))
}
override def error(
marker: LogMarker,
cause: Throwable,
template: String,
arg1: Any,
arg2: Any,
arg3: Any,
arg4: Any): Unit = {
if (isErrorEnabled)
notifyError(format(template, arg1, arg2, arg3, arg4), OptionVal.Some(cause), OptionVal.Some(marker))
}
override def error(marker: LogMarker, message: String): Unit = {
if (isErrorEnabled) notifyError(message, OptionVal.None, OptionVal.Some(marker))
}
override def error(marker: LogMarker, template: String, arg1: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1), OptionVal.None, OptionVal.Some(marker))
}
override def error(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2), OptionVal.None, OptionVal.Some(marker))
}
override def error(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3), OptionVal.None, OptionVal.Some(marker))
}
override def error(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isErrorEnabled) notifyError(format(template, arg1, arg2, arg3, arg4), OptionVal.None, OptionVal.Some(marker))
}
override def warning(message: String): Unit = {
if (isWarningEnabled) notifyWarning(message, OptionVal.None, OptionVal.None)
}
override def warning(template: String, arg1: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1), OptionVal.None, OptionVal.None)
}
override def warning(template: String, arg1: Any, arg2: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1, arg2), OptionVal.None, OptionVal.None)
}
override def warning(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3), OptionVal.None, OptionVal.None)
}
override def warning(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3, arg4), OptionVal.None, OptionVal.None)
}
override def warning(cause: Throwable, message: String): Unit = {
if (isWarningEnabled) notifyWarning(message, OptionVal.Some(cause), OptionVal.None)
}
override def warning(cause: Throwable, template: String, arg1: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1), OptionVal.Some(cause), OptionVal.None)
}
override def warning(cause: Throwable, template: String, arg1: Any, arg2: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1, arg2), OptionVal.Some(cause), OptionVal.None)
}
override def warning(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3), OptionVal.Some(cause), OptionVal.None)
}
override def warning(cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3, arg4), OptionVal.Some(cause), OptionVal.None)
}
override def warning(marker: LogMarker, cause: Throwable, template: String, arg1: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1), OptionVal.Some(cause), OptionVal.Some(marker))
}
override def warning(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1, arg2), OptionVal.Some(cause), OptionVal.Some(marker))
}
override def warning(marker: LogMarker, cause: Throwable, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isWarningEnabled)
notifyWarning(format(template, arg1, arg2, arg3), OptionVal.Some(cause), OptionVal.Some(marker))
}
override def warning(
marker: LogMarker,
cause: Throwable,
template: String,
arg1: Any,
arg2: Any,
arg3: Any,
arg4: Any): Unit = {
if (isWarningEnabled)
notifyWarning(format(template, arg1, arg2, arg3, arg4), OptionVal.Some(cause), OptionVal.Some(marker))
}
override def warning(marker: LogMarker, cause: Throwable, message: String): Unit = {
if (isWarningEnabled) notifyWarning(message, OptionVal.Some(cause), OptionVal.Some(marker))
}
override def warning(marker: LogMarker, message: String): Unit = {
if (isWarningEnabled) notifyWarning(message, OptionVal.None, OptionVal.Some(marker))
}
override def warning(marker: LogMarker, template: String, arg1: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1), OptionVal.None, OptionVal.Some(marker))
}
override def warning(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1, arg2), OptionVal.None, OptionVal.Some(marker))
}
override def warning(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isWarningEnabled) notifyWarning(format(template, arg1, arg2, arg3), OptionVal.None, OptionVal.Some(marker))
}
override def warning(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isWarningEnabled)
notifyWarning(format(template, arg1, arg2, arg3, arg4), OptionVal.None, OptionVal.Some(marker))
}
override def info(message: String): Unit = {
if (isInfoEnabled) notifyInfo(message, OptionVal.None)
}
override def info(template: String, arg1: Any): Unit = {
if (isInfoEnabled) notifyInfo(format(template, arg1), OptionVal.None)
}
override def info(template: String, arg1: Any, arg2: Any): Unit = {
if (isInfoEnabled) notifyInfo(format(template, arg1, arg2), OptionVal.None)
}
override def info(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3), OptionVal.None)
}
override def info(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3, arg4), OptionVal.None)
}
override def info(marker: LogMarker, message: String): Unit = {
if (isInfoEnabled) notifyInfo(message, OptionVal.Some(marker))
}
override def info(marker: LogMarker, template: String, arg1: Any): Unit = {
if (isInfoEnabled) notifyInfo(format(template, arg1), OptionVal.Some(marker))
}
override def info(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit = {
if (isInfoEnabled) notifyInfo(format(template, arg1, arg2), OptionVal.Some(marker))
}
override def info(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3), OptionVal.Some(marker))
}
override def info(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isInfoEnabled) notifyInfo(format(template, arg1, arg2, arg3, arg4), OptionVal.Some(marker))
}
override def debug(message: String): Unit = {
if (isDebugEnabled) notifyDebug(message, OptionVal.None)
}
override def debug(template: String, arg1: Any): Unit = {
if (isDebugEnabled) notifyDebug(format(template, arg1), OptionVal.None)
}
override def debug(template: String, arg1: Any, arg2: Any): Unit = {
if (isDebugEnabled) notifyDebug(format(template, arg1, arg2), OptionVal.None)
}
override def debug(template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3), OptionVal.None)
}
override def debug(template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3, arg4), OptionVal.None)
}
override def debug(marker: LogMarker, message: String): Unit = {
if (isDebugEnabled) notifyDebug(message, OptionVal.Some(marker))
}
override def debug(marker: LogMarker, template: String, arg1: Any): Unit = {
if (isDebugEnabled) notifyDebug(format(template, arg1), OptionVal.Some(marker))
}
override def debug(marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit = {
if (isDebugEnabled) notifyDebug(format(template, arg1, arg2), OptionVal.Some(marker))
}
override def debug(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3), OptionVal.Some(marker))
}
override def debug(marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isDebugEnabled) notifyDebug(format(template, arg1, arg2, arg3, arg4), OptionVal.Some(marker))
}
override def log(level: LogLevel, message: String): Unit = {
if (isLevelEnabled(level)) notify(level, message, OptionVal.None)
}
override def log(level: LogLevel, template: String, arg1: Any): Unit = {
if (isLevelEnabled(level)) notify(level, format(template, arg1), OptionVal.None)
}
override def log(level: LogLevel, template: String, arg1: Any, arg2: Any): Unit = {
if (isLevelEnabled(level)) notify(level, format(template, arg1, arg2), OptionVal.None)
}
override def log(level: LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isLevelEnabled(level)) notify(level, format(template, arg1, arg2, arg3), OptionVal.None)
}
override def log(level: LogLevel, template: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): Unit = {
if (isLevelEnabled(level)) notify(level, format(template, arg1, arg2, arg3, arg4), OptionVal.None)
}
override def log(level: LogLevel, marker: LogMarker, message: String): Unit = {
if (isLevelEnabled(level)) notify(level, message, OptionVal.Some(marker))
}
override def log(level: LogLevel, marker: LogMarker, template: String, arg1: Any): Unit = {
if (isLevelEnabled(level)) notify(level, format(template, arg1), OptionVal.Some(marker))
}
override def log(level: LogLevel, marker: LogMarker, template: String, arg1: Any, arg2: Any): Unit = {
if (isLevelEnabled(level)) notify(level, format(template, arg1, arg2), OptionVal.Some(marker))
}
override def log(level: LogLevel, marker: LogMarker, template: String, arg1: Any, arg2: Any, arg3: Any): Unit = {
if (isLevelEnabled(level)) notify(level, format(template, arg1, arg2, arg3), OptionVal.Some(marker))
}
override def log(
level: LogLevel,
marker: LogMarker,
template: String,
arg1: Any,
arg2: Any,
arg3: Any,
arg4: Any): Unit = {
if (isLevelEnabled(level)) notify(level, format(template, arg1, arg2, arg3, arg4), OptionVal.Some(marker))
}
protected def notify(level: LogLevel, message: String, marker: OptionVal[LogMarker]): Unit = level match {
case ErrorLevel => notifyDebug(message, marker)
case WarningLevel => notifyWarning(message, OptionVal.None, marker)
case InfoLevel => notifyInfo(message, marker)
case DebugLevel => notifyDebug(message, marker)
case _ => ()
}
/**
* If `arg` is an `Array` it will be expanded into replacement arguments, which is useful when
* there are more than four arguments.
*/
private def format(t: String, arg1: Any): String = arg1 match {
case a: Array[_] if !a.getClass.getComponentType.isPrimitive => formatArrayImpl(t, a.toSeq)
case a: Array[_] => formatArrayImpl(t, a.map(_.asInstanceOf[AnyRef]).toSeq)
case x => formatArray(t, x)
}
private def format(t: String, arg1: Any, arg2: Any): String = formatArray(t, arg1, arg2)
private def format(t: String, arg1: Any, arg2: Any, arg3: Any): String = formatArray(t, arg1, arg2, arg3)
private def format(t: String, arg1: Any, arg2: Any, arg3: Any, arg4: Any): String =
formatArray(t, arg1, arg2, arg3, arg4)
private def formatArray(t: String, arg: Any*): String =
formatArrayImpl(t, arg)
private def formatArrayImpl(t: String, arg: Seq[Any]): String = {
val sb = new java.lang.StringBuilder(64)
var p = 0
var startIndex = 0
while (p < arg.length) {
val index = t.indexOf("{}", startIndex)
if (index == -1) {
sb.append(t.substring(startIndex, t.length)).append(" WARNING arguments left: ").append(arg.length - p)
p = arg.length
startIndex = t.length
} else {
sb.append(t.substring(startIndex, index)).append(arg(p))
startIndex = index + 2
p += 1
}
}
sb.append(t.substring(startIndex, t.length)).toString
}
}
/**
* INTERNAL API
*/
@InternalApi
private[akka] final class LoggerAdapterImpl(
bus: LoggingBus,
logClass: Class[_],
logSource: String,
loggingFilter: LoggingFilterWithMarker)
extends AbstractLogger {
override def isErrorEnabled = loggingFilter.isErrorEnabled(logClass, logSource)
override def isWarningEnabled = loggingFilter.isWarningEnabled(logClass, logSource)
override def isInfoEnabled = loggingFilter.isInfoEnabled(logClass, logSource)
override def isDebugEnabled = loggingFilter.isDebugEnabled(logClass, logSource)
override def isErrorEnabled(marker: LogMarker): Boolean =
loggingFilter.isErrorEnabled(logClass, logSource, marker.asInstanceOf[ClassicLM])
override def isWarningEnabled(marker: LogMarker): Boolean =
loggingFilter.isWarningEnabled(logClass, logSource, marker.asInstanceOf[ClassicLM])
override def isInfoEnabled(marker: LogMarker): Boolean =
loggingFilter.isInfoEnabled(logClass, logSource, marker.asInstanceOf[ClassicLM])
override def isDebugEnabled(marker: LogMarker): Boolean =
loggingFilter.isDebugEnabled(logClass, logSource, marker.asInstanceOf[ClassicLM])
override def withMdc(mdc: Map[String, Any]): Logger = {
val mdcAdapter = new LoggerAdapterImpl(bus, logClass, logSource, loggingFilter)
mdcAdapter.mdc = mdc
mdcAdapter
}
def withLoggerClass(clazz: Class[_]): Logger = {
val withClass = new LoggerAdapterImpl(bus, clazz, logSource, loggingFilter)
withClass.mdc = mdc
withClass
}
def withLogSource(logSource: String): Logger = {
val withSource = new LoggerAdapterImpl(bus, logClass, logSource, loggingFilter)
withSource.mdc = mdc
withSource
}
private[akka] def notifyError(message: String, cause: OptionVal[Throwable], marker: OptionVal[LogMarker]): Unit = {
val error = cause match {
case OptionVal.Some(cause) =>
marker match {
case OptionVal.Some(m) => Error(cause, logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Error(cause, logSource, logClass, message, mdc)
}
case OptionVal.None =>
marker match {
case OptionVal.Some(m) => Error(logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Error(logSource, logClass, message, mdc)
}
}
bus.publish(error)
}
private[akka] def notifyWarning(message: String, cause: OptionVal[Throwable], marker: OptionVal[LogMarker]): Unit = {
val warning =
if (cause.isDefined) Warning(cause.get, logSource, logClass, message, mdc, marker.orNull.asInstanceOf[ClassicLM])
else
marker match {
case OptionVal.Some(m) => Warning(logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Warning(logSource, logClass, message, mdc)
}
bus.publish(warning)
}
private[akka] def notifyInfo(message: String, marker: OptionVal[LogMarker]): Unit = {
val info = marker match {
case OptionVal.Some(m) => Info(logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Info(logSource, logClass, message, mdc)
}
bus.publish(info)
}
private[akka] def notifyDebug(message: String, marker: OptionVal[LogMarker]): Unit = {
val debug = marker match {
case OptionVal.Some(m) => Debug(logSource, logClass, message, mdc, m.asInstanceOf[ClassicLM])
case OptionVal.None => Debug(logSource, logClass, message, mdc)
}
bus.publish(debug)
}
}

View file

@ -98,7 +98,7 @@ private[akka] object LocalReceptionist extends ReceptionistBehaviorProvider {
def onCommand(ctx: ActorContext[Any], cmd: Command): Behavior[Any] = cmd match {
case ReceptionistMessages.Register(key, serviceInstance, maybeReplyTo) =>
ctx.log.debug("Actor was registered: {} {}", key, serviceInstance)
ctx.log.debug("Actor was registered: {} {}", Array(key, serviceInstance))
watchWith(ctx, serviceInstance, RegisteredActorTerminated(key, serviceInstance))
maybeReplyTo match {
case Some(replyTo) => replyTo ! ReceptionistMessages.Registered(key, serviceInstance)
@ -121,7 +121,7 @@ private[akka] object LocalReceptionist extends ReceptionistBehaviorProvider {
def onInternal(ctx: ActorContext[Any], cmd: InternalCommand): Behavior[Any] = cmd match {
case RegisteredActorTerminated(key, serviceInstance) =>
ctx.log.debug("Registered actor terminated: {} {}", key, serviceInstance)
ctx.log.debug("Registered actor terminated: {} {}", Array(key, serviceInstance))
updateRegistry(Set(key), _.removed(key)(serviceInstance))
case SubscriberTerminated(key, subscriber) =>

View file

@ -8,11 +8,12 @@ import java.time.Duration
import java.util.function.{ BiFunction, Function => JFunction }
import akka.annotation.DoNotInherit
import akka.actor.ClassicActorContextProvider
import akka.actor.typed._
import java.util.Optional
import java.util.concurrent.CompletionStage
import akka.actor.ClassicActorContextProvider
import org.slf4j.Logger
import scala.concurrent.ExecutionContextExecutor

View file

@ -325,13 +325,12 @@ object Behaviors {
* @param behavior The actual behavior handling the messages, the MDC is used for the log entries logged through
* `ActorContext.log`
*
* See also [[akka.actor.typed.Logger.withMdc]]
*/
def withMdc[T](
interceptMessageClass: Class[T],
mdcForMessage: akka.japi.function.Function[T, java.util.Map[String, Any]],
mdcForMessage: akka.japi.function.Function[T, java.util.Map[String, String]],
behavior: Behavior[T]): Behavior[T] =
withMdc(interceptMessageClass, Collections.emptyMap[String, Any], mdcForMessage, behavior)
withMdc(interceptMessageClass, Collections.emptyMap[String, String], mdcForMessage, behavior)
/**
* Static MDC (Mapped Diagnostic Context)
@ -343,11 +342,10 @@ object Behaviors {
* @param behavior The actual behavior handling the messages, the MDC is used for the log entries logged through
* `ActorContext.log`
*
* See also [[akka.actor.typed.Logger.withMdc]]
*/
def withMdc[T](
interceptMessageClass: Class[T],
staticMdc: java.util.Map[String, Any],
staticMdc: java.util.Map[String, String],
behavior: Behavior[T]): Behavior[T] =
withMdc(interceptMessageClass, staticMdc, null, behavior)
@ -369,20 +367,19 @@ object Behaviors {
* @param behavior The actual behavior handling the messages, the MDC is used for the log entries logged through
* `ActorContext.log`
*
* See also [[akka.actor.typed.Logger.withMdc]]
*/
def withMdc[T](
interceptMessageClass: Class[T],
staticMdc: java.util.Map[String, Any],
mdcForMessage: akka.japi.function.Function[T, java.util.Map[String, Any]],
staticMdc: java.util.Map[String, String],
mdcForMessage: akka.japi.function.Function[T, java.util.Map[String, String]],
behavior: Behavior[T]): Behavior[T] = {
def asScalaMap(m: java.util.Map[String, Any]): Map[String, Any] = {
if (m == null || m.isEmpty) Map.empty[String, Any]
def asScalaMap(m: java.util.Map[String, String]): Map[String, String] = {
if (m == null || m.isEmpty) Map.empty[String, String]
else m.asScala.toMap
}
val mdcForMessageFun: T => Map[String, Any] =
val mdcForMessageFun: T => Map[String, String] =
if (mdcForMessage == null) Map.empty
else { message =>
asScalaMap(mdcForMessage.apply(message))

View file

@ -14,6 +14,7 @@ import scala.concurrent.duration.FiniteDuration
import scala.reflect.ClassTag
import scala.util.Try
import akka.annotation.InternalApi
import org.slf4j.Logger
/**
* An Actor is given by the combination of a [[Behavior]] and a context in

View file

@ -252,10 +252,9 @@ object Behaviors {
* @param behavior The actual behavior handling the messages, the MDC is used for the log entries logged through
* `ActorContext.log`
*
* See also [[akka.actor.typed.Logger.withMdc]]
*/
def withMdc[T: ClassTag](mdcForMessage: T => Map[String, Any])(behavior: Behavior[T]): Behavior[T] =
withMdc[T](Map.empty[String, Any], mdcForMessage)(behavior)
def withMdc[T: ClassTag](mdcForMessage: T => Map[String, String])(behavior: Behavior[T]): Behavior[T] =
withMdc[T](Map.empty[String, String], mdcForMessage)(behavior)
/**
* Static MDC (Mapped Diagnostic Context)
@ -268,10 +267,9 @@ object Behaviors {
* @param behavior The actual behavior handling the messages, the MDC is used for the log entries logged through
* `ActorContext.log`
*
* See also [[akka.actor.typed.Logger.withMdc]]
*/
def withMdc[T: ClassTag](staticMdc: Map[String, Any])(behavior: Behavior[T]): Behavior[T] =
withMdc[T](staticMdc, (_: T) => Map.empty[String, Any])(behavior)
def withMdc[T: ClassTag](staticMdc: Map[String, String])(behavior: Behavior[T]): Behavior[T] =
withMdc[T](staticMdc, (_: T) => Map.empty[String, String])(behavior)
/**
* Combination of static and per message MDC (Mapped Diagnostic Context).
@ -292,9 +290,8 @@ object Behaviors {
* @param behavior The actual behavior handling the messages, the MDC is used for the log entries logged through
* `ActorContext.log`
*
* See also [[akka.actor.typed.Logger.withMdc]]
*/
def withMdc[T: ClassTag](staticMdc: Map[String, Any], mdcForMessage: T => Map[String, Any])(
def withMdc[T: ClassTag](staticMdc: Map[String, String], mdcForMessage: T => Map[String, String])(
behavior: Behavior[T]): Behavior[T] =
WithMdcBehaviorInterceptor[T](staticMdc, mdcForMessage, behavior)

View file

@ -426,6 +426,7 @@ lazy val testkit = akkaModule("akka-testkit")
lazy val actorTyped = akkaModule("akka-actor-typed")
.dependsOn(actor)
.settings(AutomaticModuleName.settings("akka.actor.typed")) // fine for now, eventually new module name to become typed.actor
.settings(Dependencies.actorTyped)
.settings(OSGi.actorTyped)
.settings(initialCommands :=
"""
@ -495,7 +496,9 @@ lazy val streamTyped = akkaModule("akka-stream-typed")
.enablePlugins(ScaladocNoVerificationOfDiagrams)
lazy val actorTestkitTyped = akkaModule("akka-actor-testkit-typed")
.dependsOn(actorTyped, testkit % "compile->compile;test->test")
.dependsOn(actorTyped,
slf4j,
testkit % "compile->compile;test->test")
.settings(AutomaticModuleName.settings("akka.actor.testkit.typed"))
.settings(Dependencies.actorTestkitTyped)

View file

@ -15,7 +15,7 @@ object Dependencies {
lazy val java8CompatVersion = settingKey[String]("The version of scala-java8-compat to use.")
val junitVersion = "4.12"
val slf4jVersion = "1.7.25"
val slf4jVersion = "1.7.27"
// check agrona version when updating this
val aeronVersion = "1.19.1"
// needs to be inline with the aeron version
@ -164,11 +164,13 @@ object Dependencies {
val actor = l ++= Seq(config, java8Compat.value)
val actorTyped = l ++= Seq(slf4jApi)
val discovery = l ++= Seq(Test.junit, Test.scalatest.value)
val coordination = l ++= Seq(Test.junit, Test.scalatest.value)
val testkit = l ++= Seq(Test.junit, Test.scalatest.value) ++ Test.metricsAll
val testkit = l ++= Seq(Test.logback, Test.junit, Test.scalatest.value) ++ Test.metricsAll
val actorTests = l ++= Seq(
Test.junit,