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 {
override def run {
// 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
// Clear Thread.subclassAudits
log.info("Clearing subclass audits")
log.slf4j.info("Clearing subclass audits")
val tf = classOf[java.lang.Thread].getDeclaredField("subclassAudits")
tf.setAccessible(true)
val subclassAudits = tf.get(null).asInstanceOf[java.util.Map[_,_]]
subclassAudits.synchronized {subclassAudits.clear}
// 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
val lf = classOf[Level].getDeclaredField("known")
lf.setAccessible(true)
@ -351,14 +351,14 @@ trait Actor extends Logging {
* <pre>
* def receive = {
* case Ping =&gt;
* log.info("got a 'Ping' message")
* log.slf4j.info("got a 'Ping' message")
* self.reply("pong")
*
* case OneWay =&gt;
* log.info("got a 'OneWay' message")
* log.slf4j.info("got a 'OneWay' message")
*
* case unknown =&gt;
* log.warning("unknown message [%s], ignoring", unknown)
* log.slf4j.warn("unknown message [%s], ignoring", unknown)
* }
* </pre>
*/

View file

@ -578,7 +578,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
protected[akka] def checkReceiveTimeout = {
cancelReceiveTimeout
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))
}
}
@ -587,7 +587,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
if (_futureTimeout.isDefined) {
_futureTimeout.get.cancel(true)
_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")
linkedActors.put(actorRef.uuid, actorRef)
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")
linkedActors.remove(actorRef.uuid)
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.
*/
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 {
currentMessage = messageHandle
try {
dispatch(messageHandle)
} catch {
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
} finally {
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]) {
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
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)
val freshActor = newActor
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 _ =>
}
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)
}
def tooManyRestarts {
Actor.log.warning(
Actor.log.slf4j.warn(
"Maximum number of restarts [%s] within time range [%s] reached." +
"\n\tWill *not* restart actor [%s] anymore." +
"\n\tLast exception causing restart was" +
"\n\t[%s].",
maxNrOfRetries, withinTimeRange, this, reason)
Array(maxNrOfRetries, withinTimeRange, this, reason))
_supervisor.foreach { sup =>
// can supervisor handle the notification?
val notification = MaximumNumberOfRestartsWithinTimeRangeReached(this, maxNrOfRetries, withinTimeRange, reason)
if (sup.isDefinedAt(notification)) notifySupervisorWithMessage(notification)
else Actor.log.warning(
else Actor.log.slf4j.warn(
"No message handler defined for system message [MaximumNumberOfRestartsWithinTimeRangeReached]" +
"\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
}
Actor.log.debug("Restart: %s for [%s].", success, id)
Actor.log.slf4j.debug("Restart: %s for [%s].", success, id)
if (success) {
_status = ActorRefInternals.RUNNING
@ -1079,7 +1080,7 @@ class LocalActorRef private[akka] (
}
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)
try {
@ -1095,12 +1096,12 @@ class LocalActorRef private[akka] (
}
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
linkedActors.remove(temporaryActor.uuid) // remove the temporary actor
// if last temporary actor is gone, then unlink me from supervisor
if (linkedActors.isEmpty) {
Actor.log.info(
Actor.log.slf4j.info(
"All linked actors have died permanently (they were all configured as TEMPORARY)" +
"\n\tshutting down and unlinking supervisor actor as well [%s].",
temporaryActor.id)
@ -1111,7 +1112,8 @@ class LocalActorRef private[akka] (
}
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
dispatcher.suspend(this)
@ -1168,7 +1170,7 @@ class LocalActorRef private[akka] (
private def initializeActorInstance = {
actor.preStart // run actor preStart
Actor.log.trace("[%s] has started", toString)
Actor.log.slf4j.trace("[%s] has started", toString)
ActorRegistry.register(this)
}

View file

@ -254,7 +254,7 @@ object ActorRegistry extends ListenerManagement {
* Shuts down and unregisters all actors in the system.
*/
def shutdownAll() {
log.info("Shutting down all actors in the system...")
log.slf4j.info("Shutting down all actors in the system...")
if (TypedActorModule.isEnabled) {
val elements = actorsByUUID.elements
while (elements.hasMoreElements) {
@ -266,7 +266,7 @@ object ActorRegistry extends ListenerManagement {
} else foreach(_.stop)
actorsByUUID.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_DIR = new File(DEPLOY)
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)
}
val filesToDeploy = DEPLOY_DIR.listFiles.toArray.toList
@ -42,8 +42,8 @@ trait BootableActorLoaderService extends Bootable with Logging {
}
}
val toDeploy = filesToDeploy.map(_.toURI.toURL)
log.info("Deploying applications from [%s]: [%s]", DEPLOY, toDeploy)
log.debug("Loading dependencies [%s]", dependencyJars)
log.slf4j.info("Deploying applications from [%s]: [%s]", DEPLOY, toDeploy)
log.slf4j.debug("Loading dependencies [%s]", dependencyJars)
val allJars = toDeploy ::: dependencyJars
new URLClassLoader(allJars.toArray,Thread.currentThread.getContextClassLoader)
@ -51,12 +51,12 @@ trait BootableActorLoaderService extends Bootable with Logging {
}
abstract override def onLoad = {
applicationLoader.foreach(_ => log.info("Creating /deploy class-loader"))
applicationLoader.foreach(_ => log.slf4j.info("Creating /deploy class-loader"))
super.onLoad
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
}
}

View file

@ -69,17 +69,17 @@ trait FSM[S, D] {
private var handleEvent: StateFunction = {
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
}
private var terminateEvent: PartialFunction[Reason, Unit] = {
case failure@Failure(_) => log.error("Stopping because of a %s", failure)
case reason => log.info("Stopping because of reason: %s", reason)
case failure@Failure(_) => log.slf4j.error("Stopping because of a %s", failure)
case reason => log.slf4j.info("Stopping because of reason: %s", reason)
}
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 = {
@ -87,7 +87,7 @@ trait FSM[S, D] {
terminateEvent.apply(reason)
self.stop
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
case value => {
timeoutFuture = timeoutFuture.flatMap {ref => ref.cancel(true); None}
@ -123,7 +123,7 @@ trait FSM[S, D] {
def replying(replyValue:Any): State = {
self.sender match {
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
}

View file

@ -29,15 +29,15 @@ object Scheduler extends Logging {
@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
*/
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]",
message, receiver, initialDelay, delay, timeUnit)
Array(message, receiver, initialDelay, delay, timeUnit))
try {
service.scheduleAtFixedRate(
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
*/
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]",
runnable, initialDelay, delay, timeUnit)
Array(runnable, initialDelay, delay, timeUnit))
try {
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
*/
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]",
message, receiver, delay, timeUnit)
Array(message, receiver, delay, timeUnit))
try {
service.schedule(
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
*/
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]",
runnable, delay, timeUnit)
Array(runnable, delay, timeUnit))
try {
service.schedule(runnable,delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]]
} catch {
@ -109,12 +109,12 @@ object Scheduler extends Logging {
}
def shutdown: Unit = synchronized {
log.info("Shutting down Scheduler")
log.slf4j.info("Shutting down Scheduler")
service.shutdown
}
def restart: Unit = synchronized {
log.info("Restarting Scheduler")
log.slf4j.info("Restarting Scheduler")
shutdown
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
case MaximumNumberOfRestartsWithinTimeRangeReached(
victim, maxNrOfRetries, withinTimeRange, lastExceptionCausingRestart) =>
Actor.log.warning(
Actor.log.slf4j.warn(
"Declaratively configured supervisor received a [MaximumNumberOfRestartsWithinTimeRangeReached] notification," +
"\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." +

View file

@ -6,9 +6,6 @@ package akka.config
import akka.AkkaException
import akka.util.Logging
import akka.actor.{ActorRef, IllegalActorStateException}
import akka.dispatch.CompletableFuture
import net.lag.configgy.{Config => CConfig, Configgy, ParseException}
import java.net.InetSocketAddress
@ -17,14 +14,12 @@ import java.lang.reflect.Method
class ConfigurationException(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).
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Config {
object Config extends Logging {
val VERSION = "1.0-SNAPSHOT"
val HOME = {
@ -62,7 +57,7 @@ object Config {
val configFile = System.getProperty("akka.config", "")
try {
Configgy.configure(configFile)
ConfigLogger.log.info("Config loaded from -Dakka.config=%s", configFile)
log.slf4j.info("Config loaded from -Dakka.config=%s", configFile)
} catch {
case e: ParseException => throw new ConfigurationException(
"Config could not be loaded from -Dakka.config=" + configFile +
@ -73,7 +68,7 @@ object Config {
try {
val configFile = HOME.getOrElse(throwNoAkkaHomeException) + "/config/" + confName
Configgy.configure(configFile)
ConfigLogger.log.info(
log.slf4j.info(
"AKKA_HOME is defined as [%s], config loaded from [%s].",
HOME.getOrElse(throwNoAkkaHomeException),
configFile)
@ -87,7 +82,7 @@ object Config {
} else if (getClass.getClassLoader.getResource(confName) ne null) {
try {
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 {
case e: ParseException => throw new ConfigurationException(
"Can't load '" + confName + "' config file from application classpath," +
@ -95,7 +90,7 @@ object Config {
}
Configgy.config
} else {
ConfigLogger.log.warning(
log.slf4j.warn(
"\nCan't load '" + confName + "'." +
"\nOne of the three ways of locating the '" + confName + "' file needs to be defined:" +
"\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 =
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 {
val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory))
if (old ne null) {
log.debug("Shutting down %s", toString)
log.slf4j.debug("Shutting down %s", toString)
old.shutdownNow()
}
}
@ -144,17 +144,17 @@ class ExecutorBasedEventDrivenDispatcher(
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 + "]"
def suspend(actorRef: ActorRef) {
log.debug("Suspending %s",actorRef.uuid)
log.slf4j.debug("Suspending %s",actorRef.uuid)
getMailbox(actorRef).suspended.switchOn
}
def resume(actorRef: ActorRef) {
log.debug("Resuming %s",actorRef.uuid)
log.slf4j.debug("Resuming %s",actorRef.uuid)
val mbox = getMailbox(actorRef)
mbox.suspended.switchOff
registerForExecution(mbox)

View file

@ -171,12 +171,12 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
} 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 {
val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory))
if (old ne null) {
log.debug("Shutting down %s", toString)
log.slf4j.debug("Shutting down %s", toString)
old.shutdownNow()
}
}

View file

@ -135,7 +135,7 @@ trait MessageDispatcher extends MailboxFactory with Logging {
ActorRegistry.actorFor(uuid) match {
case Some(actor) => actor.stop
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() {
def uncaughtException(thread: Thread, cause: Throwable) =
log.error(cause, "UNCAUGHT in thread [%s]", thread.getName)
log.slf4j.error("Thread.UncaughtException", cause)
})
override def run = {
val debug = MonitorableThread.debugLifecycle
log.debug("Created thread %s", getName)
log.slf4j.debug("Created thread %s", getName)
try {
MonitorableThread.alive.incrementAndGet
super.run
} finally {
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 =>
semaphore.release
case e =>
log.error(e,"Unexpected exception")
log.slf4j.error("Unexpected exception", e)
throw e
}
}
@ -254,7 +254,7 @@ trait LazyExecutorService extends ExecutorServiceDelegate {
def createExecutor: ExecutorService
lazy val executor = {
log.info("Lazily initializing ExecutorService for ",this)
log.slf4j.info("Lazily initializing ExecutorService for ",this)
createExecutor
}
}

View file

@ -36,7 +36,7 @@ import java.net.{InetAddress, UnknownHostException}
}
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 generateSecureCookie: String = {
log.info("Generating secure cookie...")
log.slf4j.info("Generating secure cookie...")
val bytes = Array.fill(32)(0.byteValue)
random.nextBytes(bytes)
sha1(bytes)

View file

@ -42,7 +42,8 @@ object Helpers extends Logging {
narrow(o)
} catch {
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
}

View file

@ -57,7 +57,7 @@ trait ListenerManagement extends Logging {
while (iterator.hasNext) {
val listener = iterator.next
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) {
val listener = iterator.next
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:
* <pre>
* class Foo extends Logging {
* log.info("My foo is %s","alive")
* log.error(new Exception(),"My foo is %s","broken")
* log.slf4j.info("My foo is %s","alive")
* log.slf4j.error(new Exception(),"My foo is %s","broken")
* }
* </pre>
*
* 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...)
*/
class Logger(val logger: SLFLogger) {
def name = logger.getName
class Logger(val slf4j: SLFLogger) {
final def name = logger.getName
final def logger = slf4j
def trace_? = logger.isTraceEnabled
def debug_? = logger.isDebugEnabled
def info_? = logger.isInfoEnabled
def warning_? = logger.isWarnEnabled
def error_? = logger.isErrorEnabled
final def trace_? = logger.isTraceEnabled
final def debug_? = logger.isDebugEnabled
final def info_? = logger.isInfoEnabled
final def warning_? = logger.isWarnEnabled
final def error_? = logger.isErrorEnabled
//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:_*))
}
def trace(t: Throwable, msg: => String) {
final def trace(t: Throwable, msg: => String) {
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:_*))
}
def trace(msg: => String) {
final def trace(msg: => String) {
if (trace_?) logger trace msg
}
//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:_*))
}
def debug(t: Throwable, msg: => String) {
final def debug(t: Throwable, msg: => String) {
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:_*))
}
def debug(msg: => String) {
final def debug(msg: => String) {
if (debug_?) logger debug msg
}
//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:_*))
}
def info(t: Throwable, msg: => String) {
final def info(t: Throwable, msg: => String) {
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:_*))
}
def info(msg: => String) {
final def info(msg: => String) {
if (info_?) logger info msg
}
//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:_*))
}
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)
}
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:_*))
}
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
}
def warn(msg: => String) = warning(msg)
final def warn(msg: => String) = warning(msg)
//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:_*))
}
def error(t: Throwable, msg: => String) {
final def error(t: Throwable, msg: => String) {
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:_*))
}
def error(msg: => String) {
final def error(msg: => String) {
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)
else fmt.format((arg +: argN):_*)
}

View file

@ -185,7 +185,7 @@ object ReflectiveAccess extends Logging {
Some(ctor.newInstance(args: _*).asInstanceOf[T])
} catch {
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
}
@ -202,7 +202,7 @@ object ReflectiveAccess extends Logging {
Some(ctor.newInstance(args: _*).asInstanceOf[T])
} catch {
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
}
@ -214,7 +214,7 @@ object ReflectiveAccess extends Logging {
Option(instance.get(null).asInstanceOf[T])
} catch {
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
}

View file

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

View file

@ -498,7 +498,7 @@ class SupervisorSpec extends JUnitSuite {
val inits = new AtomicInteger(0)
val dyingActor = actorOf(new Actor {
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))
throw new IllegalStateException("Don't wanna!")

View file

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

View file

@ -35,7 +35,7 @@ trait EmbeddedAppServer extends Bootable with Logging {
abstract override def onLoad = {
super.onLoad
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.host", REST_HOSTNAME)
@ -60,14 +60,14 @@ trait EmbeddedAppServer extends Bootable with Logging {
s.start()
s
}
log.info("Akka HTTP service started")
log.slf4j.info("Akka HTTP service started")
}
}
abstract override def onUnload = {
super.onUnload
server foreach { t =>
log.info("Shutting down REST service (Jersey)")
log.slf4j.info("Shutting down REST service (Jersey)")
t.stop()
}
}

View file

@ -117,16 +117,16 @@ trait Mist extends Logging {
val server = context.getServerInfo
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) {
log.info("Supporting Java asynchronous contexts.")
log.slf4j.info("Supporting Java asynchronous contexts.")
Some(Servlet30ContextMethodFactory)
} else if (server.toLowerCase startsWith JettyServer) {
log.info("Supporting Jetty asynchronous continuations.")
log.slf4j.info("Supporting Jetty asynchronous continuations.")
Some(JettyContinuationMethodFactory)
} else {
log.error("No asynchronous request handling can be supported.")
log.slf4j.error("No asynchronous request handling can be supported.")
None
}
}
@ -184,7 +184,7 @@ class AkkaMistFilter extends Filter with Mist {
case "POST" => mistify(hreq, hres)(_factory.get.Post)
case "PUT" => mistify(hreq, hres)(_factory.get.Put)
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)
case _ => chain.doFilter(req,res)
@ -269,7 +269,7 @@ trait Endpoint { this: Actor =>
*/
protected def _na(uri: String, req: RequestMethod) = {
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 = {
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) => {
try {
if (!suspended) {
log.warning("Attempt to complete an expired connection.")
log.slf4j.warn("Attempt to complete an expired connection.")
false
}
else {
@ -396,13 +396,13 @@ trait RequestMethod extends Logging
}
} catch {
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
}
}
case None =>
log.error("Attempt to complete request with no context.")
log.slf4j.error("Attempt to complete request with no context.")
false
}
@ -411,7 +411,7 @@ trait RequestMethod extends Logging
case Some(pipe) => {
try {
if (!suspended) {
log.warning("Attempt to complete an expired connection.")
log.slf4j.warn("Attempt to complete an expired connection.")
}
else {
response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, "Failed to write data to connection on resume")
@ -419,15 +419,15 @@ trait RequestMethod extends Logging
}
}
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 {
log.error(t, "Request completed with internal error.")
log.slf4j.error("Request completed with internal error.", t)
}
}
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])
case None => throw new WebApplicationException(408)
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)
}
}
@ -258,7 +258,7 @@ trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] w
val ts = System.currentTimeMillis
nonceMap.filter(tuple => (ts - tuple._2) < nonceValidityPeriod)
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
@ -371,7 +371,7 @@ trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] w
Some(UserInfo(user, null, rolesFor(user)))
} catch {
case e: PrivilegedActionException => {
log.error(e, "Action not allowed")
log.slf4j.error("Action not allowed", e)
return None
}
}

View file

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

View file

@ -27,18 +27,18 @@ trait BootableRemoteActorService extends Bootable with Logging {
abstract override def onLoad = {
if (RemoteServer.isRemotingEnabled) {
log.info("Initializing Remote Actors Service...")
log.slf4j.info("Initializing Remote Actors Service...")
startRemoteService
log.info("Remote Actors Service initialized")
log.slf4j.info("Remote Actors Service initialized")
}
super.onLoad
}
abstract override def onUnload = {
log.info("Shutting down Remote Actors Service")
log.slf4j.info("Shutting down Remote Actors Service")
RemoteNode.shutdown
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
}
}

View file

@ -238,7 +238,7 @@ class RemoteClient private[akka] (
bootstrap.setOption("tcpNoDelay", 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.
connection = bootstrap.connect(remoteAddress)
@ -247,13 +247,14 @@ class RemoteClient private[akka] (
if (!connection.isSuccess) {
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))
}
def shutdown = runSwitch switchOff {
log.info("Shutting down %s", name)
log.slf4j.info("Shutting down %s", name)
notifyListeners(RemoteClientShutdown(this))
timer.stop
timer = null
@ -262,7 +263,7 @@ class RemoteClient private[akka] (
bootstrap.releaseExternalResources
bootstrap = null
connection = null
log.info("%s has been shut down", name)
log.slf4j.info("%s has been shut down", name)
}
@deprecated("Use addListener instead")
@ -341,7 +342,7 @@ class RemoteClient private[akka] (
} else {
val timeLeft = reconnectionTimeWindow - (System.currentTimeMillis - reconnectionTimeWindowStart)
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
} else false
}
@ -406,7 +407,7 @@ class RemoteClientHandler(
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
if (event.isInstanceOf[ChannelStateEvent] &&
event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) {
log.debug(event.toString)
log.slf4j.debug(event.toString)
}
super.handleUpstream(ctx, event)
}
@ -417,7 +418,7 @@ class RemoteClientHandler(
if (result.isInstanceOf[RemoteMessageProtocol]) {
val reply = result.asInstanceOf[RemoteMessageProtocol]
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]]
if (reply.hasMessage) {
if (future eq null) throw new IllegalActorStateException("Future mapped to UUID " + replyUuid + " does not exist")
@ -445,7 +446,7 @@ class RemoteClientHandler(
} catch {
case e: Exception =>
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
}
}
@ -456,12 +457,13 @@ class RemoteClientHandler(
def run(timeout: Timeout) = {
client.openChannels.remove(event.getChannel)
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.awaitUninterruptibly // Wait until the connection attempt succeeds or fails.
if (!client.connection.isSuccess) {
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)
@ -471,7 +473,7 @@ class RemoteClientHandler(
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
def connect = {
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
}
@ -488,12 +490,16 @@ class RemoteClientHandler(
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
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) = {
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
}

View file

@ -233,7 +233,7 @@ class RemoteServer extends Logging with ListenerManagement {
try {
if (!_isRunning) {
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)
val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, loader, this)
@ -249,7 +249,7 @@ class RemoteServer extends Logging with ListenerManagement {
}
} catch {
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))
}
this
@ -265,7 +265,7 @@ class RemoteServer extends Logging with ListenerManagement {
notifyListeners(RemoteServerShutdown(this))
} catch {
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
*/
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)
else registerTypedActor(id, typedActor, typedActors)
}
@ -297,7 +297,7 @@ class RemoteServer extends Logging with ListenerManagement {
* @param typedActor typed actor to register
*/
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)
}
@ -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.
*/
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)
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.
*/
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)
}
@ -354,7 +354,7 @@ class RemoteServer extends Logging with ListenerManagement {
*/
def unregister(actorRef: ActorRef):Unit = synchronized {
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)
actorsByUuid.remove(actorRef.uuid, actorRef)
}
@ -367,7 +367,7 @@ class RemoteServer extends Logging with ListenerManagement {
*/
def unregister(id: String):Unit = synchronized {
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))
else {
val actorRef = actors get id
@ -384,7 +384,7 @@ class RemoteServer extends Logging with ListenerManagement {
*/
def unregisterPerSession(id: String):Unit = {
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)
}
}
@ -396,7 +396,7 @@ class RemoteServer extends Logging with ListenerManagement {
*/
def unregisterTypedActor(id: String):Unit = synchronized {
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))
else typedActors.remove(id)
}
@ -506,7 +506,7 @@ class RemoteServerHandler(
val clientAddress = getClientAddress(ctx)
sessionActors.set(event.getChannel(), new ConcurrentHashMap[String, ActorRef]())
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) {
val sslHandler: SslHandler = ctx.getPipeline.get(classOf[SslHandler])
// Begin handshake.
@ -524,7 +524,7 @@ class RemoteServerHandler(
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
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
val channelActors = sessionActors.remove(event.getChannel)
if (channelActors ne null) {
@ -546,13 +546,13 @@ class RemoteServerHandler(
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
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))
}
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
if (event.isInstanceOf[ChannelStateEvent] && event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) {
log.debug(event.toString)
log.slf4j.debug(event.toString)
}
super.handleUpstream(ctx, event)
}
@ -568,7 +568,7 @@ class RemoteServerHandler(
}
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
server.notifyListeners(RemoteServerError(event.getCause, server))
}
@ -580,7 +580,7 @@ class RemoteServerHandler(
}
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 {
case SCALA_ACTOR => dispatchToActor(request, channel)
case TYPED_ACTOR => dispatchToTypedActor(request, channel)
@ -591,7 +591,7 @@ class RemoteServerHandler(
private def dispatchToActor(request: RemoteMessageProtocol, channel: Channel) {
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 =
try {
@ -627,7 +627,7 @@ class RemoteServerHandler(
val exception = f.exception
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 {
channel.write(createErrorReplyMessage(exception.get, request, AkkaActorType.ScalaActor))
} catch {
@ -635,7 +635,7 @@ class RemoteServerHandler(
}
}
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(
Some(actorRef),
Right(request.getUuid),
@ -667,7 +667,7 @@ class RemoteServerHandler(
private def dispatchToTypedActor(request: RemoteMessageProtocol, channel: Channel) = {
val actorInfo = request.getActorInfo
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 args = MessageSerializer.deserialize(request.getMessage).asInstanceOf[Array[AnyRef]].toList
@ -693,7 +693,7 @@ class RemoteServerHandler(
None)
if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid)
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 {
case e: Throwable => server.notifyListeners(RemoteServerError(e, server))
}
@ -798,7 +798,7 @@ class RemoteServerHandler(
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")
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)
else Class.forName(name)
val actorRef = Actor.actorOf(clazz.asInstanceOf[Class[_ <: Actor]])
@ -810,7 +810,7 @@ class RemoteServerHandler(
actorRef
} catch {
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))
throw e
}
@ -874,7 +874,7 @@ class RemoteServerHandler(
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")
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) =
if (applicationLoader.isDefined) (applicationLoader.get.loadClass(interfaceClassname),
@ -887,7 +887,7 @@ class RemoteServerHandler(
newInstance
} catch {
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))
throw e
}
@ -913,7 +913,8 @@ class RemoteServerHandler(
private def createErrorReplyMessage(exception: Throwable, request: RemoteMessageProtocol, actorType: AkkaActorType): RemoteMessageProtocol = {
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(
None,
Right(request.getUuid),
@ -941,7 +942,7 @@ class RemoteServerHandler(
"The remote client [" + clientAddress + "] does not have a secure cookie.")
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")
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](
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 =
if (format.isInstanceOf[SerializerBasedActorFormat[_]])
@ -226,7 +226,7 @@ object RemoteActorSerialization {
* Deserializes a RemoteActorRefProtocol Protocol Buffers (protobuf) Message into an RemoteActorRef instance.
*/
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(
protocol.getClassOrServiceName,
protocol.getActorClassname,
@ -244,7 +244,7 @@ object RemoteActorSerialization {
val host = homeAddress.getHostName
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)
ActorRegistry.registerActorByUuid(homeAddress, uuid.toString, ar)
@ -367,7 +367,7 @@ object TypedActorSerialization {
private def fromProtobufToLocalTypedActorRef[T <: Actor, U <: AnyRef](
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 intfClass = toClass(loader, protocol.getInterfaceName)
TypedActor.newInstance(intfClass, actorRef).asInstanceOf[U]
@ -407,7 +407,7 @@ object RemoteTypedActorSerialization {
* Deserializes a RemoteTypedActorRefProtocol Protocol Buffers (protobuf) Message into AW RemoteActorRef proxy.
*/
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 intfClass = TypedActorSerialization.toClass(loader, protocol.getInterfaceName)
TypedActor.createProxyForRemoteActorRef(intfClass, actorRef).asInstanceOf[T]

View file

@ -29,7 +29,7 @@ object ServerInitiatedRemoteActorClient extends Logging {
def run = {
val actor = RemoteClient.actorFor("hello-service", "localhost", 2552)
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

View file

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

View file

@ -12,7 +12,7 @@ import akka.util.Logging
class HelloWorldActor extends Actor {
def receive = {
case "Hello" =>
log.info("Received 'Hello'")
log.slf4j.info("Received 'Hello'")
self.reply("World")
}
}
@ -21,9 +21,9 @@ object ServerManagedRemoteActorServer extends Logging {
def run = {
RemoteNode.start("localhost", 2552)
log.info("Remote node started")
log.slf4j.info("Remote node started")
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
@ -33,10 +33,10 @@ object ServerManagedRemoteActorClient extends Logging {
def run = {
val actor = RemoteClient.actorFor("hello-service", "localhost", 2552)
log.info("Remote client created")
log.info("Sending 'Hello' to remote actor")
log.slf4j.info("Remote client created")
log.slf4j.info("Sending 'Hello' to remote actor")
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

View file

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

View file

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

View file

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

View file

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

View file

@ -43,7 +43,7 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa
* @return the typed actors for the class
*/
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(
"inject() and/or supervise() must be called before invoking getInstance(clazz)")
val (proxy, targetInstance, component) =

View file

@ -19,7 +19,7 @@
<pattern>[%t] [%4p] [%d{ISO8601}] %c{1}: %m%n</pattern>
</encoder>
<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>
</appender>
<logger name="akka" level="INFO"/>

View file

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