2011-09-15 10:20:18 +02:00
|
|
|
/**
|
|
|
|
|
* Copyright (C) 2009-2011 Typesafe Inc. <http://www.typesafe.com>
|
|
|
|
|
*/
|
|
|
|
|
|
2011-09-20 21:44:50 +02:00
|
|
|
package akka.remote
|
2011-09-15 10:20:18 +02:00
|
|
|
|
2011-11-10 20:08:00 +01:00
|
|
|
import akka.actor.ActorSystem
|
2011-09-15 10:20:18 +02:00
|
|
|
import akka.actor._
|
2011-10-27 12:23:01 +02:00
|
|
|
import akka.event.Logging
|
2011-09-30 14:52:07 +02:00
|
|
|
import akka.actor.Status._
|
2011-09-15 10:20:18 +02:00
|
|
|
import akka.util._
|
2011-09-30 14:52:07 +02:00
|
|
|
import akka.util.duration._
|
|
|
|
|
import akka.util.Helpers._
|
|
|
|
|
import akka.actor.DeploymentConfig._
|
2011-10-19 12:25:16 +02:00
|
|
|
import akka.serialization.Compression.LZF
|
|
|
|
|
import akka.remote.RemoteProtocol._
|
|
|
|
|
import akka.remote.RemoteProtocol.RemoteSystemDaemonMessageType._
|
2011-09-15 10:20:18 +02:00
|
|
|
import java.net.InetSocketAddress
|
|
|
|
|
import com.eaio.uuid.UUID
|
2011-11-03 14:53:38 +01:00
|
|
|
import akka.serialization.{ JavaSerializer, Serialization, Serializer, Compression }
|
2011-11-09 12:41:37 +01:00
|
|
|
import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher }
|
2011-11-16 17:18:36 +01:00
|
|
|
import java.util.concurrent.atomic.AtomicLong
|
2011-11-21 10:48:21 +01:00
|
|
|
import java.util.concurrent.TimeUnit.MILLISECONDS
|
2011-11-22 13:04:10 +01:00
|
|
|
import akka.serialization.SerializationExtension
|
2011-12-08 14:44:05 +01:00
|
|
|
import akka.dispatch.SystemMessage
|
|
|
|
|
import akka.event.LoggingAdapter
|
2011-09-15 10:20:18 +02:00
|
|
|
|
|
|
|
|
/**
|
2011-10-19 12:25:16 +02:00
|
|
|
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
|
|
|
|
|
*
|
2011-09-15 10:20:18 +02:00
|
|
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
|
|
|
*/
|
2011-12-08 14:44:05 +01:00
|
|
|
class Remote(val system: ActorSystemImpl, val nodename: String, val remoteSettings: RemoteSettings) {
|
2011-10-12 09:10:05 +02:00
|
|
|
|
2011-11-18 11:59:43 +01:00
|
|
|
val log = Logging(system, "Remote")
|
2011-10-27 12:23:01 +02:00
|
|
|
|
2011-11-17 12:36:35 +01:00
|
|
|
import system._
|
2011-11-18 11:16:23 +01:00
|
|
|
import settings._
|
2011-10-12 09:10:05 +02:00
|
|
|
|
2011-11-24 18:53:18 +01:00
|
|
|
private[remote] val serialization = SerializationExtension(system)
|
2011-11-22 13:04:10 +01:00
|
|
|
private[remote] val remoteAddress = {
|
2011-12-08 14:44:05 +01:00
|
|
|
RemoteAddress(system.name, remoteSettings.serverSettings.Hostname, remoteSettings.serverSettings.Port)
|
2011-11-22 13:04:10 +01:00
|
|
|
}
|
2011-09-15 10:20:18 +02:00
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize)
|
2011-10-19 12:25:16 +02:00
|
|
|
|
2011-10-20 15:40:05 +02:00
|
|
|
// val gossiper = new Gossiper(this)
|
2011-10-19 12:25:16 +02:00
|
|
|
|
|
|
|
|
val remoteDaemonServiceName = "akka-system-remote-daemon".intern
|
2011-09-15 10:20:18 +02:00
|
|
|
|
2011-11-30 10:20:57 +01:00
|
|
|
val computeGridDispatcher = dispatcherFactory.fromConfig("akka.remote.compute-grid-dispatcher")
|
2011-09-15 10:20:18 +02:00
|
|
|
|
2011-11-30 10:20:57 +01:00
|
|
|
// FIXME it is probably better to create another supervisor for handling the children created by handle_*, ticket #1408
|
2011-12-08 14:44:05 +01:00
|
|
|
val remoteDaemonSupervisor =
|
|
|
|
|
system.provider.actorOf(system,
|
|
|
|
|
Props(OneForOneStrategy(List(classOf[Exception]), None, None)),
|
|
|
|
|
system.provider.rootGuardian,
|
|
|
|
|
"akka-system-remote-supervisor",
|
|
|
|
|
systemService = true) // is infinite restart what we want?
|
2011-09-30 14:52:07 +02:00
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
val remoteDaemon =
|
2011-11-17 12:36:35 +01:00
|
|
|
system.provider.actorOf(system,
|
2011-10-28 23:11:35 +02:00
|
|
|
Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
|
2011-12-08 14:44:05 +01:00
|
|
|
remoteDaemonSupervisor,
|
2011-10-18 15:39:26 +02:00
|
|
|
remoteDaemonServiceName,
|
2011-09-30 14:52:07 +02:00
|
|
|
systemService = true)
|
2011-09-15 10:20:18 +02:00
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
val remoteClientLifeCycleHandler = system.actorOf(Props(new Actor {
|
2011-09-15 10:20:18 +02:00
|
|
|
def receive = {
|
2011-11-10 12:23:39 +01:00
|
|
|
case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address)
|
|
|
|
|
case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address)
|
2011-09-15 10:20:18 +02:00
|
|
|
case _ ⇒ //ignore other
|
|
|
|
|
}
|
2011-10-19 12:25:16 +02:00
|
|
|
}), "akka.remote.RemoteClientLifeCycleListener")
|
2011-10-12 11:34:35 +02:00
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
val eventStream = new NetworkEventStream(system)
|
2011-09-15 10:20:18 +02:00
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
val server: RemoteSupport = {
|
2011-12-05 22:46:34 +01:00
|
|
|
val arguments = Seq(
|
|
|
|
|
classOf[ActorSystem] -> system,
|
|
|
|
|
classOf[Remote] -> this)
|
|
|
|
|
val types: Array[Class[_]] = arguments map (_._1) toArray
|
|
|
|
|
val values: Array[AnyRef] = arguments map (_._2) toArray
|
|
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
ReflectiveAccess.createInstance[RemoteSupport](remoteSettings.RemoteTransport, types, values) match {
|
2011-12-02 18:08:13 +01:00
|
|
|
case Left(problem) ⇒
|
|
|
|
|
log.error(problem, "Could not load remote transport layer")
|
|
|
|
|
throw problem
|
|
|
|
|
case Right(remote) ⇒
|
|
|
|
|
remote.start(None) //TODO Any application loader here?
|
|
|
|
|
|
|
|
|
|
system.eventStream.subscribe(eventStream.sender, classOf[RemoteLifeCycleEvent])
|
|
|
|
|
system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
|
|
|
|
|
|
|
|
|
|
remote
|
|
|
|
|
}
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
log.info("Starting remote server on [{}] and starting remoteDaemon {}", remoteAddress, remoteDaemon)
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
2011-10-19 12:25:16 +02:00
|
|
|
* Internal system "daemon" actor for remote internal communication.
|
2011-09-15 10:20:18 +02:00
|
|
|
*
|
2011-10-19 12:25:16 +02:00
|
|
|
* It acts as the brain of the remote that responds to system remote events (messages) and undertakes action.
|
2011-09-15 10:20:18 +02:00
|
|
|
*
|
|
|
|
|
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
|
|
|
|
*/
|
2011-10-19 12:25:16 +02:00
|
|
|
class RemoteSystemDaemon(remote: Remote) extends Actor {
|
2011-10-12 11:34:35 +02:00
|
|
|
|
2011-10-12 09:10:05 +02:00
|
|
|
import remote._
|
2011-11-17 12:36:35 +01:00
|
|
|
import remote.{ system ⇒ systemImpl }
|
2011-10-12 11:34:35 +02:00
|
|
|
|
2011-09-15 10:20:18 +02:00
|
|
|
override def preRestart(reason: Throwable, msg: Option[Any]) {
|
2011-10-27 12:23:01 +02:00
|
|
|
log.debug("RemoteSystemDaemon failed due to [{}] - restarting...", reason)
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-12 09:10:05 +02:00
|
|
|
def receive: Actor.Receive = {
|
2011-10-19 12:25:16 +02:00
|
|
|
case message: RemoteSystemDaemonMessageProtocol ⇒
|
2011-11-09 14:56:05 +01:00
|
|
|
log.debug("Received command [\n{}] to RemoteSystemDaemon on [{}]", message.getMessageType, nodename)
|
2011-09-15 10:20:18 +02:00
|
|
|
|
|
|
|
|
message.getMessageType match {
|
|
|
|
|
case USE ⇒ handleUse(message)
|
|
|
|
|
case RELEASE ⇒ handleRelease(message)
|
|
|
|
|
// case STOP ⇒ cluster.shutdown()
|
|
|
|
|
// case DISCONNECT ⇒ cluster.disconnect()
|
|
|
|
|
// case RECONNECT ⇒ cluster.reconnect()
|
|
|
|
|
// case RESIGN ⇒ cluster.resign()
|
|
|
|
|
// case FAIL_OVER_CONNECTIONS ⇒ handleFailover(message)
|
2011-10-19 12:25:16 +02:00
|
|
|
case GOSSIP ⇒ handleGossip(message)
|
2011-09-15 10:20:18 +02:00
|
|
|
case FUNCTION_FUN0_UNIT ⇒ handle_fun0_unit(message)
|
|
|
|
|
case FUNCTION_FUN0_ANY ⇒ handle_fun0_any(message)
|
|
|
|
|
case FUNCTION_FUN1_ARG_UNIT ⇒ handle_fun1_arg_unit(message)
|
|
|
|
|
case FUNCTION_FUN1_ARG_ANY ⇒ handle_fun1_arg_any(message)
|
|
|
|
|
//TODO: should we not deal with unrecognized message types?
|
|
|
|
|
}
|
|
|
|
|
|
2011-10-27 12:23:01 +02:00
|
|
|
case unknown ⇒ log.warning("Unknown message to RemoteSystemDaemon [{}]", unknown)
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-19 12:25:16 +02:00
|
|
|
def handleUse(message: RemoteSystemDaemonMessageProtocol) {
|
2011-09-15 10:20:18 +02:00
|
|
|
try {
|
2011-11-08 14:30:33 +01:00
|
|
|
if (message.hasActorPath) {
|
2011-09-19 14:43:28 +02:00
|
|
|
|
|
|
|
|
val actorFactoryBytes =
|
2011-12-08 14:44:05 +01:00
|
|
|
if (remoteSettings.ShouldCompressData) LZF.uncompress(message.getPayload.toByteArray) else message.getPayload.toByteArray
|
2011-09-19 14:43:28 +02:00
|
|
|
|
|
|
|
|
val actorFactory =
|
2011-11-24 18:53:18 +01:00
|
|
|
serialization.deserialize(actorFactoryBytes, classOf[() ⇒ Actor], None) match {
|
2011-09-15 10:20:18 +02:00
|
|
|
case Left(error) ⇒ throw error
|
2011-09-19 14:43:28 +02:00
|
|
|
case Right(instance) ⇒ instance.asInstanceOf[() ⇒ Actor]
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-11-29 16:32:50 +01:00
|
|
|
message.getActorPath match {
|
2011-12-05 21:43:58 +01:00
|
|
|
case RemoteActorPath(`remoteAddress`, elems) if elems.size > 0 ⇒
|
2011-11-29 16:32:50 +01:00
|
|
|
val name = elems.last
|
2011-12-03 11:06:38 +01:00
|
|
|
systemImpl.provider.actorFor(systemImpl.lookupRoot, elems.dropRight(1)) match {
|
2011-11-29 16:32:50 +01:00
|
|
|
case x if x eq system.deadLetters ⇒
|
|
|
|
|
log.error("Parent actor does not exist, ignoring remote system daemon command [{}]", message)
|
|
|
|
|
case parent ⇒
|
|
|
|
|
systemImpl.provider.actorOf(systemImpl, Props(creator = actorFactory), parent, name)
|
|
|
|
|
}
|
|
|
|
|
case _ ⇒
|
|
|
|
|
log.error("remote path does not match path from message [{}]", message)
|
2011-11-08 11:56:46 +01:00
|
|
|
}
|
|
|
|
|
|
2011-09-15 10:20:18 +02:00
|
|
|
} else {
|
2011-10-27 12:23:01 +02:00
|
|
|
log.error("Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [{}]", message)
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-11-22 13:04:10 +01:00
|
|
|
sender ! Success(remoteAddress)
|
2011-09-15 10:20:18 +02:00
|
|
|
} catch {
|
2011-11-30 10:20:57 +01:00
|
|
|
case exc: Exception ⇒
|
|
|
|
|
sender ! Failure(exc)
|
|
|
|
|
throw exc
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
2011-11-22 13:04:10 +01:00
|
|
|
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-19 12:25:16 +02:00
|
|
|
// FIXME implement handleRelease
|
|
|
|
|
def handleRelease(message: RemoteSystemDaemonMessageProtocol) {
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def handleGossip(message: RemoteSystemDaemonMessageProtocol) {
|
2011-10-20 15:40:05 +02:00
|
|
|
// try {
|
|
|
|
|
// val gossip = serialization.deserialize(message.getPayload.toByteArray, classOf[Gossip], None) match {
|
|
|
|
|
// case Left(error) ⇒ throw error
|
|
|
|
|
// case Right(instance) ⇒ instance.asInstanceOf[Gossip]
|
|
|
|
|
// }
|
|
|
|
|
|
|
|
|
|
// gossiper tell gossip
|
|
|
|
|
|
2011-10-22 16:06:20 +02:00
|
|
|
// sender ! Success(address.toString)
|
2011-10-20 15:40:05 +02:00
|
|
|
// } catch {
|
|
|
|
|
// case error: Throwable ⇒
|
2011-10-22 16:06:20 +02:00
|
|
|
// sender ! Failure(error)
|
2011-10-20 15:40:05 +02:00
|
|
|
// throw error
|
|
|
|
|
// }
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-11-16 17:18:36 +01:00
|
|
|
/*
|
|
|
|
|
* generate name for temporary actor refs
|
|
|
|
|
*/
|
|
|
|
|
private val tempNumber = new AtomicLong
|
2011-11-17 13:27:33 +01:00
|
|
|
def tempName = "$_" + Helpers.base64(tempNumber.getAndIncrement())
|
2011-11-16 17:18:36 +01:00
|
|
|
def tempPath = remoteDaemon.path / tempName
|
|
|
|
|
|
2011-11-30 10:20:57 +01:00
|
|
|
// FIXME: handle real remote supervision, ticket #1408
|
2011-10-19 12:25:16 +02:00
|
|
|
def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
|
2011-11-17 12:36:35 +01:00
|
|
|
new LocalActorRef(systemImpl,
|
2011-09-15 10:20:18 +02:00
|
|
|
Props(
|
2011-09-29 12:44:52 +02:00
|
|
|
context ⇒ {
|
|
|
|
|
case f: Function0[_] ⇒ try { f() } finally { context.self.stop() }
|
2011-11-16 17:18:36 +01:00
|
|
|
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-11-30 10:20:57 +01:00
|
|
|
// FIXME: handle real remote supervision, ticket #1408
|
2011-10-19 12:25:16 +02:00
|
|
|
def handle_fun0_any(message: RemoteSystemDaemonMessageProtocol) {
|
2011-11-17 12:36:35 +01:00
|
|
|
new LocalActorRef(systemImpl,
|
2011-09-15 10:20:18 +02:00
|
|
|
Props(
|
2011-09-29 12:44:52 +02:00
|
|
|
context ⇒ {
|
2011-10-22 16:06:20 +02:00
|
|
|
case f: Function0[_] ⇒ try { sender ! f() } finally { context.self.stop() }
|
2011-11-16 17:18:36 +01:00
|
|
|
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-11-30 10:20:57 +01:00
|
|
|
// FIXME: handle real remote supervision, ticket #1408
|
2011-10-19 12:25:16 +02:00
|
|
|
def handle_fun1_arg_unit(message: RemoteSystemDaemonMessageProtocol) {
|
2011-11-17 12:36:35 +01:00
|
|
|
new LocalActorRef(systemImpl,
|
2011-09-15 10:20:18 +02:00
|
|
|
Props(
|
2011-09-29 12:44:52 +02:00
|
|
|
context ⇒ {
|
|
|
|
|
case (fun: Function[_, _], param: Any) ⇒ try { fun.asInstanceOf[Any ⇒ Unit].apply(param) } finally { context.self.stop() }
|
2011-11-16 17:18:36 +01:00
|
|
|
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-11-30 10:20:57 +01:00
|
|
|
// FIXME: handle real remote supervision, ticket #1408
|
2011-10-19 12:25:16 +02:00
|
|
|
def handle_fun1_arg_any(message: RemoteSystemDaemonMessageProtocol) {
|
2011-11-17 12:36:35 +01:00
|
|
|
new LocalActorRef(systemImpl,
|
2011-09-15 10:20:18 +02:00
|
|
|
Props(
|
2011-09-29 12:44:52 +02:00
|
|
|
context ⇒ {
|
2011-10-22 16:06:20 +02:00
|
|
|
case (fun: Function[_, _], param: Any) ⇒ try { sender ! fun.asInstanceOf[Any ⇒ Any](param) } finally { context.self.stop() }
|
2011-11-16 17:18:36 +01:00
|
|
|
}).copy(dispatcher = computeGridDispatcher), remoteDaemon, tempPath, systemService = true) forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-19 12:25:16 +02:00
|
|
|
def handleFailover(message: RemoteSystemDaemonMessageProtocol) {
|
2011-09-22 03:36:59 +02:00
|
|
|
// val (from, to) = payloadFor(message, classOf[(InetSocketremoteDaemonServiceName, InetSocketremoteDaemonServiceName)])
|
2011-09-15 10:20:18 +02:00
|
|
|
// cluster.failOverClusterActorRefConnections(from, to)
|
|
|
|
|
}
|
|
|
|
|
|
2011-10-19 12:25:16 +02:00
|
|
|
private def payloadFor[T](message: RemoteSystemDaemonMessageProtocol, clazz: Class[T]): T = {
|
2011-11-24 18:53:18 +01:00
|
|
|
serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
|
2011-09-15 10:20:18 +02:00
|
|
|
case Left(error) ⇒ throw error
|
|
|
|
|
case Right(instance) ⇒ instance.asInstanceOf[T]
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2011-11-03 14:53:38 +01:00
|
|
|
|
|
|
|
|
class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) {
|
2011-11-16 17:18:36 +01:00
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
def provider = remote.system.asInstanceOf[ActorSystemImpl].provider
|
2011-11-16 17:18:36 +01:00
|
|
|
|
2011-11-03 14:53:38 +01:00
|
|
|
lazy val sender: ActorRef =
|
2011-12-03 11:06:38 +01:00
|
|
|
if (input.hasSender) provider.actorFor(provider.rootGuardian, input.getSender.getPath)
|
2011-12-01 14:29:33 +01:00
|
|
|
else remote.system.deadLetters
|
2011-11-08 11:56:46 +01:00
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
lazy val recipient: InternalActorRef = provider.actorFor(provider.rootGuardian, input.getRecipient.getPath)
|
2011-11-03 14:53:38 +01:00
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
lazy val payload: AnyRef = MessageSerializer.deserialize(remote.system, input.getMessage, classLoader)
|
2011-11-03 14:53:38 +01:00
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
override def toString = "RemoteMessage: " + payload + " to " + recipient + " from " + sender
|
2011-11-03 14:53:38 +01:00
|
|
|
}
|
2011-11-03 15:42:46 +01:00
|
|
|
|
|
|
|
|
trait RemoteMarshallingOps {
|
|
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
def log: LoggingAdapter
|
|
|
|
|
|
2011-11-17 12:36:35 +01:00
|
|
|
def system: ActorSystem
|
2011-11-03 15:42:46 +01:00
|
|
|
|
2011-11-24 10:38:36 +01:00
|
|
|
protected def useUntrustedMode: Boolean
|
|
|
|
|
|
2011-11-03 15:42:46 +01:00
|
|
|
def createMessageSendEnvelope(rmp: RemoteMessageProtocol): AkkaRemoteProtocol = {
|
|
|
|
|
val arp = AkkaRemoteProtocol.newBuilder
|
|
|
|
|
arp.setMessage(rmp)
|
|
|
|
|
arp.build
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def createControlEnvelope(rcp: RemoteControlProtocol): AkkaRemoteProtocol = {
|
|
|
|
|
val arp = AkkaRemoteProtocol.newBuilder
|
|
|
|
|
arp.setInstruction(rcp)
|
|
|
|
|
arp.build
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/**
|
|
|
|
|
* Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
|
|
|
|
|
*/
|
|
|
|
|
def toRemoteActorRefProtocol(actor: ActorRef): ActorRefProtocol = {
|
2011-12-01 14:29:33 +01:00
|
|
|
ActorRefProtocol.newBuilder.setPath(actor.path.toString).build
|
2011-11-03 15:42:46 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
|
|
def createRemoteMessageProtocolBuilder(
|
2011-12-08 14:44:05 +01:00
|
|
|
recipient: ActorRef,
|
|
|
|
|
message: Any,
|
2011-11-03 15:42:46 +01:00
|
|
|
senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = {
|
|
|
|
|
|
2011-12-08 14:44:05 +01:00
|
|
|
val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient))
|
|
|
|
|
messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
|
2011-11-03 15:42:46 +01:00
|
|
|
|
|
|
|
|
if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get))
|
|
|
|
|
|
|
|
|
|
messageBuilder
|
|
|
|
|
}
|
|
|
|
|
|
2011-11-24 10:38:36 +01:00
|
|
|
def receiveMessage(remoteMessage: RemoteMessage) {
|
2011-12-08 14:44:05 +01:00
|
|
|
log.debug("received message {}", remoteMessage)
|
|
|
|
|
|
|
|
|
|
remoteMessage.recipient match {
|
|
|
|
|
case l @ (_: LocalActorRef | _: MinimalActorRef) ⇒
|
|
|
|
|
remoteMessage.payload match {
|
|
|
|
|
case msg: SystemMessage ⇒
|
|
|
|
|
if (useUntrustedMode)
|
|
|
|
|
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not send system message")
|
|
|
|
|
else l.sendSystemMessage(msg)
|
|
|
|
|
case _: AutoReceivedMessage if (useUntrustedMode) ⇒
|
|
|
|
|
throw new SecurityException("RemoteModule server is operating is untrusted mode, can not pass on a AutoReceivedMessage to the remote actor")
|
|
|
|
|
case m ⇒ l.!(m)(remoteMessage.sender)
|
|
|
|
|
}
|
|
|
|
|
case r ⇒ log.error("dropping message {} for non-local recipient {}", remoteMessage.payload, r)
|
2011-11-09 12:41:37 +01:00
|
|
|
}
|
|
|
|
|
}
|
2011-11-03 15:42:46 +01:00
|
|
|
}
|