make remote lookup work
- create RemoteActorRef in actorFor - simplify send/receive because Futures/Exceptions do not go over the wire anymore - add RemoteCommunicationSpec which uses two ActorSystems communicating in the same JVM via TCP socket
This commit is contained in:
parent
25e23a3378
commit
fac840adfc
19 changed files with 376 additions and 435 deletions
|
|
@ -22,26 +22,27 @@ import akka.dispatch.{ Terminate, Dispatchers, Future, PinnedDispatcher }
|
|||
import java.util.concurrent.atomic.AtomicLong
|
||||
import java.util.concurrent.TimeUnit.MILLISECONDS
|
||||
import akka.serialization.SerializationExtension
|
||||
import akka.dispatch.SystemMessage
|
||||
import akka.event.LoggingAdapter
|
||||
|
||||
/**
|
||||
* Remote module - contains remote client and server config, remote server instance, remote daemon, remote dispatchers etc.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class Remote(val system: ActorSystemImpl, val nodename: String) {
|
||||
class Remote(val system: ActorSystemImpl, val nodename: String, val remoteSettings: RemoteSettings) {
|
||||
|
||||
val log = Logging(system, "Remote")
|
||||
|
||||
import system._
|
||||
import settings._
|
||||
|
||||
private[remote] val remoteExtension = RemoteExtension(system)
|
||||
private[remote] val serialization = SerializationExtension(system)
|
||||
private[remote] val remoteAddress = {
|
||||
RemoteAddress(system.name, remoteExtension.serverSettings.Hostname, remoteExtension.serverSettings.Port)
|
||||
RemoteAddress(system.name, remoteSettings.serverSettings.Hostname, remoteSettings.serverSettings.Port)
|
||||
}
|
||||
|
||||
val failureDetector = new AccrualFailureDetector(system)
|
||||
val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize)
|
||||
|
||||
// val gossiper = new Gossiper(this)
|
||||
|
||||
|
|
@ -50,17 +51,21 @@ class Remote(val system: ActorSystemImpl, val nodename: String) {
|
|||
val computeGridDispatcher = dispatcherFactory.fromConfig("akka.remote.compute-grid-dispatcher")
|
||||
|
||||
// FIXME it is probably better to create another supervisor for handling the children created by handle_*, ticket #1408
|
||||
private[remote] lazy val remoteDaemonSupervisor = system.actorOf(Props(
|
||||
OneForOneStrategy(List(classOf[Exception]), None, None)), "akka-system-remote-supervisor") // is infinite restart what we want?
|
||||
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?
|
||||
|
||||
private[remote] lazy val remoteDaemon =
|
||||
val remoteDaemon =
|
||||
system.provider.actorOf(system,
|
||||
Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
|
||||
remoteDaemonSupervisor.asInstanceOf[InternalActorRef],
|
||||
remoteDaemonSupervisor,
|
||||
remoteDaemonServiceName,
|
||||
systemService = true)
|
||||
|
||||
private[remote] lazy val remoteClientLifeCycleHandler = system.actorOf(Props(new Actor {
|
||||
val remoteClientLifeCycleHandler = system.actorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
|
|
@ -68,16 +73,16 @@ class Remote(val system: ActorSystemImpl, val nodename: String) {
|
|||
}
|
||||
}), "akka.remote.RemoteClientLifeCycleListener")
|
||||
|
||||
lazy val eventStream = new NetworkEventStream(system)
|
||||
val eventStream = new NetworkEventStream(system)
|
||||
|
||||
lazy val server: RemoteSupport = {
|
||||
val server: RemoteSupport = {
|
||||
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
|
||||
|
||||
ReflectiveAccess.createInstance[RemoteSupport](remoteExtension.RemoteTransport, types, values) match {
|
||||
ReflectiveAccess.createInstance[RemoteSupport](remoteSettings.RemoteTransport, types, values) match {
|
||||
case Left(problem) ⇒
|
||||
log.error(problem, "Could not load remote transport layer")
|
||||
throw problem
|
||||
|
|
@ -91,10 +96,7 @@ class Remote(val system: ActorSystemImpl, val nodename: String) {
|
|||
}
|
||||
}
|
||||
|
||||
def start() {
|
||||
val daemonPath = remoteDaemon.path //Force init of daemon
|
||||
log.info("Starting remote server on [{}] and starting remoteDaemon with path [{}]", remoteAddress, daemonPath)
|
||||
}
|
||||
log.info("Starting remote server on [{}] and starting remoteDaemon {}", remoteAddress, remoteDaemon)
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
@ -141,7 +143,7 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
if (message.hasActorPath) {
|
||||
|
||||
val actorFactoryBytes =
|
||||
if (remoteExtension.ShouldCompressData) LZF.uncompress(message.getPayload.toByteArray) else message.getPayload.toByteArray
|
||||
if (remoteSettings.ShouldCompressData) LZF.uncompress(message.getPayload.toByteArray) else message.getPayload.toByteArray
|
||||
|
||||
val actorFactory =
|
||||
serialization.deserialize(actorFactoryBytes, classOf[() ⇒ Actor], None) match {
|
||||
|
|
@ -254,39 +256,23 @@ class RemoteSystemDaemon(remote: Remote) extends Actor {
|
|||
|
||||
class RemoteMessage(input: RemoteMessageProtocol, remote: RemoteSupport, classLoader: Option[ClassLoader] = None) {
|
||||
|
||||
val provider = remote.system.asInstanceOf[ActorSystemImpl].provider
|
||||
def provider = remote.system.asInstanceOf[ActorSystemImpl].provider
|
||||
|
||||
lazy val sender: ActorRef =
|
||||
if (input.hasSender) provider.actorFor(provider.rootGuardian, input.getSender.getPath)
|
||||
else remote.system.deadLetters
|
||||
|
||||
lazy val recipient: ActorRef = remote.system.actorFor(input.getRecipient.getPath)
|
||||
lazy val recipient: InternalActorRef = provider.actorFor(provider.rootGuardian, input.getRecipient.getPath)
|
||||
|
||||
lazy val payload: Either[Throwable, AnyRef] =
|
||||
if (input.hasException) Left(parseException())
|
||||
else Right(MessageSerializer.deserialize(remote.system, input.getMessage, classLoader))
|
||||
lazy val payload: AnyRef = MessageSerializer.deserialize(remote.system, input.getMessage, classLoader)
|
||||
|
||||
protected def parseException(): Throwable = {
|
||||
val exception = input.getException
|
||||
val classname = exception.getClassname
|
||||
try {
|
||||
val exceptionClass =
|
||||
if (classLoader.isDefined) classLoader.get.loadClass(classname) else Class.forName(classname)
|
||||
exceptionClass
|
||||
.getConstructor(Array[Class[_]](classOf[String]): _*)
|
||||
.newInstance(exception.getMessage).asInstanceOf[Throwable]
|
||||
} catch {
|
||||
case problem: Exception ⇒
|
||||
remote.system.eventStream.publish(Logging.Error(problem, "RemoteMessage", problem.getMessage))
|
||||
CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(problem, classname, exception.getMessage)
|
||||
}
|
||||
}
|
||||
|
||||
override def toString = "RemoteMessage: " + recipient + "(" + input.getRecipient.getPath + ") from " + sender
|
||||
override def toString = "RemoteMessage: " + payload + " to " + recipient + " from " + sender
|
||||
}
|
||||
|
||||
trait RemoteMarshallingOps {
|
||||
|
||||
def log: LoggingAdapter
|
||||
|
||||
def system: ActorSystem
|
||||
|
||||
protected def useUntrustedMode: Boolean
|
||||
|
|
@ -311,21 +297,12 @@ trait RemoteMarshallingOps {
|
|||
}
|
||||
|
||||
def createRemoteMessageProtocolBuilder(
|
||||
recipient: Either[ActorRef, ActorRefProtocol],
|
||||
message: Either[Throwable, Any],
|
||||
recipient: ActorRef,
|
||||
message: Any,
|
||||
senderOption: Option[ActorRef]): RemoteMessageProtocol.Builder = {
|
||||
|
||||
val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(recipient.fold(toRemoteActorRefProtocol _, identity))
|
||||
|
||||
message match {
|
||||
case Right(message) ⇒
|
||||
messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
|
||||
case Left(exception) ⇒
|
||||
messageBuilder.setException(ExceptionProtocol.newBuilder
|
||||
.setClassname(exception.getClass.getName)
|
||||
.setMessage(Option(exception.getMessage).getOrElse(""))
|
||||
.build)
|
||||
}
|
||||
val messageBuilder = RemoteMessageProtocol.newBuilder.setRecipient(toRemoteActorRefProtocol(recipient))
|
||||
messageBuilder.setMessage(MessageSerializer.serialize(system, message.asInstanceOf[AnyRef]))
|
||||
|
||||
if (senderOption.isDefined) messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get))
|
||||
|
||||
|
|
@ -333,15 +310,20 @@ trait RemoteMarshallingOps {
|
|||
}
|
||||
|
||||
def receiveMessage(remoteMessage: RemoteMessage) {
|
||||
val recipient = remoteMessage.recipient
|
||||
log.debug("received message {}", remoteMessage)
|
||||
|
||||
remoteMessage.payload match {
|
||||
case Left(t) ⇒ throw t
|
||||
case Right(r) ⇒ r match {
|
||||
case _: Terminate ⇒ if (useUntrustedMode) throw new SecurityException("RemoteModule server is operating is untrusted mode, can not stop the actor") else recipient.stop()
|
||||
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 ⇒ recipient.!(m)(remoteMessage.sender)
|
||||
}
|
||||
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)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue