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-10-12 09:10:05 +02:00
|
|
|
import akka.AkkaApplication
|
2011-09-15 10:20:18 +02:00
|
|
|
import akka.actor._
|
|
|
|
|
import akka.event.EventHandler
|
|
|
|
|
import akka.dispatch.{ Dispatchers, Future, PinnedDispatcher }
|
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-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-10-12 09:10:05 +02:00
|
|
|
class Remote(val app: AkkaApplication) extends RemoteService {
|
|
|
|
|
|
2011-10-19 12:25:16 +02:00
|
|
|
import app._
|
2011-10-12 09:10:05 +02:00
|
|
|
import app.config
|
2011-10-19 12:25:16 +02:00
|
|
|
import app.AkkaConfig._
|
2011-10-12 09:10:05 +02:00
|
|
|
|
2011-10-19 12:25:16 +02:00
|
|
|
// TODO move to AkkaConfig?
|
2011-09-19 15:21:18 +02:00
|
|
|
val shouldCompressData = config.getBool("akka.remote.use-compression", false)
|
2011-10-19 12:25:16 +02:00
|
|
|
val remoteSystemDaemonAckTimeout = Duration(config.getInt("akka.remote.remote-daemon-ack-timeout", 30), DefaultTimeUnit).toMillis.toInt
|
2011-09-15 10:20:18 +02:00
|
|
|
|
2011-10-19 12:25:16 +02:00
|
|
|
val failureDetector = new AccrualFailureDetector(FailureDetectorThreshold, FailureDetectorMaxSampleSize)
|
|
|
|
|
|
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
|
|
|
|
|
|
|
|
// FIXME configure computeGridDispatcher to what?
|
2011-10-19 12:25:16 +02:00
|
|
|
val computeGridDispatcher = dispatcherFactory.newDispatcher("akka:compute-grid").build
|
2011-09-15 10:20:18 +02:00
|
|
|
|
2011-10-18 17:56:23 +02:00
|
|
|
private[remote] lazy val remoteDaemonSupervisor = app.actorOf(Props(
|
2011-10-12 09:10:05 +02:00
|
|
|
OneForOneStrategy(List(classOf[Exception]), None, None))) // is infinite restart what we want?
|
2011-09-30 14:52:07 +02:00
|
|
|
|
|
|
|
|
private[remote] lazy val remoteDaemon =
|
2011-11-03 14:53:38 +01:00
|
|
|
app.provider.actorOf(
|
2011-10-28 23:11:35 +02:00
|
|
|
Props(new RemoteSystemDaemon(this)).withDispatcher(dispatcherFactory.newPinnedDispatcher(remoteDaemonServiceName)),
|
2011-10-21 15:11:43 +02: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-10-18 17:56:23 +02:00
|
|
|
private[remote] lazy val remoteClientLifeCycleHandler = app.actorOf(Props(new Actor {
|
2011-09-15 10:20:18 +02:00
|
|
|
def receive = {
|
|
|
|
|
case RemoteClientError(cause, client, address) ⇒ client.shutdownClientModule()
|
|
|
|
|
case RemoteClientDisconnected(client, address) ⇒ client.shutdownClientModule()
|
|
|
|
|
case _ ⇒ //ignore other
|
|
|
|
|
}
|
2011-10-19 12:25:16 +02:00
|
|
|
}), "akka.remote.RemoteClientLifeCycleListener")
|
2011-10-12 11:34:35 +02:00
|
|
|
|
2011-10-12 09:10:05 +02:00
|
|
|
lazy val eventStream = new NetworkEventStream(app)
|
2011-09-15 10:20:18 +02:00
|
|
|
|
|
|
|
|
lazy val server: RemoteSupport = {
|
2011-10-12 09:10:05 +02:00
|
|
|
val remote = new akka.remote.netty.NettyRemoteSupport(app)
|
2011-09-15 10:20:18 +02:00
|
|
|
remote.start(hostname, port)
|
2011-10-22 16:06:20 +02:00
|
|
|
|
|
|
|
|
app.eventHandler.addListener(eventStream.sender)
|
2011-10-23 00:40:09 +02:00
|
|
|
app.eventHandler.addListener(remoteClientLifeCycleHandler)
|
2011-10-22 16:06:20 +02:00
|
|
|
|
2011-10-13 14:23:44 +02:00
|
|
|
// TODO actually register this provider in app in remote mode
|
2011-10-19 12:25:16 +02:00
|
|
|
//provider.register(ActorRefProvider.RemoteProvider, new RemoteActorRefProvider)
|
2011-09-15 10:20:18 +02:00
|
|
|
remote
|
|
|
|
|
}
|
|
|
|
|
|
2011-11-03 14:53:38 +01:00
|
|
|
def start(): Unit = {
|
|
|
|
|
val serverAddress = server.app.defaultAddress //Force init of server
|
|
|
|
|
val daemonAddress = remoteDaemon.address //Force init of daemon
|
|
|
|
|
eventHandler.info(this, "Starting remote server on [%s] and starting remoteDaemon with address [%s]".format(serverAddress, daemonAddress))
|
2011-09-22 03:36:59 +02:00
|
|
|
}
|
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-10-19 12:25:16 +02:00
|
|
|
import remote.app._
|
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-19 12:25:16 +02:00
|
|
|
eventHandler.debug(this, "RemoteSystemDaemon failed due to [%s] - restarting...".format(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-03 14:53:38 +01:00
|
|
|
eventHandler.debug(this, "Received command [\n%s] to RemoteSystemDaemon on [%s]".format(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-19 12:25:16 +02:00
|
|
|
case unknown ⇒ eventHandler.warning(this, "Unknown message to RemoteSystemDaemon [%s]".format(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 {
|
|
|
|
|
if (message.hasActorAddress) {
|
2011-09-19 14:43:28 +02:00
|
|
|
|
|
|
|
|
val actorFactoryBytes =
|
2011-10-28 23:11:35 +02:00
|
|
|
if (shouldCompressData) LZF.uncompress(message.getPayload.toByteArray) else message.getPayload.toByteArray
|
2011-09-19 14:43:28 +02:00
|
|
|
|
|
|
|
|
val actorFactory =
|
2011-10-19 12:25:16 +02: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-03 14:53:38 +01:00
|
|
|
app.actorOf(Props(creator = actorFactory), message.getActorAddress)
|
2011-09-15 10:20:18 +02:00
|
|
|
} else {
|
2011-10-19 12:25:16 +02:00
|
|
|
eventHandler.error(this, "Actor 'address' for actor to instantiate is not defined, ignoring remote system daemon command [%s]".format(message))
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-11-03 14:53:38 +01:00
|
|
|
sender ! Success(app.defaultAddress)
|
2011-09-15 10:20:18 +02:00
|
|
|
} catch {
|
2011-10-28 23:11:35 +02:00
|
|
|
case error: Throwable ⇒ //FIXME doesn't seem sensible
|
2011-10-22 16:06:20 +02:00
|
|
|
sender ! Failure(error)
|
2011-09-15 10:20:18 +02:00
|
|
|
throw error
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
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-10-18 15:39:26 +02:00
|
|
|
// FIXME: handle real remote supervision
|
2011-10-19 12:25:16 +02:00
|
|
|
def handle_fun0_unit(message: RemoteSystemDaemonMessageProtocol) {
|
2011-10-12 09:10:05 +02:00
|
|
|
new LocalActorRef(app,
|
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-10-18 15:39:26 +02:00
|
|
|
}).copy(dispatcher = computeGridDispatcher), app.guardian, Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Function0[Unit]])
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-18 15:39:26 +02:00
|
|
|
// FIXME: handle real remote supervision
|
2011-10-19 12:25:16 +02:00
|
|
|
def handle_fun0_any(message: RemoteSystemDaemonMessageProtocol) {
|
2011-10-12 09:10:05 +02:00
|
|
|
new LocalActorRef(app,
|
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-10-18 15:39:26 +02:00
|
|
|
}).copy(dispatcher = computeGridDispatcher), app.guardian, Props.randomAddress, systemService = true) forward payloadFor(message, classOf[Function0[Any]])
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-18 15:39:26 +02:00
|
|
|
// FIXME: handle real remote supervision
|
2011-10-19 12:25:16 +02:00
|
|
|
def handle_fun1_arg_unit(message: RemoteSystemDaemonMessageProtocol) {
|
2011-10-12 09:10:05 +02:00
|
|
|
new LocalActorRef(app,
|
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-10-18 15:39:26 +02:00
|
|
|
}).copy(dispatcher = computeGridDispatcher), app.guardian, Props.randomAddress, systemService = true) ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
|
2011-09-15 10:20:18 +02:00
|
|
|
}
|
|
|
|
|
|
2011-10-18 15:39:26 +02:00
|
|
|
// FIXME: handle real remote supervision
|
2011-10-19 12:25:16 +02:00
|
|
|
def handle_fun1_arg_any(message: RemoteSystemDaemonMessageProtocol) {
|
2011-10-12 09:10:05 +02:00
|
|
|
new LocalActorRef(app,
|
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-10-18 15:39:26 +02:00
|
|
|
}).copy(dispatcher = computeGridDispatcher), app.guardian, Props.randomAddress, 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 = {
|
|
|
|
|
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) {
|
|
|
|
|
lazy val sender: ActorRef =
|
|
|
|
|
if (input.hasSender)
|
|
|
|
|
remote.app.provider.deserialize(
|
|
|
|
|
SerializedActorRef(input.getSender.getAddress, input.getSender.getHost, input.getSender.getPort)).getOrElse(throw new IllegalStateException("OHNOES"))
|
|
|
|
|
else
|
|
|
|
|
remote.app.deadLetters
|
|
|
|
|
lazy val recipient: ActorRef = remote.app.findActor(input.getRecipient.getAddress) match {
|
|
|
|
|
case None ⇒ remote.app.deadLetters
|
|
|
|
|
case Some(target) ⇒ target
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
lazy val payload: Either[Throwable, AnyRef] =
|
|
|
|
|
if (input.hasException) Left(parseException())
|
|
|
|
|
else Right(MessageSerializer.deserialize(remote.app, 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.app.eventHandler.error(problem, remote, problem.getMessage)
|
|
|
|
|
CannotInstantiateRemoteExceptionDueToRemoteProtocolParsingErrorException(problem, classname, exception.getMessage)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
override def toString = "RemoteMessage: " + recipient + "(" + input.getRecipient.getAddress + ") from " + sender
|
|
|
|
|
}
|