make EventHandler non-global
- add Logging trait for nicer interface - add EventHandlerLogging class for offering the nice interface from app.log - add eventHandler instance to app and use that for all internal logging (this means that some places (dispatchers, remoting) were infiltrated by app just to do logging, but I think we'll need app in there soon enough for other reasons)
This commit is contained in:
parent
e25ee9f0e2
commit
85b7accfd4
61 changed files with 590 additions and 542 deletions
|
|
@ -9,7 +9,6 @@ import akka.dispatch.{ ActorPromise, DefaultPromise, Promise }
|
|||
import akka.remote._
|
||||
import RemoteProtocol._
|
||||
import akka.util._
|
||||
import akka.event.EventHandler
|
||||
import org.jboss.netty.channel._
|
||||
import org.jboss.netty.channel.group.{ DefaultChannelGroup, ChannelGroup, ChannelGroupFuture }
|
||||
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
|
||||
|
|
@ -209,7 +208,7 @@ abstract class RemoteClient private[akka] (
|
|||
senderFuture: Option[Promise[T]]): Option[Promise[T]] = {
|
||||
|
||||
if (isRunning) {
|
||||
EventHandler.debug(this, "Sending to connection [%s] message [\n%s]".format(remoteAddress, request))
|
||||
app.eventHandler.debug(this, "Sending to connection [%s] message [\n%s]".format(remoteAddress, request))
|
||||
|
||||
// tell
|
||||
if (request.getOneWay) {
|
||||
|
|
@ -279,7 +278,7 @@ abstract class RemoteClient private[akka] (
|
|||
private[remote] def sendPendingRequests() = pendingRequests synchronized {
|
||||
// ensure only one thread at a time can flush the log
|
||||
val nrOfMessages = pendingRequests.size
|
||||
if (nrOfMessages > 0) EventHandler.info(this, "Resending [%s] previously failed messages after remote client reconnect" format nrOfMessages)
|
||||
if (nrOfMessages > 0) app.eventHandler.info(this, "Resending [%s] previously failed messages after remote client reconnect" format nrOfMessages)
|
||||
var pendingRequest = pendingRequests.peek
|
||||
|
||||
while (pendingRequest ne null) {
|
||||
|
|
@ -362,14 +361,14 @@ class ActiveRemoteClient private[akka] (
|
|||
}
|
||||
|
||||
def attemptReconnect(): Boolean = {
|
||||
EventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
|
||||
app.eventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
|
||||
|
||||
val connection = bootstrap.connect(remoteAddress)
|
||||
openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
|
||||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress))
|
||||
EventHandler.error(connection.getCause, this, "Reconnection to [%s] has failed".format(remoteAddress))
|
||||
app.eventHandler.error(connection.getCause, this, "Reconnection to [%s] has failed".format(remoteAddress))
|
||||
false
|
||||
|
||||
} else {
|
||||
|
|
@ -387,7 +386,7 @@ class ActiveRemoteClient private[akka] (
|
|||
bootstrap.setOption("tcpNoDelay", true)
|
||||
bootstrap.setOption("keepAlive", true)
|
||||
|
||||
EventHandler.debug(this, "Starting remote client connection to [%s]".format(remoteAddress))
|
||||
app.eventHandler.debug(this, "Starting remote client connection to [%s]".format(remoteAddress))
|
||||
|
||||
connection = bootstrap.connect(remoteAddress)
|
||||
|
||||
|
|
@ -396,7 +395,7 @@ class ActiveRemoteClient private[akka] (
|
|||
|
||||
if (!connection.isSuccess) {
|
||||
notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress))
|
||||
EventHandler.error(connection.getCause, this, "Remote client connection to [%s] has failed".format(remoteAddress))
|
||||
app.eventHandler.error(connection.getCause, this, "Remote client connection to [%s] has failed".format(remoteAddress))
|
||||
false
|
||||
|
||||
} else {
|
||||
|
|
@ -423,7 +422,7 @@ class ActiveRemoteClient private[akka] (
|
|||
case false if reconnectIfAlreadyConnected ⇒
|
||||
closeChannel(connection)
|
||||
|
||||
EventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
|
||||
app.eventHandler.debug(this, "Remote client reconnecting to [%s]".format(remoteAddress))
|
||||
attemptReconnect()
|
||||
|
||||
case false ⇒ false
|
||||
|
|
@ -432,7 +431,7 @@ class ActiveRemoteClient private[akka] (
|
|||
|
||||
// Please note that this method does _not_ remove the ARC from the NettyRemoteClientModule's map of clients
|
||||
def shutdown() = runSwitch switchOff {
|
||||
EventHandler.info(this, "Shutting down remote client [%s]".format(name))
|
||||
app.eventHandler.info(this, "Shutting down remote client [%s]".format(name))
|
||||
|
||||
notifyListeners(RemoteClientShutdown(module, remoteAddress))
|
||||
timer.stop()
|
||||
|
|
@ -444,7 +443,7 @@ class ActiveRemoteClient private[akka] (
|
|||
connection = null
|
||||
pendingRequests.clear()
|
||||
|
||||
EventHandler.info(this, "[%s] has been shut down".format(name))
|
||||
app.eventHandler.info(this, "[%s] has been shut down".format(name))
|
||||
}
|
||||
|
||||
private[akka] def isWithinReconnectionTimeWindow: Boolean = {
|
||||
|
|
@ -454,7 +453,7 @@ class ActiveRemoteClient private[akka] (
|
|||
} else {
|
||||
val timeLeft = (RECONNECTION_TIME_WINDOW - (System.currentTimeMillis - reconnectionTimeWindowStart)) > 0
|
||||
if (timeLeft) {
|
||||
EventHandler.info(this, "Will try to reconnect to remote server for another [%s] milliseconds".format(timeLeft))
|
||||
app.eventHandler.info(this, "Will try to reconnect to remote server for another [%s] milliseconds".format(timeLeft))
|
||||
}
|
||||
timeLeft
|
||||
}
|
||||
|
|
@ -521,7 +520,7 @@ class ActiveRemoteClientHandler(
|
|||
case arp: AkkaRemoteProtocol if arp.hasMessage ⇒
|
||||
val reply = arp.getMessage
|
||||
val replyUuid = uuidFrom(reply.getActorInfo.getUuid.getHigh, reply.getActorInfo.getUuid.getLow)
|
||||
EventHandler.debug(this, "Remote client received RemoteMessageProtocol[\n%s]\nTrying to map back to future [%s]".format(reply, replyUuid))
|
||||
app.eventHandler.debug(this, "Remote client received RemoteMessageProtocol[\n%s]\nTrying to map back to future [%s]".format(reply, replyUuid))
|
||||
|
||||
futures.remove(replyUuid).asInstanceOf[Promise[Any]] match {
|
||||
case null ⇒
|
||||
|
|
@ -543,7 +542,7 @@ class ActiveRemoteClientHandler(
|
|||
}
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
EventHandler.error(e, this, e.getMessage)
|
||||
app.eventHandler.error(e, this, e.getMessage)
|
||||
client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
|
@ -567,24 +566,24 @@ class ActiveRemoteClientHandler(
|
|||
try {
|
||||
if (client.useTransactionLog) client.sendPendingRequests() // try to send pending requests (still there after client/server crash ard reconnect
|
||||
client.notifyListeners(RemoteClientConnected(client.module, client.remoteAddress))
|
||||
EventHandler.debug(this, "Remote client connected to [%s]".format(ctx.getChannel.getRemoteAddress))
|
||||
app.eventHandler.debug(this, "Remote client connected to [%s]".format(ctx.getChannel.getRemoteAddress))
|
||||
client.resetReconnectionTimeWindow
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
EventHandler.error(e, this, e.getMessage)
|
||||
app.eventHandler.error(e, this, e.getMessage)
|
||||
client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress))
|
||||
}
|
||||
}
|
||||
|
||||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
client.notifyListeners(RemoteClientDisconnected(client.module, client.remoteAddress))
|
||||
EventHandler.debug(this, "Remote client disconnected from [%s]".format(ctx.getChannel.getRemoteAddress))
|
||||
app.eventHandler.debug(this, "Remote client disconnected from [%s]".format(ctx.getChannel.getRemoteAddress))
|
||||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
val cause = event.getCause
|
||||
if (cause ne null) {
|
||||
EventHandler.error(event.getCause, this, "Unexpected exception [%s] from downstream in remote client [%s]".format(event.getCause, event))
|
||||
app.eventHandler.error(event.getCause, this, "Unexpected exception [%s] from downstream in remote client [%s]".format(event.getCause, event))
|
||||
|
||||
cause match {
|
||||
case e: ReadTimeoutException ⇒
|
||||
|
|
@ -596,7 +595,7 @@ class ActiveRemoteClientHandler(
|
|||
event.getChannel.close //FIXME Is this the correct behavior?
|
||||
}
|
||||
|
||||
} else EventHandler.error(this, "Unexpected exception from downstream in remote client [%s]".format(event))
|
||||
} else app.eventHandler.error(this, "Unexpected exception from downstream in remote client [%s]".format(event))
|
||||
}
|
||||
|
||||
private def parseException(reply: RemoteMessageProtocol, loader: Option[ClassLoader]): Throwable = {
|
||||
|
|
@ -611,7 +610,7 @@ class ActiveRemoteClientHandler(
|
|||
.newInstance(exception.getMessage).asInstanceOf[Throwable]
|
||||
} catch {
|
||||
case problem: Exception ⇒
|
||||
EventHandler.error(problem, this, problem.getMessage)
|
||||
app.eventHandler.error(problem, this, problem.getMessage)
|
||||
CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(problem, classname, exception.getMessage)
|
||||
}
|
||||
}
|
||||
|
|
@ -646,7 +645,7 @@ class NettyRemoteSupport(_app: AkkaApplication) extends RemoteSupport(_app) with
|
|||
}
|
||||
|
||||
val remoteInetSocketAddress = new InetSocketAddress(host, port)
|
||||
EventHandler.debug(this,
|
||||
app.eventHandler.debug(this,
|
||||
"Creating RemoteActorRef with address [%s] connected to [%s]"
|
||||
.format(actorAddress, remoteInetSocketAddress))
|
||||
RemoteActorRef(app, app.remote, remoteInetSocketAddress, actorAddress, loader)
|
||||
|
|
@ -689,7 +688,7 @@ class NettyRemoteServer(app: AkkaApplication, serverModule: NettyRemoteServerMod
|
|||
serverModule.notifyListeners(RemoteServerStarted(serverModule))
|
||||
|
||||
def shutdown() {
|
||||
EventHandler.info(this, "Shutting down remote server [%s]".format(name))
|
||||
app.eventHandler.info(this, "Shutting down remote server [%s]".format(name))
|
||||
try {
|
||||
val shutdownSignal = {
|
||||
val b = RemoteControlProtocol.newBuilder.setCommandType(CommandType.SHUTDOWN)
|
||||
|
|
@ -705,7 +704,7 @@ class NettyRemoteServer(app: AkkaApplication, serverModule: NettyRemoteServerMod
|
|||
serverModule.notifyListeners(RemoteServerShutdown(serverModule))
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
EventHandler.error(e, this, e.getMessage)
|
||||
app.eventHandler.error(e, this, e.getMessage)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -736,13 +735,13 @@ trait NettyRemoteServerModule extends RemoteServerModule {
|
|||
def start(_hostname: String, _port: Int, loader: Option[ClassLoader] = None): RemoteServerModule = guard withGuard {
|
||||
try {
|
||||
_isRunning switchOn {
|
||||
EventHandler.debug(this, "Starting up remote server on %s:s".format(_hostname, _port))
|
||||
app.eventHandler.debug(this, "Starting up remote server on %s:s".format(_hostname, _port))
|
||||
|
||||
currentServer.set(Some(new NettyRemoteServer(app, this, _hostname, _port, loader)))
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
EventHandler.error(e, this, e.getMessage)
|
||||
app.eventHandler.error(e, this, e.getMessage)
|
||||
notifyListeners(RemoteServerError(e, this))
|
||||
}
|
||||
this
|
||||
|
|
@ -751,7 +750,7 @@ trait NettyRemoteServerModule extends RemoteServerModule {
|
|||
def shutdownServerModule() = guard withGuard {
|
||||
_isRunning switchOff {
|
||||
currentServer.getAndSet(None) foreach { instance ⇒
|
||||
EventHandler.debug(this, "Shutting down remote server on %s:%s".format(instance.host, instance.port))
|
||||
app.eventHandler.debug(this, "Shutting down remote server on %s:%s".format(instance.host, instance.port))
|
||||
instance.shutdown()
|
||||
}
|
||||
}
|
||||
|
|
@ -796,7 +795,7 @@ trait NettyRemoteServerModule extends RemoteServerModule {
|
|||
def unregister(actorRef: ActorRef): Unit = guard withGuard {
|
||||
|
||||
if (_isRunning.isOn) {
|
||||
EventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(actorRef.uuid))
|
||||
app.eventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(actorRef.uuid))
|
||||
|
||||
actors.remove(actorRef.address, actorRef)
|
||||
actorsByUuid.remove(actorRef.uuid.toString, actorRef)
|
||||
|
|
@ -811,7 +810,7 @@ trait NettyRemoteServerModule extends RemoteServerModule {
|
|||
def unregister(id: String): Unit = guard withGuard {
|
||||
|
||||
if (_isRunning.isOn) {
|
||||
EventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(id))
|
||||
app.eventHandler.debug(this, "Unregister server side remote actor with id [%s]".format(id))
|
||||
|
||||
if (id.startsWith(UUID_PREFIX)) actorsByUuid.remove(id.substring(UUID_PREFIX.length))
|
||||
else {
|
||||
|
|
@ -830,7 +829,7 @@ trait NettyRemoteServerModule extends RemoteServerModule {
|
|||
def unregisterPerSession(id: String) {
|
||||
|
||||
if (_isRunning.isOn) {
|
||||
EventHandler.info(this, "Unregistering server side remote actor with id [%s]".format(id))
|
||||
app.eventHandler.info(this, "Unregistering server side remote actor with id [%s]".format(id))
|
||||
|
||||
actorsFactories.remove(id)
|
||||
}
|
||||
|
|
@ -936,7 +935,7 @@ class RemoteServerHandler(
|
|||
|
||||
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx)
|
||||
EventHandler.debug(this, "Remote client [%s] connected to [%s]".format(clientAddress, server.name))
|
||||
app.eventHandler.debug(this, "Remote client [%s] connected to [%s]".format(clientAddress, server.name))
|
||||
|
||||
sessionActors.set(event.getChannel(), new ConcurrentHashMap[String, ActorRef]())
|
||||
server.notifyListeners(RemoteServerClientConnected(server, clientAddress))
|
||||
|
|
@ -945,7 +944,7 @@ class RemoteServerHandler(
|
|||
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx)
|
||||
|
||||
EventHandler.debug(this, "Remote client [%s] disconnected from [%s]".format(clientAddress, server.name))
|
||||
app.eventHandler.debug(this, "Remote client [%s] disconnected from [%s]".format(clientAddress, server.name))
|
||||
|
||||
// stop all session actors
|
||||
for (
|
||||
|
|
@ -955,7 +954,7 @@ class RemoteServerHandler(
|
|||
try {
|
||||
actor ! PoisonPill
|
||||
} catch {
|
||||
case e: Exception ⇒ EventHandler.error(e, this, "Couldn't stop %s".format(actor))
|
||||
case e: Exception ⇒ app.eventHandler.error(e, this, "Couldn't stop %s".format(actor))
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -964,7 +963,7 @@ class RemoteServerHandler(
|
|||
|
||||
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
|
||||
val clientAddress = getClientAddress(ctx)
|
||||
EventHandler.debug("Remote client [%s] channel closed from [%s]".format(clientAddress, server.name), this)
|
||||
app.eventHandler.debug("Remote client [%s] channel closed from [%s]".format(clientAddress, server.name), this)
|
||||
|
||||
server.notifyListeners(RemoteServerClientClosed(server, clientAddress))
|
||||
}
|
||||
|
|
@ -984,7 +983,7 @@ class RemoteServerHandler(
|
|||
}
|
||||
|
||||
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
||||
EventHandler.error(event.getCause, this, "Unexpected exception from remote downstream")
|
||||
app.eventHandler.error(event.getCause, this, "Unexpected exception from remote downstream")
|
||||
|
||||
event.getChannel.close
|
||||
server.notifyListeners(RemoteServerError(event.getCause, server))
|
||||
|
|
@ -997,25 +996,25 @@ class RemoteServerHandler(
|
|||
}
|
||||
|
||||
private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = try {
|
||||
EventHandler.debug(this, "Received remote message [%s]".format(request))
|
||||
app.eventHandler.debug(this, "Received remote message [%s]".format(request))
|
||||
dispatchToActor(request, channel)
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
server.notifyListeners(RemoteServerError(e, server))
|
||||
EventHandler.error(e, this, e.getMessage)
|
||||
app.eventHandler.error(e, this, e.getMessage)
|
||||
}
|
||||
|
||||
private def dispatchToActor(request: RemoteMessageProtocol, channel: Channel) {
|
||||
val actorInfo = request.getActorInfo
|
||||
|
||||
EventHandler.debug(this, "Dispatching to remote actor [%s]".format(actorInfo.getUuid))
|
||||
app.eventHandler.debug(this, "Dispatching to remote actor [%s]".format(actorInfo.getUuid))
|
||||
|
||||
val actorRef =
|
||||
try {
|
||||
createActor(actorInfo, channel)
|
||||
} catch {
|
||||
case e: SecurityException ⇒
|
||||
EventHandler.error(e, this, e.getMessage)
|
||||
app.eventHandler.error(e, this, e.getMessage)
|
||||
write(channel, createErrorReplyMessage(e, request))
|
||||
server.notifyListeners(RemoteServerError(e, server))
|
||||
return
|
||||
|
|
@ -1078,7 +1077,7 @@ class RemoteServerHandler(
|
|||
// else addr
|
||||
// }
|
||||
|
||||
EventHandler.debug(this,
|
||||
app.eventHandler.debug(this,
|
||||
"Looking up a remotely available actor for address [%s] on node [%s]"
|
||||
.format(address, app.nodename))
|
||||
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue