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:
parent
0c2ad78042
commit
84e8e41193
15 changed files with 290 additions and 22 deletions
|
|
@ -84,6 +84,10 @@ private[akka] final class FunctionRef[-T](
|
||||||
withMdc.mdc = mdc
|
withMdc.mdc = mdc
|
||||||
withMdc
|
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 {
|
@InternalApi private[akka] final class StubbedLoggerWithMdc(actual: StubbedLogger) extends AbstractLogger {
|
||||||
|
|
@ -121,6 +125,9 @@ private[akka] final class FunctionRef[-T](
|
||||||
actual.mdc = original
|
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 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
|
* The log entries logged through context.log.{debug, info, warn, error} are captured and can be inspected through
|
||||||
* this method.
|
* this method.
|
||||||
|
|
|
||||||
|
|
@ -4,12 +4,12 @@
|
||||||
|
|
||||||
package akka.actor.typed.javadsl;
|
package akka.actor.typed.javadsl;
|
||||||
|
|
||||||
|
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
|
||||||
import akka.actor.typed.ActorRef;
|
import akka.actor.typed.ActorRef;
|
||||||
import akka.actor.typed.Behavior;
|
import akka.actor.typed.Behavior;
|
||||||
import akka.event.Logging;
|
import akka.event.Logging;
|
||||||
import akka.japi.pf.PFBuilder;
|
import akka.japi.pf.PFBuilder;
|
||||||
import akka.testkit.CustomEventFilter;
|
import akka.testkit.CustomEventFilter;
|
||||||
import akka.actor.testkit.typed.javadsl.TestKitJunitResource;
|
|
||||||
import com.typesafe.config.ConfigFactory;
|
import com.typesafe.config.ConfigFactory;
|
||||||
import org.junit.ClassRule;
|
import org.junit.ClassRule;
|
||||||
import org.junit.Test;
|
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
|
@Test
|
||||||
public void loggingProvidesMDC() {
|
public void loggingProvidesMDC() {
|
||||||
Behavior<Protocol> behavior =
|
Behavior<Protocol> behavior =
|
||||||
|
|
|
||||||
|
|
@ -11,10 +11,31 @@ import akka.actor.testkit.typed.TestException
|
||||||
import akka.actor.typed.scaladsl.adapter._
|
import akka.actor.typed.scaladsl.adapter._
|
||||||
import akka.actor.typed.{ Behavior, LogMarker }
|
import akka.actor.typed.{ Behavior, LogMarker }
|
||||||
import akka.event.Logging
|
import akka.event.Logging
|
||||||
import akka.event.Logging.{ LogEventWithCause, LogEventWithMarker }
|
import akka.event.Logging.{ LogEvent, LogEventWithCause, LogEventWithMarker }
|
||||||
import akka.testkit.EventFilter
|
import akka.testkit.EventFilter
|
||||||
import org.scalatest.WordSpecLike
|
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("""
|
class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
|
||||||
akka.loglevel = DEBUG # test verifies debug
|
akka.loglevel = DEBUG # test verifies debug
|
||||||
akka.loggers = ["akka.testkit.TestEventListener"]
|
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 {
|
"pass markers to the log" in {
|
||||||
EventFilter.custom({
|
EventFilter.custom({
|
||||||
case event: LogEventWithMarker if event.marker == marker ⇒ true
|
case event: LogEventWithMarker if event.marker == marker ⇒ true
|
||||||
|
|
@ -204,7 +291,7 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
|
||||||
) {
|
) {
|
||||||
Behaviors.setup { context ⇒
|
Behaviors.setup { context ⇒
|
||||||
context.log.info("Starting")
|
context.log.info("Starting")
|
||||||
Behaviors.receiveMessage { message ⇒
|
Behaviors.receiveMessage { _ ⇒
|
||||||
context.log.info("Got message!")
|
context.log.info("Got message!")
|
||||||
Behaviors.same
|
Behaviors.same
|
||||||
}
|
}
|
||||||
|
|
@ -348,7 +435,7 @@ class ActorLoggingSpec extends ScalaTestWithActorTestKit("""
|
||||||
"provide a withMdc decorator" in {
|
"provide a withMdc decorator" in {
|
||||||
val behavior = Behaviors.withMdc[Protocol](Map("mdc" -> "outer"))(
|
val behavior = Behaviors.withMdc[Protocol](Map("mdc" -> "outer"))(
|
||||||
Behaviors.setup { context ⇒
|
Behaviors.setup { context ⇒
|
||||||
Behaviors.receiveMessage { message ⇒
|
Behaviors.receiveMessage { _ ⇒
|
||||||
context.log.withMdc(Map("mdc" -> "inner")).info("Got message log.withMDC!")
|
context.log.withMdc(Map("mdc" -> "inner")).info("Got message log.withMDC!")
|
||||||
// after log.withMdc so we know it didn't change the outer mdc
|
// after log.withMdc so we know it didn't change the outer mdc
|
||||||
context.log.info("Got message behavior.withMdc!")
|
context.log.info("Got message behavior.withMdc!")
|
||||||
|
|
|
||||||
|
|
@ -738,4 +738,14 @@ abstract class Logger private[akka] () {
|
||||||
* See also [[akka.actor.typed.javadsl.Behaviors.withMdc]]
|
* See also [[akka.actor.typed.javadsl.Behaviors.withMdc]]
|
||||||
*/
|
*/
|
||||||
def withMdc(mdc: java.util.Map[String, Any]): Logger
|
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
|
||||||
}
|
}
|
||||||
|
|
|
||||||
|
|
@ -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
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -83,18 +83,26 @@ import scala.concurrent.duration._
|
||||||
ActorRefAdapter[U](ref)
|
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 = {
|
override def log: Logger = {
|
||||||
actorLogger match {
|
actorLogger match {
|
||||||
case OptionVal.Some(logger) ⇒ logger
|
case OptionVal.Some(logger) ⇒ logger
|
||||||
case OptionVal.None ⇒
|
case OptionVal.None ⇒
|
||||||
val logSource = self.path.toString
|
val logClass = LoggerClass.detectLoggerClassFromStack(classOf[Behavior[_]])
|
||||||
val logClass = classOf[Behavior[_]] // FIXME figure out untyped better class somehow
|
initLoggerWithClass(logClass)
|
||||||
val system = untypedContext.system.asInstanceOf[ExtendedActorSystem]
|
|
||||||
val logger = new LoggerAdapterImpl(system.eventStream, logClass, logSource, system.logFilter)
|
|
||||||
actorLogger = OptionVal.Some(logger)
|
|
||||||
logger
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
override def setLoggerClass(clazz: Class[_]): Unit = {
|
||||||
|
initLoggerWithClass(clazz)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
||||||
|
|
@ -371,6 +371,18 @@ private[akka] final class LoggerAdapterImpl(bus: LoggingBus, logClass: Class[_],
|
||||||
mdcAdapter
|
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 = {
|
private[akka] def notifyError(message: String, cause: OptionVal[Throwable], marker: OptionVal[LogMarker]): Unit = {
|
||||||
val error = cause match {
|
val error = cause match {
|
||||||
case OptionVal.Some(cause) ⇒
|
case OptionVal.Some(cause) ⇒
|
||||||
|
|
|
||||||
|
|
@ -22,6 +22,11 @@ private[akka] trait ReceptionistBehaviorProvider {
|
||||||
def behavior: Behavior[Command]
|
def behavior: Behavior[Command]
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// just to provide a log class
|
||||||
|
/** INTERNAL API */
|
||||||
|
@InternalApi
|
||||||
|
private[akka] final class LocalReceptionist
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
@InternalApi
|
@InternalApi
|
||||||
private[akka] object LocalReceptionist extends ReceptionistBehaviorProvider {
|
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 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
|
final case class SubscriberTerminated[T](key: ServiceKey[T], ref: ActorRef[ReceptionistMessages.Listing[T]]) extends InternalCommand
|
||||||
|
|
||||||
override def behavior: Behavior[Command] = behavior(
|
override def behavior: Behavior[Command] = Behaviors.setup { ctx ⇒
|
||||||
TypedMultiMap.empty[AbstractServiceKey, KV],
|
ctx.setLoggerClass(classOf[LocalReceptionist])
|
||||||
TypedMultiMap.empty[AbstractServiceKey, SubscriptionsKV]
|
behavior(
|
||||||
).narrow[Command]
|
TypedMultiMap.empty[AbstractServiceKey, KV],
|
||||||
|
TypedMultiMap.empty[AbstractServiceKey, SubscriptionsKV]
|
||||||
|
).narrow[Command]
|
||||||
|
}
|
||||||
|
|
||||||
private def behavior(
|
private def behavior(
|
||||||
serviceRegistry: LocalServiceRegistry,
|
serviceRegistry: LocalServiceRegistry,
|
||||||
|
|
|
||||||
|
|
@ -74,6 +74,15 @@ trait ActorContext[T] extends TypedActorContext[T] {
|
||||||
*/
|
*/
|
||||||
def getLog: Logger
|
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
|
* The list of child Actors created by this Actor during its lifetime that
|
||||||
* are still alive, in no particular order.
|
* are still alive, in no particular order.
|
||||||
|
|
|
||||||
|
|
@ -65,13 +65,27 @@ trait ActorContext[T] extends TypedActorContext[T] { this: akka.actor.typed.java
|
||||||
def system: ActorSystem[Nothing]
|
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
|
* *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.
|
* than the ordinary actor message processing thread, such as [[scala.concurrent.Future]] callbacks.
|
||||||
*/
|
*/
|
||||||
def log: Logger
|
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
|
* The list of child Actors created by this Actor during its lifetime that
|
||||||
* are still alive, in no particular order.
|
* are still alive, in no particular order.
|
||||||
|
|
|
||||||
30
akka-bench-jmh/src/main/scala/akka/util/StackBench.scala
Normal file
30
akka-bench-jmh/src/main/scala/akka/util/StackBench.scala
Normal 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
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
|
@ -21,6 +21,11 @@ import akka.util.TypedMultiMap
|
||||||
|
|
||||||
import scala.concurrent.duration._
|
import scala.concurrent.duration._
|
||||||
|
|
||||||
|
// just to provide a log class
|
||||||
|
/** INTERNAL API */
|
||||||
|
@InternalApi
|
||||||
|
private[typed] final class ClusterReceptionist
|
||||||
|
|
||||||
/** INTERNAL API */
|
/** INTERNAL API */
|
||||||
@InternalApi
|
@InternalApi
|
||||||
private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
|
|
@ -68,6 +73,7 @@ private[typed] object ClusterReceptionist extends ReceptionistBehaviorProvider {
|
||||||
|
|
||||||
override def behavior: Behavior[Command] =
|
override def behavior: Behavior[Command] =
|
||||||
Behaviors.setup { ctx ⇒
|
Behaviors.setup { ctx ⇒
|
||||||
|
ctx.setLoggerClass(classOf[ClusterReceptionist])
|
||||||
Behaviors.withTimers { timers ⇒
|
Behaviors.withTimers { timers ⇒
|
||||||
|
|
||||||
val setup = new Setup(ctx)
|
val setup = new Setup(ctx)
|
||||||
|
|
|
||||||
|
|
@ -62,6 +62,7 @@ private[akka] final case class EventSourcedBehaviorImpl[Command, Event, State](
|
||||||
emptyState: State,
|
emptyState: State,
|
||||||
commandHandler: EventSourcedBehavior.CommandHandler[Command, Event, State],
|
commandHandler: EventSourcedBehavior.CommandHandler[Command, Event, State],
|
||||||
eventHandler: EventSourcedBehavior.EventHandler[State, Event],
|
eventHandler: EventSourcedBehavior.EventHandler[State, Event],
|
||||||
|
loggerClass: Class[_],
|
||||||
journalPluginId: Option[String] = None,
|
journalPluginId: Option[String] = None,
|
||||||
snapshotPluginId: Option[String] = None,
|
snapshotPluginId: Option[String] = None,
|
||||||
recoveryCompleted: State ⇒ Unit = ConstantFun.scalaAnyToUnit,
|
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] = {
|
override def apply(context: typed.TypedActorContext[Command]): Behavior[Command] = {
|
||||||
val ctx = context.asScala
|
val ctx = context.asScala
|
||||||
|
ctx.setLoggerClass(loggerClass)
|
||||||
val settings = EventSourcedSettings(ctx.system, journalPluginId.getOrElse(""), snapshotPluginId.getOrElse(""))
|
val settings = EventSourcedSettings(ctx.system, journalPluginId.getOrElse(""), snapshotPluginId.getOrElse(""))
|
||||||
|
|
||||||
// stashState outside supervise because StashState should survive restarts due to persist failures
|
// stashState outside supervise because StashState should survive restarts due to persist failures
|
||||||
|
|
|
||||||
|
|
@ -131,7 +131,6 @@ abstract class EventSourcedBehavior[Command, Event, State >: Null] private[akka]
|
||||||
* INTERNAL API: DeferredBehavior init
|
* INTERNAL API: DeferredBehavior init
|
||||||
*/
|
*/
|
||||||
@InternalApi override def apply(context: typed.TypedActorContext[Command]): Behavior[Command] = {
|
@InternalApi override def apply(context: typed.TypedActorContext[Command]): Behavior[Command] = {
|
||||||
|
|
||||||
val snapshotWhen: (State, Event, Long) ⇒ Boolean = { (state, event, seqNr) ⇒
|
val snapshotWhen: (State, Event, Long) ⇒ Boolean = { (state, event, seqNr) ⇒
|
||||||
val n = snapshotEvery()
|
val n = snapshotEvery()
|
||||||
if (n > 0)
|
if (n > 0)
|
||||||
|
|
@ -147,11 +146,12 @@ abstract class EventSourcedBehavior[Command, Event, State >: Null] private[akka]
|
||||||
else tags.asScala.toSet
|
else tags.asScala.toSet
|
||||||
}
|
}
|
||||||
|
|
||||||
val behavior = scaladsl.EventSourcedBehavior[Command, Event, State](
|
val behavior = new internal.EventSourcedBehaviorImpl[Command, Event, State](
|
||||||
persistenceId,
|
persistenceId,
|
||||||
emptyState,
|
emptyState,
|
||||||
(state, cmd) ⇒ commandHandler()(state, cmd).asInstanceOf[EffectImpl[Event, State]],
|
(state, cmd) ⇒ commandHandler()(state, cmd).asInstanceOf[EffectImpl[Event, State]],
|
||||||
eventHandler()(_, _))
|
eventHandler()(_, _),
|
||||||
|
getClass)
|
||||||
.onRecoveryCompleted(onRecoveryCompleted)
|
.onRecoveryCompleted(onRecoveryCompleted)
|
||||||
.snapshotWhen(snapshotWhen)
|
.snapshotWhen(snapshotWhen)
|
||||||
.withTagger(tagger)
|
.withTagger(tagger)
|
||||||
|
|
|
||||||
|
|
@ -9,6 +9,7 @@ import scala.util.Try
|
||||||
import akka.Done
|
import akka.Done
|
||||||
import akka.actor.typed.BackoffSupervisorStrategy
|
import akka.actor.typed.BackoffSupervisorStrategy
|
||||||
import akka.actor.typed.Behavior.DeferredBehavior
|
import akka.actor.typed.Behavior.DeferredBehavior
|
||||||
|
import akka.actor.typed.internal.LoggerClass
|
||||||
import akka.annotation.DoNotInherit
|
import akka.annotation.DoNotInherit
|
||||||
import akka.persistence._
|
import akka.persistence._
|
||||||
import akka.persistence.typed.EventAdapter
|
import akka.persistence.typed.EventAdapter
|
||||||
|
|
@ -43,8 +44,10 @@ object EventSourcedBehavior {
|
||||||
persistenceId: PersistenceId,
|
persistenceId: PersistenceId,
|
||||||
emptyState: State,
|
emptyState: State,
|
||||||
commandHandler: (State, Command) ⇒ Effect[Event, State],
|
commandHandler: (State, Command) ⇒ Effect[Event, State],
|
||||||
eventHandler: (State, Event) ⇒ State): EventSourcedBehavior[Command, Event, State] =
|
eventHandler: (State, Event) ⇒ State): EventSourcedBehavior[Command, Event, State] = {
|
||||||
EventSourcedBehaviorImpl(persistenceId, emptyState, commandHandler, eventHandler)
|
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.
|
* 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,
|
persistenceId: PersistenceId,
|
||||||
emptyState: State,
|
emptyState: State,
|
||||||
commandHandler: (State, Command) ⇒ ReplyEffect[Event, State],
|
commandHandler: (State, Command) ⇒ ReplyEffect[Event, State],
|
||||||
eventHandler: (State, Event) ⇒ State): EventSourcedBehavior[Command, Event, State] =
|
eventHandler: (State, Event) ⇒ State): EventSourcedBehavior[Command, Event, State] = {
|
||||||
EventSourcedBehaviorImpl(persistenceId, emptyState, commandHandler, eventHandler)
|
val loggerClass = LoggerClass.detectLoggerClassFromStack(classOf[EventSourcedBehavior[_, _, _]])
|
||||||
|
EventSourcedBehaviorImpl(persistenceId, emptyState, commandHandler, eventHandler, loggerClass)
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* The `CommandHandler` defines how to act on commands. A `CommandHandler` is
|
* The `CommandHandler` defines how to act on commands. A `CommandHandler` is
|
||||||
|
|
|
||||||
Loading…
Add table
Add a link
Reference in a new issue