Added a remote event handler that pipes remote server and client events to the standard EventHandler system

This commit is contained in:
Jonas Bonér 2011-03-24 12:28:01 +01:00
parent 03ad1acdd9
commit cb0f14a2c0
2 changed files with 60 additions and 19 deletions

View file

@ -7,44 +7,36 @@ package akka.remote.netty
import akka.dispatch.{DefaultCompletableFuture, CompletableFuture, Future}
import akka.remote.protocol.RemoteProtocol._
import akka.remote.protocol.RemoteProtocol.ActorType._
import akka.config.ConfigurationException
import akka.serialization.RemoteActorSerialization
import akka.serialization.RemoteActorSerialization._
import akka.japi.Creator
import akka.config.Config._
import akka.remoteinterface._
import akka.actor.{PoisonPill, Index,
ActorInitializationException, LocalActorRef, newUuid,
ActorRegistry, Actor, RemoteActorRef,
import akka.actor.{PoisonPill, Index, LocalActorRef, Actor, RemoteActorRef,
TypedActor, ActorRef, IllegalActorStateException,
RemoteActorSystemMessage, uuidFrom, Uuid,
Exit, LifeCycleMessage, ActorType => AkkaActorType}
import akka.AkkaException
import akka.event.EventHandler
import akka.actor.Actor._
import akka.util._
import akka.remote.{MessageSerializer, RemoteClientSettings, RemoteServerSettings}
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
import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
import org.jboss.netty.bootstrap.{ServerBootstrap,ClientBootstrap}
import org.jboss.netty.bootstrap.{ServerBootstrap, ClientBootstrap}
import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender }
import org.jboss.netty.handler.codec.compression.{ ZlibDecoder, ZlibEncoder }
import org.jboss.netty.handler.codec.protobuf.{ ProtobufDecoder, ProtobufEncoder }
import org.jboss.netty.handler.timeout.{ ReadTimeoutHandler, ReadTimeoutException }
import org.jboss.netty.handler.execution.{ OrderedMemoryAwareThreadPoolExecutor, ExecutionHandler }
import org.jboss.netty.util.{ TimerTask, Timeout, HashedWheelTimer }
import org.jboss.netty.handler.ssl.SslHandler
import scala.collection.mutable.{ HashMap }
import scala.reflect.BeanProperty
import scala.collection.mutable.HashMap
import java.net.{ SocketAddress, InetSocketAddress }
import java.net.InetSocketAddress
import java.lang.reflect.InvocationTargetException
import java.util.concurrent.{ TimeUnit, Executors, ConcurrentMap, ConcurrentHashMap, ConcurrentSkipListSet }
import java.util.concurrent.atomic.{AtomicReference, AtomicLong, AtomicBoolean}
import java.util.concurrent.{ TimeUnit, Executors, ConcurrentMap, ConcurrentHashMap }
import java.util.concurrent.atomic.{AtomicReference, AtomicBoolean}
import akka.remote.{RemoteEventHandler, MessageSerializer, RemoteClientSettings, RemoteServerSettings}
object RemoteEncoder {
def encode(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
@ -212,8 +204,9 @@ abstract class RemoteClient private[akka] (
* Sends the message across the wire
*/
def send[T](
request: RemoteMessageProtocol,
senderFuture: Option[CompletableFuture[T]]): Option[CompletableFuture[T]] = {
request: RemoteMessageProtocol,
senderFuture: Option[CompletableFuture[T]]): Option[CompletableFuture[T]] = {
if (isRunning) {
if (request.getOneWay) {
val future = currentChannel.write(RemoteEncoder.encode(request))
@ -222,7 +215,6 @@ abstract class RemoteClient private[akka] (
notifyListeners(RemoteClientWriteFailed(request, future.getCause, module, remoteAddress))
throw future.getCause
}
None
} else {
val futureResult = if (senderFuture.isDefined) senderFuture.get
@ -497,6 +489,9 @@ class NettyRemoteSupport extends RemoteSupport with NettyRemoteServerModule with
//Needed for remote testing and switching on/off under run
val optimizeLocal = new AtomicBoolean(true)
// add the remote client and server listener that pipes the events to the event handler system
addListener(Actor.actorOf[RemoteEventHandler].start)
def optimizeLocalScoped_?() = optimizeLocal.get
protected[akka] def actorFor(serviceId: String, className: String, timeout: Long, host: String, port: Int, loader: Option[ClassLoader]): ActorRef = {