2009-06-24 15:12:47 +02:00
/* *
2010-12-22 15:35:50 +01:00
* Copyright ( C ) 2009 - 2011 Scalable Solutions AB < http : //scalablesolutions.se>
2009-06-24 15:12:47 +02:00
*/
2010-10-26 12:49:25 +02:00
package akka.remote
2009-06-24 15:12:47 +02:00
2010-12-15 17:52:31 +01:00
import akka.remote.protocol.RemoteProtocol. { ActorType => ActorTypeProtocol , _ }
import akka.dispatch. { DefaultCompletableFuture , CompletableFuture , Future }
2010-10-26 12:49:25 +02:00
import akka.remote.protocol.RemoteProtocol._
import akka.remote.protocol.RemoteProtocol.ActorType._
2010-10-31 19:27:55 +01:00
import akka.config.ConfigurationException
2010-10-26 12:49:25 +02:00
import akka.serialization.RemoteActorSerialization
2010-12-15 17:52:31 +01:00
import akka.japi.Creator
import akka.config.Config._
2010-10-26 12:49:25 +02:00
import akka.serialization.RemoteActorSerialization._
2010-12-15 17:52:31 +01:00
import akka.AkkaException
import akka.actor.Actor._
import akka.util._
2009-07-18 00:16:32 +02:00
2009-06-24 15:12:47 +02:00
import org.jboss.netty.channel._
2010-12-15 17:52:31 +01:00
import org.jboss.netty.channel.group. { DefaultChannelGroup , ChannelGroup }
import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
2009-06-24 15:12:47 +02:00
import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
2010-12-15 17:52:31 +01:00
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
import org.jboss.netty.util. { TimerTask , Timeout , HashedWheelTimer }
2010-04-25 20:32:52 +02:00
import org.jboss.netty.handler.ssl.SslHandler
2010-12-15 17:52:31 +01:00
import java.net. { SocketAddress , InetSocketAddress }
import java.util.concurrent. { TimeUnit , Executors , ConcurrentMap , ConcurrentHashMap , ConcurrentSkipListSet }
import scala.collection.mutable. { HashSet , HashMap }
2010-08-21 16:37:33 +02:00
import scala.reflect.BeanProperty
2010-12-15 17:52:31 +01:00
import java.lang.reflect.InvocationTargetException
2010-12-20 16:58:05 +01:00
import akka.actor. { ActorInitializationException , LocalActorRef , newUuid , ActorRegistry , Actor , RemoteActorRef , TypedActor , ActorRef , IllegalActorStateException , RemoteActorSystemMessage , uuidFrom , Uuid , Exit , LifeCycleMessage , ActorType => AkkaActorType }
import java.util.concurrent.atomic. { AtomicReference , AtomicLong , AtomicBoolean }
2010-12-29 16:08:43 +01:00
import akka.remoteinterface._
2010-12-20 12:10:46 +01:00
2010-12-15 17:52:31 +01:00
/* *
* The RemoteClient object manages RemoteClient instances and gives you an API to lookup remote actor handles .
2009-12-18 21:26:03 +01:00
*
2010-12-15 17:52:31 +01:00
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
trait NettyRemoteClientModule extends RemoteClientModule { self : ListenerManagement with Logging =>
private val remoteClients = new HashMap [ String , RemoteClient ]
private val remoteActors = new HashMap [ Address , HashSet [ Uuid ] ]
protected [ akka ] def typedActorFor [ T ] ( intfClass : Class [ T ] , serviceId : String , implClassName : String , timeout : Long , hostname : String , port : Int , loader : Option [ ClassLoader ] ) : T =
2010-12-20 16:58:05 +01:00
TypedActor . createProxyForRemoteActorRef ( intfClass , RemoteActorRef ( serviceId , implClassName , hostname , port , timeout , loader , AkkaActorType . TypedActor ) )
2010-12-15 17:52:31 +01:00
protected [ akka ] def send [ T ] ( message : Any ,
senderOption : Option [ ActorRef ] ,
senderFuture : Option [ CompletableFuture [ T ] ] ,
remoteAddress : InetSocketAddress ,
timeout : Long ,
isOneWay : Boolean ,
actorRef : ActorRef ,
typedActorInfo : Option [ Tuple2 [ String , String ] ] ,
2010-12-20 12:10:46 +01:00
actorType : AkkaActorType ,
loader : Option [ ClassLoader ] ) : Option [ CompletableFuture [ T ] ] =
clientFor ( remoteAddress , loader ) . send [ T ] ( message , senderOption , senderFuture , remoteAddress , timeout , isOneWay , actorRef , typedActorInfo , actorType )
2010-12-15 17:52:31 +01:00
private [ akka ] def clientFor (
address : InetSocketAddress , loader : Option [ ClassLoader ] ) : RemoteClient = synchronized { //TODO: REVIST: synchronized here seems bottlenecky
val hostname = address . getHostName
val port = address . getPort
val hash = hostname + ':' + port
loader . foreach ( MessageSerializer . setClassLoader ( _ ) ) //TODO: REVISIT: THIS SMELLS FUNNY
if ( remoteClients . contains ( hash ) ) remoteClients ( hash )
else {
2010-12-29 16:08:43 +01:00
val client = new RemoteClient ( this , new InetSocketAddress ( hostname , port ) , loader , self . notifyListeners _ )
2010-12-15 17:52:31 +01:00
client . connect
remoteClients += hash -> client
client
}
}
def shutdownClientFor ( address : InetSocketAddress ) = synchronized {
val hostname = address . getHostName
val port = address . getPort
val hash = hostname + ':' + port
if ( remoteClients . contains ( hash ) ) {
val client = remoteClients ( hash )
client . shutdown
remoteClients -= hash
}
}
2010-12-20 16:58:05 +01:00
private [ akka ] def registerSupervisorForActor ( actorRef : ActorRef ) : ActorRef =
2010-12-21 14:36:47 +01:00
clientFor ( actorRef . homeAddress . get , None ) . registerSupervisorForActor ( actorRef )
2010-12-15 17:52:31 +01:00
private [ akka ] def deregisterSupervisorForActor ( actorRef : ActorRef ) : ActorRef =
2010-12-21 14:36:47 +01:00
clientFor ( actorRef . homeAddress . get , None ) . deregisterSupervisorForActor ( actorRef )
2010-12-15 17:52:31 +01:00
/* *
* Clean - up all open connections .
*/
def shutdownClientModule = synchronized {
remoteClients . foreach ( { case ( addr , client ) => client . shutdown } )
remoteClients . clear
}
def registerClientManagedActor ( hostname : String , port : Int , uuid : Uuid ) = synchronized {
actorsFor ( Address ( hostname , port ) ) += uuid
}
private [ akka ] def unregisterClientManagedActor ( hostname : String , port : Int , uuid : Uuid ) = synchronized {
val set = actorsFor ( Address ( hostname , port ) )
set -= uuid
if ( set . isEmpty ) shutdownClientFor ( new InetSocketAddress ( hostname , port ) )
}
private [ akka ] def actorsFor ( remoteServerAddress : Address ) : HashSet [ Uuid ] = {
val set = remoteActors . get ( remoteServerAddress )
if ( set . isDefined && ( set . get ne null ) ) set . get
else {
val remoteActorSet = new HashSet [ Uuid ]
remoteActors . put ( remoteServerAddress , remoteActorSet )
remoteActorSet
}
}
}
object RemoteClient {
val SECURE_COOKIE : Option [ String ] = {
val cookie = config . getString ( "akka.remote.secure-cookie" , "" )
if ( cookie == "" ) None else Some ( cookie )
}
val READ_TIMEOUT = Duration ( config . getInt ( "akka.remote.client.read-timeout" , 1 ) , TIME_UNIT )
val RECONNECT_DELAY = Duration ( config . getInt ( "akka.remote.client.reconnect-delay" , 5 ) , TIME_UNIT )
val MESSAGE_FRAME_SIZE = config . getInt ( "akka.remote.client.message-frame-size" , 1048576 )
}
/* *
* RemoteClient represents a connection to a RemoteServer . Is used to send messages to remote actors on the RemoteServer .
2009-11-24 17:41:08 +01:00
*
2009-07-23 20:01:37 +02:00
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
2010-12-15 17:52:31 +01:00
class RemoteClient private [ akka ] (
2010-12-29 16:08:43 +01:00
val module : NettyRemoteClientModule ,
val remoteAddress : InetSocketAddress ,
2010-12-15 17:52:31 +01:00
val loader : Option [ ClassLoader ] = None ,
val notifyListeners : ( => Any ) => Unit ) extends Logging {
2010-12-29 16:08:43 +01:00
val name = "RemoteClient@" + remoteAddress . getHostName + "::" + remoteAddress . getPort
2010-12-15 17:52:31 +01:00
//FIXME Should these be clear:ed on postStop?
private val futures = new ConcurrentHashMap [ Uuid , CompletableFuture [ _ ] ]
private val supervisors = new ConcurrentHashMap [ Uuid , ActorRef ]
//FIXME rewrite to a wrapper object (minimize volatile access and maximize encapsulation)
@volatile
private var bootstrap : ClientBootstrap = _
@volatile
private [ remote ] var connection : ChannelFuture = _
@volatile
private [ remote ] var openChannels : DefaultChannelGroup = _
@volatile
private var timer : HashedWheelTimer = _
private [ remote ] val runSwitch = new Switch ( )
private [ remote ] val isAuthenticated = new AtomicBoolean ( false )
private [ remote ] def isRunning = runSwitch . isOn
private val reconnectionTimeWindow = Duration ( config . getInt (
"akka.remote.client.reconnection-time-window" , 600 ) , TIME_UNIT ) . toMillis
@volatile
private var reconnectionTimeWindowStart = 0L
def connect = runSwitch switchOn {
openChannels = new DefaultChannelGroup ( classOf [ RemoteClient ] . getName )
timer = new HashedWheelTimer
bootstrap = new ClientBootstrap ( new NioClientSocketChannelFactory ( Executors . newCachedThreadPool , Executors . newCachedThreadPool ) )
bootstrap . setPipelineFactory ( new RemoteClientPipelineFactory ( name , futures , supervisors , bootstrap , remoteAddress , timer , this ) )
bootstrap . setOption ( "tcpNoDelay" , true )
bootstrap . setOption ( "keepAlive" , true )
2010-12-29 16:08:43 +01:00
log . slf4j . info ( "Starting remote client connection to [{}]" , remoteAddress )
2010-12-15 17:52:31 +01:00
// Wait until the connection attempt succeeds or fails.
connection = bootstrap . connect ( remoteAddress )
val channel = connection . awaitUninterruptibly . getChannel
openChannels . add ( channel )
if ( ! connection . isSuccess ) {
2010-12-29 16:08:43 +01:00
notifyListeners ( RemoteClientError ( connection . getCause , module , remoteAddress ) )
log . slf4j . error ( "Remote client connection to [{}] has failed" , remoteAddress )
2010-12-15 17:52:31 +01:00
log . slf4j . debug ( "Remote client connection failed" , connection . getCause )
}
2010-12-29 16:08:43 +01:00
notifyListeners ( RemoteClientStarted ( module , remoteAddress ) )
2010-12-15 17:52:31 +01:00
}
def shutdown = runSwitch switchOff {
log . slf4j . info ( "Shutting down {}" , name )
2010-12-29 16:08:43 +01:00
notifyListeners ( RemoteClientShutdown ( module , remoteAddress ) )
2010-12-15 17:52:31 +01:00
timer . stop
timer = null
openChannels . close . awaitUninterruptibly
openChannels = null
bootstrap . releaseExternalResources
bootstrap = null
connection = null
log . slf4j . info ( "{} has been shut down" , name )
}
def send [ T ] (
message : Any ,
senderOption : Option [ ActorRef ] ,
senderFuture : Option [ CompletableFuture [ T ] ] ,
remoteAddress : InetSocketAddress ,
timeout : Long ,
isOneWay : Boolean ,
actorRef : ActorRef ,
typedActorInfo : Option [ Tuple2 [ String , String ] ] ,
actorType : AkkaActorType ) : Option [ CompletableFuture [ T ] ] = {
send ( createRemoteMessageProtocolBuilder (
Some ( actorRef ) ,
Left ( actorRef . uuid ) ,
actorRef . id ,
actorRef . actorClassName ,
actorRef . timeout ,
Left ( message ) ,
isOneWay ,
senderOption ,
typedActorInfo ,
actorType ,
2010-12-17 16:09:21 +01:00
if ( isAuthenticated . compareAndSet ( false , true ) ) RemoteClient . SECURE_COOKIE else None
2010-12-15 17:52:31 +01:00
) . build , senderFuture )
}
def send [ T ] (
request : RemoteMessageProtocol ,
senderFuture : Option [ CompletableFuture [ T ] ] ) : Option [ CompletableFuture [ T ] ] = {
2010-12-20 16:58:05 +01:00
log . slf4j . debug ( "sending message: {} has future {}" , request , senderFuture )
2010-12-15 17:52:31 +01:00
if ( isRunning ) {
if ( request . getOneWay ) {
connection . getChannel . write ( request )
None
} else {
val futureResult = if ( senderFuture . isDefined ) senderFuture . get
2010-12-17 16:09:21 +01:00
else new DefaultCompletableFuture [ T ] ( request . getActorInfo . getTimeout )
val futureUuid = uuidFrom ( request . getUuid . getHigh , request . getUuid . getLow )
futures . put ( futureUuid , futureResult )
log . slf4j . debug ( "Stashing away future for {}" , futureUuid )
2010-12-15 17:52:31 +01:00
connection . getChannel . write ( request )
Some ( futureResult )
}
} else {
2010-12-29 16:08:43 +01:00
val exception = new RemoteClientException ( "Remote client is not running, make sure you have invoked 'RemoteClient.connect' before using it." , module , remoteAddress )
notifyListeners ( RemoteClientError ( exception , module , remoteAddress ) )
2010-12-15 17:52:31 +01:00
throw exception
}
}
private [ akka ] def registerSupervisorForActor ( actorRef : ActorRef ) : ActorRef =
if ( ! actorRef . supervisor . isDefined ) throw new IllegalActorStateException (
"Can't register supervisor for " + actorRef + " since it is not under supervision" )
else supervisors . putIfAbsent ( actorRef . supervisor . get . uuid , actorRef )
private [ akka ] def deregisterSupervisorForActor ( actorRef : ActorRef ) : ActorRef =
if ( ! actorRef . supervisor . isDefined ) throw new IllegalActorStateException (
"Can't unregister supervisor for " + actorRef + " since it is not under supervision" )
else supervisors . remove ( actorRef . supervisor . get . uuid )
private [ akka ] def isWithinReconnectionTimeWindow : Boolean = {
if ( reconnectionTimeWindowStart == 0L ) {
reconnectionTimeWindowStart = System . currentTimeMillis
true
} else {
val timeLeft = reconnectionTimeWindow - ( System . currentTimeMillis - reconnectionTimeWindowStart )
if ( timeLeft > 0 ) {
log . slf4j . info ( "Will try to reconnect to remote server for another [{}] milliseconds" , timeLeft )
true
} else false
}
}
private [ akka ] def resetReconnectionTimeWindow = reconnectionTimeWindowStart = 0L
}
/* *
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
class RemoteClientPipelineFactory (
name : String ,
futures : ConcurrentMap [ Uuid , CompletableFuture [ _ ] ] ,
supervisors : ConcurrentMap [ Uuid , ActorRef ] ,
bootstrap : ClientBootstrap ,
remoteAddress : SocketAddress ,
timer : HashedWheelTimer ,
client : RemoteClient ) extends ChannelPipelineFactory {
def getPipeline : ChannelPipeline = {
def join ( ch : ChannelHandler * ) = Array [ ChannelHandler ] ( ch : _ * )
lazy val engine = {
val e = RemoteServerSslContext . client . createSSLEngine ( )
e . setEnabledCipherSuites ( e . getSupportedCipherSuites ) //TODO is this sensible?
e . setUseClientMode ( true )
e
}
val ssl = if ( RemoteServer . SECURE ) join ( new SslHandler ( engine ) ) else join ( )
val timeout = new ReadTimeoutHandler ( timer , RemoteClient . READ_TIMEOUT . toMillis . toInt )
val lenDec = new LengthFieldBasedFrameDecoder ( RemoteClient . MESSAGE_FRAME_SIZE , 0 , 4 , 0 , 4 )
val lenPrep = new LengthFieldPrepender ( 4 )
val protobufDec = new ProtobufDecoder ( RemoteMessageProtocol . getDefaultInstance )
val protobufEnc = new ProtobufEncoder
val ( enc , dec ) = RemoteServer . COMPRESSION_SCHEME match {
case "zlib" => ( join ( new ZlibEncoder ( RemoteServer . ZLIB_COMPRESSION_LEVEL ) ) , join ( new ZlibDecoder ) )
case _ => ( join ( ) , join ( ) )
}
val remoteClient = new RemoteClientHandler ( name , futures , supervisors , bootstrap , remoteAddress , timer , client )
val stages = ssl ++ join ( timeout ) ++ dec ++ join ( lenDec , protobufDec ) ++ enc ++ join ( lenPrep , protobufEnc , remoteClient )
new StaticChannelPipeline ( stages : _ * )
}
}
/* *
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
@ChannelHandler . Sharable
class RemoteClientHandler (
val name : String ,
val futures : ConcurrentMap [ Uuid , CompletableFuture [ _ ] ] ,
val supervisors : ConcurrentMap [ Uuid , ActorRef ] ,
val bootstrap : ClientBootstrap ,
val remoteAddress : SocketAddress ,
val timer : HashedWheelTimer ,
val client : RemoteClient )
extends SimpleChannelUpstreamHandler with Logging {
override def handleUpstream ( ctx : ChannelHandlerContext , event : ChannelEvent ) = {
if ( event . isInstanceOf [ ChannelStateEvent ] &&
event . asInstanceOf [ ChannelStateEvent ] . getState != ChannelState . INTEREST_OPS ) {
log . slf4j . debug ( event . toString )
}
super . handleUpstream ( ctx , event )
}
override def messageReceived ( ctx : ChannelHandlerContext , event : MessageEvent ) {
try {
2010-12-17 16:09:21 +01:00
event . getMessage match {
case reply : RemoteMessageProtocol =>
val replyUuid = uuidFrom ( reply . getActorInfo . getUuid . getHigh , reply . getActorInfo . getUuid . getLow )
log . slf4j . debug ( "Remote client received RemoteMessageProtocol[\n{}]" , reply )
log . slf4j . debug ( "Trying to map back to future: {}" , replyUuid )
val future = futures . remove ( replyUuid ) . asInstanceOf [ CompletableFuture [ Any ] ]
2010-12-20 12:10:46 +01:00
2010-12-17 16:09:21 +01:00
if ( reply . hasMessage ) {
if ( future eq null ) throw new IllegalActorStateException ( "Future mapped to UUID " + replyUuid + " does not exist" )
val message = MessageSerializer . deserialize ( reply . getMessage )
future . completeWithResult ( message )
} else {
2010-12-20 12:10:46 +01:00
val exception = parseException ( reply , client . loader )
2010-12-17 16:09:21 +01:00
if ( reply . hasSupervisorUuid ( ) ) {
val supervisorUuid = uuidFrom ( reply . getSupervisorUuid . getHigh , reply . getSupervisorUuid . getLow )
if ( ! supervisors . containsKey ( supervisorUuid ) ) throw new IllegalActorStateException (
"Expected a registered supervisor for UUID [" + supervisorUuid + "] but none was found" )
val supervisedActor = supervisors . get ( supervisorUuid )
if ( ! supervisedActor . supervisor . isDefined ) throw new IllegalActorStateException (
"Can't handle restart for remote actor " + supervisedActor + " since its supervisor has been removed" )
2010-12-20 12:10:46 +01:00
else supervisedActor . supervisor . get ! Exit ( supervisedActor , exception )
2010-12-17 16:09:21 +01:00
}
2010-12-20 12:10:46 +01:00
future . completeWithException ( exception )
2010-12-15 17:52:31 +01:00
}
2010-12-17 16:09:21 +01:00
case other =>
2010-12-29 16:08:43 +01:00
throw new RemoteClientException ( "Unknown message received in remote client handler: " + other , client . module , client . remoteAddress )
2010-12-15 17:52:31 +01:00
}
} catch {
case e : Exception =>
2010-12-29 16:08:43 +01:00
client . notifyListeners ( RemoteClientError ( e , client . module , client . remoteAddress ) )
2010-12-27 12:12:49 +01:00
log . slf4j . error ( "Unexpected exception in remote client handler" , e )
2010-12-15 17:52:31 +01:00
throw e
}
}
override def channelClosed ( ctx : ChannelHandlerContext , event : ChannelStateEvent ) = client . runSwitch ifOn {
if ( client . isWithinReconnectionTimeWindow ) {
timer . newTimeout ( new TimerTask ( ) {
def run ( timeout : Timeout ) = {
client . openChannels . remove ( event . getChannel )
client . isAuthenticated . set ( false )
log . slf4j . debug ( "Remote client reconnecting to [{}]" , remoteAddress )
client . connection = bootstrap . connect ( remoteAddress )
client . connection . awaitUninterruptibly // Wait until the connection attempt succeeds or fails.
if ( ! client . connection . isSuccess ) {
2010-12-29 16:08:43 +01:00
client . notifyListeners ( RemoteClientError ( client . connection . getCause , client . module , client . remoteAddress ) )
2010-12-15 17:52:31 +01:00
log . slf4j . error ( "Reconnection to [{}] has failed" , remoteAddress )
log . slf4j . debug ( "Reconnection failed" , client . connection . getCause )
}
}
} , RemoteClient . RECONNECT_DELAY . toMillis , TimeUnit . MILLISECONDS )
} else spawn { client . shutdown }
}
override def channelConnected ( ctx : ChannelHandlerContext , event : ChannelStateEvent ) = {
def connect = {
2010-12-29 16:08:43 +01:00
client . notifyListeners ( RemoteClientConnected ( client . module , client . remoteAddress ) )
2010-12-15 17:52:31 +01:00
log . slf4j . debug ( "Remote client connected to [{}]" , ctx . getChannel . getRemoteAddress )
client . resetReconnectionTimeWindow
}
if ( RemoteServer . SECURE ) {
val sslHandler : SslHandler = ctx . getPipeline . get ( classOf [ SslHandler ] )
sslHandler . handshake . addListener ( new ChannelFutureListener {
def operationComplete ( future : ChannelFuture ) : Unit = {
if ( future . isSuccess ) connect
2010-12-29 16:08:43 +01:00
else throw new RemoteClientException ( "Could not establish SSL handshake" , client . module , client . remoteAddress )
2010-12-15 17:52:31 +01:00
}
} )
} else connect
}
override def channelDisconnected ( ctx : ChannelHandlerContext , event : ChannelStateEvent ) = {
2010-12-29 16:08:43 +01:00
client . notifyListeners ( RemoteClientDisconnected ( client . module , client . remoteAddress ) )
2010-12-15 17:52:31 +01:00
log . slf4j . debug ( "Remote client disconnected from [{}]" , ctx . getChannel . getRemoteAddress )
}
override def exceptionCaught ( ctx : ChannelHandlerContext , event : ExceptionEvent ) = {
2010-12-29 16:08:43 +01:00
client . notifyListeners ( RemoteClientError ( event . getCause , client . module , client . remoteAddress ) )
2010-12-15 17:52:31 +01:00
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
}
private def parseException ( reply : RemoteMessageProtocol , loader : Option [ ClassLoader ] ) : Throwable = {
val exception = reply . getException
val classname = exception . getClassname
2010-12-20 12:10:46 +01:00
try {
val exceptionClass = if ( loader . isDefined ) loader . get . loadClass ( classname )
else Class . forName ( classname )
exceptionClass
. getConstructor ( Array [ Class [ _ ] ] ( classOf [ String ] ) : _ * )
. newInstance ( exception . getMessage ) . asInstanceOf [ Throwable ]
} catch {
case problem =>
log . debug ( "Couldn't parse exception returned from RemoteServer" , problem )
log . warn ( "Couldn't create instance of {} with message {}, returning UnparsableException" , classname , exception . getMessage )
UnparsableException ( classname , exception . getMessage )
}
2010-12-15 17:52:31 +01:00
}
}
2009-11-24 17:41:08 +01:00
/* *
2010-10-26 12:04:32 +02:00
* For internal use only . Holds configuration variables , remote actors , remote typed actors and remote servers .
2009-12-18 21:26:03 +01:00
*
2009-11-24 17:41:08 +01:00
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
2010-10-26 12:04:32 +02:00
object RemoteServer {
2010-11-22 15:32:54 +01:00
val isRemotingEnabled = config . getList ( "akka.enabled-modules" ) . exists ( _ == "remote" )
2010-11-11 17:23:37 +01:00
val MESSAGE_FRAME_SIZE = config . getInt ( "akka.remote.server.message-frame-size" , 1048576 )
val SECURE_COOKIE = config . getString ( "akka.remote.secure-cookie" )
val REQUIRE_COOKIE = {
2010-10-26 12:04:32 +02:00
val requireCookie = config . getBool ( "akka.remote.server.require-cookie" , true )
2010-11-22 15:32:54 +01:00
if ( isRemotingEnabled && requireCookie && RemoteServer . SECURE_COOKIE . isEmpty ) throw new ConfigurationException (
2010-10-26 12:04:32 +02:00
"Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'." )
requireCookie
}
2010-10-29 16:33:31 +02:00
2010-10-28 21:18:25 +02:00
val UNTRUSTED_MODE = config . getBool ( "akka.remote.server.untrusted-mode" , false )
2010-10-26 12:04:32 +02:00
val HOSTNAME = config . getString ( "akka.remote.server.hostname" , "localhost" )
2010-10-29 16:07:33 +02:00
val PORT = config . getInt ( "akka.remote.server.port" , 2552 )
2010-07-23 04:54:21 +02:00
val CONNECTION_TIMEOUT_MILLIS = Duration ( config . getInt ( "akka.remote.server.connection-timeout" , 1 ) , TIME_UNIT )
2010-10-26 12:04:32 +02:00
val COMPRESSION_SCHEME = config . getString ( "akka.remote.compression-scheme" , "zlib" )
val ZLIB_COMPRESSION_LEVEL = {
2009-11-22 14:32:27 +01:00
val level = config . getInt ( "akka.remote.zlib-compression-level" , 6 )
if ( level < 1 && level > 9 ) throw new IllegalArgumentException (
"zlib compression level has to be within 1-9, with 1 being fastest and 9 being the most compressed" )
level
}
2009-12-30 08:36:24 +01:00
2010-07-15 21:33:44 +02:00
val SECURE = {
2010-08-18 19:02:48 +02:00
/* if (config.getBool("akka.remote.ssl.service",false)) {
2010-07-15 21:33:44 +02:00
val properties = List (
2010-08-11 01:15:01 +02:00
( "key-store-type" , "keyStoreType" ) ,
( "key-store" , "keyStore" ) ,
( "key-store-pass" , "keyStorePassword" ) ,
( "trust-store-type" , "trustStoreType" ) ,
( "trust-store" , "trustStore" ) ,
( "trust-store-pass" , "trustStorePassword" )
) . map ( x => ( "akka.remote.ssl." + x . _1 , "javax.net.ssl." + x . _2 ) )
// If property is not set, and we have a value from our akka.conf, use that value
2010-08-21 16:13:16 +02:00
for {
2010-08-11 01:15:01 +02:00
p <- properties if System . getProperty ( p . _2 ) eq null
c <- config . getString ( p . _1 )
} System . setProperty ( p . _2 , c )
2010-08-21 16:13:16 +02:00
2010-08-11 01:15:01 +02:00
if ( config . getBool ( "akka.remote.ssl.debug" , false ) ) System . setProperty ( "javax.net.debug" , "ssl" )
2010-07-15 21:33:44 +02:00
true
2010-08-18 19:02:48 +02:00
} else */ false
2010-07-15 21:33:44 +02:00
}
2009-11-24 17:41:08 +01:00
}
2009-11-22 14:32:27 +01:00
2010-12-15 17:52:31 +01:00
2009-11-24 17:41:08 +01:00
/* *
2010-12-15 17:52:31 +01:00
* Provides the implementation of the Netty remote support
2009-11-24 17:41:08 +01:00
*/
2010-12-17 16:09:21 +01:00
class NettyRemoteSupport extends RemoteSupport with NettyRemoteServerModule with NettyRemoteClientModule {
//Needed for remote testing and switching on/off under run
private [ akka ] val optimizeLocal = new AtomicBoolean ( true )
def optimizeLocalScoped_? ( ) = optimizeLocal . get
2010-12-15 17:52:31 +01:00
2010-12-21 14:36:47 +01:00
protected [ akka ] def actorFor ( serviceId : String , className : String , timeout : Long , host : String , port : Int , loader : Option [ ClassLoader ] ) : ActorRef = {
if ( optimizeLocalScoped_? ) {
val home = this . address
2010-12-22 10:10:04 +01:00
if ( host == home . getHostName && port == home . getPort ) { //TODO: switch to InetSocketAddress.equals?
2010-12-21 14:36:47 +01:00
val localRef = findActorByIdOrUuid ( serviceId , serviceId )
if ( localRef ne null ) return localRef //Code significantly simpler with the return statement
}
}
RemoteActorRef ( serviceId , className , host , port , timeout , loader )
2010-12-15 17:52:31 +01:00
}
2010-12-21 14:36:47 +01:00
def clientManagedActorOf ( factory : ( ) => Actor , host : String , port : Int ) : ActorRef = {
2010-12-22 10:10:04 +01:00
if ( optimizeLocalScoped_? ) {
val home = this . address
if ( host == home . getHostName && port == home . getPort ) //TODO: switch to InetSocketAddress.equals?
return new LocalActorRef ( factory , None ) // Code is much simpler with return
}
2010-12-21 14:36:47 +01:00
val ref = new LocalActorRef ( factory , Some ( new InetSocketAddress ( host , port ) ) )
//ref.timeout = timeout //removed because setting default timeout should be done after construction
2010-12-20 16:58:05 +01:00
ref
2010-12-15 17:52:31 +01:00
}
}
2010-12-20 16:58:05 +01:00
class NettyRemoteServer ( serverModule : NettyRemoteServerModule , val host : String , val port : Int , val loader : Option [ ClassLoader ] ) {
2009-12-18 21:26:03 +01:00
2010-12-20 16:58:05 +01:00
val name = "NettyRemoteServer@" + host + ":" + port
2010-12-21 14:36:47 +01:00
val address = new InetSocketAddress ( host , port )
2009-06-25 23:47:30 +02:00
2010-12-14 18:22:46 +01:00
private val factory = new NioServerSocketChannelFactory ( Executors . newCachedThreadPool , Executors . newCachedThreadPool )
2009-06-24 15:12:47 +02:00
2009-06-25 13:07:58 +02:00
private val bootstrap = new ServerBootstrap ( factory )
2009-06-24 15:12:47 +02:00
2009-12-18 21:26:03 +01:00
// group of open channels, used for clean-up
2009-12-30 08:36:24 +01:00
private val openChannels : ChannelGroup = new DefaultChannelGroup ( "akka-remote-server" )
2009-12-18 21:26:03 +01:00
2010-12-20 16:58:05 +01:00
val pipelineFactory = new RemoteServerPipelineFactory ( name , openChannels , loader , serverModule )
bootstrap . setPipelineFactory ( pipelineFactory )
bootstrap . setOption ( "child.tcpNoDelay" , true )
bootstrap . setOption ( "child.keepAlive" , true )
bootstrap . setOption ( "child.reuseAddress" , true )
bootstrap . setOption ( "child.connectTimeoutMillis" , RemoteServer . CONNECTION_TIMEOUT_MILLIS . toMillis )
2010-12-21 14:36:47 +01:00
openChannels . add ( bootstrap . bind ( address ) )
2010-12-20 16:58:05 +01:00
serverModule . notifyListeners ( RemoteServerStarted ( serverModule ) )
def shutdown {
try {
openChannels . disconnect
openChannels . close . awaitUninterruptibly
bootstrap . releaseExternalResources
serverModule . notifyListeners ( RemoteServerShutdown ( serverModule ) )
} catch {
case e : java . nio . channels . ClosedChannelException => { }
case e => serverModule . log . slf4j . warn ( "Could not close remote server channel in a graceful way" )
}
}
}
trait NettyRemoteServerModule extends RemoteServerModule { self : RemoteModule =>
import RemoteServer._
private [ akka ] val currentServer = new AtomicReference [ Option [ NettyRemoteServer ] ] ( None )
2010-12-21 14:36:47 +01:00
def address = currentServer . get match {
case Some ( s ) => s . address
case None => ReflectiveAccess . Remote . configDefaultAddress
2010-12-20 16:58:05 +01:00
}
def name = currentServer . get match {
case Some ( s ) => s . name
2010-12-21 14:36:47 +01:00
case None =>
val a = ReflectiveAccess . Remote . configDefaultAddress
"NettyRemoteServer@" + a . getHostName + ":" + a . getPort
2010-12-20 16:58:05 +01:00
}
private val _isRunning = new Switch ( false )
2010-12-14 18:22:46 +01:00
def isRunning = _isRunning . isOn
2009-10-13 11:18:21 +02:00
2010-12-15 17:52:31 +01:00
def start ( _hostname : String , _port : Int , loader : Option [ ClassLoader ] = None ) : RemoteServerModule = guard withGuard {
2009-12-27 06:35:25 +01:00
try {
2010-12-14 18:22:46 +01:00
_isRunning switchOn {
2010-12-20 16:58:05 +01:00
log . slf4j . debug ( "Starting up remote server on {}:{}" , _hostname , _port )
currentServer . set ( Some ( new NettyRemoteServer ( this , _hostname , _port , loader ) ) )
2010-04-06 12:45:09 +02:00
}
2009-12-27 06:35:25 +01:00
} catch {
2010-08-18 09:52:11 +02:00
case e =>
2010-11-24 13:42:41 +01:00
log . slf4j . error ( "Could not start up remote server" , e )
2010-09-12 11:24:27 +02:00
notifyListeners ( RemoteServerError ( e , this ) )
2009-06-25 23:47:30 +02:00
}
2010-05-16 10:59:06 +02:00
this
2009-06-25 23:47:30 +02:00
}
2009-11-24 17:41:08 +01:00
2010-12-15 17:52:31 +01:00
def shutdownServerModule = guard withGuard {
2010-12-14 18:22:46 +01:00
_isRunning switchOff {
2010-12-20 16:58:05 +01:00
currentServer . getAndSet ( None ) foreach {
instance =>
log . slf4j . debug ( "Shutting down remote server on {}:{}" , instance . host , instance . port )
instance . shutdown
2010-07-14 14:38:56 +02:00
}
2010-04-06 12:45:09 +02:00
}
2009-11-24 17:41:08 +01:00
}
2010-02-16 15:39:09 +01:00
2010-02-17 15:32:17 +01:00
/* *
2010-09-06 16:33:55 +02:00
* Register remote typed actor by a specific id .
* @param id custom actor id
* @param typedActor typed actor to register
2010-02-17 15:32:17 +01:00
*/
2010-12-14 18:22:46 +01:00
def registerTypedActor ( id : String , typedActor : AnyRef ) : Unit = guard withGuard {
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Registering server side remote typed actor [{}] with id [{}]" , typedActor . getClass . getName , id )
2010-10-26 15:23:50 +02:00
if ( id . startsWith ( UUID_PREFIX ) ) registerTypedActor ( id . substring ( UUID_PREFIX . length ) , typedActor , typedActorsByUuid )
else registerTypedActor ( id , typedActor , typedActors )
2010-09-06 10:15:44 +02:00
}
2010-02-17 15:32:17 +01:00
2010-11-14 18:03:34 -06:00
/* *
* Register remote typed actor by a specific id .
* @param id custom actor id
* @param typedActor typed actor to register
*/
2010-12-14 18:22:46 +01:00
def registerTypedPerSessionActor ( id : String , factory : => AnyRef ) : Unit = guard withGuard {
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Registering server side typed remote session actor with id [{}]" , id )
2010-11-14 18:03:34 -06:00
registerTypedPerSessionActor ( id , ( ) => factory , typedActorsFactories )
}
2010-02-17 15:32:17 +01:00
/* *
2010-04-06 12:45:09 +02:00
* Register Remote Actor by a specific 'id' passed as argument .
2010-05-07 11:19:19 +02:00
* < p />
2010-05-21 20:08:49 +02:00
* NOTE : If you use this method to register your remote actor then you must unregister the actor by this ID yourself .
2010-02-17 15:32:17 +01:00
*/
2010-12-14 18:22:46 +01:00
def register ( id : String , actorRef : ActorRef ) : Unit = guard withGuard {
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Registering server side remote actor [{}] with id [{}]" , actorRef . actorClass . getName , id )
2010-10-26 15:23:50 +02:00
if ( id . startsWith ( UUID_PREFIX ) ) register ( id . substring ( UUID_PREFIX . length ) , actorRef , actorsByUuid )
else register ( id , actorRef , actors )
2010-09-20 12:33:30 +02:00
}
2010-12-14 18:22:46 +01:00
def registerByUuid ( actorRef : ActorRef ) : Unit = guard withGuard {
2010-12-17 16:09:21 +01:00
log . slf4j . debug ( "Registering remote actor {} to it's uuid {}" , actorRef , actorRef . uuid )
2010-12-14 18:22:46 +01:00
register ( actorRef . uuid . toString , actorRef , actorsByUuid )
}
private def register [ Key ] ( id : Key , actorRef : ActorRef , registry : ConcurrentHashMap [ Key , ActorRef ] ) {
if ( _isRunning . isOn ) {
registry . put ( id , actorRef ) //TODO change to putIfAbsent
if ( ! actorRef . isRunning ) actorRef . start
}
2010-11-14 13:10:13 -06:00
}
2010-11-29 20:49:15 +01:00
/* *
* Register Remote Session Actor by a specific 'id' passed as argument .
* < p />
* NOTE : If you use this method to register your remote actor then you must unregister the actor by this ID yourself .
*/
2010-12-14 18:22:46 +01:00
def registerPerSession ( id : String , factory : => ActorRef ) : Unit = synchronized {
2010-11-29 20:49:15 +01:00
log . slf4j . debug ( "Registering server side remote session actor with id [{}]" , id )
2010-12-14 18:22:46 +01:00
registerPerSession ( id , ( ) => factory , actorsFactories )
2010-09-20 12:33:30 +02:00
}
2010-11-14 13:10:13 -06:00
private def registerPerSession [ Key ] ( id : Key , factory : ( ) => ActorRef , registry : ConcurrentHashMap [ Key ,( ) => ActorRef ] ) {
2010-12-14 18:22:46 +01:00
if ( _isRunning . isOn )
2010-11-22 11:57:17 +01:00
registry . put ( id , factory ) //TODO change to putIfAbsent
2010-11-14 13:10:13 -06:00
}
2010-09-22 11:37:23 +02:00
private def registerTypedActor [ Key ] ( id : Key , typedActor : AnyRef , registry : ConcurrentHashMap [ Key , AnyRef ] ) {
2010-12-14 18:22:46 +01:00
if ( _isRunning . isOn )
2010-11-22 11:57:17 +01:00
registry . put ( id , typedActor ) //TODO change to putIfAbsent
2010-02-17 15:32:17 +01:00
}
2010-05-05 22:45:19 +02:00
2010-11-14 18:03:34 -06:00
private def registerTypedPerSessionActor [ Key ] ( id : Key , factory : ( ) => AnyRef , registry : ConcurrentHashMap [ Key ,( ) => AnyRef ] ) {
2010-12-14 18:22:46 +01:00
if ( _isRunning . isOn )
2010-11-22 11:57:17 +01:00
registry . put ( id , factory ) //TODO change to putIfAbsent
2010-02-17 15:32:17 +01:00
}
2010-05-05 22:45:19 +02:00
/* *
2010-09-07 12:54:10 +02:00
* Unregister Remote Actor that is registered using its 'id' field ( not custom ID ) .
2010-05-05 22:45:19 +02:00
*/
2010-12-14 18:22:46 +01:00
def unregister ( actorRef : ActorRef ) : Unit = guard withGuard {
if ( _isRunning . isOn ) {
2010-11-30 12:00:58 +01:00
log . slf4j . debug ( "Unregistering server side remote actor [{}] with id [{}:{}]" , Array [ AnyRef ] ( actorRef . actorClass . getName , actorRef . id , actorRef . uuid ) )
2010-10-26 15:23:50 +02:00
actors . remove ( actorRef . id , actorRef )
actorsByUuid . remove ( actorRef . uuid , actorRef )
2010-05-07 11:19:19 +02:00
}
}
/* *
2010-05-21 20:08:49 +02:00
* Unregister Remote Actor by specific 'id' .
2010-05-07 11:19:19 +02:00
* < p />
2010-05-21 20:08:49 +02:00
* NOTE : You need to call this method if you have registered an actor by a custom ID .
2010-05-07 11:19:19 +02:00
*/
2010-12-14 18:22:46 +01:00
def unregister ( id : String ) : Unit = guard withGuard {
if ( _isRunning . isOn ) {
2010-11-24 13:53:53 +01:00
log . slf4j . info ( "Unregistering server side remote actor with id [{}]" , id )
2010-10-29 16:33:31 +02:00
if ( id . startsWith ( UUID_PREFIX ) ) actorsByUuid . remove ( id . substring ( UUID_PREFIX . length ) )
2010-10-26 15:23:50 +02:00
else {
val actorRef = actors get id
actorsByUuid . remove ( actorRef . uuid , actorRef )
actors . remove ( id , actorRef )
2010-09-20 12:33:30 +02:00
}
2010-05-05 22:45:19 +02:00
}
}
2010-08-18 09:52:11 +02:00
2010-11-14 13:10:13 -06:00
/* *
* Unregister Remote Actor by specific 'id' .
* < p />
* NOTE : You need to call this method if you have registered an actor by a custom ID .
*/
2010-12-14 18:22:46 +01:00
def unregisterPerSession ( id : String ) : Unit = {
if ( _isRunning . isOn ) {
2010-11-24 13:53:53 +01:00
log . slf4j . info ( "Unregistering server side remote session actor with id [{}]" , id )
2010-11-14 13:10:13 -06:00
actorsFactories . remove ( id )
}
}
2010-09-06 16:33:55 +02:00
/* *
* Unregister Remote Typed Actor by specific 'id' .
* < p />
* NOTE : You need to call this method if you have registered an actor by a custom ID .
*/
2010-12-14 18:22:46 +01:00
def unregisterTypedActor ( id : String ) : Unit = guard withGuard {
if ( _isRunning . isOn ) {
2010-11-24 13:53:53 +01:00
log . slf4j . info ( "Unregistering server side remote typed actor with id [{}]" , id )
2010-10-26 15:23:50 +02:00
if ( id . startsWith ( UUID_PREFIX ) ) typedActorsByUuid . remove ( id . substring ( UUID_PREFIX . length ) )
else typedActors . remove ( id )
2010-09-06 16:33:55 +02:00
}
}
2010-09-06 10:15:44 +02:00
2010-11-14 18:03:34 -06:00
/* *
* Unregister Remote Typed Actor by specific 'id' .
* < p />
* NOTE : You need to call this method if you have registered an actor by a custom ID .
*/
2010-12-15 17:52:31 +01:00
def unregisterTypedPerSessionActor ( id : String ) : Unit =
if ( _isRunning . isOn ) typedActorsFactories . remove ( id )
2009-06-24 15:12:47 +02:00
}
2010-04-25 20:32:52 +02:00
object RemoteServerSslContext {
2010-07-15 21:33:44 +02:00
import javax.net.ssl.SSLContext
2010-04-25 20:32:52 +02:00
2010-08-11 01:15:01 +02:00
val ( client , server ) = {
2010-04-25 20:32:52 +02:00
val protocol = "TLS"
2010-07-15 21:33:44 +02:00
//val algorithm = Option(Security.getProperty("ssl.KeyManagerFactory.algorithm")).getOrElse("SunX509")
//val store = KeyStore.getInstance("JKS")
2010-04-25 20:32:52 +02:00
val s = SSLContext . getInstance ( protocol )
2010-08-12 15:36:05 +02:00
s . init ( null , null , null )
2010-04-25 20:32:52 +02:00
val c = SSLContext . getInstance ( protocol )
2010-08-12 15:36:05 +02:00
c . init ( null , null , null )
( c , s )
2010-04-25 20:32:52 +02:00
}
}
2009-07-27 21:21:28 +02:00
/* *
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
2009-12-27 22:56:55 +01:00
class RemoteServerPipelineFactory (
2009-12-30 08:36:24 +01:00
val name : String ,
val openChannels : ChannelGroup ,
val loader : Option [ ClassLoader ] ,
2010-12-15 17:52:31 +01:00
val server : NettyRemoteServerModule ) extends ChannelPipelineFactory {
2009-11-24 17:41:08 +01:00
import RemoteServer._
2009-12-18 21:26:03 +01:00
def getPipeline : ChannelPipeline = {
2010-07-15 21:33:44 +02:00
def join ( ch : ChannelHandler * ) = Array [ ChannelHandler ] ( ch : _ * )
2010-08-20 11:54:57 +02:00
lazy val engine = {
val e = RemoteServerSslContext . server . createSSLEngine ( )
e . setEnabledCipherSuites ( e . getSupportedCipherSuites ) //TODO is this sensible?
e . setUseClientMode ( false )
e
}
2010-04-25 20:32:52 +02:00
2010-08-11 01:15:01 +02:00
val ssl = if ( RemoteServer . SECURE ) join ( new SslHandler ( engine ) ) else join ( )
2010-11-11 17:23:37 +01:00
val lenDec = new LengthFieldBasedFrameDecoder ( RemoteServer . MESSAGE_FRAME_SIZE , 0 , 4 , 0 , 4 )
2010-07-15 21:33:44 +02:00
val lenPrep = new LengthFieldPrepender ( 4 )
2010-10-28 20:30:11 +02:00
val protobufDec = new ProtobufDecoder ( RemoteMessageProtocol . getDefaultInstance )
2010-07-15 21:33:44 +02:00
val protobufEnc = new ProtobufEncoder
2010-10-26 12:04:32 +02:00
val ( enc , dec ) = RemoteServer . COMPRESSION_SCHEME match {
2010-08-11 01:15:01 +02:00
case "zlib" => ( join ( new ZlibEncoder ( RemoteServer . ZLIB_COMPRESSION_LEVEL ) ) , join ( new ZlibDecoder ) )
case _ => ( join ( ) , join ( ) )
2009-11-22 14:32:27 +01:00
}
2010-07-15 21:33:44 +02:00
2010-09-07 10:12:26 +02:00
val remoteServer = new RemoteServerHandler ( name , openChannels , loader , server )
2010-07-15 21:33:44 +02:00
val stages = ssl ++ dec ++ join ( lenDec , protobufDec ) ++ enc ++ join ( lenPrep , protobufEnc , remoteServer )
2009-12-30 08:48:22 +01:00
new StaticChannelPipeline ( stages : _ * )
2009-07-18 00:16:32 +02:00
}
}
2009-07-27 21:21:28 +02:00
/* *
* @author < a href = "http://jonasboner.com" > Jonas Bon & # 233 ; r </ a >
*/
2010-03-29 09:33:32 +02:00
@ChannelHandler . Sharable
2009-12-27 22:56:55 +01:00
class RemoteServerHandler (
val name : String ,
2009-12-30 08:36:24 +01:00
val openChannels : ChannelGroup ,
val applicationLoader : Option [ ClassLoader ] ,
2010-12-15 17:52:31 +01:00
val server : NettyRemoteServerModule ) extends SimpleChannelUpstreamHandler with Logging {
2010-09-20 12:33:30 +02:00
import RemoteServer._
2010-10-26 15:23:50 +02:00
2009-11-22 15:25:16 +01:00
val AW_PROXY_PREFIX = "$$ProxiedByAW" . intern
2010-10-26 12:04:32 +02:00
val CHANNEL_INIT = "channel-init" . intern
2009-12-18 21:26:03 +01:00
2010-11-19 11:53:51 -06:00
val sessionActors = new ChannelLocal [ ConcurrentHashMap [ String , ActorRef ] ] ( )
val typedSessionActors = new ChannelLocal [ ConcurrentHashMap [ String , AnyRef ] ] ( )
2010-11-14 13:10:13 -06:00
2010-12-15 17:52:31 +01:00
applicationLoader . foreach ( MessageSerializer . setClassLoader ( _ ) ) //TODO: REVISIT: THIS FEELS A BIT DODGY
2009-12-27 22:56:55 +01:00
2009-12-18 21:26:03 +01:00
/* *
2010-09-13 11:08:43 +02:00
* ChannelOpen overridden to store open channels for a clean postStop of a RemoteServer .
2010-06-15 13:15:00 +02:00
* If a channel is closed before , it is automatically removed from the open channels group .
2009-12-18 21:26:03 +01:00
*/
2010-08-21 16:37:33 +02:00
override def channelOpen ( ctx : ChannelHandlerContext , event : ChannelStateEvent ) = openChannels . add ( ctx . getChannel )
2010-08-10 21:42:27 +02:00
2010-08-21 16:37:33 +02:00
override def channelConnected ( ctx : ChannelHandlerContext , event : ChannelStateEvent ) = {
2010-10-31 07:13:00 +01:00
val clientAddress = getClientAddress ( ctx )
2010-11-19 11:53:51 -06:00
sessionActors . set ( event . getChannel ( ) , new ConcurrentHashMap [ String , ActorRef ] ( ) )
typedSessionActors . set ( event . getChannel ( ) , new ConcurrentHashMap [ String , AnyRef ] ( ) )
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Remote client [{}] connected to [{}]" , clientAddress , server . name )
2010-08-11 01:15:01 +02:00
if ( RemoteServer . SECURE ) {
2010-08-21 16:37:33 +02:00
val sslHandler : SslHandler = ctx . getPipeline . get ( classOf [ SslHandler ] )
2010-07-15 21:33:44 +02:00
// Begin handshake.
2010-08-21 16:37:33 +02:00
sslHandler . handshake ( ) . addListener ( new ChannelFutureListener {
2010-08-11 01:15:01 +02:00
def operationComplete ( future : ChannelFuture ) : Unit = {
2010-08-21 16:37:33 +02:00
if ( future . isSuccess ) {
openChannels . add ( future . getChannel )
2010-10-31 07:13:00 +01:00
server . notifyListeners ( RemoteServerClientConnected ( server , clientAddress ) )
2010-08-21 16:37:33 +02:00
} else future . getChannel . close
2010-07-15 21:33:44 +02:00
}
} )
2010-10-31 07:13:00 +01:00
} else server . notifyListeners ( RemoteServerClientConnected ( server , clientAddress ) )
2010-10-26 15:23:50 +02:00
if ( RemoteServer . REQUIRE_COOKIE ) ctx . setAttachment ( CHANNEL_INIT ) // signal that this is channel initialization, which will need authentication
2010-04-25 20:32:52 +02:00
}
2010-10-31 07:13:00 +01:00
override def channelDisconnected ( ctx : ChannelHandlerContext , event : ChannelStateEvent ) = {
val clientAddress = getClientAddress ( ctx )
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Remote client [{}] disconnected from [{}]" , clientAddress , server . name )
2010-11-14 16:26:33 -06:00
// stop all session actors
2010-11-22 11:57:17 +01:00
val channelActors = sessionActors . remove ( event . getChannel )
if ( channelActors ne null ) {
val channelActorsIterator = channelActors . elements
while ( channelActorsIterator . hasMoreElements ) {
channelActorsIterator . nextElement . stop
}
2010-11-14 16:26:33 -06:00
}
2010-11-18 12:48:31 -06:00
2010-11-22 11:57:17 +01:00
val channelTypedActors = typedSessionActors . remove ( event . getChannel )
if ( channelTypedActors ne null ) {
val channelTypedActorsIterator = channelTypedActors . elements
while ( channelTypedActorsIterator . hasMoreElements ) {
TypedActor . stop ( channelTypedActorsIterator . nextElement )
}
2010-11-14 18:03:34 -06:00
}
2010-10-31 07:13:00 +01:00
server . notifyListeners ( RemoteServerClientDisconnected ( server , clientAddress ) )
}
2010-08-21 16:37:33 +02:00
override def channelClosed ( ctx : ChannelHandlerContext , event : ChannelStateEvent ) = {
2010-10-31 07:13:00 +01:00
val clientAddress = getClientAddress ( ctx )
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Remote client [{}] channel closed from [{}]" , clientAddress , server . name )
2010-10-31 07:13:00 +01:00
server . notifyListeners ( RemoteServerClientClosed ( server , clientAddress ) )
2010-08-21 16:37:33 +02:00
}
2009-06-24 15:12:47 +02:00
override def handleUpstream ( ctx : ChannelHandlerContext , event : ChannelEvent ) = {
2010-10-26 12:04:32 +02:00
if ( event . isInstanceOf [ ChannelStateEvent ] && event . asInstanceOf [ ChannelStateEvent ] . getState != ChannelState . INTEREST_OPS ) {
2010-11-24 13:42:41 +01:00
log . slf4j . debug ( event . toString )
2009-06-24 15:12:47 +02:00
}
super . handleUpstream ( ctx , event )
}
2010-12-14 18:57:45 +01:00
override def messageReceived ( ctx : ChannelHandlerContext , event : MessageEvent ) = event . getMessage match {
case null => throw new IllegalActorStateException ( "Message in remote MessageEvent is null: " + event )
case requestProtocol : RemoteMessageProtocol =>
if ( RemoteServer . REQUIRE_COOKIE ) authenticateRemoteClient ( requestProtocol , ctx )
2010-10-28 20:30:11 +02:00
handleRemoteMessageProtocol ( requestProtocol , event . getChannel )
2010-12-14 18:57:45 +01:00
case _ => //ignore
2009-06-24 15:12:47 +02:00
}
override def exceptionCaught ( ctx : ChannelHandlerContext , event : ExceptionEvent ) = {
2010-11-24 13:42:41 +01:00
log . slf4j . error ( "Unexpected exception from remote downstream" , event . getCause )
2009-06-24 15:12:47 +02:00
event . getChannel . close
2010-09-12 11:24:27 +02:00
server . notifyListeners ( RemoteServerError ( event . getCause , server ) )
2009-06-24 15:12:47 +02:00
}
2010-10-31 07:13:00 +01:00
private def getClientAddress ( ctx : ChannelHandlerContext ) : Option [ InetSocketAddress ] = {
val remoteAddress = ctx . getChannel . getRemoteAddress
if ( remoteAddress . isInstanceOf [ InetSocketAddress ] ) Some ( remoteAddress . asInstanceOf [ InetSocketAddress ] )
else None
}
2010-10-28 20:30:11 +02:00
private def handleRemoteMessageProtocol ( request : RemoteMessageProtocol , channel : Channel ) = {
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Received RemoteMessageProtocol[\n{}]" , request )
2010-09-20 17:15:54 +02:00
request . getActorInfo . getActorType match {
case SCALA_ACTOR => dispatchToActor ( request , channel )
case TYPED_ACTOR => dispatchToTypedActor ( request , channel )
case JAVA_ACTOR => throw new IllegalActorStateException ( "ActorType JAVA_ACTOR is currently not supported" )
case other => throw new IllegalActorStateException ( "Unknown ActorType [" + other + "]" )
}
2009-06-24 15:12:47 +02:00
}
2010-11-02 21:18:31 +01:00
private def dispatchToActor ( request : RemoteMessageProtocol , channel : Channel ) {
2010-07-26 18:47:25 +02:00
val actorInfo = request . getActorInfo
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Dispatching to remote actor [{}:{}]" , actorInfo . getTarget , actorInfo . getUuid )
2010-07-26 18:47:25 +02:00
2010-10-29 16:33:31 +02:00
val actorRef =
2010-10-28 21:18:25 +02:00
try {
2010-11-14 16:26:33 -06:00
createActor ( actorInfo , channel ) . start
2010-10-28 21:18:25 +02:00
} catch {
2010-10-29 16:33:31 +02:00
case e : SecurityException =>
2010-11-02 21:18:31 +01:00
channel . write ( createErrorReplyMessage ( e , request , AkkaActorType . ScalaActor ) )
2010-10-28 21:18:25 +02:00
server . notifyListeners ( RemoteServerError ( e , server ) )
return
}
2010-07-26 18:47:25 +02:00
2010-06-24 08:48:48 +02:00
val message = MessageSerializer . deserialize ( request . getMessage )
2010-06-30 16:26:15 +02:00
val sender =
2010-07-02 00:16:11 +05:30
if ( request . hasSender ) Some ( RemoteActorSerialization . fromProtobufToRemoteActorRef ( request . getSender , applicationLoader ) )
2010-06-15 13:15:00 +02:00
else None
2010-07-26 18:47:25 +02:00
2010-08-12 15:36:05 +02:00
message match { // first match on system messages
2010-10-29 16:33:31 +02:00
case RemoteActorSystemMessage . Stop =>
2010-10-28 21:18:25 +02:00
if ( RemoteServer . UNTRUSTED_MODE ) throw new SecurityException ( "Remote server is operating is untrusted mode, can not stop the actor" )
else actorRef . stop
2010-10-29 16:33:31 +02:00
case _ : LifeCycleMessage if ( RemoteServer . UNTRUSTED_MODE ) =>
2010-10-28 21:18:25 +02:00
throw new SecurityException ( "Remote server is operating is untrusted mode, can not pass on a LifeCycleMessage to the remote actor" )
2010-08-12 15:36:05 +02:00
case _ => // then match on user defined messages
2010-10-28 20:30:11 +02:00
if ( request . getOneWay ) actorRef . ! ( message ) ( sender )
2010-10-26 12:04:32 +02:00
else actorRef . postMessageToMailboxAndCreateFutureResultWithTimeout (
message ,
request . getActorInfo . getTimeout ,
None ,
2010-11-12 12:11:53 +01:00
Some ( new DefaultCompletableFuture [ AnyRef ] ( request . getActorInfo . getTimeout ) .
onComplete ( f => {
2010-12-17 16:09:21 +01:00
log . slf4j . debug ( "Future was completed, now flushing to remote!" )
2010-11-12 12:11:53 +01:00
val result = f . result
val exception = f . exception
if ( exception . isDefined ) {
2010-12-17 16:09:21 +01:00
log . slf4j . debug ( "Returning exception from actor invocation [{}]" , exception . get . getClass )
2010-11-12 12:11:53 +01:00
try {
channel . write ( createErrorReplyMessage ( exception . get , request , AkkaActorType . ScalaActor ) )
} catch {
2010-12-17 16:09:21 +01:00
case e : Throwable =>
log . slf4j . debug ( "An error occurred in sending the reply" , e )
server . notifyListeners ( RemoteServerError ( e , server ) )
2010-11-12 12:11:53 +01:00
}
2010-09-16 15:58:46 +02:00
}
2010-11-12 12:11:53 +01:00
else if ( result . isDefined ) {
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Returning result from actor invocation [{}]" , result . get )
2010-11-12 12:11:53 +01:00
val messageBuilder = RemoteActorSerialization . createRemoteMessageProtocolBuilder (
Some ( actorRef ) ,
Right ( request . getUuid ) ,
actorInfo . getId ,
actorInfo . getTarget ,
actorInfo . getTimeout ,
Left ( result . get ) ,
true ,
Some ( actorRef ) ,
None ,
AkkaActorType . ScalaActor ,
None )
// FIXME lift in the supervisor uuid management into toh createRemoteMessageProtocolBuilder method
if ( request . hasSupervisorUuid ) messageBuilder . setSupervisorUuid ( request . getSupervisorUuid )
try {
channel . write ( messageBuilder . build )
} catch {
case e : Throwable => server . notifyListeners ( RemoteServerError ( e , server ) )
}
2010-09-16 15:58:46 +02:00
}
}
2010-11-12 12:11:53 +01:00
)
) )
2009-12-18 21:26:03 +01:00
}
2009-06-25 23:47:30 +02:00
}
2010-10-28 20:30:11 +02:00
private def dispatchToTypedActor ( request : RemoteMessageProtocol , channel : Channel ) = {
2010-07-26 18:47:25 +02:00
val actorInfo = request . getActorInfo
val typedActorInfo = actorInfo . getTypedActorInfo
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Dispatching to remote typed actor [{} :: {}]" , typedActorInfo . getMethod , typedActorInfo . getInterface )
2009-06-25 23:47:30 +02:00
2010-11-14 18:03:34 -06:00
val typedActor = createTypedActor ( actorInfo , channel )
2010-06-24 08:48:48 +02:00
val args = MessageSerializer . deserialize ( request . getMessage ) . asInstanceOf [ Array [ AnyRef ] ] . toList
2009-07-01 15:29:06 +02:00
val argClasses = args . map ( _ . getClass )
2009-06-25 23:47:30 +02:00
try {
2010-07-31 00:41:43 +02:00
val messageReceiver = typedActor . getClass . getDeclaredMethod ( typedActorInfo . getMethod , argClasses : _ * )
2010-10-28 20:30:11 +02:00
if ( request . getOneWay ) messageReceiver . invoke ( typedActor , args : _ * )
2009-06-25 23:47:30 +02:00
else {
2010-11-22 11:57:17 +01:00
//Sends the response
def sendResponse ( result : Either [ Any ,Throwable ] ) : Unit = try {
val messageBuilder = RemoteActorSerialization . createRemoteMessageProtocolBuilder (
None ,
Right ( request . getUuid ) ,
actorInfo . getId ,
actorInfo . getTarget ,
actorInfo . getTimeout ,
result ,
true ,
None ,
None ,
AkkaActorType . TypedActor ,
None )
if ( request . hasSupervisorUuid ) messageBuilder . setSupervisorUuid ( request . getSupervisorUuid )
channel . write ( messageBuilder . build )
2010-11-24 13:53:53 +01:00
log . slf4j . debug ( "Returning result from remote typed actor invocation [{}]" , result )
2010-11-22 11:57:17 +01:00
} catch {
case e : Throwable => server . notifyListeners ( RemoteServerError ( e , server ) )
2010-11-11 19:41:06 +01:00
}
2010-11-02 18:11:58 +01:00
2010-11-22 11:57:17 +01:00
messageReceiver . invoke ( typedActor , args : _ * ) match {
case f : Future [ _ ] => //If it's a future, we can lift on that to defer the send to when the future is completed
f . onComplete ( future => {
val result : Either [ Any ,Throwable ] = if ( future . exception . isDefined ) Right ( future . exception . get ) else Left ( future . result . get )
sendResponse ( result )
} )
case other => sendResponse ( Left ( other ) )
}
2009-06-25 23:47:30 +02:00
}
} catch {
2010-08-18 09:52:11 +02:00
case e : InvocationTargetException =>
2010-11-02 18:11:58 +01:00
channel . write ( createErrorReplyMessage ( e . getCause , request , AkkaActorType . TypedActor ) )
2010-09-12 11:24:27 +02:00
server . notifyListeners ( RemoteServerError ( e , server ) )
2010-10-26 15:23:50 +02:00
case e : Throwable =>
2010-11-02 18:11:58 +01:00
channel . write ( createErrorReplyMessage ( e , request , AkkaActorType . TypedActor ) )
2010-09-12 11:24:27 +02:00
server . notifyListeners ( RemoteServerError ( e , server ) )
2009-06-25 23:47:30 +02:00
}
}
2010-11-14 16:26:33 -06:00
private def findSessionActor ( id : String , channel : Channel ) : ActorRef = {
2010-11-24 15:02:36 +01:00
val map = sessionActors . get ( channel )
if ( map ne null ) map . get ( id )
else null
2010-11-14 16:26:33 -06:00
}
2010-11-14 18:03:34 -06:00
private def findTypedSessionActor ( id : String , channel : Channel ) : AnyRef = {
2010-11-24 15:02:36 +01:00
val map = typedSessionActors . get ( channel )
if ( map ne null ) map . get ( id )
else null
2010-11-14 18:03:34 -06:00
}
2010-04-06 12:45:09 +02:00
/* *
2010-11-18 12:48:31 -06:00
* gets the actor from the session , or creates one if there is a factory for it
2010-04-06 12:45:09 +02:00
*/
2010-11-18 12:48:31 -06:00
private def createSessionActor ( actorInfo : ActorInfoProtocol , channel : Channel ) : ActorRef = {
2010-09-16 13:50:57 +02:00
val uuid = actorInfo . getUuid
val id = actorInfo . getId
2010-11-19 11:53:51 -06:00
val sessionActorRefOrNull = findSessionActor ( id , channel )
2010-12-20 12:10:46 +01:00
if ( sessionActorRefOrNull ne null ) {
2010-11-18 12:48:31 -06:00
sessionActorRefOrNull
2010-12-20 12:10:46 +01:00
} else {
2010-11-18 12:48:31 -06:00
// we dont have it in the session either, see if we have a factory for it
2010-12-21 14:36:47 +01:00
val actorFactoryOrNull = server . findActorFactory ( id )
2010-11-18 12:48:31 -06:00
if ( actorFactoryOrNull ne null ) {
2010-11-19 11:53:51 -06:00
val actorRef = actorFactoryOrNull ( )
2010-11-18 12:48:31 -06:00
actorRef . uuid = uuidFrom ( uuid . getHigh , uuid . getLow )
2010-11-19 11:53:51 -06:00
sessionActors . get ( channel ) . put ( id , actorRef )
2010-11-18 12:48:31 -06:00
actorRef
}
else
null
}
}
2010-11-14 16:26:33 -06:00
2010-11-18 11:09:55 -06:00
2010-11-18 12:48:31 -06:00
private def createClientManagedActor ( actorInfo : ActorInfoProtocol ) : ActorRef = {
val uuid = actorInfo . getUuid
val id = actorInfo . getId
val timeout = actorInfo . getTimeout
val name = actorInfo . getTarget
2010-11-14 16:26:33 -06:00
try {
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" )
2010-12-17 16:09:21 +01:00
log . slf4j . info ( "Creating a new client-managed remote actor [{}:{}]" , name , uuid )
2010-11-14 16:26:33 -06:00
val clazz = if ( applicationLoader . isDefined ) applicationLoader . get . loadClass ( name )
else Class . forName ( name )
2010-12-21 14:36:47 +01:00
val actorRef = Actor . actorOf ( clazz . asInstanceOf [ Class [ _ <: Actor ] ] )
2010-11-14 16:26:33 -06:00
actorRef . uuid = uuidFrom ( uuid . getHigh , uuid . getLow )
actorRef . id = id
actorRef . timeout = timeout
server . actorsByUuid . put ( actorRef . uuid . toString , actorRef ) // register by uuid
actorRef
} catch {
case e =>
2010-11-24 13:42:41 +01:00
log . slf4j . error ( "Could not create remote actor instance" , e )
2010-11-14 16:26:33 -06:00
server . notifyListeners ( RemoteServerError ( e , server ) )
throw e
}
2010-11-18 12:48:31 -06:00
2009-06-24 15:12:47 +02:00
}
2010-07-26 18:47:25 +02:00
2010-04-06 12:45:09 +02:00
/* *
* Creates a new instance of the actor with name , uuid and timeout specified as arguments .
2010-07-29 17:29:51 +02:00
*
2010-04-06 12:45:09 +02:00
* If actor already created then just return it from the registry .
2010-07-29 17:29:51 +02:00
*
2010-04-06 12:45:09 +02:00
* Does not start the actor .
*/
2010-11-18 12:48:31 -06:00
private def createActor ( actorInfo : ActorInfoProtocol , channel : Channel ) : ActorRef = {
2010-09-16 13:50:57 +02:00
val uuid = actorInfo . getUuid
val id = actorInfo . getId
2010-09-13 13:31:42 +02:00
2010-12-21 14:36:47 +01:00
val actorRefOrNull = server . findActorByIdOrUuid ( id , uuidFrom ( uuid . getHigh , uuid . getLow ) . toString )
2010-10-29 16:33:31 +02:00
2010-11-20 15:48:37 -06:00
if ( actorRefOrNull ne null )
2010-11-18 12:48:31 -06:00
actorRefOrNull
2010-12-29 16:08:43 +01:00
else { // the actor has not been registered globally. See if we have it in the session
2010-11-19 11:53:51 -06:00
val sessionActorRefOrNull = createSessionActor ( actorInfo , channel )
2010-11-24 21:05:12 +01:00
if ( sessionActorRefOrNull ne null )
2010-11-18 12:48:31 -06:00
sessionActorRefOrNull
else // maybe it is a client managed actor
2010-11-20 15:48:37 -06:00
createClientManagedActor ( actorInfo )
2010-11-18 12:48:31 -06:00
}
}
2010-10-28 21:18:25 +02:00
2010-11-18 12:48:31 -06:00
/* *
* gets the actor from the session , or creates one if there is a factory for it
*/
private def createTypedSessionActor ( actorInfo : ActorInfoProtocol , channel : Channel ) : AnyRef = {
val id = actorInfo . getId
2010-11-19 11:53:51 -06:00
val sessionActorRefOrNull = findTypedSessionActor ( id , channel )
2010-11-14 18:03:34 -06:00
if ( sessionActorRefOrNull ne null )
2010-11-18 12:48:31 -06:00
sessionActorRefOrNull
else {
2010-12-21 14:36:47 +01:00
val actorFactoryOrNull = server . findTypedActorFactory ( id )
2010-11-18 12:48:31 -06:00
if ( actorFactoryOrNull ne null ) {
2010-11-19 11:53:51 -06:00
val newInstance = actorFactoryOrNull ( )
typedSessionActors . get ( channel ) . put ( id , newInstance )
2010-11-18 12:48:31 -06:00
newInstance
2009-06-25 13:07:58 +02:00
}
2010-11-18 12:48:31 -06:00
else
2010-11-24 21:05:12 +01:00
null
2010-11-14 18:03:34 -06:00
}
2010-07-26 18:47:25 +02:00
2009-06-24 15:12:47 +02:00
}
2010-07-26 18:47:25 +02:00
2010-11-18 12:48:31 -06:00
private def createClientManagedTypedActor ( actorInfo : ActorInfoProtocol ) = {
2010-11-14 18:03:34 -06:00
val typedActorInfo = actorInfo . getTypedActorInfo
val interfaceClassname = typedActorInfo . getInterface
val targetClassname = actorInfo . getTarget
2010-09-16 13:50:57 +02:00
val uuid = actorInfo . getUuid
2010-09-13 13:31:42 +02:00
2010-11-14 18:03:34 -06:00
try {
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" )
2010-07-26 18:47:25 +02:00
2010-11-24 13:53:53 +01:00
log . slf4j . info ( "Creating a new remote typed actor:\n\t[{} :: {}]" , interfaceClassname , targetClassname )
2010-07-26 18:47:25 +02:00
2010-11-14 18:03:34 -06:00
val ( interfaceClass , targetClass ) =
if ( applicationLoader . isDefined ) ( applicationLoader . get . loadClass ( interfaceClassname ) ,
applicationLoader . get . loadClass ( targetClassname ) )
else ( Class . forName ( interfaceClassname ) , Class . forName ( targetClassname ) )
2010-10-28 21:18:25 +02:00
2010-11-14 18:03:34 -06:00
val newInstance = TypedActor . newInstance (
interfaceClass , targetClass . asInstanceOf [ Class [ _ <: TypedActor ] ] , actorInfo . getTimeout ) . asInstanceOf [ AnyRef ]
server . typedActors . put ( uuidFrom ( uuid . getHigh , uuid . getLow ) . toString , newInstance ) // register by uuid
newInstance
} catch {
case e =>
2010-11-24 13:42:41 +01:00
log . slf4j . error ( "Could not create remote typed actor instance" , e )
2010-11-14 18:03:34 -06:00
server . notifyListeners ( RemoteServerError ( e , server ) )
throw e
}
2010-07-26 18:47:25 +02:00
}
2010-11-18 12:48:31 -06:00
private def createTypedActor ( actorInfo : ActorInfoProtocol , channel : Channel ) : AnyRef = {
val uuid = actorInfo . getUuid
val id = actorInfo . getId
2010-07-29 17:29:51 +02:00
2010-12-21 14:36:47 +01:00
val typedActorOrNull = server . findTypedActorByIdOrUuid ( id , uuidFrom ( uuid . getHigh , uuid . getLow ) . toString )
2010-11-18 12:48:31 -06:00
if ( typedActorOrNull ne null )
typedActorOrNull
else
{
// the actor has not been registered globally. See if we have it in the session
val sessionActorRefOrNull = createTypedSessionActor ( actorInfo , channel )
if ( sessionActorRefOrNull ne null )
sessionActorRefOrNull
else // maybe it is a client managed actor
createClientManagedTypedActor ( actorInfo )
}
2010-07-26 18:47:25 +02:00
}
2010-07-29 17:29:51 +02:00
2010-11-02 18:11:58 +01:00
private def createErrorReplyMessage ( exception : Throwable , request : RemoteMessageProtocol , actorType : AkkaActorType ) : RemoteMessageProtocol = {
2010-07-26 18:47:25 +02:00
val actorInfo = request . getActorInfo
2010-11-24 13:53:53 +01:00
log . slf4j . error ( "Could not invoke remote actor [{}]" , actorInfo . getTarget )
2010-11-24 13:42:41 +01:00
log . slf4j . debug ( "Could not invoke remote actor" , exception )
2010-11-02 18:11:58 +01:00
val messageBuilder = RemoteActorSerialization . createRemoteMessageProtocolBuilder (
None ,
2010-11-24 21:05:12 +01:00
Right ( request . getUuid ) ,
actorInfo . getId ,
actorInfo . getTarget ,
actorInfo . getTimeout ,
Right ( exception ) ,
true ,
None ,
None ,
actorType ,
2010-11-02 18:11:58 +01:00
None )
if ( request . hasSupervisorUuid ) messageBuilder . setSupervisorUuid ( request . getSupervisorUuid )
messageBuilder . build
2010-07-26 18:47:25 +02:00
}
2010-10-26 12:04:32 +02:00
2010-10-28 20:30:11 +02:00
private def authenticateRemoteClient ( request : RemoteMessageProtocol , ctx : ChannelHandlerContext ) = {
2010-10-26 15:23:50 +02:00
val attachment = ctx . getAttachment
2010-10-29 16:33:31 +02:00
if ( ( attachment ne null ) &&
attachment . isInstanceOf [ String ] &&
2010-10-26 15:23:50 +02:00
attachment . asInstanceOf [ String ] == CHANNEL_INIT ) { // is first time around, channel initialization
2010-10-29 16:33:31 +02:00
ctx . setAttachment ( null )
2010-10-26 15:23:50 +02:00
val clientAddress = ctx . getChannel . getRemoteAddress . toString
if ( ! request . hasCookie ) throw new SecurityException (
"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" )
2010-11-24 13:53:53 +01:00
log . slf4j . info ( "Remote client [{}] successfully authenticated using secure cookie" , clientAddress )
2010-10-26 12:04:32 +02:00
}
}
2009-06-24 15:12:47 +02:00
}