add Class[_] to LogEvent

- it is customary to use class name for categorizing logs, hence we
  should support it; class is taken from logSource.getClass
- update SLF4J module to use logClass as category and set logSource in
  MDC "akkaSource"
- add docs
This commit is contained in:
Roland 2012-01-11 14:14:08 +01:00
parent 0470f5f9fd
commit d0498eb32e
25 changed files with 295 additions and 137 deletions

View file

@ -171,7 +171,7 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
system.eventStream.subscribe(testActor, classOf[Logging.Error]) system.eventStream.subscribe(testActor, classOf[Logging.Error])
fsm ! "go" fsm ! "go"
expectMsgPF(1 second, hint = "Next state 2 does not exist") { expectMsgPF(1 second, hint = "Next state 2 does not exist") {
case Logging.Error(_, `name`, "Next state 2 does not exist") true case Logging.Error(_, `name`, _, "Next state 2 does not exist") true
} }
system.eventStream.unsubscribe(testActor) system.eventStream.unsubscribe(testActor)
} }
@ -221,15 +221,15 @@ class FSMActorSpec extends AkkaSpec(Map("akka.actor.debug.fsm" -> true)) with Im
system.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
fsm ! "go" fsm ! "go"
expectMsgPF(1 second, hint = "processing Event(go,null)") { expectMsgPF(1 second, hint = "processing Event(go,null)") {
case Logging.Debug(`name`, s: String) if s.startsWith("processing Event(go,null) from Actor[") true case Logging.Debug(`name`, _, s: String) if s.startsWith("processing Event(go,null) from Actor[") true
} }
expectMsg(1 second, Logging.Debug(name, "setting timer 't'/1500 milliseconds: Shutdown")) expectMsg(1 second, Logging.Debug(name, fsm.underlyingActor.getClass, "setting timer 't'/1500 milliseconds: Shutdown"))
expectMsg(1 second, Logging.Debug(name, "transition 1 -> 2")) expectMsg(1 second, Logging.Debug(name, fsm.underlyingActor.getClass, "transition 1 -> 2"))
fsm ! "stop" fsm ! "stop"
expectMsgPF(1 second, hint = "processing Event(stop,null)") { expectMsgPF(1 second, hint = "processing Event(stop,null)") {
case Logging.Debug(`name`, s: String) if s.startsWith("processing Event(stop,null) from Actor[") true case Logging.Debug(`name`, _, s: String) if s.startsWith("processing Event(stop,null) from Actor[") true
} }
expectMsgAllOf(1 second, Logging.Debug(name, "canceling timer 't'"), Normal) expectMsgAllOf(1 second, Logging.Debug(name, fsm.underlyingActor.getClass, "canceling timer 't'"), Normal)
expectNoMsg(1 second) expectNoMsg(1 second)
system.eventStream.unsubscribe(testActor) system.eventStream.unsubscribe(testActor)
} }

View file

@ -151,7 +151,7 @@ object ActorModelSpec {
await(deadline)(stops == dispatcher.stops.get) await(deadline)(stops == dispatcher.stops.get)
} catch { } catch {
case e case e
system.eventStream.publish(Error(e, dispatcher.toString, "actual: stops=" + dispatcher.stops.get + system.eventStream.publish(Error(e, dispatcher.toString, dispatcher.getClass, "actual: stops=" + dispatcher.stops.get +
" required: stops=" + stops)) " required: stops=" + stops))
throw e throw e
} }
@ -208,7 +208,9 @@ object ActorModelSpec {
await(deadline)(stats.restarts.get() == restarts) await(deadline)(stats.restarts.get() == restarts)
} catch { } catch {
case e case e
system.eventStream.publish(Error(e, Option(dispatcher).toString, "actual: " + stats + ", required: InterceptorStats(susp=" + suspensions + system.eventStream.publish(Error(e, Option(dispatcher).toString,
if (dispatcher ne null) dispatcher.getClass else this.getClass,
"actual: " + stats + ", required: InterceptorStats(susp=" + suspensions +
",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters + ",res=" + resumes + ",reg=" + registers + ",unreg=" + unregisters +
",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts)) ",recv=" + msgsReceived + ",proc=" + msgsProcessed + ",restart=" + restarts))
throw e throw e
@ -311,7 +313,7 @@ abstract class ActorModelSpec(config: String) extends AkkaSpec(config) with Defa
try { try {
f f
} catch { } catch {
case e system.eventStream.publish(Error(e, "spawn", "error in spawned thread")) case e system.eventStream.publish(Error(e, "spawn", this.getClass, "error in spawned thread"))
} }
} }
} }

View file

@ -108,7 +108,7 @@ class EventStreamSpec extends AkkaSpec(EventStreamSpec.config) {
private def verifyLevel(bus: LoggingBus, level: Logging.LogLevel) { private def verifyLevel(bus: LoggingBus, level: Logging.LogLevel) {
import Logging._ import Logging._
val allmsg = Seq(Debug("", "debug"), Info("", "info"), Warning("", "warning"), Error("", "error")) val allmsg = Seq(Debug("", null, "debug"), Info("", null, "info"), Warning("", null, "warning"), Error("", null, "error"))
val msg = allmsg filter (_.level <= level) val msg = allmsg filter (_.level <= level)
allmsg foreach bus.publish allmsg foreach bus.publish
msg foreach (x expectMsg(x)) msg foreach (x expectMsg(x))

View file

@ -59,7 +59,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
} }
val log = LoggingReceive("funky")(r) val log = LoggingReceive("funky")(r)
log.isDefinedAt("hallo") log.isDefinedAt("hallo")
expectMsg(1 second, Logging.Debug("funky", "received unhandled message hallo")) expectMsg(1 second, Logging.Debug("funky", classOf[String], "received unhandled message hallo"))
} }
} }
@ -83,7 +83,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
val name = actor.path.toString val name = actor.path.toString
actor ! "buh" actor ! "buh"
within(1 second) { within(1 second) {
expectMsg(Logging.Debug(name, "received handled message buh")) expectMsg(Logging.Debug(name, actor.underlyingActor.getClass, "received handled message buh"))
expectMsg("x") expectMsg("x")
} }
@ -109,7 +109,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
}) })
actor ! "buh" actor ! "buh"
within(1 second) { within(1 second) {
expectMsg(Logging.Debug(actor.path.toString, "received handled message buh")) expectMsg(Logging.Debug(actor.path.toString, actor.underlyingActor.getClass, "received handled message buh"))
expectMsg("x") expectMsg("x")
} }
} }
@ -130,7 +130,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
val name = actor.path.toString val name = actor.path.toString
actor ! PoisonPill actor ! PoisonPill
expectMsgPF() { expectMsgPF() {
case Logging.Debug(`name`, msg: String) if msg startsWith "received AutoReceiveMessage Envelope(PoisonPill" true case Logging.Debug(`name`, _, msg: String) if msg startsWith "received AutoReceiveMessage Envelope(PoisonPill" true
} }
awaitCond(actor.isTerminated, 100 millis) awaitCond(actor.isTerminated, 100 millis)
} }
@ -142,7 +142,7 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
val sys = impl.systemGuardian.path.toString val sys = impl.systemGuardian.path.toString
ignoreMute(this) ignoreMute(this)
ignoreMsg { ignoreMsg {
case Logging.Debug(s, _) s.contains("MainBusReaper") || s == sys case Logging.Debug(`sys`, _, _) true
} }
system.eventStream.subscribe(testActor, classOf[Logging.Debug]) system.eventStream.subscribe(testActor, classOf[Logging.Debug])
system.eventStream.subscribe(testActor, classOf[Logging.Error]) system.eventStream.subscribe(testActor, classOf[Logging.Error])
@ -151,51 +151,53 @@ class LoggingReceiveSpec extends WordSpec with BeforeAndAfterEach with BeforeAnd
val lname = lifecycleGuardian.path.toString val lname = lifecycleGuardian.path.toString
val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000))) val supervisor = TestActorRef[TestLogActor](Props[TestLogActor].withFaultHandler(OneForOneStrategy(List(classOf[Throwable]), 5, 5000)))
val sname = supervisor.path.toString val sname = supervisor.path.toString
val sclass = classOf[TestLogActor]
val supervisorSet = receiveWhile(messages = 2) { val supervisorSet = receiveWhile(messages = 2) {
case Logging.Debug(`lname`, msg: String) if msg startsWith "now supervising" 1 case Logging.Debug(`lname`, _, msg: String) if msg startsWith "now supervising" 1
case Logging.Debug(`sname`, msg: String) if msg startsWith "started" 2 case Logging.Debug(`sname`, `sclass`, msg: String) if msg startsWith "started" 2
}.toSet }.toSet
expectNoMsg(Duration.Zero) expectNoMsg(Duration.Zero)
assert(supervisorSet == Set(1, 2), supervisorSet + " was not Set(1, 2)") assert(supervisorSet == Set(1, 2), supervisorSet + " was not Set(1, 2)")
val actor = TestActorRef[TestLogActor](Props[TestLogActor], supervisor, "none") val actor = TestActorRef[TestLogActor](Props[TestLogActor], supervisor, "none")
val aname = actor.path.toString val aname = actor.path.toString
val aclass = classOf[TestLogActor]
val set = receiveWhile(messages = 2) { val set = receiveWhile(messages = 2) {
case Logging.Debug(`sname`, msg: String) if msg startsWith "now supervising" 1 case Logging.Debug(`sname`, _, msg: String) if msg startsWith "now supervising" 1
case Logging.Debug(`aname`, msg: String) if msg startsWith "started" 2 case Logging.Debug(`aname`, `aclass`, msg: String) if msg startsWith "started" 2
}.toSet }.toSet
expectNoMsg(Duration.Zero) expectNoMsg(Duration.Zero)
assert(set == Set(1, 2), set + " was not Set(1, 2)") assert(set == Set(1, 2), set + " was not Set(1, 2)")
supervisor watch actor supervisor watch actor
expectMsgPF(hint = "now monitoring") { expectMsgPF(hint = "now monitoring") {
case Logging.Debug(ref, msg: String) case Logging.Debug(ref, `sclass`, msg: String)
ref == supervisor.underlyingActor && msg.startsWith("now monitoring") ref == supervisor.underlyingActor && msg.startsWith("now monitoring")
} }
supervisor unwatch actor supervisor unwatch actor
expectMsgPF(hint = "stopped monitoring") { expectMsgPF(hint = "stopped monitoring") {
case Logging.Debug(ref, msg: String) case Logging.Debug(ref, `sclass`, msg: String)
ref == supervisor.underlyingActor && msg.startsWith("stopped monitoring") ref == supervisor.underlyingActor && msg.startsWith("stopped monitoring")
} }
EventFilter[ActorKilledException](occurrences = 1) intercept { EventFilter[ActorKilledException](occurrences = 1) intercept {
actor ! Kill actor ! Kill
val set = receiveWhile(messages = 3) { val set = receiveWhile(messages = 3) {
case Logging.Error(_: ActorKilledException, `aname`, "Kill") 1 case Logging.Error(_: ActorKilledException, `aname`, `aclass`, "Kill") 1
case Logging.Debug(`aname`, "restarting") 2 case Logging.Debug(`aname`, `aclass`, "restarting") 2
case Logging.Debug(`aname`, "restarted") 3 case Logging.Debug(`aname`, `aclass`, "restarted") 3
}.toSet }.toSet
expectNoMsg(Duration.Zero) expectNoMsg(Duration.Zero)
assert(set == Set(1, 2, 3), set + " was not Set(1, 2, 3)") assert(set == Set(1, 2, 3), set + " was not Set(1, 2, 3)")
} }
system.stop(supervisor) system.stop(supervisor)
expectMsg(Logging.Debug(sname, "stopping")) expectMsg(Logging.Debug(sname, `sclass`, "stopping"))
expectMsg(Logging.Debug(aname, "stopped")) expectMsg(Logging.Debug(aname, `aclass`, "stopped"))
expectMsg(Logging.Debug(sname, "stopped")) expectMsg(Logging.Debug(sname, `sclass`, "stopped"))
} }
} }
} }

View file

@ -358,12 +358,12 @@ private[akka] class ActorCell(
actor = created actor = created
created.preStart() created.preStart()
checkReceiveTimeout checkReceiveTimeout
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "started (" + actor + ")")) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(created), "started (" + created + ")"))
} catch { } catch {
// FIXME catching all and continue isn't good for OOME, ticket #1418 // FIXME catching all and continue isn't good for OOME, ticket #1418
case e case e
try { try {
system.eventStream.publish(Error(e, self.path.toString, "error while creating actor")) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), "error while creating actor"))
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
} finally { } finally {
@ -373,7 +373,7 @@ private[akka] class ActorCell(
def recreate(cause: Throwable): Unit = try { def recreate(cause: Throwable): Unit = try {
val failedActor = actor val failedActor = actor
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "restarting")) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(failedActor), "restarting"))
val freshActor = newActor() val freshActor = newActor()
if (failedActor ne null) { if (failedActor ne null) {
val c = currentMessage //One read only plz val c = currentMessage //One read only plz
@ -388,7 +388,7 @@ private[akka] class ActorCell(
actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call actor = freshActor // assign it here so if preStart fails, we can null out the sef-refs next call
hotswap = Props.noHotSwap // Reset the behavior hotswap = Props.noHotSwap // Reset the behavior
freshActor.postRestart(cause) freshActor.postRestart(cause)
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "restarted")) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(freshActor), "restarted"))
dispatcher.resume(this) //FIXME should this be moved down? dispatcher.resume(this) //FIXME should this be moved down?
@ -396,7 +396,7 @@ private[akka] class ActorCell(
} catch { } catch {
// FIXME catching all and continue isn't good for OOME, ticket #1418 // FIXME catching all and continue isn't good for OOME, ticket #1418
case e try { case e try {
system.eventStream.publish(Error(e, self.path.toString, "error while creating actor")) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), "error while creating actor"))
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
} finally { } finally {
@ -417,7 +417,7 @@ private[akka] class ActorCell(
else { else {
// do not process normal messages while waiting for all children to terminate // do not process normal messages while waiting for all children to terminate
dispatcher suspend this dispatcher suspend this
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "stopping")) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopping"))
// do not use stop(child) because that would dissociate the children from us, but we still want to wait for them // do not use stop(child) because that would dissociate the children from us, but we still want to wait for them
for (child c) child.asInstanceOf[InternalActorRef].stop() for (child c) child.asInstanceOf[InternalActorRef].stop()
stopping = true stopping = true
@ -428,12 +428,12 @@ private[akka] class ActorCell(
childrenRefs.get(child.path.name) match { childrenRefs.get(child.path.name) match {
case None case None
childrenRefs = childrenRefs.updated(child.path.name, ChildRestartStats(child)) childrenRefs = childrenRefs.updated(child.path.name, ChildRestartStats(child))
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "now supervising " + child)) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now supervising " + child))
case Some(ChildRestartStats(`child`, _, _)) case Some(ChildRestartStats(`child`, _, _))
// this is the nominal case where we created the child and entered it in actorCreated() above // this is the nominal case where we created the child and entered it in actorCreated() above
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "now supervising " + child)) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now supervising " + child))
case Some(ChildRestartStats(c, _, _)) case Some(ChildRestartStats(c, _, _))
system.eventStream.publish(Warning(self.path.toString, "Already supervising other child with same name '" + child.path.name + "', old: " + c + " new: " + child)) system.eventStream.publish(Warning(self.path.toString, clazz(actor), "Already supervising other child with same name '" + child.path.name + "', old: " + c + " new: " + child))
} }
} }
@ -448,10 +448,10 @@ private[akka] class ActorCell(
case Recreate(cause) recreate(cause) case Recreate(cause) recreate(cause)
case Link(subject) case Link(subject)
system.deathWatch.subscribe(self, subject) system.deathWatch.subscribe(self, subject)
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "now monitoring " + subject)) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "now monitoring " + subject))
case Unlink(subject) case Unlink(subject)
system.deathWatch.unsubscribe(self, subject) system.deathWatch.unsubscribe(self, subject)
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "stopped monitoring " + subject)) if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped monitoring " + subject))
case Suspend() suspend() case Suspend() suspend()
case Resume() resume() case Resume() resume()
case Terminate() terminate() case Terminate() terminate()
@ -460,7 +460,7 @@ private[akka] class ActorCell(
} }
} catch { } catch {
case e //Should we really catch everything here? case e //Should we really catch everything here?
system.eventStream.publish(Error(e, self.path.toString, "error while processing " + message)) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), "error while processing " + message))
//TODO FIXME How should problems here be handled??? //TODO FIXME How should problems here be handled???
throw e throw e
} }
@ -480,7 +480,7 @@ private[akka] class ActorCell(
currentMessage = null // reset current message after successful invocation currentMessage = null // reset current message after successful invocation
} catch { } catch {
case e case e
system.eventStream.publish(Error(e, self.path.toString, e.getMessage)) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), e.getMessage))
// prevent any further messages to be processed until the actor has been restarted // prevent any further messages to be processed until the actor has been restarted
dispatcher.suspend(this) dispatcher.suspend(this)
@ -500,7 +500,7 @@ private[akka] class ActorCell(
} }
} catch { } catch {
case e case e
system.eventStream.publish(Error(e, self.path.toString, e.getMessage)) system.eventStream.publish(Error(e, self.path.toString, clazz(actor), e.getMessage))
throw e throw e
} }
} }
@ -530,7 +530,8 @@ private[akka] class ActorCell(
} }
def autoReceiveMessage(msg: Envelope) { def autoReceiveMessage(msg: Envelope) {
if (system.settings.DebugAutoReceive) system.eventStream.publish(Debug(self.path.toString, "received AutoReceiveMessage " + msg)) if (system.settings.DebugAutoReceive)
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "received AutoReceiveMessage " + msg))
msg.message match { msg.message match {
case Failed(cause) handleFailure(sender, cause) case Failed(cause) handleFailure(sender, cause)
@ -554,7 +555,8 @@ private[akka] class ActorCell(
try { try {
parent.sendSystemMessage(ChildTerminated(self)) parent.sendSystemMessage(ChildTerminated(self))
system.deathWatch.publish(Terminated(self)) system.deathWatch.publish(Terminated(self))
if (system.settings.DebugLifecycle) system.eventStream.publish(Debug(self.path.toString, "stopped")) if (system.settings.DebugLifecycle)
system.eventStream.publish(Debug(self.path.toString, clazz(actor), "stopped")) // FIXME: can actor be null?
} finally { } finally {
currentMessage = null currentMessage = null
clearActorFields() clearActorFields()
@ -565,8 +567,8 @@ private[akka] class ActorCell(
final def handleFailure(child: ActorRef, cause: Throwable): Unit = childrenRefs.get(child.path.name) match { final def handleFailure(child: ActorRef, cause: Throwable): Unit = childrenRefs.get(child.path.name) match {
case Some(stats) if stats.child == child if (!props.faultHandler.handleFailure(this, child, cause, stats, childrenRefs.values)) throw cause case Some(stats) if stats.child == child if (!props.faultHandler.handleFailure(this, child, cause, stats, childrenRefs.values)) throw cause
case Some(stats) system.eventStream.publish(Warning(self.path.toString, "dropping Failed(" + cause + ") from unknown child " + child + " matching names but not the same, was: " + stats.child)) case Some(stats) system.eventStream.publish(Warning(self.path.toString, clazz(actor), "dropping Failed(" + cause + ") from unknown child " + child + " matching names but not the same, was: " + stats.child))
case None system.eventStream.publish(Warning(self.path.toString, "dropping Failed(" + cause + ") from unknown child " + child)) case None system.eventStream.publish(Warning(self.path.toString, clazz(actor), "dropping Failed(" + cause + ") from unknown child " + child))
} }
final def handleChildTerminated(child: ActorRef): Unit = { final def handleChildTerminated(child: ActorRef): Unit = {
@ -625,4 +627,9 @@ private[akka] class ActorCell(
lookupAndSetField(a.getClass, a, "self", self) lookupAndSetField(a.getClass, a, "self", self)
} }
} }
private def clazz(o: AnyRef): Class[_] = {
if (o eq null) this.getClass
else o.getClass
}
} }

View file

@ -330,7 +330,7 @@ class ActorSystemImpl(val name: String, applicationConfig: Config) extends Actor
// this provides basic logging (to stdout) until .start() is called below // this provides basic logging (to stdout) until .start() is called below
val eventStream = new EventStream(DebugEventStream) val eventStream = new EventStream(DebugEventStream)
eventStream.startStdoutLogger(settings) eventStream.startStdoutLogger(settings)
val log = new BusLogging(eventStream, "ActorSystem") // this used only for .getClass in tagging messages val log = new BusLogging(eventStream, "ActorSystem", this.getClass)
val scheduler = createScheduler() val scheduler = createScheduler()

View file

@ -80,7 +80,7 @@ final case class TaskInvocation(eventStream: EventStream, runnable: Runnable, cl
runnable.run() runnable.run()
} catch { } catch {
// FIXME catching all and continue isn't good for OOME, ticket #1418 // FIXME catching all and continue isn't good for OOME, ticket #1418
case e eventStream.publish(Error(e, "TaskInvocation", e.getMessage)) case e eventStream.publish(Error(e, "TaskInvocation", this.getClass, e.getMessage))
} finally { } finally {
cleanup() cleanup()
} }

View file

@ -59,7 +59,7 @@ class Dispatcher(
executorService.get() execute invocation executorService.get() execute invocation
} catch { } catch {
case e2: RejectedExecutionException case e2: RejectedExecutionException
prerequisites.eventStream.publish(Warning("Dispatcher", e2.toString)) prerequisites.eventStream.publish(Warning("Dispatcher", this.getClass, e2.toString))
throw e2 throw e2
} }
} }

View file

@ -77,7 +77,7 @@ class Dispatchers(val settings: ActorSystem.Settings, val prerequisites: Dispatc
} else { } else {
// Note that the configurator of the default dispatcher will be registered for this id, // Note that the configurator of the default dispatcher will be registered for this id,
// so this will only be logged once, which is crucial. // so this will only be logged once, which is crucial.
prerequisites.eventStream.publish(Warning("Dispatchers", prerequisites.eventStream.publish(Warning("Dispatchers", this.getClass,
"Dispatcher [%s] not configured, using default-dispatcher".format(id))) "Dispatcher [%s] not configured, using default-dispatcher".format(id)))
lookupConfigurator(DefaultDispatcherId) lookupConfigurator(DefaultDispatcherId)
} }

View file

@ -325,7 +325,7 @@ object Future {
// FIXME catching all and continue isn't good for OOME, ticket #1418 // FIXME catching all and continue isn't good for OOME, ticket #1418
executor match { executor match {
case m: MessageDispatcher case m: MessageDispatcher
m.prerequisites.eventStream.publish(Error(e, "Future.dispatchTask", e.getMessage)) m.prerequisites.eventStream.publish(Error(e, "Future.dispatchTask", this.getClass, e.getMessage))
case other case other
e.printStackTrace() e.printStackTrace()
} }
@ -566,7 +566,7 @@ sealed trait Future[+T] extends japi.Future[T] with Await.Awaitable[T] {
protected def logError(msg: String, problem: Throwable): Unit = { protected def logError(msg: String, problem: Throwable): Unit = {
executor match { executor match {
case m: MessageDispatcher m.prerequisites.eventStream.publish(Error(problem, msg, problem.getMessage)) case m: MessageDispatcher m.prerequisites.eventStream.publish(Error(problem, msg, this.getClass, problem.getMessage))
case other problem.printStackTrace() case other problem.printStackTrace()
} }
} }

View file

@ -214,7 +214,7 @@ private[akka] abstract class Mailbox(val actor: ActorCell) extends MessageQueue
} }
} catch { } catch {
case e case e
actor.system.eventStream.publish(Error(e, actor.self.path.toString, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!")) actor.system.eventStream.publish(Error(e, actor.self.path.toString, actor.actor.getClass, "exception during processing system messages, dropping " + SystemMessage.size(nextMessage) + " messages!"))
throw e throw e
} }
} }

View file

@ -38,19 +38,19 @@ class EventStream(private val debug: Boolean = false) extends LoggingBus with Su
} }
override def subscribe(subscriber: ActorRef, channel: Class[_]): Boolean = { override def subscribe(subscriber: ActorRef, channel: Class[_]): Boolean = {
if (debug) publish(Logging.Debug(simpleName(this), "subscribing " + subscriber + " to channel " + channel)) if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "subscribing " + subscriber + " to channel " + channel))
super.subscribe(subscriber, channel) super.subscribe(subscriber, channel)
} }
override def unsubscribe(subscriber: ActorRef, channel: Class[_]): Boolean = { override def unsubscribe(subscriber: ActorRef, channel: Class[_]): Boolean = {
val ret = super.unsubscribe(subscriber, channel) val ret = super.unsubscribe(subscriber, channel)
if (debug) publish(Logging.Debug(simpleName(this), "unsubscribing " + subscriber + " from channel " + channel)) if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "unsubscribing " + subscriber + " from channel " + channel))
ret ret
} }
override def unsubscribe(subscriber: ActorRef) { override def unsubscribe(subscriber: ActorRef) {
super.unsubscribe(subscriber) super.unsubscribe(subscriber)
if (debug) publish(Logging.Debug(simpleName(this), "unsubscribing " + subscriber + " from all channels")) if (debug) publish(Logging.Debug(simpleName(this), this.getClass, "unsubscribing " + subscriber + " from all channels"))
} }
} }

View file

@ -16,10 +16,6 @@ import scala.util.control.NoStackTrace
import java.util.concurrent.TimeoutException import java.util.concurrent.TimeoutException
import akka.dispatch.Await import akka.dispatch.Await
object LoggingBus {
implicit def fromActorSystem(system: ActorSystem): LoggingBus = system.eventStream
}
/** /**
* This trait brings log level handling to the EventStream: it reads the log * This trait brings log level handling to the EventStream: it reads the log
* levels for the initial logging (StandardOutLogger) and the loggers & level * levels for the initial logging (StandardOutLogger) and the loggers & level
@ -75,7 +71,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), "unknown akka.stdout-loglevel " + config.StdoutLogLevel)) StandardOutLogger.print(Error(new EventHandlerException, 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)))
@ -83,7 +79,7 @@ trait LoggingBus extends ActorEventBus {
loggers = Seq(StandardOutLogger) loggers = Seq(StandardOutLogger)
_logLevel = level _logLevel = level
} }
publish(Debug(simpleName(this), "StandardOutLogger started")) publish(Debug(simpleName(this), this.getClass, "StandardOutLogger started"))
} }
/** /**
@ -91,7 +87,7 @@ trait LoggingBus extends ActorEventBus {
*/ */
private[akka] def startDefaultLoggers(system: ActorSystemImpl) { private[akka] def startDefaultLoggers(system: ActorSystemImpl) {
val level = levelFor(system.settings.LogLevel) getOrElse { val level = levelFor(system.settings.LogLevel) getOrElse {
StandardOutLogger.print(Error(new EventHandlerException, simpleName(this), "unknown akka.stdout-loglevel " + system.settings.LogLevel)) StandardOutLogger.print(Error(new EventHandlerException, simpleName(this), this.getClass, "unknown akka.stdout-loglevel " + system.settings.LogLevel))
ErrorLevel ErrorLevel
} }
try { try {
@ -119,7 +115,7 @@ trait LoggingBus extends ActorEventBus {
loggers = myloggers loggers = myloggers
_logLevel = level _logLevel = level
} }
publish(Debug(simpleName(this), "Default Loggers started")) publish(Debug(simpleName(this), this.getClass, "Default Loggers started"))
if (!(defaultLoggers contains StandardOutLoggerName)) { if (!(defaultLoggers contains StandardOutLoggerName)) {
unsubscribe(StandardOutLogger) unsubscribe(StandardOutLogger)
} }
@ -138,7 +134,7 @@ trait LoggingBus extends ActorEventBus {
val level = _logLevel // volatile access before reading loggers val level = _logLevel // volatile access before reading loggers
if (!(loggers contains StandardOutLogger)) { if (!(loggers contains StandardOutLogger)) {
AllLogLevels filter (level >= _) foreach (l subscribe(StandardOutLogger, classFor(l))) AllLogLevels filter (level >= _) foreach (l subscribe(StandardOutLogger, classFor(l)))
publish(Debug(simpleName(this), "shutting down: StandardOutLogger started")) publish(Debug(simpleName(this), this.getClass, "shutting down: StandardOutLogger started"))
} }
for { for {
logger loggers logger loggers
@ -151,7 +147,7 @@ trait LoggingBus extends ActorEventBus {
case _ case _
} }
} }
publish(Debug(simpleName(this), "all default loggers stopped")) publish(Debug(simpleName(this), this.getClass, "all default loggers stopped"))
} }
private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = { private def addLogger(system: ActorSystemImpl, clazz: Class[_ <: Actor], level: LogLevel): ActorRef = {
@ -160,12 +156,12 @@ trait LoggingBus extends ActorEventBus {
implicit val timeout = Timeout(3 seconds) implicit val timeout = Timeout(3 seconds)
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
publish(Warning(simpleName(this), "Logger " + name + " did not respond within " + timeout + " to InitializeLogger(bus)")) publish(Warning(simpleName(this), this.getClass, "Logger " + name + " did not respond within " + timeout + " to InitializeLogger(bus)"))
} }
if (response != LoggerInitialized) if (response != LoggerInitialized)
throw new LoggerInitializationException("Logger " + name + " did not respond with LoggerInitialized, sent instead " + response) throw new LoggerInitializationException("Logger " + name + " did not respond with LoggerInitialized, sent instead " + response)
AllLogLevels filter (level >= _) foreach (l subscribe(actor, classFor(l))) AllLogLevels filter (level >= _) foreach (l subscribe(actor, classFor(l)))
publish(Debug(simpleName(this), "logger " + name + " started")) publish(Debug(simpleName(this), this.getClass, "logger " + name + " started"))
actor actor
} }
@ -218,6 +214,11 @@ object LogSource {
* log.info("hello world!") * log.info("hello world!")
* </code></pre> * </code></pre>
* *
* The source object is used in two fashions: its `Class[_]` will be part of
* all log events produced by this logger, plus a string representation is
* generated which may contain per-instance information, see `apply` or `create`
* below.
*
* Loggers are attached to the level-specific channels <code>Error</code>, * Loggers are attached to the level-specific channels <code>Error</code>,
* <code>Warning</code>, <code>Info</code> and <code>Debug</code> as * <code>Warning</code>, <code>Info</code> and <code>Debug</code> as
* appropriate for the configured (or set) log level. If you want to implement * appropriate for the configured (or set) log level. If you want to implement
@ -305,13 +306,11 @@ object Logging {
val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern val debugFormat = "[DEBUG] [%s] [%s] [%s] %s".intern
/** /**
* Obtain LoggingAdapter for the given event stream (system) and source object. * Obtain LoggingAdapter for the given logging bus and source object.
* Note that there is an implicit conversion from [[akka.actor.ActorSystem]]
* to [[akka.event.LoggingBus]].
* *
* The source is used to identify the source of this logging channel and must have * The source is used to identify the source of this logging channel and must have
* a corresponding LogSource[T] instance in scope; by default these are * a corresponding implicit LogSource[T] instance in scope; by default these are
* provided for Class[_], Actor, ActorRef and String types. The source * provided for Class[_], Actor, ActorRef and String types. By these, the source
* object is translated to a String according to the following rules: * object is translated to a String according to the following rules:
* <ul> * <ul>
* <li>if it is an Actor or ActorRef, its path is used</li> * <li>if it is an Actor or ActorRef, its path is used</li>
@ -319,13 +318,34 @@ object Logging {
* <li>in case of a class an approximation of its simpleName * <li>in case of a class an approximation of its simpleName
* <li>and in all other cases the simpleName of its class</li> * <li>and in all other cases the simpleName of its class</li>
* </ul> * </ul>
*
* You can add your own rules quite easily:
*
* {{{
* trait MyType { // as an example
* def name: String
* }
*
* implicit val myLogSourceType: LogSource[MyType] = new LogSource {
* def genString(a: MyType) = a.name
* }
*
* class MyClass extends MyType {
* val log = Logging(eventStream, this) // will use "hallo" as logSource
* def name = "hallo"
* }
* }}}
*/ */
def apply[T: LogSource](eventStream: LoggingBus, logSource: T): LoggingAdapter = def apply[T: LogSource](bus: LoggingBus, logSource: T): LoggingAdapter =
new BusLogging(eventStream, implicitly[LogSource[T]].genString(logSource)) new BusLogging(bus, implicitly[LogSource[T]].genString(logSource), logSource.getClass)
/** /**
* Java API: Obtain LoggingAdapter for the given system and source object. The * Obtain LoggingAdapter for the given actor system and source object. This
* source object is used to identify the source of this logging channel. The source * will use the systems event stream.
*
* The source is used to identify the source of this logging channel and must have
* a corresponding implicit LogSource[T] instance in scope; by default these are
* provided for Class[_], Actor, ActorRef and String types. By these, the source
* object is translated to a String according to the following rules: * object is translated to a String according to the following rules:
* <ul> * <ul>
* <li>if it is an Actor or ActorRef, its path is used</li> * <li>if it is an Actor or ActorRef, its path is used</li>
@ -333,12 +353,92 @@ object Logging {
* <li>in case of a class an approximation of its simpleName * <li>in case of a class an approximation of its simpleName
* <li>and in all other cases the simpleName of its class</li> * <li>and in all other cases the simpleName of its class</li>
* </ul> * </ul>
*
* You can add your own rules quite easily:
*
* {{{
* trait MyType { // as an example
* def name: String
* }
*
* implicit val myLogSourceType: LogSource[MyType] = new LogSource {
* def genString(a: MyType) = a.name
* }
*
* class MyClass extends MyType {
* val log = Logging(eventStream, this) // will use "hallo" as logSource
* def name = "hallo"
* }
* }}}
*/ */
def getLogger(system: ActorSystem, logSource: AnyRef): LoggingAdapter = apply(system.eventStream, LogSource.fromAnyRef(logSource)) def apply[T: LogSource](system: ActorSystem, logSource: T): LoggingAdapter =
new BusLogging(system.eventStream, implicitly[LogSource[T]].genString(logSource), logSource.getClass)
/** /**
* Java API: Obtain LoggingAdapter for the given event bus and source object. The * Obtain LoggingAdapter for the given actor system and source object. This
* source object is used to identify the source of this logging channel. * will use the systems event stream.
*
* The source is used to identify the source of this logging channel and must have
* a corresponding implicit LogSource[T] instance in scope; by default these are
* provided for Class[_], Actor, ActorRef and String types. By these, the source
* object is translated to a String according to the following rules:
* <ul>
* <li>if it is an Actor or ActorRef, its path is used</li>
* <li>in case of a String it is used as is</li>
* <li>in case of a class an approximation of its simpleName
* <li>and in all other cases the simpleName of its class</li>
* </ul>
*
* You can add your own rules quite easily:
*
* {{{
* trait MyType { // as an example
* def name: String
* }
*
* implicit val myLogSourceType: LogSource[MyType] = new LogSource {
* def genString(a: MyType) = a.name
* }
*
* class MyClass extends MyType {
* val log = Logging(eventStream, this) // will use "hallo" as logSource
* def name = "hallo"
* }
* }}}
*/
def getLogger(system: ActorSystem, logSource: AnyRef): LoggingAdapter = apply(system, LogSource.fromAnyRef(logSource))
/**
* Obtain LoggingAdapter for the given logging bus and source object. This
* will use the systems event stream.
*
* The source is used to identify the source of this logging channel and must have
* a corresponding implicit LogSource[T] instance in scope; by default these are
* provided for Class[_], Actor, ActorRef and String types. By these, the source
* object is translated to a String according to the following rules:
* <ul>
* <li>if it is an Actor or ActorRef, its path is used</li>
* <li>in case of a String it is used as is</li>
* <li>in case of a class an approximation of its simpleName
* <li>and in all other cases the simpleName of its class</li>
* </ul>
*
* You can add your own rules quite easily:
*
* {{{
* trait MyType { // as an example
* def name: String
* }
*
* implicit val myLogSourceType: LogSource[MyType] = new LogSource {
* def genString(a: MyType) = a.name
* }
*
* class MyClass extends MyType {
* val log = Logging(eventStream, this) // will use "hallo" as logSource
* def name = "hallo"
* }
* }}}
*/ */
def getLogger(bus: LoggingBus, logSource: AnyRef): LoggingAdapter = apply(bus, LogSource.fromAnyRef(logSource)) def getLogger(bus: LoggingBus, logSource: AnyRef): LoggingAdapter = apply(bus, LogSource.fromAnyRef(logSource))
@ -362,19 +462,34 @@ object Logging {
* The LogLevel of this LogEvent * The LogLevel of this LogEvent
*/ */
def level: LogLevel def level: LogLevel
/**
* The source of this event
*/
def logSource: String
/**
* The class of the source of this event
*/
def logClass: Class[_]
/**
* The message, may be any object or null.
*/
def message: Any
} }
/** /**
* For ERROR Logging * For ERROR Logging
*/ */
case class Error(cause: Throwable, logSource: String, message: Any = "") extends LogEvent { case class Error(cause: Throwable, logSource: String, logClass: Class[_], message: Any = "") extends LogEvent {
def this(logSource: String, message: Any) = this(Error.NoCause, logSource, message) def this(logSource: String, logClass: Class[_], message: Any) = this(Error.NoCause, logSource, logClass, message)
override def level = ErrorLevel override def level = ErrorLevel
} }
object Error { object Error {
def apply(logSource: String, message: Any) = new Error(NoCause, logSource, message) def apply(logSource: String, logClass: Class[_], message: Any) = new Error(NoCause, logSource, logClass, message)
/** Null Object used for errors without cause Throwable */ /** Null Object used for errors without cause Throwable */
object NoCause extends NoStackTrace object NoCause extends NoStackTrace
@ -383,21 +498,21 @@ object Logging {
/** /**
* For WARNING Logging * For WARNING Logging
*/ */
case class Warning(logSource: String, message: Any = "") extends LogEvent { case class Warning(logSource: String, logClass: Class[_], message: Any = "") extends LogEvent {
override def level = WarningLevel override def level = WarningLevel
} }
/** /**
* For INFO Logging * For INFO Logging
*/ */
case class Info(logSource: String, message: Any = "") extends LogEvent { case class Info(logSource: String, logClass: Class[_], message: Any = "") extends LogEvent {
override def level = InfoLevel override def level = InfoLevel
} }
/** /**
* For DEBUG Logging * For DEBUG Logging
*/ */
case class Debug(logSource: String, message: Any = "") extends LogEvent { case class Debug(logSource: String, logClass: Class[_], message: Any = "") extends LogEvent {
override def level = DebugLevel override def level = DebugLevel
} }
@ -439,7 +554,7 @@ object Logging {
case e: Warning warning(e) case e: Warning warning(e)
case e: Info info(e) case e: Info info(e)
case e: Debug debug(e) case e: Debug debug(e)
case e warning(Warning(simpleName(this), "received unexpected event of class " + e.getClass + ": " + e)) case e warning(Warning(simpleName(this), this.getClass, "received unexpected event of class " + e.getClass + ": " + e))
} }
} }
@ -626,7 +741,7 @@ trait LoggingAdapter {
} }
} }
class BusLogging(val bus: LoggingBus, val logSource: String) extends LoggingAdapter { class BusLogging(val bus: LoggingBus, val logSource: String, val logClass: Class[_]) extends LoggingAdapter {
import Logging._ import Logging._
@ -635,14 +750,14 @@ class BusLogging(val bus: LoggingBus, val logSource: String) extends LoggingAdap
def isInfoEnabled = bus.logLevel >= InfoLevel def isInfoEnabled = bus.logLevel >= InfoLevel
def isDebugEnabled = bus.logLevel >= DebugLevel def isDebugEnabled = bus.logLevel >= DebugLevel
protected def notifyError(message: String) { bus.publish(Error(logSource, message)) } protected def notifyError(message: String) { bus.publish(Error(logSource, logClass, message)) }
protected def notifyError(cause: Throwable, message: String) { bus.publish(Error(cause, logSource, message)) } protected def notifyError(cause: Throwable, message: String) { bus.publish(Error(cause, logSource, logClass, message)) }
protected def notifyWarning(message: String) { bus.publish(Warning(logSource, message)) } protected def notifyWarning(message: String) { bus.publish(Warning(logSource, logClass, message)) }
protected def notifyInfo(message: String) { bus.publish(Info(logSource, message)) } protected def notifyInfo(message: String) { bus.publish(Info(logSource, logClass, message)) }
protected def notifyDebug(message: String) { bus.publish(Debug(logSource, message)) } protected def notifyDebug(message: String) { bus.publish(Debug(logSource, logClass, message)) }
} }

View file

@ -36,7 +36,7 @@ object LoggingReceive {
class LoggingReceive(source: AnyRef, r: Receive)(implicit system: ActorSystem) extends Receive { class LoggingReceive(source: AnyRef, r: Receive)(implicit system: ActorSystem) extends Receive {
def isDefinedAt(o: Any) = { def isDefinedAt(o: Any) = {
val handled = r.isDefinedAt(o) val handled = r.isDefinedAt(o)
system.eventStream.publish(Debug(LogSource.fromAnyRef(source), "received " + (if (handled) "handled" else "unhandled") + " message " + o)) system.eventStream.publish(Debug(LogSource.fromAnyRef(source), source.getClass, "received " + (if (handled) "handled" else "unhandled") + " message " + o))
handled handled
} }
def apply(o: Any): Unit = r(o) def apply(o: Any): Unit = r(o)

View file

@ -21,7 +21,7 @@ object JMX {
case e: InstanceAlreadyExistsException case e: InstanceAlreadyExistsException
Some(mbeanServer.getObjectInstance(name)) Some(mbeanServer.getObjectInstance(name))
case e: Exception case e: Exception
system.eventStream.publish(Error(e, "JMX", "Error when registering mbean [%s]".format(mbean))) system.eventStream.publish(Error(e, "JMX", this.getClass, "Error when registering mbean [%s]".format(mbean)))
None None
} }
@ -29,6 +29,6 @@ object JMX {
mbeanServer.unregisterMBean(mbean) mbeanServer.unregisterMBean(mbean)
} catch { } catch {
case e: InstanceNotFoundException {} case e: InstanceNotFoundException {}
case e: Exception system.eventStream.publish(Error(e, "JMX", "Error while unregistering mbean [%s]".format(mbean))) case e: Exception system.eventStream.publish(Error(e, "JMX", this.getClass, "Error while unregistering mbean [%s]".format(mbean)))
} }
} }

View file

@ -83,8 +83,8 @@ creating the ``LoggingAdapter`` correspond to the name of the SL4FJ logger.
loglevel = "DEBUG" loglevel = "DEBUG"
} }
Logging thread in MDC Logging Thread and Akka Source in MDC
--------------------- -------------------------------------
Since the logging is done asynchronously the thread in which the logging was performed is captured in Since the logging is done asynchronously the thread in which the logging was performed is captured in
Mapped Diagnostic Context (MDC) with attribute name ``sourceThread``. Mapped Diagnostic Context (MDC) with attribute name ``sourceThread``.
@ -96,3 +96,16 @@ With Logback the thread name is available with ``%X{sourceThread}`` specifier wi
</layout> </layout>
</appender> </appender>
Another helpful facility is that Akka captures the actors address when
instantiating a logger within it, meaning that the full instance identification
is available for associating log messages e.g. with members of a router. This
information is available in the MDC with attribute name ``akkaSource``::
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
<layout>
<pattern>%date{ISO8601} %-5level %logger{36} %X{akkaSource} - %msg%n</pattern>
</layout>
</appender>
For more details on what this attribute contains—also for non-actors—please see
`How to Log`_.

View file

@ -162,10 +162,10 @@ class ActorDocSpec extends AkkaSpec(Map("akka.loglevel" -> "INFO")) {
system.eventStream.subscribe(testActor, classOf[Logging.Info]) system.eventStream.subscribe(testActor, classOf[Logging.Info])
myActor ! "test" myActor ! "test"
expectMsgPF(1 second) { case Logging.Info(_, "received test") true } expectMsgPF(1 second) { case Logging.Info(_, _, "received test") true }
myActor ! "unknown" myActor ! "unknown"
expectMsgPF(1 second) { case Logging.Info(_, "received unknown message") true } expectMsgPF(1 second) { case Logging.Info(_, _, "received unknown message") true }
system.eventStream.unsubscribe(testActor) system.eventStream.unsubscribe(testActor)
system.eventStream.publish(TestEvent.UnMute(filter)) system.eventStream.publish(TestEvent.UnMute(filter))

View file

@ -39,10 +39,10 @@ object LoggingDocSpec {
class MyEventListener extends Actor { class MyEventListener extends Actor {
def receive = { def receive = {
case InitializeLogger(_) sender ! LoggerInitialized case InitializeLogger(_) sender ! LoggerInitialized
case Error(cause, logSource, message) // ... case Error(cause, logSource, logClass, message) // ...
case Warning(logSource, message) // ... case Warning(logSource, logClass, message) // ...
case Info(logSource, message) // ... case Info(logSource, logClass, message) // ...
case Debug(logSource, message) // ... case Debug(logSource, logClass, message) // ...
} }
} }
//#my-event-listener //#my-event-listener

View file

@ -85,8 +85,8 @@ creating the ``LoggingAdapter`` correspond to the name of the SL4FJ logger.
loglevel = "DEBUG" loglevel = "DEBUG"
} }
Logging thread in MDC Logging Thread and Akka Source in MDC
--------------------- -------------------------------------
Since the logging is done asynchronously the thread in which the logging was performed is captured in Since the logging is done asynchronously the thread in which the logging was performed is captured in
Mapped Diagnostic Context (MDC) with attribute name ``sourceThread``. Mapped Diagnostic Context (MDC) with attribute name ``sourceThread``.
@ -98,3 +98,16 @@ With Logback the thread name is available with ``%X{sourceThread}`` specifier wi
</layout> </layout>
</appender> </appender>
Another helpful facility is that Akka captures the actors address when
instantiating a logger within it, meaning that the full instance identification
is available for associating log messages e.g. with members of a router. This
information is available in the MDC with attribute name ``akkaSource``::
<appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
<layout>
<pattern>%date{ISO8601} %-5level %logger{36} %X{akkaSource} - %msg%n</pattern>
</layout>
</appender>
For more details on what this attribute contains—also for non-actors—please see
`How to Log`_.

View file

@ -153,7 +153,7 @@ class RemoteSystemDaemon(system: ActorSystemImpl, remote: Remote, _path: ActorPa
override def !(msg: Any)(implicit sender: ActorRef = null): Unit = msg match { override def !(msg: Any)(implicit sender: ActorRef = null): Unit = msg match {
case message: DaemonMsg case message: DaemonMsg
log.debug("Received command [\n{}] to RemoteSystemDaemon on [{}]", message, remote.remoteSettings.NodeName) log.debug("Received command [{}] to RemoteSystemDaemon on [{}]", message, remote.remoteSettings.NodeName)
message match { message match {
case DaemonMsgCreate(factory, path, supervisor) case DaemonMsgCreate(factory, path, supervisor)
import remote.remoteAddress import remote.remoteAddress

View file

@ -60,7 +60,7 @@ abstract class RemoteClient private[akka] (
* Converts the message to the wireprotocol and sends the message across the wire * Converts the message to the wireprotocol and sends the message across the wire
*/ */
def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) { def send(message: Any, senderOption: Option[ActorRef], recipient: ActorRef): Unit = if (isRunning) {
log.debug("Sending message: {}", message) log.debug("Sending message {} from {} to {}", message, senderOption, recipient)
send((message, senderOption, recipient)) send((message, senderOption, recipient))
} else { } else {
val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", remoteSupport, remoteAddress) val exception = new RemoteClientException("RemoteModule client is not running, make sure you have invoked 'RemoteClient.connect()' before using it.", remoteSupport, remoteAddress)

View file

@ -15,8 +15,10 @@ akka {
deployment { deployment {
/watchers.remote = "akka://other@127.0.0.1:2666" /watchers.remote = "akka://other@127.0.0.1:2666"
} }
debug.lifecycle = on
} }
cluster.nodename = buh cluster.nodename = buh
loglevel = DEBUG
remote.server { remote.server {
hostname = "127.0.0.1" hostname = "127.0.0.1"
port = 2665 port = 2665

View file

@ -19,6 +19,7 @@ trait SLF4JLogging {
object Logger { object Logger {
def apply(logger: String): SLFLogger = SLFLoggerFactory getLogger logger def apply(logger: String): SLFLogger = SLFLoggerFactory getLogger logger
def apply(logClass: Class[_]): SLFLogger = SLFLoggerFactory getLogger logClass
def root: SLFLogger = apply(SLFLogger.ROOT_LOGGER_NAME) def root: SLFLogger = apply(SLFLogger.ROOT_LOGGER_NAME)
} }
@ -31,30 +32,31 @@ object Logger {
class Slf4jEventHandler extends Actor with SLF4JLogging { class Slf4jEventHandler extends Actor with SLF4JLogging {
val mdcThreadAttributeName = "sourceThread" val mdcThreadAttributeName = "sourceThread"
val mdcAkkaSourceAttributeName = "akkaSource"
def receive = { def receive = {
case event @ Error(cause, logSource, message) case event @ Error(cause, logSource, logClass, message)
withMdc(mdcThreadAttributeName, event.thread.getName) { withMdc(logSource, event.thread.getName) {
cause match { cause match {
case Error.NoCause Logger(logSource).error(message.toString) case Error.NoCause Logger(logClass).error(message.toString)
case _ Logger(logSource).error(message.toString, cause) case _ Logger(logClass).error(message.toString, cause)
} }
} }
case event @ Warning(logSource, message) case event @ Warning(logSource, logClass, message)
withMdc(mdcThreadAttributeName, event.thread.getName) { withMdc(logSource, event.thread.getName) {
Logger(logSource).warn("{}", message.asInstanceOf[AnyRef]) Logger(logClass).warn("{}", message.asInstanceOf[AnyRef])
} }
case event @ Info(logSource, message) case event @ Info(logSource, logClass, message)
withMdc(mdcThreadAttributeName, event.thread.getName) { withMdc(logSource, event.thread.getName) {
Logger(logSource).info("{}", message.asInstanceOf[AnyRef]) Logger(logClass).info("{}", message.asInstanceOf[AnyRef])
} }
case event @ Debug(logSource, message) case event @ Debug(logSource, logClass, message)
withMdc(mdcThreadAttributeName, event.thread.getName) { withMdc(logSource, event.thread.getName) {
Logger(logSource).debug("{}", message.asInstanceOf[AnyRef]) Logger(logClass).debug("{}", message.asInstanceOf[AnyRef])
} }
case InitializeLogger(_) case InitializeLogger(_)
@ -63,12 +65,14 @@ class Slf4jEventHandler extends Actor with SLF4JLogging {
} }
@inline @inline
final def withMdc(name: String, value: String)(logStatement: Unit) { final def withMdc(logSource: String, thread: String)(logStatement: Unit) {
MDC.put(name, value) MDC.put(mdcAkkaSourceAttributeName, logSource)
MDC.put(mdcThreadAttributeName, thread)
try { try {
logStatement logStatement
} finally { } finally {
MDC.remove(name) MDC.remove(mdcAkkaSourceAttributeName)
MDC.remove(mdcThreadAttributeName)
} }
} }

View file

@ -254,7 +254,7 @@ case class ErrorFilter(
def matches(event: LogEvent) = { def matches(event: LogEvent) = {
event match { event match {
case Error(cause, src, msg) if throwable isInstance cause case Error(cause, src, _, msg) if throwable isInstance cause
(msg == null && cause.getMessage == null && cause.getStackTrace.length == 0) || (msg == null && cause.getMessage == null && cause.getStackTrace.length == 0) ||
doMatch(src, msg) || doMatch(src, cause.getMessage) doMatch(src, msg) || doMatch(src, cause.getMessage)
case _ false case _ false
@ -305,7 +305,7 @@ case class WarningFilter(
def matches(event: LogEvent) = { def matches(event: LogEvent) = {
event match { event match {
case Warning(src, msg) doMatch(src, msg) case Warning(src, _, msg) doMatch(src, msg)
case _ false case _ false
} }
} }
@ -348,7 +348,7 @@ case class InfoFilter(
def matches(event: LogEvent) = { def matches(event: LogEvent) = {
event match { event match {
case Info(src, msg) doMatch(src, msg) case Info(src, _, msg) doMatch(src, msg)
case _ false case _ false
} }
} }
@ -391,7 +391,7 @@ case class DebugFilter(
def matches(event: LogEvent) = { def matches(event: LogEvent) = {
event match { event match {
case Debug(src, msg) doMatch(src, msg) case Debug(src, _, msg) doMatch(src, msg)
case _ false case _ false
} }
} }
@ -456,15 +456,15 @@ class TestEventListener extends Logging.DefaultLogger {
case event: LogEvent if (!filter(event)) print(event) case event: LogEvent if (!filter(event)) print(event)
case DeadLetter(msg: SystemMessage, _, rcp) case DeadLetter(msg: SystemMessage, _, rcp)
if (!msg.isInstanceOf[Terminate]) { if (!msg.isInstanceOf[Terminate]) {
val event = Warning(rcp.path.toString, "received dead system message: " + msg) val event = Warning(rcp.path.toString, rcp.getClass, "received dead system message: " + msg)
if (!filter(event)) print(event) if (!filter(event)) print(event)
} }
case DeadLetter(msg, snd, rcp) case DeadLetter(msg, snd, rcp)
if (!msg.isInstanceOf[Terminated]) { if (!msg.isInstanceOf[Terminated]) {
val event = Warning(rcp.path.toString, "received dead letter from " + snd + ": " + msg) val event = Warning(rcp.path.toString, rcp.getClass, "received dead letter from " + snd + ": " + msg)
if (!filter(event)) print(event) if (!filter(event)) print(event)
} }
case m print(Debug(context.system.name, m)) case m print(Debug(context.system.name, this.getClass, m))
} }
def filter(event: LogEvent): Boolean = filters exists (f try { f(event) } catch { case e: Exception false }) def filter(event: LogEvent): Boolean = filters exists (f try { f(event) } catch { case e: Exception false })

View file

@ -81,7 +81,7 @@ object TestActorRefSpec {
var count = 0 var count = 0
var msg: String = _ var msg: String = _
def receive = { def receive = {
case Warning(_, m: String) count += 1; msg = m case Warning(_, _, m: String) count += 1; msg = m
} }
} }