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:
Patrik Nordwall 2013-02-01 08:02:53 +01:00
parent c6f08fb935
commit 2476831705
37 changed files with 347 additions and 216 deletions

View file

@ -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 { "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() implicit val dispatcher = interceptedDispatcher()
val a = newTestActor(dispatcher.id) val a = newTestActor(dispatcher.id)
val f1 = a ? Reply("foo") val f1 = a ? Reply("foo")

View file

@ -5,12 +5,12 @@
package akka.config package akka.config
import language.postfixOps import language.postfixOps
import akka.testkit.AkkaSpec import akka.testkit.AkkaSpec
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.concurrent.duration._ import scala.concurrent.duration._
import akka.actor.{ IOManager, ActorSystem } import akka.actor.{ IOManager, ActorSystem }
import akka.event.Logging.DefaultLogger
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class ConfigSpec extends AkkaSpec(ConfigFactory.defaultReference(ActorSystem.findClassLoader())) { 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) getMilliseconds("akka.actor.unstarted-push-timeout") must be(10.seconds.toMillis)
settings.UnstartedPushTimeout.duration must be(10.seconds) 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)
} }
{ {

View file

@ -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"))
}
}
}
}

View file

@ -19,7 +19,7 @@ object EventStreamSpec {
akka { akka {
stdout-loglevel = WARNING stdout-loglevel = WARNING
loglevel = INFO loglevel = INFO
event-handlers = ["akka.event.EventStreamSpec$MyLog", "%s"] loggers = ["akka.event.EventStreamSpec$MyLog", "%s"]
} }
""".format(Logging.StandardOutLogger.getClass.getName)) """.format(Logging.StandardOutLogger.getClass.getName))

View file

@ -4,7 +4,7 @@ import com.typesafe.config.ConfigFactory
object BenchmarkConfig { object BenchmarkConfig {
private val benchmarkConfig = ConfigFactory.parseString(""" private val benchmarkConfig = ConfigFactory.parseString("""
akka { akka {
event-handlers = ["akka.testkit.TestEventListener"] loggers = ["akka.testkit.TestEventListener"]
loglevel = "WARNING" loglevel = "WARNING"
} }

View file

@ -12,15 +12,22 @@ akka {
# Home directory of Akka, modules in the deploy directory will be loaded # Home directory of Akka, modules in the deploy directory will be loaded
home = "" home = ""
# Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT) # Loggers to register at boot time (akka.event.Logging$DefaultLogger logs
event-handlers = ["akka.event.Logging$DefaultLogger"] # 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 # start-up, and since they are actors, this timeout is used to bound the
# waiting time # 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" # as they have been started; before that, see "stdout-loglevel"
# Options: ERROR, WARNING, INFO, DEBUG # Options: ERROR, WARNING, INFO, DEBUG
loglevel = "INFO" loglevel = "INFO"

View file

@ -143,7 +143,11 @@ object ActorSystem {
final val LogLevel: String = getString("akka.loglevel") final val LogLevel: String = getString("akka.loglevel")
final val StdoutLogLevel: String = getString("akka.stdout-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")) 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 EventHandlerStartTimeout: Timeout = Timeout(Duration(getMilliseconds("akka.event-handler-startup-timeout"), MILLISECONDS))
final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start") final val LogConfigOnStart: Boolean = config.getBoolean("akka.log-config-on-start")

View file

@ -72,7 +72,7 @@ trait LoggingBus extends ActorEventBus {
*/ */
private[akka] def startStdoutLogger(config: Settings) { private[akka] def startStdoutLogger(config: Settings) {
val level = levelFor(config.StdoutLogLevel) getOrElse { 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 ErrorLevel
} }
AllLogLevels filter (level >= _) foreach (l subscribe(StandardOutLogger, classFor(l))) AllLogLevels filter (level >= _) foreach (l subscribe(StandardOutLogger, classFor(l)))
@ -89,14 +89,19 @@ trait LoggingBus extends ActorEventBus {
private[akka] def startDefaultLoggers(system: ActorSystemImpl) { private[akka] def startDefaultLoggers(system: ActorSystemImpl) {
val logName = simpleName(this) + "(" + system + ")" val logName = simpleName(this) + "(" + system + ")"
val level = levelFor(system.settings.LogLevel) getOrElse { 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 ErrorLevel
} }
try { try {
val defaultLoggers = system.settings.EventHandlers match { val defaultLoggers = system.settings.EventHandlers match {
case Nil "akka.event.Logging$DefaultLogger" :: Nil case Nil system.settings.Loggers match {
case Nil classOf[DefaultLogger].getName :: Nil
case loggers loggers case loggers loggers
} }
case loggers
StandardOutLogger.print(Warning(logName, this.getClass, "[akka.event-handlers] config is deprecated, use [akka.loggers]"))
loggers
}
val myloggers = val myloggers =
for { for {
loggerName defaultLoggers loggerName defaultLoggers
@ -106,7 +111,7 @@ trait LoggingBus extends ActorEventBus {
case actorClass addLogger(system, actorClass, level, logName) case actorClass addLogger(system, actorClass, level, logName)
}).recover({ }).recover({
case e throw new ConfigurationException( 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) "] due to [" + e.toString + "]", e)
}).get }).get
} }
@ -166,7 +171,13 @@ trait LoggingBus extends ActorEventBus {
private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel, logName: String): ActorRef = { private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel, logName: String): ActorRef = {
val name = "log" + Extension(system).id() + "-" + simpleName(clazz) val name = "log" + Extension(system).id() + "-" + simpleName(clazz)
val actor = system.systemActorOf(Props(clazz), name) 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 import akka.pattern.ask
val response = try Await.result(actor ? InitializeLogger(this), timeout.duration) catch { val response = try Await.result(actor ? InitializeLogger(this), timeout.duration) catch {
case _: TimeoutException case _: TimeoutException
@ -353,8 +364,8 @@ object LogSource {
* *
* <pre><code> * <pre><code>
* akka { * akka {
* event-handlers = ["akka.slf4j.Slf4jEventHandler"] # for example * loggers = ["akka.slf4j.Slf4jLogger"] # for example
* loglevel = "INFO" # used when normal logging ("event-handlers") has been started * loglevel = "INFO" # used when normal logging ("loggers") has been started
* stdout-loglevel = "WARN" # used during application start-up until normal logging is available * stdout-loglevel = "WARN" # used during application start-up until normal logging is available
* } * }
* </code></pre> * </code></pre>
@ -531,7 +542,7 @@ object Logging {
* Artificial exception injected into Error events if no Throwable is * Artificial exception injected into Error events if no Throwable is
* supplied; used for getting a stack dump of error locations. * supplied; used for getting a stack dump of error locations.
*/ */
class EventHandlerException extends AkkaException("") class LoggerException extends AkkaException("")
/** /**
* Exception that wraps a LogEvent. * Exception that wraps a LogEvent.
@ -716,7 +727,7 @@ object Logging {
/** /**
* Actor wrapper around the standard output logger. If * 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. * logger.
*/ */
class DefaultLogger extends Actor with StdOutLogger { class DefaultLogger extends Actor with StdOutLogger {

View file

@ -43,7 +43,7 @@ object MultiNodeClusterSpec {
} }
akka.loglevel = INFO akka.loglevel = INFO
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
akka.event-handlers = ["akka.testkit.TestEventListener"] akka.loggers = ["akka.testkit.TestEventListener"]
akka.test { akka.test {
single-expect-default = 5 s single-expect-default = 5 s
} }

View file

@ -101,7 +101,7 @@ object StressMultiJvmSpec extends MultiNodeConfig {
auto-down = on auto-down = on
publish-stats-interval = 0 s # always, when it happens publish-stats-interval = 0 s # always, when it happens
} }
akka.event-handlers = ["akka.testkit.TestEventListener"] akka.loggers = ["akka.testkit.TestEventListener"]
akka.loglevel = INFO akka.loglevel = INFO
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off

View file

@ -28,7 +28,7 @@ object SunnyWeatherMultiJvmSpec extends MultiNodeConfig {
akka.cluster { akka.cluster {
auto-join = off auto-join = off
} }
akka.event-handlers = ["akka.testkit.TestEventListener"] akka.loggers = ["akka.testkit.TestEventListener"]
akka.loglevel = INFO akka.loglevel = INFO
akka.remote.log-remote-lifecycle-events = off akka.remote.log-remote-lifecycle-events = off
""")) """))

View file

@ -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: 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`. To access the `akka.event.Logging` API from non-Actor code, mix in `akka.contrib.jul.JavaLogging`.

View file

@ -1,3 +1,6 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.contrib.jul package akka.contrib.jul
import akka.event.Logging._ 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 = { def receive = {
case event @ Error(cause, logSource, logClass, message) case event @ Error(cause, _, _, _) log(logging.Level.SEVERE, cause, event)
log(logging.Level.SEVERE, cause, logSource, logClass, message, event) case event: Warning log(logging.Level.WARNING, null, event)
case event: Info log(logging.Level.INFO, null, event)
case event @ Warning(logSource, logClass, message) case event: Debug log(logging.Level.CONFIG, null, event)
log(logging.Level.WARNING, null, logSource, logClass, message, event) case InitializeLogger(_) sender ! LoggerInitialized
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
} }
@inline @inline
def log(level: logging.Level, cause: Throwable, logSource: String, logClass: Class[_], message: Any, event: LogEvent) { def log(level: logging.Level, cause: Throwable, event: LogEvent) {
val logger = logging.Logger.getLogger(logSource) val logger = logging.Logger.getLogger(event.logSource)
val record = new logging.LogRecord(level, message.toString) val record = new logging.LogRecord(level, String.valueOf(event.message))
record.setLoggerName(logger.getName) record.setLoggerName(logger.getName)
record.setThrown(cause) record.setThrown(cause)
record.setThreadID(event.thread.getId.toInt) record.setThreadID(event.thread.getId.toInt)
record.setSourceClassName(logClass.getName) record.setSourceClassName(event.logClass.getName)
record.setSourceMethodName(null) // lost forever record.setSourceMethodName(null) // lost forever
logger.log(record) logger.log(record)
} }

View file

@ -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")
}

View file

@ -1,3 +1,6 @@
/**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/
package akka.contrib.mailbox package akka.contrib.mailbox
import java.util.concurrent.{ ConcurrentHashMap, ConcurrentLinkedQueue } import java.util.concurrent.{ ConcurrentHashMap, ConcurrentLinkedQueue }
@ -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 // this logic does not work if maxRetries==0, but then you could also use a normal mailbox
override def dequeue(): Envelope = tries match { override def dequeue(): Envelope = tries match {
case -1 queue.poll() case -1
case 0 | Marker val e = queue.peek(); tries = if (e eq null) 0 else 1; e queue.poll()
case `maxRetries` tries = Marker; queue.poll() case 0 | Marker
case n tries = n + 1; queue.peek() 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 = { def ack(): Unit = {

View file

@ -6,12 +6,12 @@ import akka.testkit.AkkaSpec
import java.util.logging import java.util.logging
import java.io.ByteArrayInputStream import java.io.ByteArrayInputStream
object JavaLoggingEventHandlerSpec { object JavaLoggerSpec {
val config = ConfigFactory.parseString(""" val config = ConfigFactory.parseString("""
akka { akka {
loglevel = INFO loglevel = INFO
event-handlers = ["akka.contrib.jul.JavaLoggingEventHandler"] loggers = ["akka.contrib.jul.JavaLogger"]
}""") }""")
class LogProducer extends Actor with ActorLogging { class LogProducer extends Actor with ActorLogging {
@ -25,9 +25,9 @@ object JavaLoggingEventHandlerSpec {
} }
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @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.setUseParentHandlers(false) // turn off output of test LogRecords
logger.addHandler(new logging.Handler { logger.addHandler(new logging.Handler {
def publish(record: logging.LogRecord) { def publish(record: logging.LogRecord) {
@ -38,9 +38,9 @@ class JavaLoggingEventHandlerSpec extends AkkaSpec(JavaLoggingEventHandlerSpec.c
def close() {} 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 { "log error with stackTrace" in {
producer ! new RuntimeException("Simulated error") producer ! new RuntimeException("Simulated error")
@ -53,7 +53,7 @@ class JavaLoggingEventHandlerSpec extends AkkaSpec(JavaLoggingEventHandlerSpec.c
record.getLevel must be(logging.Level.SEVERE) record.getLevel must be(logging.Level.SEVERE)
record.getMessage must be("Simulated error") record.getMessage must be("Simulated error")
record.getThrown.isInstanceOf[RuntimeException] must be(true) 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) record.getSourceMethodName must be(null)
} }
@ -68,7 +68,7 @@ class JavaLoggingEventHandlerSpec extends AkkaSpec(JavaLoggingEventHandlerSpec.c
record.getLevel must be(logging.Level.INFO) record.getLevel must be(logging.Level.INFO)
record.getMessage must be("3 is the magic number") record.getMessage must be("3 is the magic number")
record.getThrown must be(null) 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) record.getSourceMethodName must be(null)
} }
} }

View file

@ -78,10 +78,11 @@ A custom ``application.conf`` might look like this::
akka { akka {
# Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT) # Loggers to register at boot time (akka.event.Logging$DefaultLogger logs
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] # 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" # as they have been started; before that, see "stdout-loglevel"
# Options: ERROR, WARNING, INFO, DEBUG # Options: ERROR, WARNING, INFO, DEBUG
loglevel = DEBUG loglevel = DEBUG

View file

@ -39,7 +39,7 @@ Akka is very modular and consists of several JARs containing different features.
- ``akka-agent`` -- Agents, integrated with Scala STM - ``akka-agent`` -- Agents, integrated with Scala STM
- ``akka-camel`` -- Apache Camel integration - ``akka-camel`` -- Apache Camel integration
- ``akka-zeromq`` -- ZeroMQ 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) - ``akka-filebased-mailbox`` -- Akka durable mailbox (find more among community projects)
The filename of the actual JAR is for example ``@jarName@`` (and analog for The filename of the actual JAR is for example ``@jarName@`` (and analog for

View file

@ -50,7 +50,7 @@ public class TestKitDocTest {
@BeforeClass @BeforeClass
public static void setup() { public static void setup() {
final Config config = ConfigFactory.parseString( final Config config = ConfigFactory.parseString(
"akka.event-handlers = [akka.testkit.TestEventListener]"); "akka.loggers = [akka.testkit.TestEventListener]");
system = ActorSystem.create("demoSystem", config); system = ActorSystem.create("demoSystem", config);
} }

View file

@ -175,7 +175,7 @@ stream for logging: these are the handlers which are configured for example in
.. code-block:: text .. code-block:: text
akka { 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 The handlers listed here by fully-qualified class name will be subscribed to

View file

@ -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`. 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 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 'event-handlers' element in the :ref:`configuration`. subscribe to the logging events. That is done using the 'loggers' element in the :ref:`configuration`.
Here you can also define the log level. Here you can also define the log level.
.. code-block:: ruby .. code-block:: ruby
akka { akka {
# Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT) # Loggers to register at boot time (akka.event.Logging$DefaultLogger logs
event-handlers = ["akka.event.Logging$DefaultLogger"] # to STDOUT)
loggers = ["akka.event.Logging$DefaultLogger"]
# Options: ERROR, WARNING, INFO, DEBUG # Options: ERROR, WARNING, INFO, DEBUG
loglevel = "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` 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: Example of creating a listener:
@ -186,7 +187,7 @@ Example of creating a listener:
SLF4J 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/>`_: 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 .. 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> <version>1.0.7</version>
</dependency> </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. 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 More fine grained log levels can be defined in the configuration of the SLF4J backend
(e.g. logback.xml). (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 .. code-block:: ruby
akka { akka {
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] loggers = ["akka.event.slf4j.Slf4jLogger"]
loglevel = "DEBUG" loglevel = "DEBUG"
} }

View file

@ -305,11 +305,11 @@ fails.
.. note:: .. 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 :class:`TestEventListener` in your ``application.conf`` to enable this
function:: function::
akka.event-handlers = [akka.testkit.TestEventListener] akka.loggers = [akka.testkit.TestEventListener]
.. _JavaTestKit.within: .. _JavaTestKit.within:

View file

@ -98,3 +98,17 @@ Old Scala API New Scala API
``agent.resume()`` ``No replacement, pointless feature`` ``agent.resume()`` ``No replacement, pointless feature``
``agent.close()`` ``No replacement, not needed in new implementation`` ``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``.

View file

@ -255,7 +255,7 @@ class TestkitDocSpec extends AkkaSpec with DefaultTimeout with ImplicitSender {
import com.typesafe.config.ConfigFactory import com.typesafe.config.ConfigFactory
implicit val system = ActorSystem("testsystem", ConfigFactory.parseString(""" implicit val system = ActorSystem("testsystem", ConfigFactory.parseString("""
akka.event-handlers = ["akka.testkit.TestEventListener"] akka.loggers = ["akka.testkit.TestEventListener"]
""")) """))
try { try {
val actor = system.actorOf(Props.empty) val actor = system.actorOf(Props.empty)

View file

@ -170,7 +170,7 @@ stream for logging: these are the handlers which are configured for example in
.. code-block:: text .. code-block:: text
akka { 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 The handlers listed here by fully-qualified class name will be subscribed to

View file

@ -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 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. 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 Logging is performed asynchronously through an event bus. You can configure
which event handlers that should subscribe to the logging events. That is done which loggers that should subscribe to the logging events. That is done
using the ``event-handlers`` element in the :ref:`configuration`. Here you can using the ``loggers`` element in the :ref:`configuration`. Here you can
also define the log level. also define the log level.
.. code-block:: ruby .. code-block:: ruby
akka { akka {
# Event handlers to register at boot time (Logging$DefaultLogger logs to STDOUT) # Loggers to register at boot time (akka.event.Logging$DefaultLogger logs
event-handlers = ["akka.event.Logging$DefaultLogger"] # to STDOUT)
loggers = ["akka.event.Logging$DefaultLogger"]
# Options: ERROR, WARNING, INFO, DEBUG # Options: ERROR, WARNING, INFO, DEBUG
loglevel = "DEBUG" loglevel = "DEBUG"
} }
The default one logs to STDOUT and is registered by default. It is not intended 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` 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: Example of creating a listener:
@ -227,7 +228,7 @@ Example of creating a listener:
SLF4J 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/>`_: 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 .. 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" 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. 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 More fine grained log levels can be defined in the configuration of the SLF4J backend
(e.g. logback.xml). (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 .. code-block:: ruby
akka { akka {
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] loggers = ["akka.event.slf4j.Slf4jLogger"]
loglevel = "DEBUG" loglevel = "DEBUG"
} }

View file

@ -337,11 +337,11 @@ test fails.
.. note:: .. 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 :class:`TestEventListener` in your ``application.conf`` to enable this
function:: function::
akka.event-handlers = [akka.testkit.TestEventListener] akka.loggers = [akka.testkit.TestEventListener]
.. _TestKit.within: .. _TestKit.within:
@ -679,8 +679,7 @@ options:
The logging feature is coupled to this specific local mark-up because 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 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` would lead to endless loops if it were applied to event bus logger listeners.
listeners.
* *Logging of special messages* * *Logging of special messages*

View file

@ -40,7 +40,7 @@ object DurableMailboxSpecActorFactory {
object DurableMailboxSpec { object DurableMailboxSpec {
def fallbackConfig: Config = ConfigFactory.parseString(""" def fallbackConfig: Config = ConfigFactory.parseString("""
akka { akka {
event-handlers = ["akka.testkit.TestEventListener"] loggers = ["akka.testkit.TestEventListener"]
loglevel = "WARNING" loglevel = "WARNING"
stdout-loglevel = "WARNING" stdout-loglevel = "WARNING"
} }

View file

@ -9,7 +9,7 @@ import akka.event.Logging.{ DefaultLogger, LogEvent }
import akka.event.Logging.Error.NoCause import akka.event.Logging.Error.NoCause
/** /**
* EventHandler for OSGi environment. * Logger for OSGi environment.
* Stands for an interface between akka and the OSGi LogService * Stands for an interface between akka and the OSGi LogService
* It uses the OSGi LogService to log the received LogEvents * 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 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 * @return Receive: Store LogEvent or become initialised
*/ */
def uninitialisedReceive: Receive = { def uninitialisedReceive: Receive = {
@ -29,7 +29,7 @@ class DefaultOSGiLogger extends DefaultLogger {
context.system.eventStream.subscribe(self, classOf[LogService]) context.system.eventStream.subscribe(self, classOf[LogService])
context.system.eventStream.unsubscribe(self, UnregisteringLogService.getClass) 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, * @param logService OSGi LogService that has been registered,
*/ */
@ -78,6 +78,6 @@ class DefaultOSGiLogger extends DefaultLogger {
/** /**
* Message sent when LogService is unregistred. * 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 case object UnregisteringLogService

View file

@ -199,7 +199,7 @@ object MultiNodeSpec {
private[testkit] val baseConfig: Config = ConfigFactory.parseString(""" private[testkit] val baseConfig: Config = ConfigFactory.parseString("""
akka { akka {
event-handlers = ["akka.testkit.TestEventListener"] loggers = ["akka.testkit.TestEventListener"]
loglevel = "WARNING" loglevel = "WARNING"
stdout-loglevel = "WARNING" stdout-loglevel = "WARNING"
actor { actor {

View file

@ -1,4 +1,4 @@
akka { akka {
loglevel = INFO loglevel = INFO
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] loggers = ["akka.event.slf4j.Slf4jLogger"]
} }

View file

@ -1,95 +1,14 @@
/** /**
* Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com> * Copyright (C) 2009-2013 Typesafe Inc. <http://www.typesafe.com>
*/ */
package akka.event.slf4j package akka.event.slf4j
import org.slf4j.{ Logger SLFLogger, LoggerFactory SLFLoggerFactory } import akka.event.Logging.Warning
import org.slf4j.MDC
import akka.event.Logging._
import akka.actor._
import akka.event.DummyClassForStringSources
/** @deprecated("use akka.event.slf4j.Slf4jLogger)", "2.2")
* Base trait for all classes that wants to be able use the SLF4J logging infrastructure. class Slf4jEventHandler extends Slf4jLogger {
*/
trait SLF4JLogging {
@transient
lazy val log = Logger(this.getClass.getName)
}
/** self ! Warning(getClass.getName, getClass,
* Logger is a factory for obtaining SLF4J-Loggers s"[${getClass.getName}] is depreceated, use [${classOf[Slf4jLogger].getName}] instead")
*/
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)
}
}
} }

View 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)
}
}
}

View file

@ -9,13 +9,13 @@
</appender> </appender>
<appender name="TEST" <appender name="TEST"
class="akka.event.slf4j.Slf4jEventHandlerSpec$TestAppender"> class="akka.event.slf4j.Slf4jLoggerSpec$TestAppender">
<encoder> <encoder>
<pattern>%date{ISO8601} level=[%level] logger=[%logger] akkaSource=[%X{akkaSource}] sourceThread=[%X{sourceThread}] - msg=[%msg]%n----%n</pattern> <pattern>%date{ISO8601} level=[%level] logger=[%logger] akkaSource=[%X{akkaSource}] sourceThread=[%X{sourceThread}] - msg=[%msg]%n----%n</pattern>
</encoder> </encoder>
</appender> </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" /> <appender-ref ref="TEST" />
</logger> </logger>

View file

@ -16,15 +16,15 @@ import java.io.StringWriter
import java.io.ByteArrayOutputStream import java.io.ByteArrayOutputStream
import org.scalatest.BeforeAndAfterEach import org.scalatest.BeforeAndAfterEach
object Slf4jEventHandlerSpec { object Slf4jLoggerSpec {
// This test depends on logback configuration in src/test/resources/logback-test.xml // This test depends on logback configuration in src/test/resources/logback-test.xml
val config = """ val config = """
akka { akka {
loglevel = INFO loglevel = INFO
event-handlers = ["akka.event.slf4j.Slf4jEventHandler"] loggers = ["akka.event.slf4j.Slf4jLogger"]
event-handler-startup-timeout = 30s logger-startup-timeout = 30s
} }
""" """
@ -53,8 +53,8 @@ object Slf4jEventHandlerSpec {
} }
@org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner]) @org.junit.runner.RunWith(classOf[org.scalatest.junit.JUnitRunner])
class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with BeforeAndAfterEach { class Slf4jLoggerSpec extends AkkaSpec(Slf4jLoggerSpec.config) with BeforeAndAfterEach {
import Slf4jEventHandlerSpec._ import Slf4jLoggerSpec._
val producer = system.actorOf(Props[LogProducer], name = "logProducer") val producer = system.actorOf(Props[LogProducer], name = "logProducer")
@ -62,22 +62,22 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with
output.reset() 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 { "log error with stackTrace" in {
producer ! new RuntimeException("Simulated error") producer ! new RuntimeException("Simulated error")
awaitCond(outputString.contains("----"), 5 seconds) awaitCond(outputString.contains("----"), 5 seconds)
val s = outputString 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("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 regex (sourceThreadRegex)
s must include("msg=[Simulated error]") s must include("msg=[Simulated error]")
s must include("java.lang.RuntimeException: 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 { "log info with parameters" in {
@ -85,29 +85,29 @@ class Slf4jEventHandlerSpec extends AkkaSpec(Slf4jEventHandlerSpec.config) with
awaitCond(outputString.contains("----"), 5 seconds) awaitCond(outputString.contains("----"), 5 seconds)
val s = outputString 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("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 regex (sourceThreadRegex)
s must include("msg=[test x=3 y=17]") s must include("msg=[test x=3 y=17]")
} }
"include system info in akkaSource when creating Logging with system" in { "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") log.info("test")
awaitCond(outputString.contains("----"), 5 seconds) awaitCond(outputString.contains("----"), 5 seconds)
val s = outputString val s = outputString
s must include("akkaSource=[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.Slf4jEventHandlerSpec.MyLogSource(akka://Slf4jEventHandlerSpec)]") s must include("logger=[akka.event.slf4j.Slf4jLoggerSpec.MyLogSource(akka://Slf4jLoggerSpec)]")
} }
"not include system info in akkaSource when creating Logging with system.eventStream" in { "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") log.info("test")
awaitCond(outputString.contains("----"), 5 seconds) awaitCond(outputString.contains("----"), 5 seconds)
val s = outputString val s = outputString
s must include("akkaSource=[akka.event.slf4j.Slf4jEventHandlerSpec.MyLogSource]") s must include("akkaSource=[akka.event.slf4j.Slf4jLoggerSpec.MyLogSource]")
s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec.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 { "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") log.info("test")
awaitCond(outputString.contains("----"), 5 seconds) awaitCond(outputString.contains("----"), 5 seconds)
val s = outputString val s = outputString
s must include("akkaSource=[Slf4jEventHandlerSpec$MyLogSource(akka://Slf4jEventHandlerSpec)]") s must include("akkaSource=[Slf4jLoggerSpec$MyLogSource(akka://Slf4jLoggerSpec)]")
s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$MyLogSource]") s must include("logger=[akka.event.slf4j.Slf4jLoggerSpec$MyLogSource]")
} }
"use short class name in akkaSource when creating Logging with system.eventStream and class" in { "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") log.info("test")
awaitCond(outputString.contains("----"), 5 seconds) awaitCond(outputString.contains("----"), 5 seconds)
val s = outputString val s = outputString
s must include("akkaSource=[Slf4jEventHandlerSpec$MyLogSource]") s must include("akkaSource=[Slf4jLoggerSpec$MyLogSource]")
s must include("logger=[akka.event.slf4j.Slf4jEventHandlerSpec$MyLogSource]") s must include("logger=[akka.event.slf4j.Slf4jLoggerSpec$MyLogSource]")
} }
} }

View file

@ -453,7 +453,7 @@ case class CustomEventFilter(test: PartialFunction[LogEvent, Boolean])(occurrenc
* *
* <pre><code> * <pre><code>
* akka { * akka {
* event-handlers = ["akka.testkit.TestEventListener"] * loggers = ["akka.testkit.TestEventListener"]
* } * }
* </code></pre> * </code></pre>
*/ */

View file

@ -19,7 +19,7 @@ import akka.pattern.ask
object AkkaSpec { object AkkaSpec {
val testConf: Config = ConfigFactory.parseString(""" val testConf: Config = ConfigFactory.parseString("""
akka { akka {
event-handlers = ["akka.testkit.TestEventListener"] loggers = ["akka.testkit.TestEventListener"]
loglevel = "WARNING" loglevel = "WARNING"
stdout-loglevel = "WARNING" stdout-loglevel = "WARNING"
actor { actor {