2009-06-24 15:12:47 +02:00
|
|
|
/**
|
2009-12-27 16:01:53 +01:00
|
|
|
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
|
2009-06-24 15:12:47 +02:00
|
|
|
*/
|
|
|
|
|
|
2009-12-18 21:26:03 +01:00
|
|
|
package se.scalablesolutions.akka.remote
|
2009-06-24 15:12:47 +02:00
|
|
|
|
|
|
|
|
import java.lang.reflect.InvocationTargetException
|
|
|
|
|
import java.net.InetSocketAddress
|
|
|
|
|
import java.util.concurrent.{ConcurrentHashMap, Executors}
|
2009-12-30 08:36:24 +01:00
|
|
|
import java.util.{Map => JMap}
|
2009-07-02 13:23:03 +02:00
|
|
|
|
2009-10-06 00:07:27 +02:00
|
|
|
import se.scalablesolutions.akka.actor._
|
|
|
|
|
import se.scalablesolutions.akka.util._
|
2009-12-18 21:26:03 +01:00
|
|
|
import se.scalablesolutions.akka.remote.protobuf.RemoteProtocol.{RemoteReply, RemoteRequest}
|
2009-10-06 00:07:27 +02:00
|
|
|
import se.scalablesolutions.akka.Config.config
|
2009-07-18 00:16:32 +02:00
|
|
|
|
2009-06-24 15:12:47 +02:00
|
|
|
import org.jboss.netty.bootstrap.ServerBootstrap
|
|
|
|
|
import org.jboss.netty.channel._
|
2009-12-18 21:26:03 +01:00
|
|
|
import org.jboss.netty.channel.group.{DefaultChannelGroup, ChannelGroup}
|
2009-06-24 15:12:47 +02:00
|
|
|
import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
|
2009-07-12 23:08:17 +02:00
|
|
|
import org.jboss.netty.handler.codec.frame.{LengthFieldBasedFrameDecoder, LengthFieldPrepender}
|
|
|
|
|
import org.jboss.netty.handler.codec.protobuf.{ProtobufDecoder, ProtobufEncoder}
|
2009-11-21 22:04:10 +01:00
|
|
|
import org.jboss.netty.handler.codec.compression.{ZlibEncoder, ZlibDecoder}
|
2009-07-12 23:08:17 +02:00
|
|
|
|
2009-07-23 20:01:37 +02:00
|
|
|
/**
|
2009-11-24 17:41:08 +01:00
|
|
|
* Use this object if you need a single remote server on a specific node.
|
|
|
|
|
*
|
|
|
|
|
* <pre>
|
2009-12-18 21:26:03 +01:00
|
|
|
* // takes hostname and port from 'akka.conf'
|
2009-11-25 12:42:50 +01:00
|
|
|
* RemoteNode.start
|
2009-11-24 17:41:08 +01:00
|
|
|
* </pre>
|
|
|
|
|
*
|
2009-12-18 21:26:03 +01:00
|
|
|
* <pre>
|
|
|
|
|
* RemoteNode.start(hostname, port)
|
|
|
|
|
* </pre>
|
2010-02-16 09:39:56 +01:00
|
|
|
*
|
|
|
|
|
* You can specify the class loader to use to load the remote actors.
|
|
|
|
|
* <pre>
|
|
|
|
|
* RemoteNode.start(hostname, port, classLoader)
|
|
|
|
|
* </pre>
|
2009-12-18 21:26:03 +01:00
|
|
|
*
|
2009-11-24 17:41:08 +01:00
|
|
|
* If you need to create more than one, then you can use the RemoteServer:
|
2009-12-18 21:26:03 +01:00
|
|
|
*
|
2009-11-24 17:41:08 +01:00
|
|
|
* <pre>
|
|
|
|
|
* val server = new RemoteServer
|
2009-12-18 21:26:03 +01:00
|
|
|
* server.start(hostname, port)
|
2009-11-24 17:41:08 +01:00
|
|
|
* </pre>
|
|
|
|
|
*
|
2009-07-23 20:01:37 +02:00
|
|
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
|
|
|
*/
|
2010-01-04 21:54:54 +01:00
|
|
|
object RemoteNode extends RemoteServer
|
2009-11-24 17:41:08 +01:00
|
|
|
|
|
|
|
|
/**
|
2010-01-05 08:57:12 +01:00
|
|
|
* For internal use only.
|
|
|
|
|
* Holds configuration variables, remote actors, remote active objects 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ér</a>
|
|
|
|
|
*/
|
|
|
|
|
object RemoteServer {
|
2009-10-22 11:14:36 +02:00
|
|
|
val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost")
|
2010-03-04 23:25:55 +01:00
|
|
|
val PORT = config.getInt("akka.remote.server.port", 9966)
|
2009-11-24 17:41:08 +01:00
|
|
|
|
|
|
|
|
val CONNECTION_TIMEOUT_MILLIS = config.getInt("akka.remote.server.connection-timeout", 1000)
|
|
|
|
|
|
2009-11-22 14:32:27 +01:00
|
|
|
val COMPRESSION_SCHEME = config.getString("akka.remote.compression-scheme", "zlib")
|
|
|
|
|
val ZLIB_COMPRESSION_LEVEL = {
|
|
|
|
|
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
|
|
|
|
|
|
|
|
object Address {
|
|
|
|
|
def apply(hostname: String, port: Int) = new Address(hostname, port)
|
|
|
|
|
}
|
|
|
|
|
class Address(val hostname: String, val port: Int) {
|
|
|
|
|
override def hashCode: Int = {
|
|
|
|
|
var result = HashCode.SEED
|
|
|
|
|
result = HashCode.hash(result, hostname)
|
|
|
|
|
result = HashCode.hash(result, port)
|
|
|
|
|
result
|
|
|
|
|
}
|
|
|
|
|
override def equals(that: Any): Boolean = {
|
|
|
|
|
that != null &&
|
|
|
|
|
that.isInstanceOf[Address] &&
|
|
|
|
|
that.asInstanceOf[Address].hostname == hostname &&
|
|
|
|
|
that.asInstanceOf[Address].port == port
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
class RemoteActorSet {
|
|
|
|
|
val actors = new ConcurrentHashMap[String, Actor]
|
|
|
|
|
val activeObjects = new ConcurrentHashMap[String, AnyRef]
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private val remoteActorSets = new ConcurrentHashMap[Address, RemoteActorSet]
|
2010-01-04 21:54:54 +01:00
|
|
|
private val remoteServers = new ConcurrentHashMap[Address, RemoteServer]
|
|
|
|
|
|
2010-01-05 08:57:12 +01:00
|
|
|
private[akka] def actorsFor(remoteServerAddress: RemoteServer.Address): RemoteActorSet = {
|
2009-12-30 08:36:24 +01:00
|
|
|
val set = remoteActorSets.get(remoteServerAddress)
|
|
|
|
|
if (set ne null) set
|
|
|
|
|
else {
|
|
|
|
|
val remoteActorSet = new RemoteActorSet
|
|
|
|
|
remoteActorSets.put(remoteServerAddress, remoteActorSet)
|
|
|
|
|
remoteActorSet
|
|
|
|
|
}
|
|
|
|
|
}
|
2010-01-04 21:54:54 +01:00
|
|
|
|
2010-01-05 08:57:12 +01:00
|
|
|
private[remote] def serverFor(hostname: String, port: Int): Option[RemoteServer] = {
|
2010-01-04 21:54:54 +01:00
|
|
|
val server = remoteServers.get(Address(hostname, port))
|
|
|
|
|
if (server eq null) None
|
|
|
|
|
else Some(server)
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private[remote] def register(hostname: String, port: Int, server: RemoteServer) =
|
|
|
|
|
remoteServers.put(Address(hostname, port), server)
|
|
|
|
|
|
|
|
|
|
private[remote] def unregister(hostname: String, port: Int) =
|
|
|
|
|
remoteServers.remove(Address(hostname, port))
|
2009-11-24 17:41:08 +01:00
|
|
|
}
|
2009-11-22 14:32:27 +01:00
|
|
|
|
2009-11-24 17:41:08 +01:00
|
|
|
/**
|
|
|
|
|
* Use this class if you need a more than one remote server on a specific node.
|
|
|
|
|
*
|
|
|
|
|
* <pre>
|
|
|
|
|
* val server = new RemoteServer
|
|
|
|
|
* server.start
|
|
|
|
|
* </pre>
|
|
|
|
|
*
|
|
|
|
|
* If you need to create more than one, then you can use the RemoteServer:
|
|
|
|
|
*
|
|
|
|
|
* <pre>
|
2009-11-25 12:42:50 +01:00
|
|
|
* RemoteNode.start
|
2009-11-24 17:41:08 +01:00
|
|
|
* </pre>
|
|
|
|
|
*
|
|
|
|
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
|
|
|
*/
|
2010-01-04 21:54:54 +01:00
|
|
|
class RemoteServer extends Logging {
|
2009-11-24 17:41:08 +01:00
|
|
|
val name = "RemoteServer@" + hostname + ":" + port
|
2009-08-11 12:16:50 +02:00
|
|
|
|
2009-11-24 17:41:08 +01:00
|
|
|
private var hostname = RemoteServer.HOSTNAME
|
2009-12-30 08:36:24 +01:00
|
|
|
private var port = RemoteServer.PORT
|
2009-12-18 21:26:03 +01:00
|
|
|
|
2009-06-25 23:47:30 +02:00
|
|
|
@volatile private var isRunning = false
|
2009-08-11 12:16:50 +02:00
|
|
|
@volatile private var isConfigured = false
|
2009-06-25 23:47:30 +02:00
|
|
|
|
2009-06-25 13:07:58 +02:00
|
|
|
private val factory = new NioServerSocketChannelFactory(
|
2009-06-24 15:12:47 +02:00
|
|
|
Executors.newCachedThreadPool,
|
|
|
|
|
Executors.newCachedThreadPool)
|
|
|
|
|
|
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
|
|
|
|
2009-10-08 19:01:04 +02:00
|
|
|
def start: Unit = start(None)
|
2009-10-13 11:18:21 +02:00
|
|
|
|
2009-11-24 17:41:08 +01:00
|
|
|
def start(loader: Option[ClassLoader]): Unit = start(hostname, port, loader)
|
2009-10-13 11:18:21 +02:00
|
|
|
|
2009-11-24 17:41:08 +01:00
|
|
|
def start(_hostname: String, _port: Int): Unit = start(_hostname, _port, None)
|
2009-10-13 11:18:21 +02:00
|
|
|
|
|
|
|
|
def start(_hostname: String, _port: Int, loader: Option[ClassLoader]): Unit = synchronized {
|
2009-12-27 06:35:25 +01:00
|
|
|
try {
|
|
|
|
|
if (!isRunning) {
|
|
|
|
|
hostname = _hostname
|
|
|
|
|
port = _port
|
|
|
|
|
log.info("Starting remote server at [%s:%s]", hostname, port)
|
2010-01-04 21:54:54 +01:00
|
|
|
RemoteServer.register(hostname, port, this)
|
2009-12-30 08:36:24 +01:00
|
|
|
val remoteActorSet = RemoteServer.actorsFor(RemoteServer.Address(hostname, port))
|
|
|
|
|
val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, loader, remoteActorSet.actors, remoteActorSet.activeObjects)
|
|
|
|
|
bootstrap.setPipelineFactory(pipelineFactory)
|
2009-12-27 06:35:25 +01:00
|
|
|
bootstrap.setOption("child.tcpNoDelay", true)
|
|
|
|
|
bootstrap.setOption("child.keepAlive", true)
|
|
|
|
|
bootstrap.setOption("child.reuseAddress", true)
|
|
|
|
|
bootstrap.setOption("child.connectTimeoutMillis", RemoteServer.CONNECTION_TIMEOUT_MILLIS)
|
|
|
|
|
openChannels.add(bootstrap.bind(new InetSocketAddress(hostname, port)))
|
|
|
|
|
isRunning = true
|
2010-01-04 21:54:54 +01:00
|
|
|
Cluster.registerLocalNode(hostname, port)
|
2009-12-27 06:35:25 +01:00
|
|
|
}
|
|
|
|
|
} catch {
|
|
|
|
|
case e => log.error(e, "Could not start up remote server")
|
2009-06-25 23:47:30 +02:00
|
|
|
}
|
|
|
|
|
}
|
2009-11-24 17:41:08 +01:00
|
|
|
|
2010-02-16 15:39:09 +01:00
|
|
|
def shutdown = if (isRunning) {
|
2010-01-04 21:54:54 +01:00
|
|
|
RemoteServer.unregister(hostname, port)
|
2010-01-02 11:03:06 +01:00
|
|
|
openChannels.disconnect
|
2010-02-15 16:21:26 +01:00
|
|
|
openChannels.close.awaitUninterruptibly
|
2009-11-24 17:41:08 +01:00
|
|
|
bootstrap.releaseExternalResources
|
2010-01-04 21:54:54 +01:00
|
|
|
Cluster.deregisterLocalNode(hostname, port)
|
2009-11-24 17:41:08 +01:00
|
|
|
}
|
2010-02-16 15:39:09 +01:00
|
|
|
|
|
|
|
|
// TODO: register active object in RemoteServer as well
|
2010-02-17 15:32:17 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Register Remote Actor by the Actor's 'id' field.
|
|
|
|
|
*/
|
2010-02-16 15:39:09 +01:00
|
|
|
def register(actor: Actor) = if (isRunning) {
|
2010-02-26 17:42:44 +01:00
|
|
|
log.info("Registering server side remote actor [%s] with id [%s]", actor.getClass.getName, actor.getId)
|
|
|
|
|
RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors.put(actor.getId, actor)
|
2010-02-16 15:39:09 +01:00
|
|
|
}
|
2010-02-17 15:32:17 +01:00
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Register Remote Actor by a specific 'id' passed as argument.
|
|
|
|
|
*/
|
|
|
|
|
def register(id: String, actor: Actor) = if (isRunning) {
|
|
|
|
|
log.info("Registering server side remote actor [%s] with id [%s]", actor.getClass.getName, id)
|
|
|
|
|
RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors.put(id, actor)
|
|
|
|
|
}
|
2009-06-24 15:12:47 +02:00
|
|
|
}
|
|
|
|
|
|
2010-01-05 08:57:12 +01:00
|
|
|
case class Codec(encoder : ChannelHandler, decoder : ChannelHandler)
|
2009-12-29 20:12:46 +01:00
|
|
|
|
2009-07-27 21:21:28 +02:00
|
|
|
/**
|
|
|
|
|
* @author <a href="http://jonasboner.com">Jonas Boné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],
|
|
|
|
|
val actors: JMap[String, Actor],
|
|
|
|
|
val activeObjects: JMap[String, AnyRef]) extends ChannelPipelineFactory {
|
2009-11-24 17:41:08 +01:00
|
|
|
import RemoteServer._
|
|
|
|
|
|
2009-12-18 21:26:03 +01:00
|
|
|
def getPipeline: ChannelPipeline = {
|
2009-12-29 20:12:46 +01:00
|
|
|
val lenDec = new LengthFieldBasedFrameDecoder(1048576, 0, 4, 0, 4)
|
|
|
|
|
val lenPrep = new LengthFieldPrepender(4)
|
|
|
|
|
val protobufDec = new ProtobufDecoder(RemoteRequest.getDefaultInstance)
|
|
|
|
|
val protobufEnc = new ProtobufEncoder
|
2009-12-30 08:48:22 +01:00
|
|
|
val zipCodec = RemoteServer.COMPRESSION_SCHEME match {
|
2009-12-30 09:24:10 +01:00
|
|
|
case "zlib" => Some(Codec(new ZlibEncoder(RemoteServer.ZLIB_COMPRESSION_LEVEL), new ZlibDecoder))
|
2009-12-29 20:12:46 +01:00
|
|
|
//case "lzf" => Some(Codec(new LzfEncoder, new LzfDecoder))
|
|
|
|
|
case _ => None
|
2009-11-22 14:32:27 +01:00
|
|
|
}
|
2009-12-30 08:48:22 +01:00
|
|
|
val remoteServer = new RemoteServerHandler(name, openChannels, loader, actors, activeObjects)
|
|
|
|
|
|
2009-12-30 09:24:10 +01:00
|
|
|
val stages: Array[ChannelHandler] =
|
|
|
|
|
zipCodec.map(codec => Array(codec.decoder, lenDec, protobufDec, codec.encoder, lenPrep, protobufEnc, remoteServer))
|
|
|
|
|
.getOrElse(Array(lenDec, protobufDec, 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ér</a>
|
|
|
|
|
*/
|
2009-12-18 21:26:03 +01:00
|
|
|
@ChannelPipelineCoverage {val value = "all"}
|
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],
|
|
|
|
|
val actors: JMap[String, Actor],
|
|
|
|
|
val activeObjects: JMap[String, AnyRef]) extends SimpleChannelUpstreamHandler with Logging {
|
2009-11-22 15:25:16 +01:00
|
|
|
val AW_PROXY_PREFIX = "$$ProxiedByAW".intern
|
2009-12-18 21:26:03 +01:00
|
|
|
|
2009-12-27 22:56:55 +01:00
|
|
|
applicationLoader.foreach(RemoteProtocolBuilder.setClassLoader(_))
|
|
|
|
|
|
2009-12-18 21:26:03 +01:00
|
|
|
/**
|
|
|
|
|
* ChannelOpen overridden to store open channels for a clean shutdown
|
|
|
|
|
* of a RemoteServer. If a channel is closed before, it is
|
|
|
|
|
* automatically removed from the open channels group.
|
|
|
|
|
*/
|
|
|
|
|
override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) {
|
|
|
|
|
openChannels.add(ctx.getChannel)
|
|
|
|
|
}
|
|
|
|
|
|
2009-06-24 15:12:47 +02:00
|
|
|
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
|
2009-11-21 20:51:03 +01:00
|
|
|
if (event.isInstanceOf[ChannelStateEvent] &&
|
|
|
|
|
event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) {
|
2009-06-24 15:12:47 +02:00
|
|
|
log.debug(event.toString)
|
|
|
|
|
}
|
|
|
|
|
super.handleUpstream(ctx, event)
|
|
|
|
|
}
|
|
|
|
|
|
2009-07-03 17:15:36 +02:00
|
|
|
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = {
|
2009-06-24 15:12:47 +02:00
|
|
|
val message = event.getMessage
|
2010-02-16 15:39:09 +01:00
|
|
|
if (message eq null) throw new IllegalStateException("Message in remote MessageEvent is null: " + event)
|
2009-11-21 20:51:03 +01:00
|
|
|
if (message.isInstanceOf[RemoteRequest]) {
|
|
|
|
|
handleRemoteRequest(message.asInstanceOf[RemoteRequest], event.getChannel)
|
|
|
|
|
}
|
2009-06-24 15:12:47 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
|
2009-11-22 15:25:16 +01:00
|
|
|
log.error(event.getCause, "Unexpected exception from remote downstream")
|
2009-06-24 15:12:47 +02:00
|
|
|
event.getChannel.close
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def handleRemoteRequest(request: RemoteRequest, channel: Channel) = {
|
2009-07-18 00:16:32 +02:00
|
|
|
log.debug("Received RemoteRequest[\n%s]", request.toString)
|
|
|
|
|
if (request.getIsActor) dispatchToActor(request, channel)
|
2009-07-01 15:29:06 +02:00
|
|
|
else dispatchToActiveObject(request, channel)
|
2009-06-24 15:12:47 +02:00
|
|
|
}
|
|
|
|
|
|
2009-06-25 23:47:30 +02:00
|
|
|
private def dispatchToActor(request: RemoteRequest, channel: Channel) = {
|
2009-07-18 00:16:32 +02:00
|
|
|
log.debug("Dispatching to remote actor [%s]", request.getTarget)
|
2009-11-21 20:51:03 +01:00
|
|
|
val actor = createActor(request.getTarget, request.getUuid, request.getTimeout)
|
2009-12-18 21:26:03 +01:00
|
|
|
|
2009-07-23 20:01:37 +02:00
|
|
|
val message = RemoteProtocolBuilder.getMessage(request)
|
2009-11-23 15:19:53 +01:00
|
|
|
if (request.getIsOneWay) {
|
2009-12-18 21:26:03 +01:00
|
|
|
if (request.hasSourceHostname && request.hasSourcePort) {
|
|
|
|
|
// re-create the sending actor
|
|
|
|
|
val targetClass = if (request.hasSourceTarget) request.getSourceTarget
|
|
|
|
|
else request.getTarget
|
|
|
|
|
|
|
|
|
|
val remoteActor = createActor(targetClass, request.getSourceUuid, request.getTimeout)
|
|
|
|
|
if (!remoteActor.isRunning) {
|
|
|
|
|
remoteActor.makeRemote(request.getSourceHostname, request.getSourcePort)
|
|
|
|
|
remoteActor.start
|
|
|
|
|
}
|
2009-12-26 22:14:06 +01:00
|
|
|
actor.!(message)(Some(remoteActor))
|
2009-12-18 21:26:03 +01:00
|
|
|
} else {
|
|
|
|
|
// couldn't find a way to reply, send the message without a source/sender
|
|
|
|
|
actor.send(message)
|
|
|
|
|
}
|
|
|
|
|
} else {
|
2009-06-25 23:47:30 +02:00
|
|
|
try {
|
2009-07-13 00:17:57 +02:00
|
|
|
val resultOrNone = actor !! message
|
2009-07-01 15:29:06 +02:00
|
|
|
val result: AnyRef = if (resultOrNone.isDefined) resultOrNone.get else null
|
2009-06-25 23:47:30 +02:00
|
|
|
log.debug("Returning result from actor invocation [%s]", result)
|
2009-07-18 00:16:32 +02:00
|
|
|
val replyBuilder = RemoteReply.newBuilder
|
2009-12-18 21:26:03 +01:00
|
|
|
.setId(request.getId)
|
|
|
|
|
.setIsSuccessful(true)
|
|
|
|
|
.setIsActor(true)
|
2009-07-23 20:01:37 +02:00
|
|
|
RemoteProtocolBuilder.setMessage(result, replyBuilder)
|
2009-07-18 00:16:32 +02:00
|
|
|
if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
2009-08-11 12:16:50 +02:00
|
|
|
val replyMessage = replyBuilder.build
|
|
|
|
|
channel.write(replyMessage)
|
2009-06-25 23:47:30 +02:00
|
|
|
} catch {
|
2009-07-01 15:29:06 +02:00
|
|
|
case e: Throwable =>
|
2009-11-22 15:25:16 +01:00
|
|
|
log.error(e, "Could not invoke remote actor [%s]", request.getTarget)
|
2009-07-18 00:16:32 +02:00
|
|
|
val replyBuilder = RemoteReply.newBuilder
|
2009-12-18 21:26:03 +01:00
|
|
|
.setId(request.getId)
|
|
|
|
|
.setException(e.getClass.getName + "$" + e.getMessage)
|
|
|
|
|
.setIsSuccessful(false)
|
|
|
|
|
.setIsActor(true)
|
2009-07-18 00:16:32 +02:00
|
|
|
if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
2009-08-11 12:16:50 +02:00
|
|
|
val replyMessage = replyBuilder.build
|
|
|
|
|
channel.write(replyMessage)
|
2009-06-25 23:47:30 +02:00
|
|
|
}
|
2009-12-18 21:26:03 +01:00
|
|
|
}
|
2009-06-25 23:47:30 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
private def dispatchToActiveObject(request: RemoteRequest, channel: Channel) = {
|
2009-07-18 00:16:32 +02:00
|
|
|
log.debug("Dispatching to remote active object [%s :: %s]", request.getMethod, request.getTarget)
|
|
|
|
|
val activeObject = createActiveObject(request.getTarget, request.getTimeout)
|
2009-06-25 23:47:30 +02:00
|
|
|
|
2009-07-24 00:41:42 +02:00
|
|
|
val args = RemoteProtocolBuilder.getMessage(request).asInstanceOf[Array[AnyRef]].toList
|
2009-07-01 15:29:06 +02:00
|
|
|
val argClasses = args.map(_.getClass)
|
2009-07-18 00:16:32 +02:00
|
|
|
val (unescapedArgs, unescapedArgClasses) = unescapeArgs(args, argClasses, request.getTimeout)
|
2009-06-25 23:47:30 +02:00
|
|
|
|
2009-07-06 23:45:15 +02:00
|
|
|
//continueTransaction(request)
|
2009-06-25 23:47:30 +02:00
|
|
|
try {
|
2009-11-21 20:51:03 +01:00
|
|
|
val messageReceiver = activeObject.getClass.getDeclaredMethod(
|
|
|
|
|
request.getMethod, unescapedArgClasses: _*)
|
2009-07-18 00:16:32 +02:00
|
|
|
if (request.getIsOneWay) messageReceiver.invoke(activeObject, unescapedArgs: _*)
|
2009-06-25 23:47:30 +02:00
|
|
|
else {
|
2009-07-06 23:45:15 +02:00
|
|
|
val result = messageReceiver.invoke(activeObject, unescapedArgs: _*)
|
2009-07-01 15:29:06 +02:00
|
|
|
log.debug("Returning result from remote active object invocation [%s]", result)
|
2009-07-18 00:16:32 +02:00
|
|
|
val replyBuilder = RemoteReply.newBuilder
|
2009-12-18 21:26:03 +01:00
|
|
|
.setId(request.getId)
|
|
|
|
|
.setIsSuccessful(true)
|
|
|
|
|
.setIsActor(false)
|
2009-07-23 20:01:37 +02:00
|
|
|
RemoteProtocolBuilder.setMessage(result, replyBuilder)
|
2009-07-18 00:16:32 +02:00
|
|
|
if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
2009-08-11 12:16:50 +02:00
|
|
|
val replyMessage = replyBuilder.build
|
|
|
|
|
channel.write(replyMessage)
|
2009-06-25 23:47:30 +02:00
|
|
|
}
|
|
|
|
|
} catch {
|
|
|
|
|
case e: InvocationTargetException =>
|
2009-11-22 15:25:16 +01:00
|
|
|
log.error(e.getCause, "Could not invoke remote active object [%s :: %s]", request.getMethod, request.getTarget)
|
2009-07-18 00:16:32 +02:00
|
|
|
val replyBuilder = RemoteReply.newBuilder
|
2009-12-18 21:26:03 +01:00
|
|
|
.setId(request.getId)
|
|
|
|
|
.setException(e.getCause.getClass.getName + "$" + e.getCause.getMessage)
|
|
|
|
|
.setIsSuccessful(false)
|
|
|
|
|
.setIsActor(false)
|
2009-08-11 12:16:50 +02:00
|
|
|
if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
|
|
|
|
val replyMessage = replyBuilder.build
|
|
|
|
|
channel.write(replyMessage)
|
2009-07-01 15:29:06 +02:00
|
|
|
case e: Throwable =>
|
2009-11-22 15:25:16 +01:00
|
|
|
log.error(e.getCause, "Could not invoke remote active object [%s :: %s]", request.getMethod, request.getTarget)
|
2009-07-18 00:16:32 +02:00
|
|
|
val replyBuilder = RemoteReply.newBuilder
|
2009-12-18 21:26:03 +01:00
|
|
|
.setId(request.getId)
|
|
|
|
|
.setException(e.getClass.getName + "$" + e.getMessage)
|
|
|
|
|
.setIsSuccessful(false)
|
|
|
|
|
.setIsActor(false)
|
2009-08-11 12:16:50 +02:00
|
|
|
if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
|
|
|
|
val replyMessage = replyBuilder.build
|
|
|
|
|
channel.write(replyMessage)
|
2009-06-25 23:47:30 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
2009-07-12 23:08:17 +02:00
|
|
|
/*
|
2009-06-25 23:47:30 +02:00
|
|
|
private def continueTransaction(request: RemoteRequest) = {
|
|
|
|
|
val tx = request.tx
|
|
|
|
|
if (tx.isDefined) {
|
|
|
|
|
tx.get.reinit
|
2009-06-29 15:01:20 +02:00
|
|
|
TransactionManagement.threadBoundTx.set(tx)
|
2009-08-15 22:44:29 +02:00
|
|
|
setThreadLocalTransaction(tx.transaction)
|
|
|
|
|
} else {
|
|
|
|
|
TransactionManagement.threadBoundTx.set(None)
|
|
|
|
|
setThreadLocalTransaction(null)
|
|
|
|
|
}
|
2009-06-25 23:47:30 +02:00
|
|
|
}
|
2009-07-12 23:08:17 +02:00
|
|
|
*/
|
2009-07-01 15:29:06 +02:00
|
|
|
private def unescapeArgs(args: scala.List[AnyRef], argClasses: scala.List[Class[_]], timeout: Long) = {
|
2009-06-24 15:12:47 +02:00
|
|
|
val unescapedArgs = new Array[AnyRef](args.size)
|
|
|
|
|
val unescapedArgClasses = new Array[Class[_]](args.size)
|
|
|
|
|
|
|
|
|
|
val escapedArgs = for (i <- 0 until args.size) {
|
2009-07-18 00:16:32 +02:00
|
|
|
val arg = args(i)
|
2009-11-22 15:25:16 +01:00
|
|
|
if (arg.isInstanceOf[String] && arg.asInstanceOf[String].startsWith(AW_PROXY_PREFIX)) {
|
2009-07-18 00:16:32 +02:00
|
|
|
val argString = arg.asInstanceOf[String]
|
2009-11-22 15:25:16 +01:00
|
|
|
val proxyName = argString.replace(AW_PROXY_PREFIX, "") //argString.substring(argString.indexOf("$$ProxiedByAW"), argString.length)
|
2009-07-01 15:29:06 +02:00
|
|
|
val activeObject = createActiveObject(proxyName, timeout)
|
2009-06-24 15:12:47 +02:00
|
|
|
unescapedArgs(i) = activeObject
|
2009-12-18 21:26:03 +01:00
|
|
|
unescapedArgClasses(i) = Class.forName(proxyName)
|
2009-06-24 15:12:47 +02:00
|
|
|
} else {
|
|
|
|
|
unescapedArgs(i) = args(i)
|
2009-12-18 21:26:03 +01:00
|
|
|
unescapedArgClasses(i) = argClasses(i)
|
2009-06-24 15:12:47 +02:00
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
(unescapedArgs, unescapedArgClasses)
|
|
|
|
|
}
|
|
|
|
|
|
2009-07-01 15:29:06 +02:00
|
|
|
private def createActiveObject(name: String, timeout: Long): AnyRef = {
|
2009-06-24 15:12:47 +02:00
|
|
|
val activeObjectOrNull = activeObjects.get(name)
|
2009-12-27 08:24:11 +01:00
|
|
|
if (activeObjectOrNull eq null) {
|
2009-06-25 13:07:58 +02:00
|
|
|
try {
|
2009-10-06 00:07:27 +02:00
|
|
|
log.info("Creating a new remote active object [%s]", name)
|
2009-08-11 12:16:50 +02:00
|
|
|
val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name)
|
2009-12-18 21:26:03 +01:00
|
|
|
else Class.forName(name)
|
2009-10-17 00:37:56 +02:00
|
|
|
val newInstance = ActiveObject.newInstance(clazz, timeout).asInstanceOf[AnyRef]
|
2009-06-25 13:07:58 +02:00
|
|
|
activeObjects.put(name, newInstance)
|
|
|
|
|
newInstance
|
|
|
|
|
} catch {
|
|
|
|
|
case e =>
|
2009-11-22 15:25:16 +01:00
|
|
|
log.error(e, "Could not create remote active object instance")
|
2009-06-25 23:47:30 +02:00
|
|
|
throw e
|
|
|
|
|
}
|
|
|
|
|
} else activeObjectOrNull
|
|
|
|
|
}
|
|
|
|
|
|
2009-11-21 20:51:03 +01:00
|
|
|
private def createActor(name: String, uuid: String, timeout: Long): Actor = {
|
|
|
|
|
val actorOrNull = actors.get(uuid)
|
2009-12-27 08:24:11 +01:00
|
|
|
if (actorOrNull eq null) {
|
2009-06-25 23:47:30 +02:00
|
|
|
try {
|
2009-11-21 20:51:03 +01:00
|
|
|
log.info("Creating a new remote actor [%s:%s]", name, uuid)
|
2009-08-11 12:16:50 +02:00
|
|
|
val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name)
|
2009-12-18 21:26:03 +01:00
|
|
|
else Class.forName(name)
|
2009-06-25 23:47:30 +02:00
|
|
|
val newInstance = clazz.newInstance.asInstanceOf[Actor]
|
2009-11-21 20:51:03 +01:00
|
|
|
newInstance._uuid = uuid
|
2009-07-01 15:29:06 +02:00
|
|
|
newInstance.timeout = timeout
|
2009-11-18 10:02:33 +01:00
|
|
|
newInstance._remoteAddress = None
|
2009-11-21 20:51:03 +01:00
|
|
|
actors.put(uuid, newInstance)
|
2009-12-18 21:26:03 +01:00
|
|
|
newInstance.start
|
2009-06-25 23:47:30 +02:00
|
|
|
newInstance
|
|
|
|
|
} catch {
|
|
|
|
|
case e =>
|
2010-01-05 08:57:12 +01:00
|
|
|
log.error(e, "Could not create remote actor instance")
|
2009-06-25 23:47:30 +02:00
|
|
|
throw e
|
2009-06-25 13:07:58 +02:00
|
|
|
}
|
2009-06-25 23:47:30 +02:00
|
|
|
} else actorOrNull
|
2009-06-24 15:12:47 +02:00
|
|
|
}
|
|
|
|
|
}
|