Logger class for typed actors #25677

* Walk stack to figure out logger class
* User API for getting a customized logger
* API to set logger class, use in receptionists, auto-good-class for EventSourcedBehavior
This commit is contained in:
Johan Andrén 2019-02-11 17:18:17 +01:00 committed by GitHub
parent 0c2ad78042
commit 84e8e41193
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
15 changed files with 290 additions and 22 deletions

View file

@ -84,6 +84,10 @@ private[akka] final class FunctionRef[-T](
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 {
@ -121,6 +125,9 @@ private[akka] final class FunctionRef[-T](
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
}
/**
@ -244,6 +251,8 @@ private[akka] final class FunctionRef[-T](
override def log: Logger = loggingAdapter
override def setLoggerClass(clazz: Class[_]): Unit = () // nop as we dont track logger class
/**
* The log entries logged through context.log.{debug, info, warn, error} are captured and can be inspected through
* this method.

View file

@ -4,12 +4,12 @@
package akka.actor.typed.javadsl;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import akka.actor.typed.ActorRef;
import akka.actor.typed.Behavior;
import akka.event.Logging;
import akka.japi.pf.PFBuilder;
import akka.testkit.CustomEventFilter;
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
import com.typesafe.config.ConfigFactory;
import org.junit.ClassRule;
import org.junit.Test;
@ -44,6 +44,28 @@ public class ActorLoggingTest extends JUnitSuite {
}
}
@Test
public void loggingProvidesClassWhereLogWasCalled() {
CustomEventFilter eventFilter =
new CustomEventFilter(
new PFBuilder<Logging.LogEvent, Object>()
.match(
Logging.LogEvent.class, (event) -> event.logClass() == ActorLoggingTest.class)
.build(),
1);
Behavior<String> behavior =
Behaviors.setup(
(context) -> {
context.getLog().info("Starting up");
return Behaviors.empty();
});
testKit.spawn(behavior);
eventFilter.awaitDone(FiniteDuration.create(3, TimeUnit.SECONDS));
}
@Test
public void loggingProvidesMDC() {
Behavior<Protocol> behavior =

View file

@ -11,10 +11,31 @@ import akka.actor.testkit.typed.TestException
import akka.actor.typed.scaladsl.adapter._
import akka.actor.typed.{ Behavior, LogMarker }
import akka.event.Logging
import akka.event.Logging.{ LogEventWithCause, LogEventWithMarker }
import akka.event.Logging.{ LogEvent, LogEventWithCause, LogEventWithMarker }
import akka.testkit.EventFilter
import org.scalatest.WordSpecLike
class SomeClass
object WhereTheBehaviorIsDefined {
def behavior: Behavior[String] = Behaviors.setup { context
context.log.info("Starting up")
Behaviors.stopped
}
}
object BehaviorWhereTheLoggerIsUsed {
def behavior: Behavior[String] = Behaviors.setup(ctx new BehaviorWhereTheLoggerIsUsed(ctx))
}
class BehaviorWhereTheLoggerIsUsed(context: ActorContext[String]) extends AbstractBehavior[String] {
context.log.info("Starting up")
override def onMessage(msg: String): Behavior[String] = {
Behaviors.same
}
}
class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
akka.loglevel = DEBUG # test verifies debug
akka.loggers = ["akka.testkit.TestEventListener"]
@ -44,6 +65,72 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
}
}
"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)
false
}, occurrences = 1)
eventFilter.intercept {
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")
}
}
"contain the object class name where the first log was called" in {
val eventFilter = EventFilter.custom({
case l: LogEvent if l.logClass == WhereTheBehaviorIsDefined.getClass true
case l: LogEvent
println(l.logClass)
false
}, occurrences = 1)
eventFilter.intercept {
spawn(WhereTheBehaviorIsDefined.behavior, "the-actor-with-object")
}
}
"contain the abstract behavior class name where the first log was called" in {
val eventFilter = EventFilter.custom({
case l: LogEvent if l.logClass == classOf[BehaviorWhereTheLoggerIsUsed] true
case l: LogEvent
println(l.logClass)
false
}, occurrences = 1)
eventFilter.intercept {
spawn(BehaviorWhereTheLoggerIsUsed.behavior, "the-actor-with-behavior")
}
}
"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
}, 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")
Behaviors.empty
}, "the-actor-with-custom-class")
}
}
"pass markers to the log" in {
EventFilter.custom({
case event: LogEventWithMarker if event.marker == marker true
@ -204,7 +291,7 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
) {
Behaviors.setup { context
context.log.info("Starting")
Behaviors.receiveMessage { message
Behaviors.receiveMessage { _
context.log.info("Got message!")
Behaviors.same
}
@ -348,7 +435,7 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
"provide a withMdc decorator" in {
val behavior = Behaviors.withMdc[Protocol](Map("mdc" -> "outer"))(
Behaviors.setup { context
Behaviors.receiveMessage { message
Behaviors.receiveMessage { _
context.log.withMdc(Map("mdc" -> "inner")).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!")

View file

@ -738,4 +738,14 @@ abstract class Logger private[akka] () {
* See also [[akka.actor.typed.javadsl.Behaviors.withMdc]]
*/
def withMdc(mdc: java.util.Map[String, Any]): Logger
/**
* Return a new logger sharing properties of this logger except the logger class
*/
def withLoggerClass(clazz: Class[_]): Logger
/**
* Return a new logger sharing properties of this logger except the log source
*/
def withLogSource(logSource: String): Logger
}

View file

@ -0,0 +1,46 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.actor.typed.internal
import akka.annotation.InternalApi
import akka.util.OptionVal
import scala.util.control.NonFatal
/**
* INTERNAL API
*/
@InternalApi
private[akka] object LoggerClass {
// just to get access to the class context
private final class TrickySecurityManager extends SecurityManager {
def getClassStack: Array[Class[_]] = getClassContext
}
/**
* Try to extract a logger class from the call stack, if not possible the provided default is used
*/
def detectLoggerClassFromStack(default: Class[_]): Class[_] = {
// TODO use stack walker API when we no longer need to support Java 8
try {
val trace = new TrickySecurityManager().getClassStack
var suitableClass: OptionVal[Class[_]] = OptionVal.None
var idx = 1 // skip this method/class and right away
while (suitableClass.isEmpty && idx < trace.length) {
val clazz = trace(idx)
val name = clazz.getName
if (!name.startsWith("scala.runtime") && !name.startsWith("akka.actor.typed.internal"))
suitableClass = OptionVal.Some(clazz)
idx += 1
}
suitableClass.getOrElse(default)
} catch {
case NonFatal(_) default
}
}
}

View file

@ -83,18 +83,26 @@ import scala.concurrent.duration._
ActorRefAdapter[U](ref)
}
private def initLoggerWithClass(logClass: Class[_]): LoggerAdapterImpl = {
val logSource = self.path.toString
val system = untypedContext.system.asInstanceOf[ExtendedActorSystem]
val logger = new LoggerAdapterImpl(system.eventStream, logClass, logSource, system.logFilter)
actorLogger = OptionVal.Some(logger)
logger
}
override def log: Logger = {
actorLogger match {
case OptionVal.Some(logger) logger
case OptionVal.None
val logSource = self.path.toString
val logClass = classOf[Behavior[_]] // FIXME figure out untyped better class somehow
val system = untypedContext.system.asInstanceOf[ExtendedActorSystem]
val logger = new LoggerAdapterImpl(system.eventStream, logClass, logSource, system.logFilter)
actorLogger = OptionVal.Some(logger)
logger
val logClass = LoggerClass.detectLoggerClassFromStack(classOf[Behavior[_]])
initLoggerWithClass(logClass)
}
}
override def setLoggerClass(clazz: Class[_]): Unit = {
initLoggerWithClass(clazz)
}
}
/**

View file

@ -371,6 +371,18 @@ private[akka] final class LoggerAdapterImpl(bus: LoggingBus, logClass: Class[_],
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)

View file

@ -22,6 +22,11 @@ private[akka] trait ReceptionistBehaviorProvider {
def behavior: Behavior[Command]
}
// just to provide a log class
/** INTERNAL API */
@InternalApi
private[akka] final class LocalReceptionist
/** INTERNAL API */
@InternalApi
private[akka] object LocalReceptionist extends ReceptionistBehaviorProvider {
@ -35,10 +40,13 @@ private[akka] object LocalReceptionist extends ReceptionistBehaviorProvider {
final case class RegisteredActorTerminated[T](key: ServiceKey[T], ref: ActorRef[T]) extends InternalCommand
final case class SubscriberTerminated[T](key: ServiceKey[T], ref: ActorRef[ReceptionistMessages.Listing[T]]) extends InternalCommand
override def behavior: Behavior[Command] = behavior(
TypedMultiMap.empty[AbstractServiceKey, KV],
TypedMultiMap.empty[AbstractServiceKey, SubscriptionsKV]
).narrow[Command]
override def behavior: Behavior[Command] = Behaviors.setup { ctx
ctx.setLoggerClass(classOf[LocalReceptionist])
behavior(
TypedMultiMap.empty[AbstractServiceKey, KV],
TypedMultiMap.empty[AbstractServiceKey, SubscriptionsKV]
).narrow[Command]
}
private def behavior(
serviceRegistry: LocalServiceRegistry,

View file

@ -74,6 +74,15 @@ trait ActorContext[T] extends TypedActorContext[T] {
*/
def getLog: Logger
/**
* Replace the current logger (or initialize a new logger if the logger was not touched before) with one that
* has ghe given class as logging class. Logger source will be actor path.
*
* *Warning*: This method is not thread-safe and must not be accessed from threads other
* than the ordinary actor message processing thread, such as [[java.util.concurrent.CompletionStage]] callbacks.
*/
def setLoggerClass(clazz: Class[_]): Unit
/**
* The list of child Actors created by this Actor during its lifetime that
* are still alive, in no particular order.

View file

@ -65,13 +65,27 @@ trait ActorContext[T] extends TypedActorContext[T] { this: akka.actor.typed.java
def system: ActorSystem[Nothing]
/**
* An actor specific logger
* An actor specific logger.
*
* The logger will have the actor path as `logSource` and will an estimated source class for the actor
* which is calculated when the logger is first used (the logger is lazily created upon first use). If this
* yields the wrong class or another class is preferred this can be achieved through `Logger.withLoggerClass`
* or `setLoggerClass`.
*
* *Warning*: This method is not thread-safe and must not be accessed from threads other
* than the ordinary actor message processing thread, such as [[scala.concurrent.Future]] callbacks.
*/
def log: Logger
/**
* Replace the current logger (or initialize a new logger if the logger was not touched before) with one that
* has ghe given class as logging class. Logger source will be actor path.
*
* *Warning*: This method is not thread-safe and must not be accessed from threads other
* than the ordinary actor message processing thread, such as [[scala.concurrent.Future]] callbacks.
*/
def setLoggerClass(clazz: Class[_]): Unit
/**
* The list of child Actors created by this Actor during its lifetime that
* are still alive, in no particular order.

View file

@ -0,0 +1,30 @@
/*
* Copyright (C) 2009-2019 Lightbend Inc. <https://www.lightbend.com>
*/
package akka.util
import java.util.concurrent.TimeUnit
import org.openjdk.jmh.annotations.{ Benchmark, Measurement, Scope, State }
@State(Scope.Benchmark)
@Measurement(timeUnit = TimeUnit.MICROSECONDS)
class StackBench {
class CustomSecurtyManager extends SecurityManager {
def getTrace: Array[Class[_]] =
getClassContext
}
@Benchmark
def currentThread(): Array[StackTraceElement] = {
Thread.currentThread().getStackTrace
}
@Benchmark
def securityManager(): Array[Class[_]] = {
(new CustomSecurtyManager).getTrace
}
}

View file

@ -21,6 +21,11 @@ import akka.util.TypedMultiMap
import scala.concurrent.duration._
// just to provide a log class
/** INTERNAL API */
@InternalApi
private[typed] final class ClusterReceptionist
/** INTERNAL API */
@InternalApi
private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
@ -68,6 +73,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
override def behavior: Behavior[Command] =
Behaviors.setup { ctx
ctx.setLoggerClass(classOf[ClusterReceptionist])
Behaviors.withTimers { timers
val setup = new Setup(ctx)

View file

@ -62,6 +62,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
emptyState: State,
commandHandler: EventSourcedBehavior.CommandHandler[Command, Event, State],
eventHandler: EventSourcedBehavior.EventHandler[State, Event],
loggerClass: Class[_],
journalPluginId: Option[String] = None,
snapshotPluginId: Option[String] = None,
recoveryCompleted: State Unit = ConstantFun.scalaAnyToUnit,
@ -78,6 +79,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
override def apply(context: typed.TypedActorContext[Command]): Behavior[Command] = {
val ctx = context.asScala
ctx.setLoggerClass(loggerClass)
val settings = EventSourcedSettings(ctx.system, journalPluginId.getOrElse(""), snapshotPluginId.getOrElse(""))
// stashState outside supervise because StashState should survive restarts due to persist failures

View file

@ -131,7 +131,6 @@ abstract class EventSourcedBehavior[Command, Event, State >: Null] private[akka]
* INTERNAL API: DeferredBehavior init
*/
@InternalApi override def apply(context: typed.TypedActorContext[Command]): Behavior[Command] = {
val snapshotWhen: (State, Event, Long) Boolean = { (state, event, seqNr)
val n = snapshotEvery()
if (n > 0)
@ -147,11 +146,12 @@ abstract class EventSourcedBehavior[Command, Event, State >: Null] private[akka]
else tags.asScala.toSet
}
val behavior = scaladsl.EventSourcedBehavior[Command, Event, State](
val behavior = new internal.EventSourcedBehaviorImpl[Command, Event, State](
persistenceId,
emptyState,
(state, cmd) commandHandler()(state, cmd).asInstanceOf[EffectImpl[Event, State]],
eventHandler()(_, _))
eventHandler()(_, _),
getClass)
.onRecoveryCompleted(onRecoveryCompleted)
.snapshotWhen(snapshotWhen)
.withTagger(tagger)

View file

@ -9,6 +9,7 @@ import scala.util.Try
import akka.Done
import akka.actor.typed.BackoffSupervisorStrategy
import akka.actor.typed.Behavior.DeferredBehavior
import akka.actor.typed.internal.LoggerClass
import akka.annotation.DoNotInherit
import akka.persistence._
import akka.persistence.typed.EventAdapter
@ -43,8 +44,10 @@ object EventSourcedBehavior {
persistenceId: PersistenceId,
emptyState: State,
commandHandler: (State, Command) Effect[Event, State],
eventHandler: (State, Event) State): EventSourcedBehavior[Command, Event, State] =
EventSourcedBehaviorImpl(persistenceId, emptyState, commandHandler, eventHandler)
eventHandler: (State, Event) State): EventSourcedBehavior[Command, Event, State] = {
val loggerClass = LoggerClass.detectLoggerClassFromStack(classOf[EventSourcedBehavior[_, _, _]])
EventSourcedBehaviorImpl(persistenceId, emptyState, commandHandler, eventHandler, loggerClass)
}
/**
* Create a `Behavior` for a persistent actor that is enforcing that replies to commands are not forgotten.
@ -55,8 +58,10 @@ object EventSourcedBehavior {
persistenceId: PersistenceId,
emptyState: State,
commandHandler: (State, Command) ReplyEffect[Event, State],
eventHandler: (State, Event) State): EventSourcedBehavior[Command, Event, State] =
EventSourcedBehaviorImpl(persistenceId, emptyState, commandHandler, eventHandler)
eventHandler: (State, Event) State): EventSourcedBehavior[Command, Event, State] = {
val loggerClass = LoggerClass.detectLoggerClassFromStack(classOf[EventSourcedBehavior[_, _, _]])
EventSourcedBehaviorImpl(persistenceId, emptyState, commandHandler, eventHandler, loggerClass)
}
/**
* The `CommandHandler` defines how to act on commands. A `CommandHandler` is