Switching to raw SLF4J on internals

This commit is contained in:
Viktor Klang 2010-11-24 13:42:41 +01:00
parent f9f65a7ddd
commit 40e40a578a
39 changed files with 253 additions and 247 deletions

View file

@ -95,18 +95,18 @@ object Actor extends Logging {
val hook = new Runnable { val hook = new Runnable {
override def run { override def run {
// Shutdown HawtDispatch GlobalQueue // Shutdown HawtDispatch GlobalQueue
log.info("Shutting down Hawt Dispatch global queue") log.slf4j.info("Shutting down Hawt Dispatch global queue")
org.fusesource.hawtdispatch.ScalaDispatch.globalQueue.asInstanceOf[org.fusesource.hawtdispatch.internal.GlobalDispatchQueue].shutdown org.fusesource.hawtdispatch.ScalaDispatch.globalQueue.asInstanceOf[org.fusesource.hawtdispatch.internal.GlobalDispatchQueue].shutdown
// Clear Thread.subclassAudits // Clear Thread.subclassAudits
log.info("Clearing subclass audits") log.slf4j.info("Clearing subclass audits")
val tf = classOf[java.lang.Thread].getDeclaredField("subclassAudits") val tf = classOf[java.lang.Thread].getDeclaredField("subclassAudits")
tf.setAccessible(true) tf.setAccessible(true)
val subclassAudits = tf.get(null).asInstanceOf[java.util.Map[_,_]] val subclassAudits = tf.get(null).asInstanceOf[java.util.Map[_,_]]
subclassAudits.synchronized {subclassAudits.clear} subclassAudits.synchronized {subclassAudits.clear}
// Clear and reset j.u.l.Level.known (due to Configgy) // Clear and reset j.u.l.Level.known (due to Configgy)
log.info("Removing Configgy-installed log levels") log.slf4j.info("Removing Configgy-installed log levels")
import java.util.logging.Level import java.util.logging.Level
val lf = classOf[Level].getDeclaredField("known") val lf = classOf[Level].getDeclaredField("known")
lf.setAccessible(true) lf.setAccessible(true)
@ -351,14 +351,14 @@ trait Actor extends Logging {
* <pre> * <pre>
* def receive = { * def receive = {
* case Ping =&gt; * case Ping =&gt;
* log.info("got a 'Ping' message") * log.slf4j.info("got a 'Ping' message")
* self.reply("pong") * self.reply("pong")
* *
* case OneWay =&gt; * case OneWay =&gt;
* log.info("got a 'OneWay' message") * log.slf4j.info("got a 'OneWay' message")
* *
* case unknown =&gt; * case unknown =&gt;
* log.warning("unknown message [%s], ignoring", unknown) * log.slf4j.warn("unknown message [%s], ignoring", unknown)
* } * }
* </pre> * </pre>
*/ */

View file

@ -578,7 +578,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
protected[akka] def checkReceiveTimeout = { protected[akka] def checkReceiveTimeout = {
cancelReceiveTimeout cancelReceiveTimeout
if (receiveTimeout.isDefined && dispatcher.mailboxSize(this) <= 0) { //Only reschedule if desired and there are currently no more messages to be processed if (receiveTimeout.isDefined && dispatcher.mailboxSize(this) <= 0) { //Only reschedule if desired and there are currently no more messages to be processed
log.debug("Scheduling timeout for %s", this) log.slf4j.debug("Scheduling timeout for %s", this)
_futureTimeout = Some(Scheduler.scheduleOnce(this, ReceiveTimeout, receiveTimeout.get, TimeUnit.MILLISECONDS)) _futureTimeout = Some(Scheduler.scheduleOnce(this, ReceiveTimeout, receiveTimeout.get, TimeUnit.MILLISECONDS))
} }
} }
@ -587,7 +587,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
if (_futureTimeout.isDefined) { if (_futureTimeout.isDefined) {
_futureTimeout.get.cancel(true) _futureTimeout.get.cancel(true)
_futureTimeout = None _futureTimeout = None
log.debug("Timeout canceled for %s", this) log.slf4j.debug("Timeout canceled for %s", this)
} }
} }
} }
@ -764,7 +764,7 @@ class LocalActorRef private[akka] (
"Actor can only have one supervisor [" + actorRef + "], e.g. link(actor) fails") "Actor can only have one supervisor [" + actorRef + "], e.g. link(actor) fails")
linkedActors.put(actorRef.uuid, actorRef) linkedActors.put(actorRef.uuid, actorRef)
actorRef.supervisor = Some(this) actorRef.supervisor = Some(this)
Actor.log.debug("Linking actor [%s] to actor [%s]", actorRef, this) Actor.log.slf4j.debug("Linking actor [%s] to actor [%s]", actorRef, this)
} }
/** /**
@ -777,7 +777,7 @@ class LocalActorRef private[akka] (
"Actor [" + actorRef + "] is not a linked actor, can't unlink") "Actor [" + actorRef + "] is not a linked actor, can't unlink")
linkedActors.remove(actorRef.uuid) linkedActors.remove(actorRef.uuid)
actorRef.supervisor = None actorRef.supervisor = None
Actor.log.debug("Unlinking actor [%s] from actor [%s]", actorRef, this) Actor.log.slf4j.debug("Unlinking actor [%s] from actor [%s]", actorRef, this)
} }
/** /**
@ -911,14 +911,15 @@ class LocalActorRef private[akka] (
* Callback for the dispatcher. This is the single entry point to the user Actor implementation. * Callback for the dispatcher. This is the single entry point to the user Actor implementation.
*/ */
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = guard.withGuard { protected[akka] def invoke(messageHandle: MessageInvocation): Unit = guard.withGuard {
if (isShutdown) Actor.log.warning("Actor [%s] is shut down,\n\tignoring message [%s]", toString, messageHandle) if (isShutdown) Actor.log.slf4j.warn("Actor [%s] is shut down,\n\tignoring message [%s]", toString, messageHandle)
else { else {
currentMessage = messageHandle currentMessage = messageHandle
try { try {
dispatch(messageHandle) dispatch(messageHandle)
} catch { } catch {
case e => case e =>
Actor.log.error(e, "Could not invoke actor [%s]", this) Actor.log.slf4j.error("Could not invoke actor [%s]", this)
Actor.log.slf4j.error("Problem", e)
throw e throw e
} finally { } finally {
currentMessage = null //TODO: Don't reset this, we might want to resend the message currentMessage = null //TODO: Don't reset this, we might want to resend the message
@ -972,9 +973,9 @@ class LocalActorRef private[akka] (
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) { protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) {
def performRestart { def performRestart {
Actor.log.info("Restarting actor [%s] configured as PERMANENT.", id) Actor.log.slf4j.info("Restarting actor [%s] configured as PERMANENT.", id)
val failedActor = actorInstance.get val failedActor = actorInstance.get
Actor.log.debug("Invoking 'preRestart' for failed actor instance [%s].", id) Actor.log.slf4j.debug("Invoking 'preRestart' for failed actor instance [%s].", id)
failedActor.preRestart(reason) failedActor.preRestart(reason)
val freshActor = newActor val freshActor = newActor
setActorSelfFields(failedActor,null) //Only null out the references if we could instantiate the new actor setActorSelfFields(failedActor,null) //Only null out the references if we could instantiate the new actor
@ -984,22 +985,22 @@ class LocalActorRef private[akka] (
case p: Proxyable => p.swapProxiedActor(freshActor) case p: Proxyable => p.swapProxiedActor(freshActor)
case _ => case _ =>
} }
Actor.log.debug("Invoking 'postRestart' for new actor instance [%s].", id) Actor.log.slf4j.debug("Invoking 'postRestart' for new actor instance [%s].", id)
freshActor.postRestart(reason) freshActor.postRestart(reason)
} }
def tooManyRestarts { def tooManyRestarts {
Actor.log.warning( Actor.log.slf4j.warn(
"Maximum number of restarts [%s] within time range [%s] reached." + "Maximum number of restarts [%s] within time range [%s] reached." +
"\n\tWill *not* restart actor [%s] anymore." + "\n\tWill *not* restart actor [%s] anymore." +
"\n\tLast exception causing restart was" + "\n\tLast exception causing restart was" +
"\n\t[%s].", "\n\t[%s].",
maxNrOfRetries, withinTimeRange, this, reason) Array(maxNrOfRetries, withinTimeRange, this, reason))
_supervisor.foreach { sup => _supervisor.foreach { sup =>
// can supervisor handle the notification? // can supervisor handle the notification?
val notification = MaximumNumberOfRestartsWithinTimeRangeReached(this, maxNrOfRetries, withinTimeRange, reason) val notification = MaximumNumberOfRestartsWithinTimeRangeReached(this, maxNrOfRetries, withinTimeRange, reason)
if (sup.isDefinedAt(notification)) notifySupervisorWithMessage(notification) if (sup.isDefinedAt(notification)) notifySupervisorWithMessage(notification)
else Actor.log.warning( else Actor.log.slf4j.warn(
"No message handler defined for system message [MaximumNumberOfRestartsWithinTimeRangeReached]" + "No message handler defined for system message [MaximumNumberOfRestartsWithinTimeRangeReached]" +
"\n\tCan't send the message to the supervisor [%s].", sup) "\n\tCan't send the message to the supervisor [%s].", sup)
} }
@ -1024,7 +1025,7 @@ class LocalActorRef private[akka] (
case e => false //An error or exception here should trigger a retry case e => false //An error or exception here should trigger a retry
} }
Actor.log.debug("Restart: %s for [%s].", success, id) Actor.log.slf4j.debug("Restart: %s for [%s].", success, id)
if (success) { if (success) {
_status = ActorRefInternals.RUNNING _status = ActorRefInternals.RUNNING
@ -1079,7 +1080,7 @@ class LocalActorRef private[akka] (
} }
private def dispatch[T](messageHandle: MessageInvocation) = { private def dispatch[T](messageHandle: MessageInvocation) = {
Actor.log.trace("Invoking actor with message: %s\n", messageHandle) Actor.log.slf4j.trace("Invoking actor with message: %s\n", messageHandle)
val message = messageHandle.message //serializeMessage(messageHandle.message) val message = messageHandle.message //serializeMessage(messageHandle.message)
try { try {
@ -1095,12 +1096,12 @@ class LocalActorRef private[akka] (
} }
private def shutDownTemporaryActor(temporaryActor: ActorRef) { private def shutDownTemporaryActor(temporaryActor: ActorRef) {
Actor.log.info("Actor [%s] configured as TEMPORARY and will not be restarted.", temporaryActor.id) Actor.log.slf4j.info("Actor [%s] configured as TEMPORARY and will not be restarted.", temporaryActor.id)
temporaryActor.stop temporaryActor.stop
linkedActors.remove(temporaryActor.uuid) // remove the temporary actor linkedActors.remove(temporaryActor.uuid) // remove the temporary actor
// if last temporary actor is gone, then unlink me from supervisor // if last temporary actor is gone, then unlink me from supervisor
if (linkedActors.isEmpty) { if (linkedActors.isEmpty) {
Actor.log.info( Actor.log.slf4j.info(
"All linked actors have died permanently (they were all configured as TEMPORARY)" + "All linked actors have died permanently (they were all configured as TEMPORARY)" +
"\n\tshutting down and unlinking supervisor actor as well [%s].", "\n\tshutting down and unlinking supervisor actor as well [%s].",
temporaryActor.id) temporaryActor.id)
@ -1111,7 +1112,8 @@ class LocalActorRef private[akka] (
} }
private def handleExceptionInDispatch(reason: Throwable, message: Any) = { private def handleExceptionInDispatch(reason: Throwable, message: Any) = {
Actor.log.error(reason, "Exception when invoking \n\tactor [%s] \n\twith message [%s]", this, message) Actor.log.slf4j.error("Exception when invoking \n\tactor [%s] \n\twith message [%s]", this, message)
Actor.log.slf4j.error("Problem", reason)
//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)
@ -1168,7 +1170,7 @@ class LocalActorRef private[akka] (
private def initializeActorInstance = { private def initializeActorInstance = {
actor.preStart // run actor preStart actor.preStart // run actor preStart
Actor.log.trace("[%s] has started", toString) Actor.log.slf4j.trace("[%s] has started", toString)
ActorRegistry.register(this) ActorRegistry.register(this)
} }

View file

@ -254,7 +254,7 @@ object ActorRegistry extends ListenerManagement {
* Shuts down and unregisters all actors in the system. * Shuts down and unregisters all actors in the system.
*/ */
def shutdownAll() { def shutdownAll() {
log.info("Shutting down all actors in the system...") log.slf4j.info("Shutting down all actors in the system...")
if (TypedActorModule.isEnabled) { if (TypedActorModule.isEnabled) {
val elements = actorsByUUID.elements val elements = actorsByUUID.elements
while (elements.hasMoreElements) { while (elements.hasMoreElements) {
@ -266,7 +266,7 @@ object ActorRegistry extends ListenerManagement {
} else foreach(_.stop) } else foreach(_.stop)
actorsByUUID.clear actorsByUUID.clear
actorsById.clear actorsById.clear
log.info("All actors have been shut down and unregistered from ActorRegistry") log.slf4j.info("All actors have been shut down and unregistered from ActorRegistry")
} }
/** /**

View file

@ -26,7 +26,7 @@ trait BootableActorLoaderService extends Bootable with Logging {
val DEPLOY = HOME.getOrElse(throwNoAkkaHomeException) + "/deploy" val DEPLOY = HOME.getOrElse(throwNoAkkaHomeException) + "/deploy"
val DEPLOY_DIR = new File(DEPLOY) val DEPLOY_DIR = new File(DEPLOY)
if (!DEPLOY_DIR.exists) { if (!DEPLOY_DIR.exists) {
log.error("Could not find a deploy directory at [%s]", DEPLOY) log.slf4j.error("Could not find a deploy directory at [%s]", DEPLOY)
System.exit(-1) System.exit(-1)
} }
val filesToDeploy = DEPLOY_DIR.listFiles.toArray.toList val filesToDeploy = DEPLOY_DIR.listFiles.toArray.toList
@ -42,8 +42,8 @@ trait BootableActorLoaderService extends Bootable with Logging {
} }
} }
val toDeploy = filesToDeploy.map(_.toURI.toURL) val toDeploy = filesToDeploy.map(_.toURI.toURL)
log.info("Deploying applications from [%s]: [%s]", DEPLOY, toDeploy) log.slf4j.info("Deploying applications from [%s]: [%s]", DEPLOY, toDeploy)
log.debug("Loading dependencies [%s]", dependencyJars) log.slf4j.debug("Loading dependencies [%s]", dependencyJars)
val allJars = toDeploy ::: dependencyJars val allJars = toDeploy ::: dependencyJars
new URLClassLoader(allJars.toArray,Thread.currentThread.getContextClassLoader) new URLClassLoader(allJars.toArray,Thread.currentThread.getContextClassLoader)
@ -51,12 +51,12 @@ trait BootableActorLoaderService extends Bootable with Logging {
} }
abstract override def onLoad = { abstract override def onLoad = {
applicationLoader.foreach(_ => log.info("Creating /deploy class-loader")) applicationLoader.foreach(_ => log.slf4j.info("Creating /deploy class-loader"))
super.onLoad super.onLoad
for (loader <- applicationLoader; clazz <- BOOT_CLASSES) { for (loader <- applicationLoader; clazz <- BOOT_CLASSES) {
log.info("Loading boot class [%s]", clazz) log.slf4j.info("Loading boot class [%s]", clazz)
loader.loadClass(clazz).newInstance loader.loadClass(clazz).newInstance
} }
} }

View file

@ -69,17 +69,17 @@ trait FSM[S, D] {
private var handleEvent: StateFunction = { private var handleEvent: StateFunction = {
case Event(value, stateData) => case Event(value, stateData) =>
log.warning("Event %s not handled in state %s, staying at current state", value, currentState.stateName) log.slf4j.warn("Event %s not handled in state %s, staying at current state", value, currentState.stateName)
stay stay
} }
private var terminateEvent: PartialFunction[Reason, Unit] = { private var terminateEvent: PartialFunction[Reason, Unit] = {
case failure@Failure(_) => log.error("Stopping because of a %s", failure) case failure@Failure(_) => log.slf4j.error("Stopping because of a %s", failure)
case reason => log.info("Stopping because of reason: %s", reason) case reason => log.slf4j.info("Stopping because of reason: %s", reason)
} }
private var transitionEvent: PartialFunction[Transition, Unit] = { private var transitionEvent: PartialFunction[Transition, Unit] = {
case Transition(from, to) => log.debug("Transitioning from state %s to %s", from, to) case Transition(from, to) => log.slf4j.debug("Transitioning from state %s to %s", from, to)
} }
override final protected def receive: Receive = { override final protected def receive: Receive = {
@ -87,7 +87,7 @@ trait FSM[S, D] {
terminateEvent.apply(reason) terminateEvent.apply(reason)
self.stop self.stop
case StateTimeout if (self.dispatcher.mailboxSize(self) > 0) => case StateTimeout if (self.dispatcher.mailboxSize(self) > 0) =>
log.trace("Ignoring StateTimeout - ") log.slf4j.trace("Ignoring StateTimeout - ")
// state timeout when new message in queue, skip this timeout // state timeout when new message in queue, skip this timeout
case value => { case value => {
timeoutFuture = timeoutFuture.flatMap {ref => ref.cancel(true); None} timeoutFuture = timeoutFuture.flatMap {ref => ref.cancel(true); None}
@ -123,7 +123,7 @@ trait FSM[S, D] {
def replying(replyValue:Any): State = { def replying(replyValue:Any): State = {
self.sender match { self.sender match {
case Some(sender) => sender ! replyValue case Some(sender) => sender ! replyValue
case None => log.error("Unable to send reply value %s, no sender reference to reply to", replyValue) case None => log.slf4j.error("Unable to send reply value %s, no sender reference to reply to", replyValue)
} }
this this
} }

View file

@ -29,15 +29,15 @@ object Scheduler extends Logging {
@volatile private var service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory) @volatile private var service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory)
log.info("Starting up Scheduler") log.slf4j.info("Starting up Scheduler")
/** /**
* Schedules to send the specified message to the receiver after initialDelay and then repeated after delay * Schedules to send the specified message to the receiver after initialDelay and then repeated after delay
*/ */
def schedule(receiver: ActorRef, message: AnyRef, initialDelay: Long, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = { def schedule(receiver: ActorRef, message: AnyRef, initialDelay: Long, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = {
log.trace( log.slf4j.trace(
"Schedule scheduled event\n\tevent = [%s]\n\treceiver = [%s]\n\tinitialDelay = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]", "Schedule scheduled event\n\tevent = [%s]\n\treceiver = [%s]\n\tinitialDelay = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]",
message, receiver, initialDelay, delay, timeUnit) Array(message, receiver, initialDelay, delay, timeUnit))
try { try {
service.scheduleAtFixedRate( service.scheduleAtFixedRate(
new Runnable { def run = receiver ! message }, new Runnable { def run = receiver ! message },
@ -59,9 +59,9 @@ object Scheduler extends Logging {
* avoid blocking operations since this is executed in the schedulers thread * avoid blocking operations since this is executed in the schedulers thread
*/ */
def schedule(runnable: Runnable, initialDelay: Long, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = { def schedule(runnable: Runnable, initialDelay: Long, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = {
log.trace( log.slf4j.trace(
"Schedule scheduled event\n\trunnable = [%s]\n\tinitialDelay = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]", "Schedule scheduled event\n\trunnable = [%s]\n\tinitialDelay = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]",
runnable, initialDelay, delay, timeUnit) Array(runnable, initialDelay, delay, timeUnit))
try { try {
service.scheduleAtFixedRate(runnable,initialDelay, delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]] service.scheduleAtFixedRate(runnable,initialDelay, delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]]
@ -74,9 +74,9 @@ object Scheduler extends Logging {
* Schedules to send the specified message to the receiver after delay * Schedules to send the specified message to the receiver after delay
*/ */
def scheduleOnce(receiver: ActorRef, message: AnyRef, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = { def scheduleOnce(receiver: ActorRef, message: AnyRef, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = {
log.trace( log.slf4j.trace(
"Schedule one-time event\n\tevent = [%s]\n\treceiver = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]", "Schedule one-time event\n\tevent = [%s]\n\treceiver = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]",
message, receiver, delay, timeUnit) Array(message, receiver, delay, timeUnit))
try { try {
service.schedule( service.schedule(
new Runnable { def run = receiver ! message }, new Runnable { def run = receiver ! message },
@ -98,9 +98,9 @@ object Scheduler extends Logging {
* avoid blocking operations since the runnable is executed in the schedulers thread * avoid blocking operations since the runnable is executed in the schedulers thread
*/ */
def scheduleOnce(runnable: Runnable, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = { def scheduleOnce(runnable: Runnable, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = {
log.trace( log.slf4j.trace(
"Schedule one-time event\n\trunnable = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]", "Schedule one-time event\n\trunnable = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]",
runnable, delay, timeUnit) Array(runnable, delay, timeUnit))
try { try {
service.schedule(runnable,delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]] service.schedule(runnable,delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]]
} catch { } catch {
@ -109,12 +109,12 @@ object Scheduler extends Logging {
} }
def shutdown: Unit = synchronized { def shutdown: Unit = synchronized {
log.info("Shutting down Scheduler") log.slf4j.info("Shutting down Scheduler")
service.shutdown service.shutdown
} }
def restart: Unit = synchronized { def restart: Unit = synchronized {
log.info("Restarting Scheduler") log.slf4j.info("Restarting Scheduler")
shutdown shutdown
service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory) service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory)
} }

View file

@ -168,7 +168,7 @@ final class SupervisorActor private[akka] (handler: FaultHandlingStrategy) exten
// FIXME add a way to respond to MaximumNumberOfRestartsWithinTimeRangeReached in declaratively configured Supervisor // FIXME add a way to respond to MaximumNumberOfRestartsWithinTimeRangeReached in declaratively configured Supervisor
case MaximumNumberOfRestartsWithinTimeRangeReached( case MaximumNumberOfRestartsWithinTimeRangeReached(
victim, maxNrOfRetries, withinTimeRange, lastExceptionCausingRestart) => victim, maxNrOfRetries, withinTimeRange, lastExceptionCausingRestart) =>
Actor.log.warning( Actor.log.slf4j.warn(
"Declaratively configured supervisor received a [MaximumNumberOfRestartsWithinTimeRangeReached] notification," + "Declaratively configured supervisor received a [MaximumNumberOfRestartsWithinTimeRangeReached] notification," +
"\n\tbut there is currently no way of handling it in a declaratively configured supervisor." + "\n\tbut there is currently no way of handling it in a declaratively configured supervisor." +
"\n\tIf you want to be able to handle this error condition then you need to create the supervision tree programatically." + "\n\tIf you want to be able to handle this error condition then you need to create the supervision tree programatically." +

View file

@ -6,9 +6,6 @@ package akka.config
import akka.AkkaException import akka.AkkaException
import akka.util.Logging import akka.util.Logging
import akka.actor.{ActorRef, IllegalActorStateException}
import akka.dispatch.CompletableFuture
import net.lag.configgy.{Config => CConfig, Configgy, ParseException} import net.lag.configgy.{Config => CConfig, Configgy, ParseException}
import java.net.InetSocketAddress import java.net.InetSocketAddress
@ -17,14 +14,12 @@ import java.lang.reflect.Method
class ConfigurationException(message: String) extends AkkaException(message) class ConfigurationException(message: String) extends AkkaException(message)
class ModuleNotAvailableException(message: String) extends AkkaException(message) class ModuleNotAvailableException(message: String) extends AkkaException(message)
object ConfigLogger extends Logging
/** /**
* Loads up the configuration (from the akka.conf file). * Loads up the configuration (from the akka.conf file).
* *
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
object Config { object Config extends Logging {
val VERSION = "1.0-SNAPSHOT" val VERSION = "1.0-SNAPSHOT"
val HOME = { val HOME = {
@ -62,7 +57,7 @@ object Config {
val configFile = System.getProperty("akka.config", "") val configFile = System.getProperty("akka.config", "")
try { try {
Configgy.configure(configFile) Configgy.configure(configFile)
ConfigLogger.log.info("Config loaded from -Dakka.config=%s", configFile) log.slf4j.info("Config loaded from -Dakka.config=%s", configFile)
} catch { } catch {
case e: ParseException => throw new ConfigurationException( case e: ParseException => throw new ConfigurationException(
"Config could not be loaded from -Dakka.config=" + configFile + "Config could not be loaded from -Dakka.config=" + configFile +
@ -73,7 +68,7 @@ object Config {
try { try {
val configFile = HOME.getOrElse(throwNoAkkaHomeException) + "/config/" + confName val configFile = HOME.getOrElse(throwNoAkkaHomeException) + "/config/" + confName
Configgy.configure(configFile) Configgy.configure(configFile)
ConfigLogger.log.info( log.slf4j.info(
"AKKA_HOME is defined as [%s], config loaded from [%s].", "AKKA_HOME is defined as [%s], config loaded from [%s].",
HOME.getOrElse(throwNoAkkaHomeException), HOME.getOrElse(throwNoAkkaHomeException),
configFile) configFile)
@ -87,7 +82,7 @@ object Config {
} else if (getClass.getClassLoader.getResource(confName) ne null) { } else if (getClass.getClassLoader.getResource(confName) ne null) {
try { try {
Configgy.configureFromResource(confName, getClass.getClassLoader) Configgy.configureFromResource(confName, getClass.getClassLoader)
ConfigLogger.log.info("Config [%s] loaded from the application classpath.",confName) log.slf4j.info("Config [%s] loaded from the application classpath.",confName)
} catch { } catch {
case e: ParseException => throw new ConfigurationException( case e: ParseException => throw new ConfigurationException(
"Can't load '" + confName + "' config file from application classpath," + "Can't load '" + confName + "' config file from application classpath," +
@ -95,7 +90,7 @@ object Config {
} }
Configgy.config Configgy.config
} else { } else {
ConfigLogger.log.warning( log.slf4j.warn(
"\nCan't load '" + confName + "'." + "\nCan't load '" + confName + "'." +
"\nOne of the three ways of locating the '" + confName + "' file needs to be defined:" + "\nOne of the three ways of locating the '" + confName + "' file needs to be defined:" +
"\n\t1. Define the '-Dakka.config=...' system property option." + "\n\t1. Define the '-Dakka.config=...' system property option." +

View file

@ -123,12 +123,12 @@ class ExecutorBasedEventDrivenDispatcher(
private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef =
createMailbox(mailboxType.mailboxImplClassname, actorRef) createMailbox(mailboxType.mailboxImplClassname, actorRef)
private[akka] def start = log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput) private[akka] def start = log.slf4j.debug("Starting up %s\n\twith throughput [%d]", toString, throughput)
private[akka] def shutdown { private[akka] def shutdown {
val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory))
if (old ne null) { if (old ne null) {
log.debug("Shutting down %s", toString) log.slf4j.debug("Shutting down %s", toString)
old.shutdownNow() old.shutdownNow()
} }
} }
@ -144,17 +144,17 @@ class ExecutorBasedEventDrivenDispatcher(
throw e throw e
} }
} }
} else log.warning("%s is shut down,\n\tignoring the rest of the messages in the mailbox of\n\t%s", this, mbox) } else log.slf4j.warn("%s is shut down,\n\tignoring the rest of the messages in the mailbox of\n\t%s", this, mbox)
override val toString = getClass.getSimpleName + "[" + name + "]" override val toString = getClass.getSimpleName + "[" + name + "]"
def suspend(actorRef: ActorRef) { def suspend(actorRef: ActorRef) {
log.debug("Suspending %s",actorRef.uuid) log.slf4j.debug("Suspending %s",actorRef.uuid)
getMailbox(actorRef).suspended.switchOn getMailbox(actorRef).suspended.switchOn
} }
def resume(actorRef: ActorRef) { def resume(actorRef: ActorRef) {
log.debug("Resuming %s",actorRef.uuid) log.slf4j.debug("Resuming %s",actorRef.uuid)
val mbox = getMailbox(actorRef) val mbox = getMailbox(actorRef)
mbox.suspended.switchOff mbox.suspended.switchOff
registerForExecution(mbox) registerForExecution(mbox)

View file

@ -171,12 +171,12 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
} else false } else false
} }
private[akka] def start = log.debug("Starting up %s",toString) private[akka] def start = log.slf4j.debug("Starting up %s",toString)
private[akka] def shutdown { private[akka] def shutdown {
val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory)) val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory))
if (old ne null) { if (old ne null) {
log.debug("Shutting down %s", toString) log.slf4j.debug("Shutting down %s", toString)
old.shutdownNow() old.shutdownNow()
} }
} }

View file

@ -135,7 +135,7 @@ trait MessageDispatcher extends MailboxFactory with Logging {
ActorRegistry.actorFor(uuid) match { ActorRegistry.actorFor(uuid) match {
case Some(actor) => actor.stop case Some(actor) => actor.stop
case None => case None =>
log.error("stopAllLinkedActors couldn't find linked actor: " + uuid) log.slf4j.error("stopAllLinkedActors couldn't find linked actor: " + uuid)
} }
} }
} }

View file

@ -174,18 +174,18 @@ class MonitorableThread(runnable: Runnable, name: String)
setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
def uncaughtException(thread: Thread, cause: Throwable) = def uncaughtException(thread: Thread, cause: Throwable) =
log.error(cause, "UNCAUGHT in thread [%s]", thread.getName) log.slf4j.error("Thread.UncaughtException", cause)
}) })
override def run = { override def run = {
val debug = MonitorableThread.debugLifecycle val debug = MonitorableThread.debugLifecycle
log.debug("Created thread %s", getName) log.slf4j.debug("Created thread %s", getName)
try { try {
MonitorableThread.alive.incrementAndGet MonitorableThread.alive.incrementAndGet
super.run super.run
} finally { } finally {
MonitorableThread.alive.decrementAndGet MonitorableThread.alive.decrementAndGet
log.debug("Exiting thread %s", getName) log.slf4j.debug("Exiting thread %s", getName)
} }
} }
} }
@ -212,7 +212,7 @@ class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extend
case e: RejectedExecutionException => case e: RejectedExecutionException =>
semaphore.release semaphore.release
case e => case e =>
log.error(e,"Unexpected exception") log.slf4j.error("Unexpected exception", e)
throw e throw e
} }
} }
@ -254,7 +254,7 @@ trait LazyExecutorService extends ExecutorServiceDelegate {
def createExecutor: ExecutorService def createExecutor: ExecutorService
lazy val executor = { lazy val executor = {
log.info("Lazily initializing ExecutorService for ",this) log.slf4j.info("Lazily initializing ExecutorService for ",this)
createExecutor createExecutor
} }
} }

View file

@ -36,7 +36,7 @@ import java.net.{InetAddress, UnknownHostException}
} }
private lazy val _log = { private lazy val _log = {
AkkaException.log.error(toString) AkkaException.log.slf4j.error(toString)
() ()
} }

View file

@ -24,7 +24,7 @@ object Crypt extends Logging {
def sha1(bytes: Array[Byte]): String = digest(bytes, MessageDigest.getInstance("SHA1")) def sha1(bytes: Array[Byte]): String = digest(bytes, MessageDigest.getInstance("SHA1"))
def generateSecureCookie: String = { def generateSecureCookie: String = {
log.info("Generating secure cookie...") log.slf4j.info("Generating secure cookie...")
val bytes = Array.fill(32)(0.byteValue) val bytes = Array.fill(32)(0.byteValue)
random.nextBytes(bytes) random.nextBytes(bytes)
sha1(bytes) sha1(bytes)

View file

@ -42,7 +42,8 @@ object Helpers extends Logging {
narrow(o) narrow(o)
} catch { } catch {
case e: ClassCastException => case e: ClassCastException =>
log.warning(e, "Cannot narrow %s to expected type %s!", o, implicitly[Manifest[T]].erasure.getName) log.slf4j.warn("Cannot narrow %s to expected type %s!", o, implicitly[Manifest[T]].erasure.getName)
log.slf4j.trace("narrowSilently", e)
None None
} }

View file

@ -57,7 +57,7 @@ trait ListenerManagement extends Logging {
while (iterator.hasNext) { while (iterator.hasNext) {
val listener = iterator.next val listener = iterator.next
if (listener.isRunning) listener ! msg if (listener.isRunning) listener ! msg
else log.warning("Can't notify [%s] since it is not running.", listener) else log.slf4j.warn("Can't notify [%s] since it is not running.", listener)
} }
} }
} }
@ -70,7 +70,7 @@ trait ListenerManagement extends Logging {
while (iterator.hasNext) { while (iterator.hasNext) {
val listener = iterator.next val listener = iterator.next
if (listener.isRunning) f(listener) if (listener.isRunning) f(listener)
else log.warning("Can't notify [%s] since it is not running.", listener) else log.slf4j.warn("Can't notify [%s] since it is not running.", listener)
} }
} }
} }

View file

@ -26,117 +26,118 @@ trait Logging {
* Example: * Example:
* <pre> * <pre>
* class Foo extends Logging { * class Foo extends Logging {
* log.info("My foo is %s","alive") * log.slf4j.info("My foo is %s","alive")
* log.error(new Exception(),"My foo is %s","broken") * log.slf4j.error(new Exception(),"My foo is %s","broken")
* } * }
* </pre> * </pre>
* *
* The logger uses String.format: * The logger uses String.format:
* http://download-llnw.oracle.com/javase/6/docs/api/java/lang/String.html#format(java.lang.String,%20java.lang.Object...) * http://download-llnw.oracle.com/javase/6/docs/api/java/lang/String.html#format(java.lang.String,%20java.lang.Object...)
*/ */
class Logger(val logger: SLFLogger) { class Logger(val slf4j: SLFLogger) {
def name = logger.getName final def name = logger.getName
final def logger = slf4j
def trace_? = logger.isTraceEnabled final def trace_? = logger.isTraceEnabled
def debug_? = logger.isDebugEnabled final def debug_? = logger.isDebugEnabled
def info_? = logger.isInfoEnabled final def info_? = logger.isInfoEnabled
def warning_? = logger.isWarnEnabled final def warning_? = logger.isWarnEnabled
def error_? = logger.isErrorEnabled final def error_? = logger.isErrorEnabled
//Trace //Trace
def trace(t: Throwable, fmt: => String, arg: Any, argN: Any*) { final def trace(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
trace(t,message(fmt,arg,argN:_*)) trace(t,message(fmt,arg,argN:_*))
} }
def trace(t: Throwable, msg: => String) { final def trace(t: Throwable, msg: => String) {
if (trace_?) logger.trace(msg,t) if (trace_?) logger.trace(msg,t)
} }
def trace(fmt: => String, arg: Any, argN: Any*) { final def trace(fmt: => String, arg: Any, argN: Any*) {
trace(message(fmt,arg,argN:_*)) trace(message(fmt,arg,argN:_*))
} }
def trace(msg: => String) { final def trace(msg: => String) {
if (trace_?) logger trace msg if (trace_?) logger trace msg
} }
//Debug //Debug
def debug(t: Throwable, fmt: => String, arg: Any, argN: Any*) { final def debug(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
debug(t,message(fmt,arg,argN:_*)) debug(t,message(fmt,arg,argN:_*))
} }
def debug(t: Throwable, msg: => String) { final def debug(t: Throwable, msg: => String) {
if (debug_?) logger.debug(msg,t) if (debug_?) logger.debug(msg,t)
} }
def debug(fmt: => String, arg: Any, argN: Any*) { final def debug(fmt: => String, arg: Any, argN: Any*) {
debug(message(fmt,arg,argN:_*)) debug(message(fmt,arg,argN:_*))
} }
def debug(msg: => String) { final def debug(msg: => String) {
if (debug_?) logger debug msg if (debug_?) logger debug msg
} }
//Info //Info
def info(t: Throwable, fmt: => String, arg: Any, argN: Any*) { final def info(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
info(t,message(fmt,arg,argN:_*)) info(t,message(fmt,arg,argN:_*))
} }
def info(t: Throwable, msg: => String) { final def info(t: Throwable, msg: => String) {
if (info_?) logger.info(msg,t) if (info_?) logger.info(msg,t)
} }
def info(fmt: => String, arg: Any, argN: Any*) { final def info(fmt: => String, arg: Any, argN: Any*) {
info(message(fmt,arg,argN:_*)) info(message(fmt,arg,argN:_*))
} }
def info(msg: => String) { final def info(msg: => String) {
if (info_?) logger info msg if (info_?) logger info msg
} }
//Warning //Warning
def warning(t: Throwable, fmt: => String, arg: Any, argN: Any*) { final def warning(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
warning(t,message(fmt,arg,argN:_*)) warning(t,message(fmt,arg,argN:_*))
} }
def warn(t: Throwable, fmt: => String, arg: Any, argN: Any*) = warning(t, fmt, arg, argN) final def warn(t: Throwable, fmt: => String, arg: Any, argN: Any*) = warning(t, fmt, arg, argN)
def warning(t: Throwable, msg: => String) { final def warning(t: Throwable, msg: => String) {
if (warning_?) logger.warn(msg,t) if (warning_?) logger.warn(msg,t)
} }
def warn(t: Throwable, msg: => String) = warning(t, msg) final def warn(t: Throwable, msg: => String) = warning(t, msg)
def warning(fmt: => String, arg: Any, argN: Any*) { final def warning(fmt: => String, arg: Any, argN: Any*) {
warning(message(fmt,arg,argN:_*)) warning(message(fmt,arg,argN:_*))
} }
def warn(fmt: => String, arg: Any, argN: Any*) = warning(fmt, arg, argN:_*) final def warn(fmt: => String, arg: Any, argN: Any*) = warning(fmt, arg, argN:_*)
def warning(msg: => String) { final def warning(msg: => String) {
if (warning_?) logger warn msg if (warning_?) logger warn msg
} }
def warn(msg: => String) = warning(msg) final def warn(msg: => String) = warning(msg)
//Error //Error
def error(t: Throwable, fmt: => String, arg: Any, argN: Any*) { final def error(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
error(t,message(fmt,arg,argN:_*)) error(t,message(fmt,arg,argN:_*))
} }
def error(t: Throwable, msg: => String) { final def error(t: Throwable, msg: => String) {
if (error_?) logger.error(msg,t) if (error_?) logger.error(msg,t)
} }
def error(fmt: => String, arg: Any, argN: Any*) { final def error(fmt: => String, arg: Any, argN: Any*) {
error(message(fmt,arg,argN:_*)) error(message(fmt,arg,argN:_*))
} }
def error(msg: => String) { final def error(msg: => String) {
if (error_?) logger error msg if (error_?) logger error msg
} }
protected def message(fmt: String, arg: Any, argN: Any*) : String = { protected final def message(fmt: String, arg: Any, argN: Any*) : String = {
if ((argN eq null) || argN.isEmpty) fmt.format(arg) if ((argN eq null) || argN.isEmpty) fmt.format(arg)
else fmt.format((arg +: argN):_*) else fmt.format((arg +: argN):_*)
} }

View file

@ -185,7 +185,7 @@ object ReflectiveAccess extends Logging {
Some(ctor.newInstance(args: _*).asInstanceOf[T]) Some(ctor.newInstance(args: _*).asInstanceOf[T])
} catch { } catch {
case e => case e =>
log.warning("Could not instantiate class [%s] due to [%s]", clazz.getName, e.getCause) log.slf4j.warn("Could not instantiate class [%s] due to [%s]", clazz.getName, e.getCause)
None None
} }
@ -202,7 +202,7 @@ object ReflectiveAccess extends Logging {
Some(ctor.newInstance(args: _*).asInstanceOf[T]) Some(ctor.newInstance(args: _*).asInstanceOf[T])
} catch { } catch {
case e => case e =>
log.warning("Could not instantiate class [%s] due to [%s]", fqn, e.getCause) log.slf4j.warn("Could not instantiate class [%s] due to [%s]", fqn, e.getCause)
None None
} }
@ -214,7 +214,7 @@ object ReflectiveAccess extends Logging {
Option(instance.get(null).asInstanceOf[T]) Option(instance.get(null).asInstanceOf[T])
} catch { } catch {
case e: ClassNotFoundException => case e: ClassNotFoundException =>
log.debug("Could not get object [%s] due to [%s]", fqn, e) log.slf4j.debug("Could not get object [%s] due to [%s]", fqn, e)
None None
} }

View file

@ -40,7 +40,7 @@ object FSMActorSpec {
goto(Open) using CodeState("", code) until timeout goto(Open) using CodeState("", code) until timeout
} }
case wrong => { case wrong => {
log.error("Wrong code %s", wrong) log.slf4j.error("Wrong code %s", wrong)
stay using CodeState("", code) stay using CodeState("", code)
} }
} }
@ -60,7 +60,7 @@ object FSMActorSpec {
whenUnhandled { whenUnhandled {
case Event(_, stateData) => { case Event(_, stateData) => {
log.info("Unhandled") log.slf4j.info("Unhandled")
unhandledLatch.open unhandledLatch.open
stay stay
} }
@ -71,12 +71,12 @@ object FSMActorSpec {
} }
private def doLock() { private def doLock() {
log.info("Locked") log.slf4j.info("Locked")
lockedLatch.open lockedLatch.open
} }
private def doUnlock = { private def doUnlock = {
log.info("Unlocked") log.slf4j.info("Unlocked")
unlockedLatch.open unlockedLatch.open
} }
} }

View file

@ -498,7 +498,7 @@ class SupervisorSpec extends JUnitSuite {
val inits = new AtomicInteger(0) val inits = new AtomicInteger(0)
val dyingActor = actorOf(new Actor { val dyingActor = actorOf(new Actor {
self.lifeCycle = Permanent self.lifeCycle = Permanent
log.debug("Creating dying actor, attempt: " + inits.incrementAndGet) log.slf4j.debug("Creating dying actor, attempt: " + inits.incrementAndGet)
if (!(inits.get % 2 != 0)) if (!(inits.get % 2 != 0))
throw new IllegalStateException("Don't wanna!") throw new IllegalStateException("Don't wanna!")

View file

@ -24,10 +24,10 @@ class AkkaLoader extends Logging {
def boot(withBanner: Boolean, b : Bootable): Unit = synchronized { def boot(withBanner: Boolean, b : Bootable): Unit = synchronized {
if (!hasBooted) { if (!hasBooted) {
if (withBanner) printBanner if (withBanner) printBanner
log.info("Starting Akka...") log.slf4j.info("Starting Akka...")
b.onLoad b.onLoad
Thread.currentThread.setContextClassLoader(getClass.getClassLoader) Thread.currentThread.setContextClassLoader(getClass.getClassLoader)
log.info("Akka started successfully") log.slf4j.info("Akka started successfully")
hasBooted = true hasBooted = true
_bundles = Some(b) _bundles = Some(b)
} }
@ -38,40 +38,40 @@ class AkkaLoader extends Logging {
*/ */
def shutdown = synchronized { def shutdown = synchronized {
if (hasBooted) { if (hasBooted) {
log.info("Shutting down Akka...") log.slf4j.info("Shutting down Akka...")
_bundles.foreach(_.onUnload) _bundles.foreach(_.onUnload)
_bundles = None _bundles = None
Actor.shutdownHook.run Actor.shutdownHook.run
log.info("Akka succesfully shut down") log.slf4j.info("Akka succesfully shut down")
} }
} }
private def printBanner = { private def printBanner = {
log.info("==================================================") log.slf4j.info("==================================================")
log.info(" t") log.slf4j.info(" t")
log.info(" t t t") log.slf4j.info(" t t t")
log.info(" t t tt t") log.slf4j.info(" t t tt t")
log.info(" tt t t tt t") log.slf4j.info(" tt t t tt t")
log.info(" t ttttttt t ttt t") log.slf4j.info(" t ttttttt t ttt t")
log.info(" t tt ttt t ttt t") log.slf4j.info(" t tt ttt t ttt t")
log.info(" t t ttt t ttt t t") log.slf4j.info(" t t ttt t ttt t t")
log.info(" tt t ttt ttt ttt t") log.slf4j.info(" tt t ttt ttt ttt t")
log.info(" t t ttt ttt t tt t") log.slf4j.info(" t t ttt ttt t tt t")
log.info(" t ttt ttt t t") log.slf4j.info(" t ttt ttt t t")
log.info(" tt ttt ttt t") log.slf4j.info(" tt ttt ttt t")
log.info(" ttt ttt") log.slf4j.info(" ttt ttt")
log.info(" tttttttt ttt ttt ttt ttt tttttttt") log.slf4j.info(" tttttttt ttt ttt ttt ttt tttttttt")
log.info(" ttt tt ttt ttt ttt ttt ttt ttt") log.slf4j.info(" ttt tt ttt ttt ttt ttt ttt ttt")
log.info(" ttt ttt ttt ttt ttt ttt ttt ttt") log.slf4j.info(" ttt ttt ttt ttt ttt ttt ttt ttt")
log.info(" ttt ttt ttt ttt ttt tt ttt ttt") log.slf4j.info(" ttt ttt ttt ttt ttt tt ttt ttt")
log.info(" tttt ttttttttt tttttttt tttt") log.slf4j.info(" tttt ttttttttt tttttttt tttt")
log.info(" ttttttttt ttt ttt ttt ttt ttttttttt") log.slf4j.info(" ttttttttt ttt ttt ttt ttt ttttttttt")
log.info(" ttt ttt ttt ttt ttt ttt ttt ttt") log.slf4j.info(" ttt ttt ttt ttt ttt ttt ttt ttt")
log.info(" ttt ttt ttt ttt ttt ttt ttt ttt") log.slf4j.info(" ttt ttt ttt ttt ttt ttt ttt ttt")
log.info(" ttt tt ttt ttt ttt ttt ttt ttt") log.slf4j.info(" ttt tt ttt ttt ttt ttt ttt ttt")
log.info(" tttttttt ttt ttt ttt ttt tttttttt") log.slf4j.info(" tttttttt ttt ttt ttt ttt tttttttt")
log.info("==================================================") log.slf4j.info("==================================================")
log.info(" Running version %s", Config.VERSION) log.slf4j.info(" Running version %s", Config.VERSION)
log.info("==================================================") log.slf4j.info("==================================================")
} }
} }

View file

@ -35,7 +35,7 @@ trait EmbeddedAppServer extends Bootable with Logging {
abstract override def onLoad = { abstract override def onLoad = {
super.onLoad super.onLoad
if (isRestEnabled) { if (isRestEnabled) {
log.info("Attempting to start Akka HTTP service") log.slf4j.info("Attempting to start Akka HTTP service")
System.setProperty("jetty.port", REST_PORT.toString) System.setProperty("jetty.port", REST_PORT.toString)
System.setProperty("jetty.host", REST_HOSTNAME) System.setProperty("jetty.host", REST_HOSTNAME)
@ -60,14 +60,14 @@ trait EmbeddedAppServer extends Bootable with Logging {
s.start() s.start()
s s
} }
log.info("Akka HTTP service started") log.slf4j.info("Akka HTTP service started")
} }
} }
abstract override def onUnload = { abstract override def onUnload = {
super.onUnload super.onUnload
server foreach { t => server foreach { t =>
log.info("Shutting down REST service (Jersey)") log.slf4j.info("Shutting down REST service (Jersey)")
t.stop() t.stop()
} }
} }

View file

@ -117,16 +117,16 @@ trait Mist extends Logging {
val server = context.getServerInfo val server = context.getServerInfo
val (major, minor) = (context.getMajorVersion, context.getMinorVersion) val (major, minor) = (context.getMajorVersion, context.getMinorVersion)
log.info("Initializing Akka HTTP on "+server+" with Servlet API "+major+"."+minor) log.slf4j.info("Initializing Akka HTTP on {} with Servlet API {}.{}",Array(server, major, minor))
_factory = if (major >= 3) { _factory = if (major >= 3) {
log.info("Supporting Java asynchronous contexts.") log.slf4j.info("Supporting Java asynchronous contexts.")
Some(Servlet30ContextMethodFactory) Some(Servlet30ContextMethodFactory)
} else if (server.toLowerCase startsWith JettyServer) { } else if (server.toLowerCase startsWith JettyServer) {
log.info("Supporting Jetty asynchronous continuations.") log.slf4j.info("Supporting Jetty asynchronous continuations.")
Some(JettyContinuationMethodFactory) Some(JettyContinuationMethodFactory)
} else { } else {
log.error("No asynchronous request handling can be supported.") log.slf4j.error("No asynchronous request handling can be supported.")
None None
} }
} }
@ -184,7 +184,7 @@ class AkkaMistFilter extends Filter with Mist {
case "POST" => mistify(hreq, hres)(_factory.get.Post) case "POST" => mistify(hreq, hres)(_factory.get.Post)
case "PUT" => mistify(hreq, hres)(_factory.get.Put) case "PUT" => mistify(hreq, hres)(_factory.get.Put)
case "TRACE" => mistify(hreq, hres)(_factory.get.Trace) case "TRACE" => mistify(hreq, hres)(_factory.get.Trace)
case unknown => log.warn("Unknown http method: %s",unknown) case unknown => log.slf4j.warn("Unknown http method: {}",unknown)
} }
chain.doFilter(req,res) chain.doFilter(req,res)
case _ => chain.doFilter(req,res) case _ => chain.doFilter(req,res)
@ -269,7 +269,7 @@ trait Endpoint { this: Actor =>
*/ */
protected def _na(uri: String, req: RequestMethod) = { protected def _na(uri: String, req: RequestMethod) = {
req.NotFound("No endpoint available for [" + uri + "]") req.NotFound("No endpoint available for [" + uri + "]")
log.debug("No endpoint available for [" + uri + "]") log.slf4j.debug("No endpoint available for [{}]", uri)
} }
} }
@ -299,7 +299,7 @@ class RootEndpoint extends Actor with Endpoint {
def recv: Receive = { def recv: Receive = {
case NoneAvailable(uri, req) => _na(uri, req) case NoneAvailable(uri, req) => _na(uri, req)
case unknown => log.error("Unexpected message sent to root endpoint. [" + unknown + "]") case unknown => log.slf4j.error("Unexpected message sent to root endpoint. [{}]", unknown)
} }
/** /**
@ -386,7 +386,7 @@ trait RequestMethod extends Logging
case Some(pipe) => { case Some(pipe) => {
try { try {
if (!suspended) { if (!suspended) {
log.warning("Attempt to complete an expired connection.") log.slf4j.warn("Attempt to complete an expired connection.")
false false
} }
else { else {
@ -396,13 +396,13 @@ trait RequestMethod extends Logging
} }
} catch { } catch {
case io => case io =>
log.error(io, "Failed to write data to connection on resume - the client probably disconnected") log.slf4j.error("Failed to write data to connection on resume - the client probably disconnected", io)
false false
} }
} }
case None => case None =>
log.error("Attempt to complete request with no context.") log.slf4j.error("Attempt to complete request with no context.")
false false
} }
@ -411,7 +411,7 @@ trait RequestMethod extends Logging
case Some(pipe) => { case Some(pipe) => {
try { try {
if (!suspended) { if (!suspended) {
log.warning("Attempt to complete an expired connection.") log.slf4j.warn("Attempt to complete an expired connection.")
} }
else { else {
response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, "Failed to write data to connection on resume") response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, "Failed to write data to connection on resume")
@ -419,15 +419,15 @@ trait RequestMethod extends Logging
} }
} }
catch { catch {
case io: IOException => log.error(io, "Request completed with internal error.") case io: IOException => log.slf4j.error("Request completed with internal error.", io)
} }
finally { finally {
log.error(t, "Request completed with internal error.") log.slf4j.error("Request completed with internal error.", t)
} }
} }
case None => case None =>
log.error(t, "Attempt to complete request with no context") log.slf4j.error("Attempt to complete request with no context", t)
} }
} }

View file

@ -91,7 +91,7 @@ class AkkaSecurityFilterFactory extends ResourceFilterFactory with Logging {
throw new WebApplicationException(r.asInstanceOf[Response]) throw new WebApplicationException(r.asInstanceOf[Response])
case None => throw new WebApplicationException(408) case None => throw new WebApplicationException(408)
case unknown => { case unknown => {
log.warning("Authenticator replied with unexpected result [%s]", unknown); log.slf4j.warn("Authenticator replied with unexpected result [%s]", unknown);
throw new WebApplicationException(Response.Status.INTERNAL_SERVER_ERROR) throw new WebApplicationException(Response.Status.INTERNAL_SERVER_ERROR)
} }
} }
@ -258,7 +258,7 @@ trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] w
val ts = System.currentTimeMillis val ts = System.currentTimeMillis
nonceMap.filter(tuple => (ts - tuple._2) < nonceValidityPeriod) nonceMap.filter(tuple => (ts - tuple._2) < nonceValidityPeriod)
case unknown => case unknown =>
log.error("Don't know what to do with: ", unknown) log.slf4j.error("Don't know what to do with: ", unknown)
} }
//Schedule the invalidation of nonces //Schedule the invalidation of nonces
@ -371,7 +371,7 @@ trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] w
Some(UserInfo(user, null, rolesFor(user))) Some(UserInfo(user, null, rolesFor(user)))
} catch { } catch {
case e: PrivilegedActionException => { case e: PrivilegedActionException => {
log.error(e, "Action not allowed") log.slf4j.error("Action not allowed", e)
return None return None
} }
} }

View file

@ -36,7 +36,7 @@ trait Servlet30Context extends AsyncListener with akka.util.Logging
} }
catch { catch {
case ex: IllegalStateException => case ex: IllegalStateException =>
log.info("Cannot update timeout - already returned to container") log.slf4j.info("Cannot update timeout - already returned to container")
false false
} }
} }
@ -46,8 +46,8 @@ trait Servlet30Context extends AsyncListener with akka.util.Logging
// //
def onComplete(e: AsyncEvent) {} def onComplete(e: AsyncEvent) {}
def onError(e: AsyncEvent) = e.getThrowable match { def onError(e: AsyncEvent) = e.getThrowable match {
case null => log.warning("Error occured...") case null => log.slf4j.warn("Error occured...")
case t => log.warning(t, "Error occured") case t => log.slf4j.warn("Error occured", t)
} }
def onStartAsync(e: AsyncEvent) {} def onStartAsync(e: AsyncEvent) {}
def onTimeout(e: AsyncEvent) = { def onTimeout(e: AsyncEvent) = {

View file

@ -27,18 +27,18 @@ trait BootableRemoteActorService extends Bootable with Logging {
abstract override def onLoad = { abstract override def onLoad = {
if (RemoteServer.isRemotingEnabled) { if (RemoteServer.isRemotingEnabled) {
log.info("Initializing Remote Actors Service...") log.slf4j.info("Initializing Remote Actors Service...")
startRemoteService startRemoteService
log.info("Remote Actors Service initialized") log.slf4j.info("Remote Actors Service initialized")
} }
super.onLoad super.onLoad
} }
abstract override def onUnload = { abstract override def onUnload = {
log.info("Shutting down Remote Actors Service") log.slf4j.info("Shutting down Remote Actors Service")
RemoteNode.shutdown RemoteNode.shutdown
if (remoteServerThread.isAlive) remoteServerThread.join(1000) if (remoteServerThread.isAlive) remoteServerThread.join(1000)
log.info("Remote Actors Service has been shut down") log.slf4j.info("Remote Actors Service has been shut down")
super.onUnload super.onUnload
} }
} }

View file

@ -238,7 +238,7 @@ class RemoteClient private[akka] (
bootstrap.setOption("tcpNoDelay", true) bootstrap.setOption("tcpNoDelay", true)
bootstrap.setOption("keepAlive", true) bootstrap.setOption("keepAlive", true)
log.info("Starting remote client connection to [%s:%s]", hostname, port) log.slf4j.info("Starting remote client connection to [%s:%s]", hostname, port)
// Wait until the connection attempt succeeds or fails. // Wait until the connection attempt succeeds or fails.
connection = bootstrap.connect(remoteAddress) connection = bootstrap.connect(remoteAddress)
@ -247,13 +247,14 @@ class RemoteClient private[akka] (
if (!connection.isSuccess) { if (!connection.isSuccess) {
notifyListeners(RemoteClientError(connection.getCause, this)) notifyListeners(RemoteClientError(connection.getCause, this))
log.error(connection.getCause, "Remote client connection to [%s:%s] has failed", hostname, port) log.slf4j.error("Remote client connection to [%s:%s] has failed", hostname, port)
log.slf4j.debug("Remote client connection failed", connection.getCause)
} }
notifyListeners(RemoteClientStarted(this)) notifyListeners(RemoteClientStarted(this))
} }
def shutdown = runSwitch switchOff { def shutdown = runSwitch switchOff {
log.info("Shutting down %s", name) log.slf4j.info("Shutting down %s", name)
notifyListeners(RemoteClientShutdown(this)) notifyListeners(RemoteClientShutdown(this))
timer.stop timer.stop
timer = null timer = null
@ -262,7 +263,7 @@ class RemoteClient private[akka] (
bootstrap.releaseExternalResources bootstrap.releaseExternalResources
bootstrap = null bootstrap = null
connection = null connection = null
log.info("%s has been shut down", name) log.slf4j.info("%s has been shut down", name)
} }
@deprecated("Use addListener instead") @deprecated("Use addListener instead")
@ -341,7 +342,7 @@ class RemoteClient private[akka] (
} else { } else {
val timeLeft = reconnectionTimeWindow - (System.currentTimeMillis - reconnectionTimeWindowStart) val timeLeft = reconnectionTimeWindow - (System.currentTimeMillis - reconnectionTimeWindowStart)
if (timeLeft > 0) { if (timeLeft > 0) {
log.info("Will try to reconnect to remote server for another [%s] milliseconds", timeLeft) log.slf4j.info("Will try to reconnect to remote server for another [%s] milliseconds", timeLeft)
true true
} else false } else false
} }
@ -406,7 +407,7 @@ class RemoteClientHandler(
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = { override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
if (event.isInstanceOf[ChannelStateEvent] && if (event.isInstanceOf[ChannelStateEvent] &&
event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) { event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) {
log.debug(event.toString) log.slf4j.debug(event.toString)
} }
super.handleUpstream(ctx, event) super.handleUpstream(ctx, event)
} }
@ -417,7 +418,7 @@ class RemoteClientHandler(
if (result.isInstanceOf[RemoteMessageProtocol]) { if (result.isInstanceOf[RemoteMessageProtocol]) {
val reply = result.asInstanceOf[RemoteMessageProtocol] val reply = result.asInstanceOf[RemoteMessageProtocol]
val replyUuid = uuidFrom(reply.getUuid.getHigh, reply.getUuid.getLow) val replyUuid = uuidFrom(reply.getUuid.getHigh, reply.getUuid.getLow)
log.debug("Remote client received RemoteMessageProtocol[\n%s]".format(reply.toString)) log.slf4j.debug("Remote client received RemoteMessageProtocol[\n%s]".format(reply.toString))
val future = futures.get(replyUuid).asInstanceOf[CompletableFuture[Any]] val future = futures.get(replyUuid).asInstanceOf[CompletableFuture[Any]]
if (reply.hasMessage) { if (reply.hasMessage) {
if (future eq null) throw new IllegalActorStateException("Future mapped to UUID " + replyUuid + " does not exist") if (future eq null) throw new IllegalActorStateException("Future mapped to UUID " + replyUuid + " does not exist")
@ -445,7 +446,7 @@ class RemoteClientHandler(
} catch { } catch {
case e: Exception => case e: Exception =>
client.notifyListeners(RemoteClientError(e, client)) client.notifyListeners(RemoteClientError(e, client))
log.error("Unexpected exception in remote client handler: %s", e) log.slf4j.error("Unexpected exception in remote client handler: %s", e)
throw e throw e
} }
} }
@ -456,12 +457,13 @@ class RemoteClientHandler(
def run(timeout: Timeout) = { def run(timeout: Timeout) = {
client.openChannels.remove(event.getChannel) client.openChannels.remove(event.getChannel)
client.isAuthenticated.set(false) client.isAuthenticated.set(false)
log.debug("Remote client reconnecting to [%s]", remoteAddress) log.slf4j.debug("Remote client reconnecting to [%s]", remoteAddress)
client.connection = bootstrap.connect(remoteAddress) client.connection = bootstrap.connect(remoteAddress)
client.connection.awaitUninterruptibly // Wait until the connection attempt succeeds or fails. client.connection.awaitUninterruptibly // Wait until the connection attempt succeeds or fails.
if (!client.connection.isSuccess) { if (!client.connection.isSuccess) {
client.notifyListeners(RemoteClientError(client.connection.getCause, client)) client.notifyListeners(RemoteClientError(client.connection.getCause, client))
log.error(client.connection.getCause, "Reconnection to [%s] has failed", remoteAddress) log.slf4j.error("Reconnection to [%s] has failed", remoteAddress)
log.slf4j.debug("Reconnection failed", client.connection.getCause)
} }
} }
}, RemoteClient.RECONNECT_DELAY.toMillis, TimeUnit.MILLISECONDS) }, RemoteClient.RECONNECT_DELAY.toMillis, TimeUnit.MILLISECONDS)
@ -471,7 +473,7 @@ class RemoteClientHandler(
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
def connect = { def connect = {
client.notifyListeners(RemoteClientConnected(client)) client.notifyListeners(RemoteClientConnected(client))
log.debug("Remote client connected to [%s]", ctx.getChannel.getRemoteAddress) log.slf4j.debug("Remote client connected to [%s]", ctx.getChannel.getRemoteAddress)
client.resetReconnectionTimeWindow client.resetReconnectionTimeWindow
} }
@ -488,12 +490,16 @@ class RemoteClientHandler(
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
client.notifyListeners(RemoteClientDisconnected(client)) client.notifyListeners(RemoteClientDisconnected(client))
log.debug("Remote client disconnected from [%s]", ctx.getChannel.getRemoteAddress) log.slf4j.debug("Remote client disconnected from [%s]", ctx.getChannel.getRemoteAddress)
} }
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
client.notifyListeners(RemoteClientError(event.getCause, client)) client.notifyListeners(RemoteClientError(event.getCause, client))
log.error(event.getCause, "Unexpected exception from downstream in remote client") if (event.getCause ne null)
log.slf4j.error("Unexpected exception from downstream in remote client", event.getCause)
else
log.slf4j.error("Unexpected exception from downstream in remote client: {}", event)
event.getChannel.close event.getChannel.close
} }

View file

@ -233,7 +233,7 @@ class RemoteServer extends Logging with ListenerManagement {
try { try {
if (!_isRunning) { if (!_isRunning) {
address = Address(_hostname,_port) address = Address(_hostname,_port)
log.info("Starting remote server at [%s:%s]", hostname, port) log.slf4j.info("Starting remote server at [%s:%s]", hostname, port)
RemoteServer.register(hostname, port, this) RemoteServer.register(hostname, port, this)
val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, loader, this) val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, loader, this)
@ -249,7 +249,7 @@ class RemoteServer extends Logging with ListenerManagement {
} }
} catch { } catch {
case e => case e =>
log.error(e, "Could not start up remote server") log.slf4j.error("Could not start up remote server", e)
notifyListeners(RemoteServerError(e, this)) notifyListeners(RemoteServerError(e, this))
} }
this this
@ -265,7 +265,7 @@ class RemoteServer extends Logging with ListenerManagement {
notifyListeners(RemoteServerShutdown(this)) notifyListeners(RemoteServerShutdown(this))
} catch { } catch {
case e: java.nio.channels.ClosedChannelException => {} case e: java.nio.channels.ClosedChannelException => {}
case e => log.warning("Could not close remote server channel in a graceful way") case e => log.slf4j.warn("Could not close remote server channel in a graceful way")
} }
} }
} }
@ -281,7 +281,7 @@ class RemoteServer extends Logging with ListenerManagement {
* @param typedActor typed actor to register * @param typedActor typed actor to register
*/ */
def registerTypedActor(id: String, typedActor: AnyRef): Unit = synchronized { def registerTypedActor(id: String, typedActor: AnyRef): Unit = synchronized {
log.debug("Registering server side remote typed actor [%s] with id [%s]", typedActor.getClass.getName, id) log.slf4j.debug("Registering server side remote typed actor [%s] with id [%s]", typedActor.getClass.getName, id)
if (id.startsWith(UUID_PREFIX)) registerTypedActor(id.substring(UUID_PREFIX.length), typedActor, typedActorsByUuid) if (id.startsWith(UUID_PREFIX)) registerTypedActor(id.substring(UUID_PREFIX.length), typedActor, typedActorsByUuid)
else registerTypedActor(id, typedActor, typedActors) else registerTypedActor(id, typedActor, typedActors)
} }
@ -297,7 +297,7 @@ class RemoteServer extends Logging with ListenerManagement {
* @param typedActor typed actor to register * @param typedActor typed actor to register
*/ */
def registerTypedPerSessionActor(id: String, factory: => AnyRef): Unit = synchronized { def registerTypedPerSessionActor(id: String, factory: => AnyRef): Unit = synchronized {
log.debug("Registering server side typed remote session actor with id [%s]", id) log.slf4j.debug("Registering server side typed remote session actor with id [%s]", id)
registerTypedPerSessionActor(id, () => factory, typedActorsFactories) registerTypedPerSessionActor(id, () => factory, typedActorsFactories)
} }
@ -312,7 +312,7 @@ class RemoteServer extends Logging with ListenerManagement {
* NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself. * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
*/ */
def register(id: String, actorRef: ActorRef): Unit = synchronized { def register(id: String, actorRef: ActorRef): Unit = synchronized {
log.debug("Registering server side remote actor [%s] with id [%s]", actorRef.actorClass.getName, id) log.slf4j.debug("Registering server side remote actor [%s] with id [%s]", actorRef.actorClass.getName, id)
if (id.startsWith(UUID_PREFIX)) register(id.substring(UUID_PREFIX.length), actorRef, actorsByUuid) if (id.startsWith(UUID_PREFIX)) register(id.substring(UUID_PREFIX.length), actorRef, actorsByUuid)
else register(id, actorRef, actors) else register(id, actorRef, actors)
} }
@ -323,7 +323,7 @@ class RemoteServer extends Logging with ListenerManagement {
* NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself. * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
*/ */
def registerPerSession(id: String, factory: => ActorRef): Unit = synchronized { def registerPerSession(id: String, factory: => ActorRef): Unit = synchronized {
log.debug("Registering server side remote session actor with id [%s]", id) log.slf4j.debug("Registering server side remote session actor with id [%s]", id)
registerPerSession(id, () => factory, actorsFactories) registerPerSession(id, () => factory, actorsFactories)
} }
@ -354,7 +354,7 @@ class RemoteServer extends Logging with ListenerManagement {
*/ */
def unregister(actorRef: ActorRef):Unit = synchronized { def unregister(actorRef: ActorRef):Unit = synchronized {
if (_isRunning) { if (_isRunning) {
log.debug("Unregistering server side remote actor [%s] with id [%s:%s]", actorRef.actorClass.getName, actorRef.id, actorRef.uuid) log.slf4j.debug("Unregistering server side remote actor [%s] with id [%s:%s]", Array(actorRef.actorClass.getName, actorRef.id, actorRef.uuid))
actors.remove(actorRef.id, actorRef) actors.remove(actorRef.id, actorRef)
actorsByUuid.remove(actorRef.uuid, actorRef) actorsByUuid.remove(actorRef.uuid, actorRef)
} }
@ -367,7 +367,7 @@ class RemoteServer extends Logging with ListenerManagement {
*/ */
def unregister(id: String):Unit = synchronized { def unregister(id: String):Unit = synchronized {
if (_isRunning) { if (_isRunning) {
log.info("Unregistering server side remote actor with id [%s]", id) log.slf4j.info("Unregistering server side remote actor with id [%s]", id)
if (id.startsWith(UUID_PREFIX)) actorsByUuid.remove(id.substring(UUID_PREFIX.length)) if (id.startsWith(UUID_PREFIX)) actorsByUuid.remove(id.substring(UUID_PREFIX.length))
else { else {
val actorRef = actors get id val actorRef = actors get id
@ -384,7 +384,7 @@ class RemoteServer extends Logging with ListenerManagement {
*/ */
def unregisterPerSession(id: String):Unit = { def unregisterPerSession(id: String):Unit = {
if (_isRunning) { if (_isRunning) {
log.info("Unregistering server side remote session actor with id [%s]", id) log.slf4j.info("Unregistering server side remote session actor with id [%s]", id)
actorsFactories.remove(id) actorsFactories.remove(id)
} }
} }
@ -396,7 +396,7 @@ class RemoteServer extends Logging with ListenerManagement {
*/ */
def unregisterTypedActor(id: String):Unit = synchronized { def unregisterTypedActor(id: String):Unit = synchronized {
if (_isRunning) { if (_isRunning) {
log.info("Unregistering server side remote typed actor with id [%s]", id) log.slf4j.info("Unregistering server side remote typed actor with id [%s]", id)
if (id.startsWith(UUID_PREFIX)) typedActorsByUuid.remove(id.substring(UUID_PREFIX.length)) if (id.startsWith(UUID_PREFIX)) typedActorsByUuid.remove(id.substring(UUID_PREFIX.length))
else typedActors.remove(id) else typedActors.remove(id)
} }
@ -506,7 +506,7 @@ class RemoteServerHandler(
val clientAddress = getClientAddress(ctx) val clientAddress = getClientAddress(ctx)
sessionActors.set(event.getChannel(), new ConcurrentHashMap[String, ActorRef]()) sessionActors.set(event.getChannel(), new ConcurrentHashMap[String, ActorRef]())
typedSessionActors.set(event.getChannel(), new ConcurrentHashMap[String, AnyRef]()) typedSessionActors.set(event.getChannel(), new ConcurrentHashMap[String, AnyRef]())
log.debug("Remote client [%s] connected to [%s]", clientAddress, server.name) log.slf4j.debug("Remote client [%s] connected to [%s]", clientAddress, server.name)
if (RemoteServer.SECURE) { if (RemoteServer.SECURE) {
val sslHandler: SslHandler = ctx.getPipeline.get(classOf[SslHandler]) val sslHandler: SslHandler = ctx.getPipeline.get(classOf[SslHandler])
// Begin handshake. // Begin handshake.
@ -524,7 +524,7 @@ class RemoteServerHandler(
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
val clientAddress = getClientAddress(ctx) val clientAddress = getClientAddress(ctx)
log.debug("Remote client [%s] disconnected from [%s]", clientAddress, server.name) log.slf4j.debug("Remote client [%s] disconnected from [%s]", clientAddress, server.name)
// stop all session actors // stop all session actors
val channelActors = sessionActors.remove(event.getChannel) val channelActors = sessionActors.remove(event.getChannel)
if (channelActors ne null) { if (channelActors ne null) {
@ -546,13 +546,13 @@ class RemoteServerHandler(
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = { override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
val clientAddress = getClientAddress(ctx) val clientAddress = getClientAddress(ctx)
log.debug("Remote client [%s] channel closed from [%s]", clientAddress, server.name) log.slf4j.debug("Remote client [%s] channel closed from [%s]", clientAddress, server.name)
server.notifyListeners(RemoteServerClientClosed(server, clientAddress)) server.notifyListeners(RemoteServerClientClosed(server, clientAddress))
} }
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = { override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
if (event.isInstanceOf[ChannelStateEvent] && event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) { if (event.isInstanceOf[ChannelStateEvent] && event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) {
log.debug(event.toString) log.slf4j.debug(event.toString)
} }
super.handleUpstream(ctx, event) super.handleUpstream(ctx, event)
} }
@ -568,7 +568,7 @@ class RemoteServerHandler(
} }
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
log.error(event.getCause, "Unexpected exception from remote downstream") log.slf4j.error("Unexpected exception from remote downstream", event.getCause)
event.getChannel.close event.getChannel.close
server.notifyListeners(RemoteServerError(event.getCause, server)) server.notifyListeners(RemoteServerError(event.getCause, server))
} }
@ -580,7 +580,7 @@ class RemoteServerHandler(
} }
private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = { private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = {
log.debug("Received RemoteMessageProtocol[\n%s]".format(request.toString)) log.slf4j.debug("Received RemoteMessageProtocol[\n%s]".format(request.toString))
request.getActorInfo.getActorType match { request.getActorInfo.getActorType match {
case SCALA_ACTOR => dispatchToActor(request, channel) case SCALA_ACTOR => dispatchToActor(request, channel)
case TYPED_ACTOR => dispatchToTypedActor(request, channel) case TYPED_ACTOR => dispatchToTypedActor(request, channel)
@ -591,7 +591,7 @@ class RemoteServerHandler(
private def dispatchToActor(request: RemoteMessageProtocol, channel: Channel) { private def dispatchToActor(request: RemoteMessageProtocol, channel: Channel) {
val actorInfo = request.getActorInfo val actorInfo = request.getActorInfo
log.debug("Dispatching to remote actor [%s:%s]", actorInfo.getTarget, actorInfo.getUuid) log.slf4j.debug("Dispatching to remote actor [%s:%s]", actorInfo.getTarget, actorInfo.getUuid)
val actorRef = val actorRef =
try { try {
@ -627,7 +627,7 @@ class RemoteServerHandler(
val exception = f.exception val exception = f.exception
if (exception.isDefined) { if (exception.isDefined) {
log.debug("Returning exception from actor invocation [%s]",exception.get) log.slf4j.debug("Returning exception from actor invocation [%s]",exception.get)
try { try {
channel.write(createErrorReplyMessage(exception.get, request, AkkaActorType.ScalaActor)) channel.write(createErrorReplyMessage(exception.get, request, AkkaActorType.ScalaActor))
} catch { } catch {
@ -635,7 +635,7 @@ class RemoteServerHandler(
} }
} }
else if (result.isDefined) { else if (result.isDefined) {
log.debug("Returning result from actor invocation [%s]".format(result.get)) log.slf4j.debug("Returning result from actor invocation [%s]".format(result.get))
val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder( val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder(
Some(actorRef), Some(actorRef),
Right(request.getUuid), Right(request.getUuid),
@ -667,7 +667,7 @@ class RemoteServerHandler(
private def dispatchToTypedActor(request: RemoteMessageProtocol, channel: Channel) = { private def dispatchToTypedActor(request: RemoteMessageProtocol, channel: Channel) = {
val actorInfo = request.getActorInfo val actorInfo = request.getActorInfo
val typedActorInfo = actorInfo.getTypedActorInfo val typedActorInfo = actorInfo.getTypedActorInfo
log.debug("Dispatching to remote typed actor [%s :: %s]", typedActorInfo.getMethod, typedActorInfo.getInterface) log.slf4j.debug("Dispatching to remote typed actor [%s :: %s]", typedActorInfo.getMethod, typedActorInfo.getInterface)
val typedActor = createTypedActor(actorInfo, channel) val typedActor = createTypedActor(actorInfo, channel)
val args = MessageSerializer.deserialize(request.getMessage).asInstanceOf[Array[AnyRef]].toList val args = MessageSerializer.deserialize(request.getMessage).asInstanceOf[Array[AnyRef]].toList
@ -693,7 +693,7 @@ class RemoteServerHandler(
None) None)
if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid) if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid)
channel.write(messageBuilder.build) channel.write(messageBuilder.build)
log.debug("Returning result from remote typed actor invocation [%s]", result) log.slf4j.debug("Returning result from remote typed actor invocation [%s]", result)
} catch { } catch {
case e: Throwable => server.notifyListeners(RemoteServerError(e, server)) case e: Throwable => server.notifyListeners(RemoteServerError(e, server))
} }
@ -798,7 +798,7 @@ class RemoteServerHandler(
if (RemoteServer.UNTRUSTED_MODE) throw new SecurityException( if (RemoteServer.UNTRUSTED_MODE) throw new SecurityException(
"Remote server is operating is untrusted mode, can not create remote actors on behalf of the remote client") "Remote server is operating is untrusted mode, can not create remote actors on behalf of the remote client")
log.info("Creating a new remote actor [%s:%s]", name, uuid) log.slf4j.info("Creating a new remote actor [%s:%s]", name, uuid)
val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name) val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name)
else Class.forName(name) else Class.forName(name)
val actorRef = Actor.actorOf(clazz.asInstanceOf[Class[_ <: Actor]]) val actorRef = Actor.actorOf(clazz.asInstanceOf[Class[_ <: Actor]])
@ -810,7 +810,7 @@ class RemoteServerHandler(
actorRef actorRef
} catch { } catch {
case e => case e =>
log.error(e, "Could not create remote actor instance") log.slf4j.error("Could not create remote actor instance", e)
server.notifyListeners(RemoteServerError(e, server)) server.notifyListeners(RemoteServerError(e, server))
throw e throw e
} }
@ -874,7 +874,7 @@ class RemoteServerHandler(
if (RemoteServer.UNTRUSTED_MODE) throw new SecurityException( if (RemoteServer.UNTRUSTED_MODE) throw new SecurityException(
"Remote server is operating is untrusted mode, can not create remote actors on behalf of the remote client") "Remote server is operating is untrusted mode, can not create remote actors on behalf of the remote client")
log.info("Creating a new remote typed actor:\n\t[%s :: %s]", interfaceClassname, targetClassname) log.slf4j.info("Creating a new remote typed actor:\n\t[%s :: %s]", interfaceClassname, targetClassname)
val (interfaceClass, targetClass) = val (interfaceClass, targetClass) =
if (applicationLoader.isDefined) (applicationLoader.get.loadClass(interfaceClassname), if (applicationLoader.isDefined) (applicationLoader.get.loadClass(interfaceClassname),
@ -887,7 +887,7 @@ class RemoteServerHandler(
newInstance newInstance
} catch { } catch {
case e => case e =>
log.error(e, "Could not create remote typed actor instance") log.slf4j.error("Could not create remote typed actor instance", e)
server.notifyListeners(RemoteServerError(e, server)) server.notifyListeners(RemoteServerError(e, server))
throw e throw e
} }
@ -913,7 +913,8 @@ class RemoteServerHandler(
private def createErrorReplyMessage(exception: Throwable, request: RemoteMessageProtocol, actorType: AkkaActorType): RemoteMessageProtocol = { private def createErrorReplyMessage(exception: Throwable, request: RemoteMessageProtocol, actorType: AkkaActorType): RemoteMessageProtocol = {
val actorInfo = request.getActorInfo val actorInfo = request.getActorInfo
log.error(exception, "Could not invoke remote actor [%s]", actorInfo.getTarget) log.slf4j.error("Could not invoke remote actor [%s]", actorInfo.getTarget)
log.slf4j.debug("Could not invoke remote actor", exception)
val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder( val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder(
None, None,
Right(request.getUuid), Right(request.getUuid),
@ -941,7 +942,7 @@ class RemoteServerHandler(
"The remote client [" + clientAddress + "] does not have a secure cookie.") "The remote client [" + clientAddress + "] does not have a secure cookie.")
if (!(request.getCookie == RemoteServer.SECURE_COOKIE.get)) throw new SecurityException( if (!(request.getCookie == RemoteServer.SECURE_COOKIE.get)) throw new SecurityException(
"The remote client [" + clientAddress + "] secure cookie is not the same as remote server secure cookie") "The remote client [" + clientAddress + "] secure cookie is not the same as remote server secure cookie")
log.info("Remote client [%s] successfully authenticated using secure cookie", clientAddress) log.slf4j.info("Remote client [%s] successfully authenticated using secure cookie", clientAddress)
} }
} }
} }

View file

@ -152,7 +152,7 @@ object ActorSerialization {
private[akka] def fromProtobufToLocalActorRef[T <: Actor]( private[akka] def fromProtobufToLocalActorRef[T <: Actor](
protocol: SerializedActorRefProtocol, format: Format[T], loader: Option[ClassLoader]): ActorRef = { protocol: SerializedActorRefProtocol, format: Format[T], loader: Option[ClassLoader]): ActorRef = {
Actor.log.debug("Deserializing SerializedActorRefProtocol to LocalActorRef:\n" + protocol) Actor.log.slf4j.debug("Deserializing SerializedActorRefProtocol to LocalActorRef:\n" + protocol)
val serializer = val serializer =
if (format.isInstanceOf[SerializerBasedActorFormat[_]]) if (format.isInstanceOf[SerializerBasedActorFormat[_]])
@ -226,7 +226,7 @@ object RemoteActorSerialization {
* Deserializes a RemoteActorRefProtocol Protocol Buffers (protobuf) Message into an RemoteActorRef instance. * Deserializes a RemoteActorRefProtocol Protocol Buffers (protobuf) Message into an RemoteActorRef instance.
*/ */
private[akka] def fromProtobufToRemoteActorRef(protocol: RemoteActorRefProtocol, loader: Option[ClassLoader]): ActorRef = { private[akka] def fromProtobufToRemoteActorRef(protocol: RemoteActorRefProtocol, loader: Option[ClassLoader]): ActorRef = {
Actor.log.debug("Deserializing RemoteActorRefProtocol to RemoteActorRef:\n %s", protocol) Actor.log.slf4j.debug("Deserializing RemoteActorRefProtocol to RemoteActorRef:\n %s", protocol)
RemoteActorRef( RemoteActorRef(
protocol.getClassOrServiceName, protocol.getClassOrServiceName,
protocol.getActorClassname, protocol.getActorClassname,
@ -244,7 +244,7 @@ object RemoteActorSerialization {
val host = homeAddress.getHostName val host = homeAddress.getHostName
val port = homeAddress.getPort val port = homeAddress.getPort
Actor.log.debug("Register serialized Actor [%s] as remote @ [%s:%s]", actorClassName, host, port) Actor.log.slf4j.debug("Register serialized Actor [%s] as remote @ [%s:%s]", Array(actorClassName, host, port))
RemoteServer.getOrCreateServer(homeAddress) RemoteServer.getOrCreateServer(homeAddress)
ActorRegistry.registerActorByUuid(homeAddress, uuid.toString, ar) ActorRegistry.registerActorByUuid(homeAddress, uuid.toString, ar)
@ -367,7 +367,7 @@ object TypedActorSerialization {
private def fromProtobufToLocalTypedActorRef[T <: Actor, U <: AnyRef]( private def fromProtobufToLocalTypedActorRef[T <: Actor, U <: AnyRef](
protocol: SerializedTypedActorRefProtocol, format: Format[T], loader: Option[ClassLoader]): U = { protocol: SerializedTypedActorRefProtocol, format: Format[T], loader: Option[ClassLoader]): U = {
Actor.log.debug("Deserializing SerializedTypedActorRefProtocol to LocalActorRef:\n" + protocol) Actor.log.slf4j.debug("Deserializing SerializedTypedActorRefProtocol to LocalActorRef:\n" + protocol)
val actorRef = ActorSerialization.fromProtobufToLocalActorRef(protocol.getActorRef, format, loader) val actorRef = ActorSerialization.fromProtobufToLocalActorRef(protocol.getActorRef, format, loader)
val intfClass = toClass(loader, protocol.getInterfaceName) val intfClass = toClass(loader, protocol.getInterfaceName)
TypedActor.newInstance(intfClass, actorRef).asInstanceOf[U] TypedActor.newInstance(intfClass, actorRef).asInstanceOf[U]
@ -407,7 +407,7 @@ object RemoteTypedActorSerialization {
* Deserializes a RemoteTypedActorRefProtocol Protocol Buffers (protobuf) Message into AW RemoteActorRef proxy. * Deserializes a RemoteTypedActorRefProtocol Protocol Buffers (protobuf) Message into AW RemoteActorRef proxy.
*/ */
private[akka] def fromProtobufToRemoteTypedActorRef[T](protocol: RemoteTypedActorRefProtocol, loader: Option[ClassLoader]): T = { private[akka] def fromProtobufToRemoteTypedActorRef[T](protocol: RemoteTypedActorRefProtocol, loader: Option[ClassLoader]): T = {
Actor.log.debug("Deserializing RemoteTypedActorRefProtocol to AW RemoteActorRef proxy:\n" + protocol) Actor.log.slf4j.debug("Deserializing RemoteTypedActorRefProtocol to AW RemoteActorRef proxy:\n" + protocol)
val actorRef = RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getActorRef, loader) val actorRef = RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getActorRef, loader)
val intfClass = TypedActorSerialization.toClass(loader, protocol.getInterfaceName) val intfClass = TypedActorSerialization.toClass(loader, protocol.getInterfaceName)
TypedActor.createProxyForRemoteActorRef(intfClass, actorRef).asInstanceOf[T] TypedActor.createProxyForRemoteActorRef(intfClass, actorRef).asInstanceOf[T]

View file

@ -29,7 +29,7 @@ object ServerInitiatedRemoteActorClient extends Logging {
def run = { def run = {
val actor = RemoteClient.actorFor("hello-service", "localhost", 2552) val actor = RemoteClient.actorFor("hello-service", "localhost", 2552)
val result = actor !! "Hello" val result = actor !! "Hello"
log.info("Result from Remote Actor: %s", result) log.slf4j.info("Result from Remote Actor: %s", result)
} }
def main(args: Array[String]) = run def main(args: Array[String]) = run

View file

@ -12,7 +12,7 @@ import akka.util.Logging
class RemoteHelloWorldActor extends RemoteActor("localhost", 2552) { class RemoteHelloWorldActor extends RemoteActor("localhost", 2552) {
def receive = { def receive = {
case "Hello" => case "Hello" =>
log.info("Received 'Hello'") log.slf4j.info("Received 'Hello'")
self.reply("World") self.reply("World")
} }
} }
@ -20,7 +20,7 @@ class RemoteHelloWorldActor extends RemoteActor("localhost", 2552) {
object ClientManagedRemoteActorServer extends Logging { object ClientManagedRemoteActorServer extends Logging {
def run = { def run = {
RemoteNode.start("localhost", 2552) RemoteNode.start("localhost", 2552)
log.info("Remote node started") log.slf4j.info("Remote node started")
} }
def main(args: Array[String]) = run def main(args: Array[String]) = run
@ -30,10 +30,10 @@ object ClientManagedRemoteActorClient extends Logging {
def run = { def run = {
val actor = actorOf[RemoteHelloWorldActor].start val actor = actorOf[RemoteHelloWorldActor].start
log.info("Remote actor created, moved to the server") log.slf4j.info("Remote actor created, moved to the server")
log.info("Sending 'Hello' to remote actor") log.slf4j.info("Sending 'Hello' to remote actor")
val result = actor !! "Hello" val result = actor !! "Hello"
log.info("Result from Remote Actor: '%s'", result.get) log.slf4j.info("Result from Remote Actor: '%s'", result.get)
} }
def main(args: Array[String]) = run def main(args: Array[String]) = run

View file

@ -12,7 +12,7 @@ import akka.util.Logging
class HelloWorldActor extends Actor { class HelloWorldActor extends Actor {
def receive = { def receive = {
case "Hello" => case "Hello" =>
log.info("Received 'Hello'") log.slf4j.info("Received 'Hello'")
self.reply("World") self.reply("World")
} }
} }
@ -21,9 +21,9 @@ object ServerManagedRemoteActorServer extends Logging {
def run = { def run = {
RemoteNode.start("localhost", 2552) RemoteNode.start("localhost", 2552)
log.info("Remote node started") log.slf4j.info("Remote node started")
RemoteNode.register("hello-service", actorOf[HelloWorldActor]) RemoteNode.register("hello-service", actorOf[HelloWorldActor])
log.info("Remote actor registered and started") log.slf4j.info("Remote actor registered and started")
} }
def main(args: Array[String]) = run def main(args: Array[String]) = run
@ -33,10 +33,10 @@ object ServerManagedRemoteActorClient extends Logging {
def run = { def run = {
val actor = RemoteClient.actorFor("hello-service", "localhost", 2552) val actor = RemoteClient.actorFor("hello-service", "localhost", 2552)
log.info("Remote client created") log.slf4j.info("Remote client created")
log.info("Sending 'Hello' to remote actor") log.slf4j.info("Sending 'Hello' to remote actor")
val result = actor !! "Hello" val result = actor !! "Hello"
log.info("Result from Remote Actor: '%s'", result.get) log.slf4j.info("Result from Remote Actor: '%s'", result.get)
} }
def main(args: Array[String]) = run def main(args: Array[String]) = run

View file

@ -100,24 +100,24 @@ object Transaction {
if (JTA_AWARE) Some(ReflectiveJtaModule.createTransactionContainer) if (JTA_AWARE) Some(ReflectiveJtaModule.createTransactionContainer)
else None else None
log.trace("Creating transaction " + toString) log.slf4j.trace("Creating transaction " + toString)
// --- public methods --------- // --- public methods ---------
def begin = synchronized { def begin = synchronized {
log.trace("Starting transaction " + toString) log.slf4j.trace("Starting transaction " + toString)
jta.foreach { _.beginWithStmSynchronization(this) } jta.foreach { _.beginWithStmSynchronization(this) }
} }
def commit = synchronized { def commit = synchronized {
log.trace("Committing transaction " + toString) log.slf4j.trace("Committing transaction " + toString)
persistentStateMap.valuesIterator.foreach(_.commit) persistentStateMap.valuesIterator.foreach(_.commit)
status = TransactionStatus.Completed status = TransactionStatus.Completed
jta.foreach(_.commit) jta.foreach(_.commit)
} }
def abort = synchronized { def abort = synchronized {
log.trace("Aborting transaction " + toString) log.slf4j.trace("Aborting transaction " + toString)
jta.foreach(_.rollback) jta.foreach(_.rollback)
persistentStateMap.valuesIterator.foreach(_.abort) persistentStateMap.valuesIterator.foreach(_.abort)
persistentStateMap.clear persistentStateMap.clear

View file

@ -18,7 +18,7 @@ object CoordinatedIncrement {
implicit val txFactory = TransactionFactory(timeout = 3 seconds) implicit val txFactory = TransactionFactory(timeout = 3 seconds)
def increment = { def increment = {
log.info(name + ": incrementing") log.slf4j.info(name + ": incrementing")
count alter (_ + 1) count alter (_ + 1)
} }

View file

@ -26,7 +26,7 @@ object FickleFriends {
implicit val txFactory = TransactionFactory(timeout = 3 seconds) implicit val txFactory = TransactionFactory(timeout = 3 seconds)
def increment = { def increment = {
log.info(name + ": incrementing") log.slf4j.info(name + ": incrementing")
count alter (_ + 1) count alter (_ + 1)
} }
@ -65,7 +65,7 @@ object FickleFriends {
implicit val txFactory = TransactionFactory(timeout = 3 seconds) implicit val txFactory = TransactionFactory(timeout = 3 seconds)
def increment = { def increment = {
log.info(name + ": incrementing") log.slf4j.info(name + ": incrementing")
count alter (_ + 1) count alter (_ + 1)
} }

View file

@ -20,7 +20,7 @@ object TransactorIncrement {
override def transactionFactory = TransactionFactory(timeout = 3 seconds) override def transactionFactory = TransactionFactory(timeout = 3 seconds)
def increment = { def increment = {
log.info(name + ": incrementing") log.slf4j.info(name + ": incrementing")
count alter (_ + 1) count alter (_ + 1)
} }
@ -32,7 +32,7 @@ object TransactorIncrement {
} }
override def before = { override def before = {
case i: Increment => log.info(name + ": before transaction") case i: Increment => log.slf4j.info(name + ": before transaction")
} }
def atomically = { def atomically = {
@ -44,7 +44,7 @@ object TransactorIncrement {
} }
override def after = { override def after = {
case i: Increment => log.info(name + ": after transaction") case i: Increment => log.slf4j.info(name + ": after transaction")
} }
override def normally = { override def normally = {

View file

@ -43,7 +43,7 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa
* @return the typed actors for the class * @return the typed actors for the class
*/ */
def getInstance[T](clazz: Class[T]): List[T] = synchronized { def getInstance[T](clazz: Class[T]): List[T] = synchronized {
log.debug("Retrieving typed actor [%s]", clazz.getName) log.slf4j.debug("Retrieving typed actor [%s]", clazz.getName)
if (injector eq null) throw new IllegalActorStateException( if (injector eq null) throw new IllegalActorStateException(
"inject() and/or supervise() must be called before invoking getInstance(clazz)") "inject() and/or supervise() must be called before invoking getInstance(clazz)")
val (proxy, targetInstance, component) = val (proxy, targetInstance, component) =

View file

@ -19,7 +19,7 @@
<pattern>[%t] [%4p] [%d{ISO8601}] %c{1}: %m%n</pattern> <pattern>[%t] [%4p] [%d{ISO8601}] %c{1}: %m%n</pattern>
</encoder> </encoder>
<rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy"> <rollingPolicy class="ch.qos.logback.core.rolling.TimeBasedRollingPolicy">
<fileNamePattern>./logs/akka.log.%d{yyyy-MM-dd-HH}</fileNamePattern> <fileNamePattern>./logs/akka.log.slf4j.%d{yyyy-MM-dd-HH}</fileNamePattern>
</rollingPolicy> </rollingPolicy>
</appender> </appender>
<logger name="akka" level="INFO"/> <logger name="akka" level="INFO"/>

View file

@ -486,7 +486,7 @@ trait DeployProject { self: BasicScalaProject =>
genJar: Boolean, genDocs: Boolean, genSource: Boolean) = task { genJar: Boolean, genDocs: Boolean, genSource: Boolean) = task {
def gen(jar: Path, toDir: Path, flag: Boolean, msg: String): Option[String] = def gen(jar: Path, toDir: Path, flag: Boolean, msg: String): Option[String] =
if (flag) { if (flag) {
log.info(msg + " " + jar) log.slf4j.info(msg + " " + jar)
FileUtilities.copyFile(jar, toDir / jar.name, log) FileUtilities.copyFile(jar, toDir / jar.name, log)
} else None } else None