Rename event-handlers to loggers, see #2979
* Rename config akka.event-handlers to akka.loggers * Rename config akka.event-handler-startup-timeout to akka.logger-startup-timeout * Rename JulEventHandler to JavaLogger * Rename Slf4jEventHandler to Slf4jLogger * Change all places in tests and docs * Deprecation, old still works, but with warnings * Migration guide * Test for the deprecated event-handler config
This commit is contained in:
parent
c6f08fb935
commit
2476831705
37 changed files with 347 additions and 216 deletions
|
|
@ -410,7 +410,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
|
|||
}
|
||||
|
||||
"continue to process messages when a thread gets interrupted and throws an exception" in {
|
||||
filterEvents(EventFilter[InterruptedException](), EventFilter[akka.event.Logging.EventHandlerException]()) {
|
||||
filterEvents(EventFilter[InterruptedException](), EventFilter[akka.event.Logging.LoggerException]()) {
|
||||
implicit val dispatcher = interceptedDispatcher()
|
||||
val a = newTestActor(dispatcher.id)
|
||||
val f1 = a ? Reply("foo")
|
||||
|
|
|
|||
|
|
@ -5,12 +5,12 @@
|
|||
package akka.config
|
||||
|
||||
import language.postfixOps
|
||||
|
||||
import akka.testkit.AkkaSpec
|
||||
import com.typesafe.config.ConfigFactory
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.concurrent.duration._
|
||||
import akka.actor.{ IOManager, ActorSystem }
|
||||
import akka.event.Logging.DefaultLogger
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.findClassLoader())) {
|
||||
|
|
@ -46,6 +46,13 @@ class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.fin
|
|||
|
||||
getMilliseconds("akka.actor.unstarted-push-timeout") must be(10.seconds.toMillis)
|
||||
settings.UnstartedPushTimeout.duration must be(10.seconds)
|
||||
|
||||
settings.Loggers.size must be(1)
|
||||
settings.Loggers.head must be(classOf[DefaultLogger].getName)
|
||||
getStringList("akka.loggers").get(0) must be(classOf[DefaultLogger].getName)
|
||||
|
||||
getMilliseconds("akka.logger-startup-timeout") must be(5.seconds.toMillis)
|
||||
settings.LoggerStartTimeout.duration must be(5.seconds)
|
||||
}
|
||||
|
||||
{
|
||||
|
|
|
|||
|
|
@ -0,0 +1,53 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.config
|
||||
|
||||
import scala.concurrent.duration._
|
||||
import akka.testkit.AkkaSpec
|
||||
import akka.actor.Actor
|
||||
import akka.event.Logging.InitializeLogger
|
||||
import akka.event.Logging.LogEvent
|
||||
import akka.event.Logging.LoggerInitialized
|
||||
import akka.event.Logging.Error
|
||||
import akka.util.Timeout
|
||||
|
||||
object DeprecatedEventHandlerSpec {
|
||||
|
||||
case class WrappedLogEvent(event: Any)
|
||||
|
||||
class TestEventHandler extends Actor {
|
||||
def receive = {
|
||||
case init: InitializeLogger ⇒
|
||||
sender ! LoggerInitialized
|
||||
case err: Error ⇒
|
||||
context.system.eventStream.publish(WrappedLogEvent(err))
|
||||
case event: LogEvent ⇒
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class DeprecatedEventHandlerSpec extends AkkaSpec("""
|
||||
akka.event-handlers = ["akka.config.DeprecatedEventHandlerSpec$TestEventHandler"]
|
||||
akka.event-handler-startup-timeout = 17s
|
||||
""") {
|
||||
|
||||
import DeprecatedEventHandlerSpec._
|
||||
|
||||
"Akka 2.2" must {
|
||||
"use deprected event-handler properties" in {
|
||||
system.settings.EventHandlers must be(List(classOf[TestEventHandler].getName))
|
||||
system.settings.EventHandlerStartTimeout must be(Timeout(17.seconds))
|
||||
|
||||
system.eventStream.subscribe(testActor, classOf[WrappedLogEvent])
|
||||
|
||||
system.log.error("test error")
|
||||
expectMsgPF(remaining) {
|
||||
case WrappedLogEvent(Error(_, _, _, "test error")) ⇒
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -19,7 +19,7 @@ object EventStreamSpec {
|
|||
akka {
|
||||
stdout-loglevel = WARNING
|
||||
loglevel = INFO
|
||||
event-handlers = ["akka.event.EventStreamSpec$MyLog", "%s"]
|
||||
loggers = ["akka.event.EventStreamSpec$MyLog", "%s"]
|
||||
}
|
||||
""".format(Logging.StandardOutLogger.getClass.getName))
|
||||
|
||||
|
|
|
|||
|
|
@ -4,7 +4,7 @@ import com.typesafe.config.ConfigFactory
|
|||
object BenchmarkConfig {
|
||||
private val benchmarkConfig = ConfigFactory.parseString("""
|
||||
akka {
|
||||
event-handlers = ["akka.testkit.TestEventListener"]
|
||||
loggers = ["akka.testkit.TestEventListener"]
|
||||
loglevel = "WARNING"
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -12,15 +12,22 @@ akka {
|
|||
# Home directory of Akka, modules in the deploy directory will be loaded
|
||||
home = ""
|
||||
|
||||
# Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT)
|
||||
event-handlers = ["akka.event.Logging$DefaultLogger"]
|
||||
# Loggers to register at boot time (akka.event.Logging$DefaultLogger logs
|
||||
# to STDOUT)
|
||||
loggers = ["akka.event.Logging$DefaultLogger"]
|
||||
|
||||
# Event handlers are created and registered synchronously during ActorSystem
|
||||
# Deprecated, use akka.loggers.
|
||||
event-handlers = []
|
||||
|
||||
# Loggers are created and registered synchronously during ActorSystem
|
||||
# start-up, and since they are actors, this timeout is used to bound the
|
||||
# waiting time
|
||||
event-handler-startup-timeout = 5s
|
||||
logger-startup-timeout = 5s
|
||||
|
||||
# Log level used by the configured loggers (see "event-handlers") as soon
|
||||
# Deprecated, use akka.loggers-startup-timeout
|
||||
event-handler-startup-timeout = -1s
|
||||
|
||||
# Log level used by the configured loggers (see "loggers") as soon
|
||||
# as they have been started; before that, see "stdout-loglevel"
|
||||
# Options: ERROR, WARNING, INFO, DEBUG
|
||||
loglevel = "INFO"
|
||||
|
|
@ -364,14 +371,14 @@ akka {
|
|||
# ticks per wheel.
|
||||
# For more information see: http://www.jboss.org/netty/
|
||||
tick-duration = 100ms
|
||||
|
||||
|
||||
# The timer uses a circular wheel of buckets to store the timer tasks.
|
||||
# This should be set such that the majority of scheduled timeouts (for high
|
||||
# scheduling frequency) will be shorter than one rotation of the wheel
|
||||
# (ticks-per-wheel * ticks-duration)
|
||||
# THIS MUST BE A POWER OF TWO!
|
||||
ticks-per-wheel = 512
|
||||
|
||||
|
||||
# This setting selects the timer implementation which shall be loaded at
|
||||
# system start-up. Built-in choices are:
|
||||
# - akka.actor.DefaultScheduler (HWT)
|
||||
|
|
@ -383,9 +390,9 @@ akka {
|
|||
# 2) akka.event.LoggingAdapter
|
||||
# 3) java.util.concurrent.ThreadFactory
|
||||
implementation = akka.actor.LightArrayRevolverScheduler
|
||||
|
||||
|
||||
# When shutting down the scheduler, there will typically be a thread which
|
||||
# needs to be stopped, and this timeout determines how long to wait for
|
||||
# needs to be stopped, and this timeout determines how long to wait for
|
||||
# that to happen. In case of timeout the shutdown of the actor system will
|
||||
# proceed without running possibly still enqueued tasks.
|
||||
shutdown-timeout = 5s
|
||||
|
|
|
|||
|
|
@ -143,7 +143,11 @@ object ActorSystem {
|
|||
|
||||
final val LogLevel: String = getString("akka.loglevel")
|
||||
final val StdoutLogLevel: String = getString("akka.stdout-loglevel")
|
||||
final val Loggers: immutable.Seq[String] = immutableSeq(getStringList("akka.loggers"))
|
||||
final val LoggerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.logger-startup-timeout"), MILLISECONDS))
|
||||
@deprecated("use Loggers)", "2.2")
|
||||
final val EventHandlers: immutable.Seq[String] = immutableSeq(getStringList("akka.event-handlers"))
|
||||
@deprecated("use LoggerStartTimeout)", "2.2")
|
||||
final val EventHandlerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.event-handler-startup-timeout"), MILLISECONDS))
|
||||
final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start")
|
||||
|
||||
|
|
|
|||
|
|
@ -72,7 +72,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
*/
|
||||
private[akka] def startStdoutLogger(config: Settings) {
|
||||
val level = levelFor(config.StdoutLogLevel) getOrElse {
|
||||
StandardOutLogger.print(Error(new EventHandlerException, simpleName(this), this.getClass, "unknown akka.stdout-loglevel " + config.StdoutLogLevel))
|
||||
StandardOutLogger.print(Error(new LoggerException, simpleName(this), this.getClass, "unknown akka.stdout-loglevel " + config.StdoutLogLevel))
|
||||
ErrorLevel
|
||||
}
|
||||
AllLogLevels filter (level >= _) foreach (l ⇒ subscribe(StandardOutLogger, classFor(l)))
|
||||
|
|
@ -89,13 +89,18 @@ trait LoggingBus extends ActorEventBus {
|
|||
private[akka] def startDefaultLoggers(system: ActorSystemImpl) {
|
||||
val logName = simpleName(this) + "(" + system + ")"
|
||||
val level = levelFor(system.settings.LogLevel) getOrElse {
|
||||
StandardOutLogger.print(Error(new EventHandlerException, logName, this.getClass, "unknown akka.stdout-loglevel " + system.settings.LogLevel))
|
||||
StandardOutLogger.print(Error(new LoggerException, logName, this.getClass, "unknown akka.stdout-loglevel " + system.settings.LogLevel))
|
||||
ErrorLevel
|
||||
}
|
||||
try {
|
||||
val defaultLoggers = system.settings.EventHandlers match {
|
||||
case Nil ⇒ "akka.event.Logging$DefaultLogger" :: Nil
|
||||
case loggers ⇒ loggers
|
||||
case Nil ⇒ system.settings.Loggers match {
|
||||
case Nil ⇒ classOf[DefaultLogger].getName :: Nil
|
||||
case loggers ⇒ loggers
|
||||
}
|
||||
case loggers ⇒
|
||||
StandardOutLogger.print(Warning(logName, this.getClass, "[akka.event-handlers] config is deprecated, use [akka.loggers]"))
|
||||
loggers
|
||||
}
|
||||
val myloggers =
|
||||
for {
|
||||
|
|
@ -106,7 +111,7 @@ trait LoggingBus extends ActorEventBus {
|
|||
case actorClass ⇒ addLogger(system, actorClass, level, logName)
|
||||
}).recover({
|
||||
case e ⇒ throw new ConfigurationException(
|
||||
"Event Handler specified in config can't be loaded [" + loggerName +
|
||||
"Logger specified in config can't be loaded [" + loggerName +
|
||||
"] due to [" + e.toString + "]", e)
|
||||
}).get
|
||||
}
|
||||
|
|
@ -166,7 +171,13 @@ trait LoggingBus extends ActorEventBus {
|
|||
private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel, logName: String): ActorRef = {
|
||||
val name = "log" + Extension(system).id() + "-" + simpleName(clazz)
|
||||
val actor = system.systemActorOf(Props(clazz), name)
|
||||
implicit def timeout = system.settings.EventHandlerStartTimeout
|
||||
implicit def timeout =
|
||||
if (system.settings.EventHandlerStartTimeout.duration >= Duration.Zero) {
|
||||
StandardOutLogger.print(Warning(logName, this.getClass,
|
||||
"[akka.event-handler-startup-timeout] config is deprecated, use [akka.logger-startup-timeout]"))
|
||||
system.settings.EventHandlerStartTimeout
|
||||
} else system.settings.LoggerStartTimeout
|
||||
|
||||
import akka.pattern.ask
|
||||
val response = try Await.result(actor ? InitializeLogger(this), timeout.duration) catch {
|
||||
case _: TimeoutException ⇒
|
||||
|
|
@ -353,8 +364,8 @@ object LogSource {
|
|||
*
|
||||
* <pre><code>
|
||||
* akka {
|
||||
* event-handlers = ["akka.slf4j.Slf4jEventHandler"] # for example
|
||||
* loglevel = "INFO" # used when normal logging ("event-handlers") has been started
|
||||
* loggers = ["akka.slf4j.Slf4jLogger"] # for example
|
||||
* loglevel = "INFO" # used when normal logging ("loggers") has been started
|
||||
* stdout-loglevel = "WARN" # used during application start-up until normal logging is available
|
||||
* }
|
||||
* </code></pre>
|
||||
|
|
@ -531,7 +542,7 @@ object Logging {
|
|||
* Artificial exception injected into Error events if no Throwable is
|
||||
* supplied; used for getting a stack dump of error locations.
|
||||
*/
|
||||
class EventHandlerException extends AkkaException("")
|
||||
class LoggerException extends AkkaException("")
|
||||
|
||||
/**
|
||||
* Exception that wraps a LogEvent.
|
||||
|
|
@ -716,7 +727,7 @@ object Logging {
|
|||
|
||||
/**
|
||||
* Actor wrapper around the standard output logger. If
|
||||
* <code>akka.event-handlers</code> is not set, it defaults to just this
|
||||
* <code>akka.loggers</code> is not set, it defaults to just this
|
||||
* logger.
|
||||
*/
|
||||
class DefaultLogger extends Actor with StdOutLogger {
|
||||
|
|
|
|||
|
|
@ -43,7 +43,7 @@ object MultiNodeClusterSpec {
|
|||
}
|
||||
akka.loglevel = INFO
|
||||
akka.remote.log-remote-lifecycle-events = off
|
||||
akka.event-handlers = ["akka.testkit.TestEventListener"]
|
||||
akka.loggers = ["akka.testkit.TestEventListener"]
|
||||
akka.test {
|
||||
single-expect-default = 5 s
|
||||
}
|
||||
|
|
|
|||
|
|
@ -101,7 +101,7 @@ object StressMultiJvmSpec extends MultiNodeConfig {
|
|||
auto-down = on
|
||||
publish-stats-interval = 0 s # always, when it happens
|
||||
}
|
||||
akka.event-handlers = ["akka.testkit.TestEventListener"]
|
||||
akka.loggers = ["akka.testkit.TestEventListener"]
|
||||
akka.loglevel = INFO
|
||||
akka.remote.log-remote-lifecycle-events = off
|
||||
|
||||
|
|
|
|||
|
|
@ -28,7 +28,7 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
|
|||
akka.cluster {
|
||||
auto-join = off
|
||||
}
|
||||
akka.event-handlers = ["akka.testkit.TestEventListener"]
|
||||
akka.loggers = ["akka.testkit.TestEventListener"]
|
||||
akka.loglevel = INFO
|
||||
akka.remote.log-remote-lifecycle-events = off
|
||||
"""))
|
||||
|
|
|
|||
|
|
@ -8,7 +8,7 @@ Provided with this module is an implementation of `akka.event.LoggingAdapter` wh
|
|||
|
||||
To enable j.u.l as the `akka.event.Logging` backend, use the following Akka config:
|
||||
|
||||
event-handlers = ["akka.contrib.jul.JavaLoggingEventHandler"]
|
||||
loggers = ["akka.contrib.jul.JavaLogger"]
|
||||
|
||||
To access the `akka.event.Logging` API from non-Actor code, mix in `akka.contrib.jul.JavaLogging`.
|
||||
|
||||
|
|
|
|||
|
|
@ -1,3 +1,6 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.contrib.jul
|
||||
|
||||
import akka.event.Logging._
|
||||
|
|
@ -25,35 +28,26 @@ trait JavaLogging {
|
|||
}
|
||||
|
||||
/**
|
||||
* `java.util.logging` EventHandler.
|
||||
* `java.util.logging` logger.
|
||||
*/
|
||||
class JavaLoggingEventHandler extends Actor {
|
||||
class JavaLogger extends Actor {
|
||||
|
||||
def receive = {
|
||||
case event @ Error(cause, logSource, logClass, message) ⇒
|
||||
log(logging.Level.SEVERE, cause, logSource, logClass, message, event)
|
||||
|
||||
case event @ Warning(logSource, logClass, message) ⇒
|
||||
log(logging.Level.WARNING, null, logSource, logClass, message, event)
|
||||
|
||||
case event @ Info(logSource, logClass, message) ⇒
|
||||
log(logging.Level.INFO, null, logSource, logClass, message, event)
|
||||
|
||||
case event @ Debug(logSource, logClass, message) ⇒
|
||||
log(logging.Level.CONFIG, null, logSource, logClass, message, event)
|
||||
|
||||
case InitializeLogger(_) ⇒
|
||||
sender ! LoggerInitialized
|
||||
case event @ Error(cause, _, _, _) ⇒ log(logging.Level.SEVERE, cause, event)
|
||||
case event: Warning ⇒ log(logging.Level.WARNING, null, event)
|
||||
case event: Info ⇒ log(logging.Level.INFO, null, event)
|
||||
case event: Debug ⇒ log(logging.Level.CONFIG, null, event)
|
||||
case InitializeLogger(_) ⇒ sender ! LoggerInitialized
|
||||
}
|
||||
|
||||
@inline
|
||||
def log(level: logging.Level, cause: Throwable, logSource: String, logClass: Class[_], message: Any, event: LogEvent) {
|
||||
val logger = logging.Logger.getLogger(logSource)
|
||||
val record = new logging.LogRecord(level, message.toString)
|
||||
def log(level: logging.Level, cause: Throwable, event: LogEvent) {
|
||||
val logger = logging.Logger.getLogger(event.logSource)
|
||||
val record = new logging.LogRecord(level, String.valueOf(event.message))
|
||||
record.setLoggerName(logger.getName)
|
||||
record.setThrown(cause)
|
||||
record.setThreadID(event.thread.getId.toInt)
|
||||
record.setSourceClassName(logClass.getName)
|
||||
record.setSourceClassName(event.logClass.getName)
|
||||
record.setSourceMethodName(null) // lost forever
|
||||
logger.log(record)
|
||||
}
|
||||
|
|
@ -0,0 +1,14 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.contrib.jul
|
||||
|
||||
import akka.event.Logging.Warning
|
||||
|
||||
@deprecated("use akka.contrib.jul.JavaLogger)", "2.2")
|
||||
class JavaLoggingEventHandler extends JavaLogger {
|
||||
|
||||
self ! Warning(getClass.getName, getClass,
|
||||
s"[${getClass.getName}] is depreceated, use [${classOf[JavaLogger].getName}] instead")
|
||||
|
||||
}
|
||||
|
|
@ -1,3 +1,6 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
package akka.contrib.mailbox
|
||||
|
||||
import java.util.concurrent.{ ConcurrentHashMap, ConcurrentLinkedQueue }
|
||||
|
|
@ -55,8 +58,8 @@ class PeekMailbox(owner: ActorRef, system: ActorSystem, maxRetries: Int)
|
|||
|
||||
/*
|
||||
* Since the queue itself is used to determine when to schedule the actor
|
||||
* (see Mailbox.hasMessages), we cannot poll() on the first try and then
|
||||
* continue handing back out that same message until ACKed, peek() must be
|
||||
* (see Mailbox.hasMessages), we cannot poll() on the first try and then
|
||||
* continue handing back out that same message until ACKed, peek() must be
|
||||
* used. The retry limit logic is then formulated in terms of the `tries`
|
||||
* field, which holds
|
||||
* 0 if clean slate (i.e. last dequeue was ack()ed)
|
||||
|
|
@ -70,10 +73,18 @@ class PeekMailbox(owner: ActorRef, system: ActorSystem, maxRetries: Int)
|
|||
|
||||
// this logic does not work if maxRetries==0, but then you could also use a normal mailbox
|
||||
override def dequeue(): Envelope = tries match {
|
||||
case -1 ⇒ queue.poll()
|
||||
case 0 | Marker ⇒ val e = queue.peek(); tries = if (e eq null) 0 else 1; e
|
||||
case `maxRetries` ⇒ tries = Marker; queue.poll()
|
||||
case n ⇒ tries = n + 1; queue.peek()
|
||||
case -1 ⇒
|
||||
queue.poll()
|
||||
case 0 | Marker ⇒
|
||||
val e = queue.peek()
|
||||
tries = if (e eq null) 0 else 1
|
||||
e
|
||||
case `maxRetries` ⇒
|
||||
tries = Marker
|
||||
queue.poll()
|
||||
case n ⇒
|
||||
tries = n + 1
|
||||
queue.peek()
|
||||
}
|
||||
|
||||
def ack(): Unit = {
|
||||
|
|
|
|||
|
|
@ -6,12 +6,12 @@ import akka.testkit.AkkaSpec
|
|||
import java.util.logging
|
||||
import java.io.ByteArrayInputStream
|
||||
|
||||
object JavaLoggingEventHandlerSpec {
|
||||
object JavaLoggerSpec {
|
||||
|
||||
val config = ConfigFactory.parseString("""
|
||||
akka {
|
||||
loglevel = INFO
|
||||
event-handlers = ["akka.contrib.jul.JavaLoggingEventHandler"]
|
||||
loggers = ["akka.contrib.jul.JavaLogger"]
|
||||
}""")
|
||||
|
||||
class LogProducer extends Actor with ActorLogging {
|
||||
|
|
@ -25,9 +25,9 @@ object JavaLoggingEventHandlerSpec {
|
|||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class JavaLoggingEventHandlerSpec extends AkkaSpec(JavaLoggingEventHandlerSpec.config) {
|
||||
class JavaLoggerSpec extends AkkaSpec(JavaLoggerSpec.config) {
|
||||
|
||||
val logger = logging.Logger.getLogger("akka://JavaLoggingEventHandlerSpec/user/log")
|
||||
val logger = logging.Logger.getLogger("akka://JavaLoggerSpec/user/log")
|
||||
logger.setUseParentHandlers(false) // turn off output of test LogRecords
|
||||
logger.addHandler(new logging.Handler {
|
||||
def publish(record: logging.LogRecord) {
|
||||
|
|
@ -38,9 +38,9 @@ class JavaLoggingEventHandlerSpec extends AkkaSpec(JavaLoggingEventHandlerSpec.c
|
|||
def close() {}
|
||||
})
|
||||
|
||||
val producer = system.actorOf(Props[JavaLoggingEventHandlerSpec.LogProducer], name = "log")
|
||||
val producer = system.actorOf(Props[JavaLoggerSpec.LogProducer], name = "log")
|
||||
|
||||
"JavaLoggingEventHandler" must {
|
||||
"JavaLogger" must {
|
||||
|
||||
"log error with stackTrace" in {
|
||||
producer ! new RuntimeException("Simulated error")
|
||||
|
|
@ -53,7 +53,7 @@ class JavaLoggingEventHandlerSpec extends AkkaSpec(JavaLoggingEventHandlerSpec.c
|
|||
record.getLevel must be(logging.Level.SEVERE)
|
||||
record.getMessage must be("Simulated error")
|
||||
record.getThrown.isInstanceOf[RuntimeException] must be(true)
|
||||
record.getSourceClassName must be("akka.contrib.jul.JavaLoggingEventHandlerSpec$LogProducer")
|
||||
record.getSourceClassName must be(classOf[JavaLoggerSpec.LogProducer].getName)
|
||||
record.getSourceMethodName must be(null)
|
||||
}
|
||||
|
||||
|
|
@ -68,7 +68,7 @@ class JavaLoggingEventHandlerSpec extends AkkaSpec(JavaLoggingEventHandlerSpec.c
|
|||
record.getLevel must be(logging.Level.INFO)
|
||||
record.getMessage must be("3 is the magic number")
|
||||
record.getThrown must be(null)
|
||||
record.getSourceClassName must be("akka.contrib.jul.JavaLoggingEventHandlerSpec$LogProducer")
|
||||
record.getSourceClassName must be(classOf[JavaLoggerSpec.LogProducer].getName)
|
||||
record.getSourceMethodName must be(null)
|
||||
}
|
||||
}
|
||||
|
|
@ -78,10 +78,11 @@ A custom ``application.conf`` might look like this::
|
|||
|
||||
akka {
|
||||
|
||||
# Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT)
|
||||
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
|
||||
# Loggers to register at boot time (akka.event.Logging$DefaultLogger logs
|
||||
# to STDOUT)
|
||||
loggers = ["akka.event.slf4j.Slf4jLogger"]
|
||||
|
||||
# Log level used by the configured loggers (see "event-handlers") as soon
|
||||
# Log level used by the configured loggers (see "loggers") as soon
|
||||
# as they have been started; before that, see "stdout-loglevel"
|
||||
# Options: ERROR, WARNING, INFO, DEBUG
|
||||
loglevel = DEBUG
|
||||
|
|
|
|||
|
|
@ -39,7 +39,7 @@ Akka is very modular and consists of several JARs containing different features.
|
|||
- ``akka-agent`` -- Agents, integrated with Scala STM
|
||||
- ``akka-camel`` -- Apache Camel integration
|
||||
- ``akka-zeromq`` -- ZeroMQ integration
|
||||
- ``akka-slf4j`` -- SLF4J Event Handler Listener
|
||||
- ``akka-slf4j`` -- SLF4J Logger (event bus listener)
|
||||
- ``akka-filebased-mailbox`` -- Akka durable mailbox (find more among community projects)
|
||||
|
||||
The filename of the actual JAR is for example ``@jarName@`` (and analog for
|
||||
|
|
|
|||
|
|
@ -50,7 +50,7 @@ public class TestKitDocTest {
|
|||
@BeforeClass
|
||||
public static void setup() {
|
||||
final Config config = ConfigFactory.parseString(
|
||||
"akka.event-handlers = [akka.testkit.TestEventListener]");
|
||||
"akka.loggers = [akka.testkit.TestEventListener]");
|
||||
system = ActorSystem.create("demoSystem", config);
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -175,7 +175,7 @@ stream for logging: these are the handlers which are configured for example in
|
|||
.. code-block:: text
|
||||
|
||||
akka {
|
||||
event-handlers = ["akka.event.Logging$DefaultLogger"]
|
||||
loggers = ["akka.event.Logging$DefaultLogger"]
|
||||
}
|
||||
|
||||
The handlers listed here by fully-qualified class name will be subscribed to
|
||||
|
|
|
|||
|
|
@ -154,24 +154,25 @@ If you want to see all messages that are received through remoting at DEBUG log
|
|||
Also see the logging options for TestKit: :ref:`actor.logging-java`.
|
||||
|
||||
|
||||
Event Handler
|
||||
=============
|
||||
Loggers
|
||||
=======
|
||||
|
||||
Logging is performed asynchronously through an event bus. You can configure which event handlers that should
|
||||
subscribe to the logging events. That is done using the 'event-handlers' element in the :ref:`configuration`.
|
||||
Logging is performed asynchronously through an event bus. You can configure which loggers that should
|
||||
subscribe to the logging events. That is done using the 'loggers' element in the :ref:`configuration`.
|
||||
Here you can also define the log level.
|
||||
|
||||
.. code-block:: ruby
|
||||
|
||||
akka {
|
||||
# Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT)
|
||||
event-handlers = ["akka.event.Logging$DefaultLogger"]
|
||||
# Loggers to register at boot time (akka.event.Logging$DefaultLogger logs
|
||||
# to STDOUT)
|
||||
loggers = ["akka.event.Logging$DefaultLogger"]
|
||||
# Options: ERROR, WARNING, INFO, DEBUG
|
||||
loglevel = "DEBUG"
|
||||
}
|
||||
|
||||
The default one logs to STDOUT and is registered by default. It is not intended to be used for production. There is also an :ref:`slf4j-java`
|
||||
event handler available in the 'akka-slf4j' module.
|
||||
logger available in the 'akka-slf4j' module.
|
||||
|
||||
Example of creating a listener:
|
||||
|
||||
|
|
@ -186,7 +187,7 @@ Example of creating a listener:
|
|||
SLF4J
|
||||
=====
|
||||
|
||||
Akka provides an event handler for `SL4FJ <http://www.slf4j.org/>`_. This module is available in the 'akka-slf4j.jar'.
|
||||
Akka provides a logger for `SL4FJ <http://www.slf4j.org/>`_. This module is available in the 'akka-slf4j.jar'.
|
||||
It has one single dependency; the slf4j-api jar. In runtime you also need a SLF4J backend, we recommend `Logback <http://logback.qos.ch/>`_:
|
||||
|
||||
.. code-block:: xml
|
||||
|
|
@ -197,7 +198,7 @@ It has one single dependency; the slf4j-api jar. In runtime you also need a SLF4
|
|||
<version>1.0.7</version>
|
||||
</dependency>
|
||||
|
||||
You need to enable the Slf4jEventHandler in the 'event-handlers' element in
|
||||
You need to enable the Slf4jLogger in the 'loggers' element in
|
||||
the :ref:`configuration`. Here you can also define the log level of the event bus.
|
||||
More fine grained log levels can be defined in the configuration of the SLF4J backend
|
||||
(e.g. logback.xml).
|
||||
|
|
@ -205,7 +206,7 @@ More fine grained log levels can be defined in the configuration of the SLF4J ba
|
|||
.. code-block:: ruby
|
||||
|
||||
akka {
|
||||
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
|
||||
loggers = ["akka.event.slf4j.Slf4jLogger"]
|
||||
loglevel = "DEBUG"
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -305,11 +305,11 @@ fails.
|
|||
|
||||
.. note::
|
||||
|
||||
Be sure to exchange the default event handler with the
|
||||
Be sure to exchange the default logger with the
|
||||
:class:`TestEventListener` in your ``application.conf`` to enable this
|
||||
function::
|
||||
|
||||
akka.event-handlers = [akka.testkit.TestEventListener]
|
||||
akka.loggers = [akka.testkit.TestEventListener]
|
||||
|
||||
.. _JavaTestKit.within:
|
||||
|
||||
|
|
|
|||
|
|
@ -97,4 +97,18 @@ Old Scala API New Scala API
|
|||
``agent.suspend()`` ``No replacement, pointless feature``
|
||||
``agent.resume()`` ``No replacement, pointless feature``
|
||||
``agent.close()`` ``No replacement, not needed in new implementation``
|
||||
======================================================== ========================================================
|
||||
======================================================== ========================================================
|
||||
|
||||
|
||||
``event-handlers`` renamed to ``loggers``
|
||||
=========================================
|
||||
|
||||
If you have defined custom event handlers (loggers) in your configuration you need to change
|
||||
``akka.event-handlers`` to ``akka.loggers`` and
|
||||
``akka.event-handler-startup-timeout`` to ``akka.logger-startup-timeout``.
|
||||
|
||||
The SLF4J logger has been renamed from ``akka.event.slf4j.Slf4jEventHandler`` to
|
||||
``akka.event.slf4j.Slf4jLogger``.
|
||||
|
||||
The ``java.util.logging`` logger has been renamed from ``akka.contrib.jul.JavaLoggingEventHandler`` to
|
||||
``akka.contrib.jul.JavaLogger``.
|
||||
|
|
|
|||
|
|
@ -255,7 +255,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
|
|||
import com.typesafe.config.ConfigFactory
|
||||
|
||||
implicit val system = ActorSystem("testsystem", ConfigFactory.parseString("""
|
||||
akka.event-handlers = ["akka.testkit.TestEventListener"]
|
||||
akka.loggers = ["akka.testkit.TestEventListener"]
|
||||
"""))
|
||||
try {
|
||||
val actor = system.actorOf(Props.empty)
|
||||
|
|
|
|||
|
|
@ -170,7 +170,7 @@ stream for logging: these are the handlers which are configured for example in
|
|||
.. code-block:: text
|
||||
|
||||
akka {
|
||||
event-handlers = ["akka.event.Logging$DefaultLogger"]
|
||||
loggers = ["akka.event.Logging$DefaultLogger"]
|
||||
}
|
||||
|
||||
The handlers listed here by fully-qualified class name will be subscribed to
|
||||
|
|
|
|||
|
|
@ -196,26 +196,27 @@ purposes as it contains exactly the default behavior.
|
|||
to look up the logger instance to use instead of the class’ name), and you
|
||||
might want to do this also in case you implement your own logging adapter.
|
||||
|
||||
Event Handler
|
||||
=============
|
||||
Loggers
|
||||
=======
|
||||
|
||||
Logging is performed asynchronously through an event bus. You can configure
|
||||
which event handlers that should subscribe to the logging events. That is done
|
||||
using the ``event-handlers`` element in the :ref:`configuration`. Here you can
|
||||
which loggers that should subscribe to the logging events. That is done
|
||||
using the ``loggers`` element in the :ref:`configuration`. Here you can
|
||||
also define the log level.
|
||||
|
||||
.. code-block:: ruby
|
||||
|
||||
akka {
|
||||
# Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT)
|
||||
event-handlers = ["akka.event.Logging$DefaultLogger"]
|
||||
# Loggers to register at boot time (akka.event.Logging$DefaultLogger logs
|
||||
# to STDOUT)
|
||||
loggers = ["akka.event.Logging$DefaultLogger"]
|
||||
# Options: ERROR, WARNING, INFO, DEBUG
|
||||
loglevel = "DEBUG"
|
||||
}
|
||||
|
||||
The default one logs to STDOUT and is registered by default. It is not intended
|
||||
to be used for production. There is also an :ref:`slf4j-scala`
|
||||
event handler available in the 'akka-slf4j' module.
|
||||
logger available in the 'akka-slf4j' module.
|
||||
|
||||
Example of creating a listener:
|
||||
|
||||
|
|
@ -227,7 +228,7 @@ Example of creating a listener:
|
|||
SLF4J
|
||||
=====
|
||||
|
||||
Akka provides an event handler for `SL4FJ <http://www.slf4j.org/>`_. This module is available in the 'akka-slf4j.jar'.
|
||||
Akka provides a logger for `SL4FJ <http://www.slf4j.org/>`_. This module is available in the 'akka-slf4j.jar'.
|
||||
It has one single dependency; the slf4j-api jar. In runtime you also need a SLF4J backend, we recommend `Logback <http://logback.qos.ch/>`_:
|
||||
|
||||
.. code-block:: scala
|
||||
|
|
@ -235,7 +236,7 @@ It has one single dependency; the slf4j-api jar. In runtime you also need a SLF4
|
|||
lazy val logback = "ch.qos.logback" % "logback-classic" % "1.0.7"
|
||||
|
||||
|
||||
You need to enable the Slf4jEventHandler in the 'event-handlers' element in
|
||||
You need to enable the Slf4jLogger in the 'loggers' element in
|
||||
the :ref:`configuration`. Here you can also define the log level of the event bus.
|
||||
More fine grained log levels can be defined in the configuration of the SLF4J backend
|
||||
(e.g. logback.xml).
|
||||
|
|
@ -243,7 +244,7 @@ More fine grained log levels can be defined in the configuration of the SLF4J ba
|
|||
.. code-block:: ruby
|
||||
|
||||
akka {
|
||||
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
|
||||
loggers = ["akka.event.slf4j.Slf4jLogger"]
|
||||
loglevel = "DEBUG"
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -337,11 +337,11 @@ test fails.
|
|||
|
||||
.. note::
|
||||
|
||||
Be sure to exchange the default event handler with the
|
||||
Be sure to exchange the default logger with the
|
||||
:class:`TestEventListener` in your ``application.conf`` to enable this
|
||||
function::
|
||||
|
||||
akka.event-handlers = [akka.testkit.TestEventListener]
|
||||
akka.loggers = [akka.testkit.TestEventListener]
|
||||
|
||||
.. _TestKit.within:
|
||||
|
||||
|
|
@ -679,8 +679,7 @@ options:
|
|||
|
||||
The logging feature is coupled to this specific local mark-up because
|
||||
enabling it uniformly on all actors is not usually what you need, and it
|
||||
would lead to endless loops if it were applied to :class:`EventHandler`
|
||||
listeners.
|
||||
would lead to endless loops if it were applied to event bus logger listeners.
|
||||
|
||||
* *Logging of special messages*
|
||||
|
||||
|
|
|
|||
|
|
@ -40,7 +40,7 @@ object DurableMailboxSpecActorFactory {
|
|||
object DurableMailboxSpec {
|
||||
def fallbackConfig: Config = ConfigFactory.parseString("""
|
||||
akka {
|
||||
event-handlers = ["akka.testkit.TestEventListener"]
|
||||
loggers = ["akka.testkit.TestEventListener"]
|
||||
loglevel = "WARNING"
|
||||
stdout-loglevel = "WARNING"
|
||||
}
|
||||
|
|
|
|||
|
|
@ -9,7 +9,7 @@ import akka.event.Logging.{ DefaultLogger, LogEvent }
|
|||
import akka.event.Logging.Error.NoCause
|
||||
|
||||
/**
|
||||
* EventHandler for OSGi environment.
|
||||
* Logger for OSGi environment.
|
||||
* Stands for an interface between akka and the OSGi LogService
|
||||
* It uses the OSGi LogService to log the received LogEvents
|
||||
*/
|
||||
|
|
@ -20,7 +20,7 @@ class DefaultOSGiLogger extends DefaultLogger {
|
|||
override def receive: Receive = uninitialisedReceive orElse super.receive
|
||||
|
||||
/**
|
||||
* Behaviour of the EventHandler that waits for its LogService
|
||||
* Behaviour of the logger that waits for its LogService
|
||||
* @return Receive: Store LogEvent or become initialised
|
||||
*/
|
||||
def uninitialisedReceive: Receive = {
|
||||
|
|
@ -29,7 +29,7 @@ class DefaultOSGiLogger extends DefaultLogger {
|
|||
context.system.eventStream.subscribe(self, classOf[LogService])
|
||||
context.system.eventStream.unsubscribe(self, UnregisteringLogService.getClass)
|
||||
/**
|
||||
* Logs every already received LogEvent and set the EventHandler ready to log every incoming LogEvent.
|
||||
* Logs every already received LogEvent and set the logger ready to log every incoming LogEvent.
|
||||
*
|
||||
* @param logService OSGi LogService that has been registered,
|
||||
*/
|
||||
|
|
@ -78,6 +78,6 @@ class DefaultOSGiLogger extends DefaultLogger {
|
|||
|
||||
/**
|
||||
* Message sent when LogService is unregistred.
|
||||
* Sent from the ActorSystemActivator to an EventHandler (as DefaultOsgiLogger).
|
||||
* Sent from the ActorSystemActivator to a logger (as DefaultOsgiLogger).
|
||||
*/
|
||||
case object UnregisteringLogService
|
||||
|
|
|
|||
|
|
@ -199,7 +199,7 @@ object MultiNodeSpec {
|
|||
|
||||
private[testkit] val baseConfig: Config = ConfigFactory.parseString("""
|
||||
akka {
|
||||
event-handlers = ["akka.testkit.TestEventListener"]
|
||||
loggers = ["akka.testkit.TestEventListener"]
|
||||
loglevel = "WARNING"
|
||||
stdout-loglevel = "WARNING"
|
||||
actor {
|
||||
|
|
|
|||
|
|
@ -1,4 +1,4 @@
|
|||
akka {
|
||||
loglevel = INFO
|
||||
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
|
||||
loggers = ["akka.event.slf4j.Slf4jLogger"]
|
||||
}
|
||||
|
|
@ -1,95 +1,14 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.event.slf4j
|
||||
|
||||
import org.slf4j.{ Logger ⇒ SLFLogger, LoggerFactory ⇒ SLFLoggerFactory }
|
||||
import org.slf4j.MDC
|
||||
import akka.event.Logging._
|
||||
import akka.actor._
|
||||
import akka.event.DummyClassForStringSources
|
||||
import akka.event.Logging.Warning
|
||||
|
||||
/**
|
||||
* Base trait for all classes that wants to be able use the SLF4J logging infrastructure.
|
||||
*/
|
||||
trait SLF4JLogging {
|
||||
@transient
|
||||
lazy val log = Logger(this.getClass.getName)
|
||||
}
|
||||
@deprecated("use akka.event.slf4j.Slf4jLogger)", "2.2")
|
||||
class Slf4jEventHandler extends Slf4jLogger {
|
||||
|
||||
/**
|
||||
* Logger is a factory for obtaining SLF4J-Loggers
|
||||
*/
|
||||
object Logger {
|
||||
/**
|
||||
* @param logger - which logger
|
||||
* @return a Logger that corresponds for the given logger name
|
||||
*/
|
||||
def apply(logger: String): SLFLogger = SLFLoggerFactory getLogger logger
|
||||
|
||||
/**
|
||||
* @param logClass - the class to log for
|
||||
* @param logSource - the textual representation of the source of this log stream
|
||||
* @return a Logger for the specified parameters
|
||||
*/
|
||||
def apply(logClass: Class[_], logSource: String): SLFLogger = logClass match {
|
||||
case c if c == classOf[DummyClassForStringSources] ⇒ apply(logSource)
|
||||
case _ ⇒ SLFLoggerFactory getLogger logClass
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the SLF4J Root Logger
|
||||
*/
|
||||
def root: SLFLogger = apply(SLFLogger.ROOT_LOGGER_NAME)
|
||||
}
|
||||
|
||||
/**
|
||||
* SLF4J Event Handler.
|
||||
*
|
||||
* The thread in which the logging was performed is captured in
|
||||
* Mapped Diagnostic Context (MDC) with attribute name "sourceThread".
|
||||
*/
|
||||
class Slf4jEventHandler extends Actor with SLF4JLogging {
|
||||
|
||||
val mdcThreadAttributeName = "sourceThread"
|
||||
val mdcAkkaSourceAttributeName = "akkaSource"
|
||||
|
||||
def receive = {
|
||||
|
||||
case event @ Error(cause, logSource, logClass, message) ⇒
|
||||
withMdc(logSource, event.thread.getName) {
|
||||
cause match {
|
||||
case Error.NoCause | null ⇒ Logger(logClass, logSource).error(if (message != null) message.toString else null)
|
||||
case cause ⇒ Logger(logClass, logSource).error(if (message != null) message.toString else cause.getLocalizedMessage, cause)
|
||||
}
|
||||
}
|
||||
|
||||
case event @ Warning(logSource, logClass, message) ⇒
|
||||
withMdc(logSource, event.thread.getName) { Logger(logClass, logSource).warn("{}", message.asInstanceOf[AnyRef]) }
|
||||
|
||||
case event @ Info(logSource, logClass, message) ⇒
|
||||
withMdc(logSource, event.thread.getName) { Logger(logClass, logSource).info("{}", message.asInstanceOf[AnyRef]) }
|
||||
|
||||
case event @ Debug(logSource, logClass, message) ⇒
|
||||
withMdc(logSource, event.thread.getName) { Logger(logClass, logSource).debug("{}", message.asInstanceOf[AnyRef]) }
|
||||
|
||||
case InitializeLogger(_) ⇒
|
||||
log.info("Slf4jEventHandler started")
|
||||
sender ! LoggerInitialized
|
||||
}
|
||||
|
||||
@inline
|
||||
final def withMdc(logSource: String, thread: String)(logStatement: ⇒ Unit) {
|
||||
MDC.put(mdcAkkaSourceAttributeName, logSource)
|
||||
MDC.put(mdcThreadAttributeName, thread)
|
||||
try {
|
||||
logStatement
|
||||
} finally {
|
||||
MDC.remove(mdcAkkaSourceAttributeName)
|
||||
MDC.remove(mdcThreadAttributeName)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
self ! Warning(getClass.getName, getClass,
|
||||
s"[${getClass.getName}] is depreceated, use [${classOf[Slf4jLogger].getName}] instead")
|
||||
|
||||
}
|
||||
95
akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jLogger.scala
Normal file
95
akka-slf4j/src/main/scala/akka/event/slf4j/Slf4jLogger.scala
Normal file
|
|
@ -0,0 +1,95 @@
|
|||
/**
|
||||
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
|
||||
*/
|
||||
|
||||
package akka.event.slf4j
|
||||
|
||||
import org.slf4j.{ Logger ⇒ SLFLogger, LoggerFactory ⇒ SLFLoggerFactory }
|
||||
import org.slf4j.MDC
|
||||
import akka.event.Logging._
|
||||
import akka.actor._
|
||||
import akka.event.DummyClassForStringSources
|
||||
|
||||
/**
|
||||
* Base trait for all classes that wants to be able use the SLF4J logging infrastructure.
|
||||
*/
|
||||
trait SLF4JLogging {
|
||||
@transient
|
||||
lazy val log = Logger(this.getClass.getName)
|
||||
}
|
||||
|
||||
/**
|
||||
* Logger is a factory for obtaining SLF4J-Loggers
|
||||
*/
|
||||
object Logger {
|
||||
/**
|
||||
* @param logger - which logger
|
||||
* @return a Logger that corresponds for the given logger name
|
||||
*/
|
||||
def apply(logger: String): SLFLogger = SLFLoggerFactory getLogger logger
|
||||
|
||||
/**
|
||||
* @param logClass - the class to log for
|
||||
* @param logSource - the textual representation of the source of this log stream
|
||||
* @return a Logger for the specified parameters
|
||||
*/
|
||||
def apply(logClass: Class[_], logSource: String): SLFLogger = logClass match {
|
||||
case c if c == classOf[DummyClassForStringSources] ⇒ apply(logSource)
|
||||
case _ ⇒ SLFLoggerFactory getLogger logClass
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the SLF4J Root Logger
|
||||
*/
|
||||
def root: SLFLogger = apply(SLFLogger.ROOT_LOGGER_NAME)
|
||||
}
|
||||
|
||||
/**
|
||||
* SLF4J logger.
|
||||
*
|
||||
* The thread in which the logging was performed is captured in
|
||||
* Mapped Diagnostic Context (MDC) with attribute name "sourceThread".
|
||||
*/
|
||||
class Slf4jLogger extends Actor with SLF4JLogging {
|
||||
|
||||
val mdcThreadAttributeName = "sourceThread"
|
||||
val mdcAkkaSourceAttributeName = "akkaSource"
|
||||
|
||||
def receive = {
|
||||
|
||||
case event @ Error(cause, logSource, logClass, message) ⇒
|
||||
withMdc(logSource, event.thread.getName) {
|
||||
cause match {
|
||||
case Error.NoCause | null ⇒ Logger(logClass, logSource).error(if (message != null) message.toString else null)
|
||||
case cause ⇒ Logger(logClass, logSource).error(if (message != null) message.toString else cause.getLocalizedMessage, cause)
|
||||
}
|
||||
}
|
||||
|
||||
case event @ Warning(logSource, logClass, message) ⇒
|
||||
withMdc(logSource, event.thread.getName) { Logger(logClass, logSource).warn("{}", message.asInstanceOf[AnyRef]) }
|
||||
|
||||
case event @ Info(logSource, logClass, message) ⇒
|
||||
withMdc(logSource, event.thread.getName) { Logger(logClass, logSource).info("{}", message.asInstanceOf[AnyRef]) }
|
||||
|
||||
case event @ Debug(logSource, logClass, message) ⇒
|
||||
withMdc(logSource, event.thread.getName) { Logger(logClass, logSource).debug("{}", message.asInstanceOf[AnyRef]) }
|
||||
|
||||
case InitializeLogger(_) ⇒
|
||||
log.info("Slf4jLogger started")
|
||||
sender ! LoggerInitialized
|
||||
}
|
||||
|
||||
@inline
|
||||
final def withMdc(logSource: String, thread: String)(logStatement: ⇒ Unit) {
|
||||
MDC.put(mdcAkkaSourceAttributeName, logSource)
|
||||
MDC.put(mdcThreadAttributeName, thread)
|
||||
try {
|
||||
logStatement
|
||||
} finally {
|
||||
MDC.remove(mdcAkkaSourceAttributeName)
|
||||
MDC.remove(mdcThreadAttributeName)
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -9,13 +9,13 @@
|
|||
</appender>
|
||||
|
||||
<appender name="TEST"
|
||||
class="akka.event.slf4j.Slf4jEventHandlerSpec$TestAppender">
|
||||
class="akka.event.slf4j.Slf4jLoggerSpec$TestAppender">
|
||||
<encoder>
|
||||
<pattern>%date{ISO8601} level=[%level] logger=[%logger] akkaSource=[%X{akkaSource}] sourceThread=[%X{sourceThread}] - msg=[%msg]%n----%n</pattern>
|
||||
</encoder>
|
||||
</appender>
|
||||
|
||||
<logger name="akka.event.slf4j.Slf4jEventHandlerSpec" level="info" additivity="false">
|
||||
<logger name="akka.event.slf4j.Slf4jLoggerSpec" level="info" additivity="false">
|
||||
<appender-ref ref="TEST" />
|
||||
</logger>
|
||||
|
||||
|
|
|
|||
|
|
@ -16,15 +16,15 @@ import java.io.StringWriter
|
|||
import java.io.ByteArrayOutputStream
|
||||
import org.scalatest.BeforeAndAfterEach
|
||||
|
||||
object Slf4jEventHandlerSpec {
|
||||
object Slf4jLoggerSpec {
|
||||
|
||||
// This test depends on logback configuration in src/test/resources/logback-test.xml
|
||||
|
||||
val config = """
|
||||
akka {
|
||||
loglevel = INFO
|
||||
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"]
|
||||
event-handler-startup-timeout = 30s
|
||||
loggers = ["akka.event.slf4j.Slf4jLogger"]
|
||||
logger-startup-timeout = 30s
|
||||
}
|
||||
"""
|
||||
|
||||
|
|
@ -53,8 +53,8 @@ object Slf4jEventHandlerSpec {
|
|||
}
|
||||
|
||||
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
|
||||
class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with BeforeAndAfterEach {
|
||||
import Slf4jEventHandlerSpec._
|
||||
class Slf4jLoggerSpec extends AkkaSpec(Slf4jLoggerSpec.config) with BeforeAndAfterEach {
|
||||
import Slf4jLoggerSpec._
|
||||
|
||||
val producer = system.actorOf(Props[LogProducer], name = "logProducer")
|
||||
|
||||
|
|
@ -62,22 +62,22 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with
|
|||
output.reset()
|
||||
}
|
||||
|
||||
val sourceThreadRegex = "sourceThread=\\[Slf4jEventHandlerSpec-akka.actor.default-dispatcher-[1-9][0-9]*\\]"
|
||||
val sourceThreadRegex = "sourceThread=\\[Slf4jLoggerSpec-akka.actor.default-dispatcher-[1-9][0-9]*\\]"
|
||||
|
||||
"Slf4jEventHandler" must {
|
||||
"Slf4jLogger" must {
|
||||
|
||||
"log error with stackTrace" in {
|
||||
producer ! new RuntimeException("Simulated error")
|
||||
|
||||
awaitCond(outputString.contains("----"), 5 seconds)
|
||||
val s = outputString
|
||||
s must include("akkaSource=[akka://Slf4jEventHandlerSpec/user/logProducer]")
|
||||
s must include("akkaSource=[akka://Slf4jLoggerSpec/user/logProducer]")
|
||||
s must include("level=[ERROR]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$LogProducer]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jLoggerSpec$LogProducer]")
|
||||
s must include regex (sourceThreadRegex)
|
||||
s must include("msg=[Simulated error]")
|
||||
s must include("java.lang.RuntimeException: Simulated error")
|
||||
s must include("at akka.event.slf4j.Slf4jEventHandlerSpec")
|
||||
s must include("at akka.event.slf4j.Slf4jLoggerSpec")
|
||||
}
|
||||
|
||||
"log info with parameters" in {
|
||||
|
|
@ -85,29 +85,29 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with
|
|||
|
||||
awaitCond(outputString.contains("----"), 5 seconds)
|
||||
val s = outputString
|
||||
s must include("akkaSource=[akka://Slf4jEventHandlerSpec/user/logProducer]")
|
||||
s must include("akkaSource=[akka://Slf4jLoggerSpec/user/logProducer]")
|
||||
s must include("level=[INFO]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$LogProducer]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jLoggerSpec$LogProducer]")
|
||||
s must include regex (sourceThreadRegex)
|
||||
s must include("msg=[test x=3 y=17]")
|
||||
}
|
||||
|
||||
"include system info in akkaSource when creating Logging with system" in {
|
||||
val log = Logging(system, "akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource")
|
||||
val log = Logging(system, "akka.event.slf4j.Slf4jLoggerSpec.MyLogSource")
|
||||
log.info("test")
|
||||
awaitCond(outputString.contains("----"), 5 seconds)
|
||||
val s = outputString
|
||||
s must include("akkaSource=[akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource(akka://Slf4jEventHandlerSpec)]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource(akka://Slf4jEventHandlerSpec)]")
|
||||
s must include("akkaSource=[akka.event.slf4j.Slf4jLoggerSpec.MyLogSource(akka://Slf4jLoggerSpec)]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jLoggerSpec.MyLogSource(akka://Slf4jLoggerSpec)]")
|
||||
}
|
||||
|
||||
"not include system info in akkaSource when creating Logging with system.eventStream" in {
|
||||
val log = Logging(system.eventStream, "akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource")
|
||||
val log = Logging(system.eventStream, "akka.event.slf4j.Slf4jLoggerSpec.MyLogSource")
|
||||
log.info("test")
|
||||
awaitCond(outputString.contains("----"), 5 seconds)
|
||||
val s = outputString
|
||||
s must include("akkaSource=[akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource]")
|
||||
s must include("akkaSource=[akka.event.slf4j.Slf4jLoggerSpec.MyLogSource]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jLoggerSpec.MyLogSource]")
|
||||
}
|
||||
|
||||
"use short class name and include system info in akkaSource when creating Logging with system and class" in {
|
||||
|
|
@ -115,8 +115,8 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with
|
|||
log.info("test")
|
||||
awaitCond(outputString.contains("----"), 5 seconds)
|
||||
val s = outputString
|
||||
s must include("akkaSource=[Slf4jEventHandlerSpec$MyLogSource(akka://Slf4jEventHandlerSpec)]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$MyLogSource]")
|
||||
s must include("akkaSource=[Slf4jLoggerSpec$MyLogSource(akka://Slf4jLoggerSpec)]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jLoggerSpec$MyLogSource]")
|
||||
}
|
||||
|
||||
"use short class name in akkaSource when creating Logging with system.eventStream and class" in {
|
||||
|
|
@ -124,8 +124,8 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with
|
|||
log.info("test")
|
||||
awaitCond(outputString.contains("----"), 5 seconds)
|
||||
val s = outputString
|
||||
s must include("akkaSource=[Slf4jEventHandlerSpec$MyLogSource]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$MyLogSource]")
|
||||
s must include("akkaSource=[Slf4jLoggerSpec$MyLogSource]")
|
||||
s must include("logger=[akka.event.slf4j.Slf4jLoggerSpec$MyLogSource]")
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -453,7 +453,7 @@ case class CustomEventFilter(test: PartialFunction[LogEvent, Boolean])(occurrenc
|
|||
*
|
||||
* <pre><code>
|
||||
* akka {
|
||||
* event-handlers = ["akka.testkit.TestEventListener"]
|
||||
* loggers = ["akka.testkit.TestEventListener"]
|
||||
* }
|
||||
* </code></pre>
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -19,7 +19,7 @@ import akka.pattern.ask
|
|||
object AkkaSpec {
|
||||
val testConf: Config = ConfigFactory.parseString("""
|
||||
akka {
|
||||
event-handlers = ["akka.testkit.TestEventListener"]
|
||||
loggers = ["akka.testkit.TestEventListener"]
|
||||
loglevel = "WARNING"
|
||||
stdout-loglevel = "WARNING"
|
||||
actor {
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue