First step of changing to SL4J in Typed logging, #26537
This commit is contained in:
parent
1d38f1579a
commit
dd96d21ffa
38 changed files with 1185 additions and 1655 deletions
|
|
@ -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")
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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()
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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 {
|
||||
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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;
|
||||
},
|
||||
|
|
|
|||
27
akka-actor-typed-tests/src/test/resources/logback-test.xml
Normal file
27
akka-actor-typed-tests/src/test/resources/logback-test.xml
Normal 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>
|
||||
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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 run’s 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 run’s console output clean and do not miss real
|
||||
* error messages.
|
||||
*
|
||||
* '''Also have a look at the `akka.testkit` package object’s `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 event’s origin</li>
|
||||
* <li><code>message</code> applies a filter on the event’s message (either
|
||||
* with String.startsWith or Regex.findFirstIn().isDefined); if the message
|
||||
* itself does not match, the match is retried with the contained Exception’s
|
||||
* message; if both are <code>null</code>, the filter always matches if at
|
||||
* the same time the Exception’s 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 event’s 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 event’s origin</li>
|
||||
* <li><code>message</code> applies a filter on the event’s 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 event’s 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 event’s origin</li>
|
||||
* <li><code>message</code> applies a filter on the event’s 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 event’s 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 event’s origin</li>
|
||||
* <li><code>message</code> applies a filter on the event’s 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 event’s 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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
@ -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
|
||||
|
||||
}
|
||||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -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 =>
|
||||
|
|
|
|||
|
|
@ -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 Actor’s [[TypedActorContext]]
|
||||
* as that ties the log entries to the actor.
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
}
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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) =>
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
||||
|
|
|
|||
|
|
@ -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))
|
||||
|
|
|
|||
|
|
@ -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
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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)
|
||||
|
||||
|
|
|
|||
|
|
@ -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,
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue