ticket #992
This commit is contained in:
parent
02aeec6b57
commit
aaec3aef77
13 changed files with 439 additions and 302 deletions
|
|
@ -1758,6 +1758,8 @@ object RemoteClusterDaemon {
|
|||
/**
|
||||
* Internal "daemon" actor for cluster internal communication.
|
||||
*
|
||||
* It acts as the brain of the cluster that responds to cluster events (messages) and undertakes action.
|
||||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
|
||||
|
|
@ -1774,185 +1776,207 @@ class RemoteClusterDaemon(cluster: ClusterNode) extends Actor {
|
|||
def receive: Receive = {
|
||||
case message: RemoteDaemonMessageProtocol ⇒
|
||||
EventHandler.debug(this,
|
||||
"Received command [\n%s] to RemoteClusterDaemon on node [%s]"
|
||||
.format(message, cluster.nodeAddress.nodeName))
|
||||
"Received command [\n%s] to RemoteClusterDaemon on node [%s]".format(message, cluster.nodeAddress.nodeName))
|
||||
|
||||
message.getMessageType match {
|
||||
|
||||
case USE ⇒
|
||||
try {
|
||||
if (message.hasActorAddress) {
|
||||
val actorAddress = message.getActorAddress
|
||||
cluster.serializerForActor(actorAddress) foreach { serializer ⇒
|
||||
cluster.use(actorAddress, serializer) foreach { newActorRef ⇒
|
||||
cluster.remoteService.register(actorAddress, newActorRef)
|
||||
|
||||
if (message.hasReplicateActorFromUuid) {
|
||||
// replication is used - fetch the messages and replay them
|
||||
import akka.remote.protocol.RemoteProtocol._
|
||||
import akka.remote.MessageSerializer
|
||||
|
||||
val replicateFromUuid = uuidProtocolToUuid(message.getReplicateActorFromUuid)
|
||||
val deployment = Deployer.deploymentFor(actorAddress)
|
||||
val replicationScheme = DeploymentConfig.replicationSchemeFor(deployment).getOrElse(
|
||||
throw new IllegalStateException(
|
||||
"Actor [" + actorAddress + "] should have been configured as a replicated actor but could not find its ReplicationScheme"))
|
||||
val isWriteBehind = DeploymentConfig.isWriteBehindReplication(replicationScheme)
|
||||
|
||||
try {
|
||||
// get the transaction log for the actor UUID
|
||||
val txLog = TransactionLog.logFor(replicateFromUuid.toString, isWriteBehind, replicationScheme)
|
||||
|
||||
// get the latest snapshot (Option[Array[Byte]]) and all the subsequent messages (Array[Byte])
|
||||
val (snapshotAsBytes, entriesAsBytes) = txLog.latestSnapshotAndSubsequentEntries
|
||||
|
||||
// deserialize and restore actor snapshot
|
||||
val actorRefToUseForReplay =
|
||||
snapshotAsBytes match {
|
||||
|
||||
// we have a new actor ref - the snapshot
|
||||
case Some(bytes) ⇒
|
||||
// stop the new actor ref and use the snapshot instead
|
||||
cluster.remoteService.unregister(actorAddress)
|
||||
|
||||
// deserialize the snapshot actor ref and register it as remote actor
|
||||
val uncompressedBytes =
|
||||
if (Cluster.shouldCompressData) LZF.uncompress(bytes)
|
||||
else bytes
|
||||
|
||||
val snapshotActorRef = fromBinary(uncompressedBytes, newActorRef.uuid).start()
|
||||
cluster.remoteService.register(actorAddress, snapshotActorRef)
|
||||
|
||||
// FIXME we should call 'stop()' here (to GC the actor), but can't since that will currently shut down the TransactionLog for this UUID - since both this actor and the new snapshotActorRef have the same UUID (which they should)
|
||||
//newActorRef.stop()
|
||||
|
||||
snapshotActorRef
|
||||
|
||||
// we have no snapshot - use the new actor ref
|
||||
case None ⇒
|
||||
newActorRef
|
||||
}
|
||||
|
||||
// deserialize the messages
|
||||
val messages: Vector[AnyRef] = entriesAsBytes map { bytes ⇒
|
||||
val messageBytes =
|
||||
if (Cluster.shouldCompressData) LZF.uncompress(bytes)
|
||||
else bytes
|
||||
MessageSerializer.deserialize(MessageProtocol.parseFrom(messageBytes), None)
|
||||
}
|
||||
|
||||
EventHandler.info(this, "Replaying [%s] messages to actor [%s]".format(messages.size, actorAddress))
|
||||
|
||||
// replay all messages
|
||||
messages foreach { message ⇒
|
||||
EventHandler.debug(this, "Replaying message [%s] to actor [%s]".format(message, actorAddress))
|
||||
|
||||
// FIXME how to handle '?' messages? We can *not* replay them with the correct semantics. Should we: 1. Ignore/drop them and log warning? 2. Throw exception when about to log them? 3. Other?
|
||||
actorRefToUseForReplay ! message
|
||||
}
|
||||
|
||||
} catch {
|
||||
case e: Throwable ⇒
|
||||
EventHandler.error(e, this, e.toString)
|
||||
throw e
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
EventHandler.error(this, "Actor 'address' is not defined, ignoring remote cluster daemon command [%s]".format(message))
|
||||
}
|
||||
self.reply(Success)
|
||||
|
||||
} catch {
|
||||
case error ⇒
|
||||
self.reply(Failure(error))
|
||||
throw error
|
||||
}
|
||||
|
||||
case RELEASE ⇒
|
||||
if (message.hasActorUuid) {
|
||||
cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address ⇒
|
||||
cluster.release(address)
|
||||
}
|
||||
} else if (message.hasActorAddress) {
|
||||
cluster release message.getActorAddress
|
||||
} else {
|
||||
EventHandler.warning(this,
|
||||
"None of 'uuid' or 'actorAddress'' is specified, ignoring remote cluster daemon command [%s]"
|
||||
.format(message))
|
||||
}
|
||||
|
||||
case START ⇒ cluster.start()
|
||||
|
||||
case STOP ⇒ cluster.shutdown()
|
||||
|
||||
case DISCONNECT ⇒ cluster.disconnect()
|
||||
|
||||
case RECONNECT ⇒ cluster.reconnect()
|
||||
|
||||
case RESIGN ⇒ cluster.resign()
|
||||
|
||||
case FAIL_OVER_CONNECTIONS ⇒
|
||||
val (from, to) = payloadFor(message, classOf[(InetSocketAddress, InetSocketAddress)])
|
||||
cluster.failOverClusterActorRefConnections(from, to)
|
||||
|
||||
case FUNCTION_FUN0_UNIT ⇒
|
||||
localActorOf(new Actor() {
|
||||
self.dispatcher = computeGridDispatcher
|
||||
|
||||
def receive = {
|
||||
case f: Function0[_] ⇒ try {
|
||||
f()
|
||||
} finally {
|
||||
self.stop()
|
||||
}
|
||||
}
|
||||
}).start ! payloadFor(message, classOf[Function0[Unit]])
|
||||
|
||||
case FUNCTION_FUN0_ANY ⇒
|
||||
localActorOf(new Actor() {
|
||||
self.dispatcher = computeGridDispatcher
|
||||
|
||||
def receive = {
|
||||
case f: Function0[_] ⇒ try {
|
||||
self.reply(f())
|
||||
} finally {
|
||||
self.stop()
|
||||
}
|
||||
}
|
||||
}).start forward payloadFor(message, classOf[Function0[Any]])
|
||||
|
||||
case FUNCTION_FUN1_ARG_UNIT ⇒
|
||||
localActorOf(new Actor() {
|
||||
self.dispatcher = computeGridDispatcher
|
||||
|
||||
def receive = {
|
||||
case (fun: Function[_, _], param: Any) ⇒ try {
|
||||
fun.asInstanceOf[Any ⇒ Unit].apply(param)
|
||||
} finally {
|
||||
self.stop()
|
||||
}
|
||||
}
|
||||
}).start ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
|
||||
|
||||
case FUNCTION_FUN1_ARG_ANY ⇒
|
||||
localActorOf(new Actor() {
|
||||
self.dispatcher = computeGridDispatcher
|
||||
|
||||
def receive = {
|
||||
case (fun: Function[_, _], param: Any) ⇒ try {
|
||||
self.reply(fun.asInstanceOf[Any ⇒ Any](param))
|
||||
} finally {
|
||||
self.stop()
|
||||
}
|
||||
}
|
||||
}).start forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
|
||||
case USE ⇒ handleUse(message)
|
||||
case RELEASE ⇒ handleRelease(message)
|
||||
case START ⇒ cluster.start()
|
||||
case STOP ⇒ cluster.shutdown()
|
||||
case DISCONNECT ⇒ cluster.disconnect()
|
||||
case RECONNECT ⇒ cluster.reconnect()
|
||||
case RESIGN ⇒ cluster.resign()
|
||||
case FAIL_OVER_CONNECTIONS ⇒ handleFailover(message)
|
||||
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?
|
||||
}
|
||||
|
||||
case unknown ⇒ EventHandler.warning(this, "Unknown message [%s]".format(unknown))
|
||||
}
|
||||
|
||||
def handleRelease(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
|
||||
if (message.hasActorUuid) {
|
||||
cluster.actorAddressForUuid(uuidProtocolToUuid(message.getActorUuid)) foreach { address ⇒
|
||||
cluster.release(address)
|
||||
}
|
||||
} else if (message.hasActorAddress) {
|
||||
cluster release message.getActorAddress
|
||||
} else {
|
||||
EventHandler.warning(this,
|
||||
"None of 'uuid' or 'actorAddress'' is specified, ignoring remote cluster daemon command [%s]".format(message))
|
||||
}
|
||||
}
|
||||
|
||||
def handleUse(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
|
||||
def deserializeMessages(entriesAsBytes: Vector[Array[Byte]]): Vector[AnyRef] = {
|
||||
import akka.remote.protocol.RemoteProtocol._
|
||||
import akka.remote.MessageSerializer
|
||||
|
||||
entriesAsBytes map { bytes ⇒
|
||||
val messageBytes =
|
||||
if (Cluster.shouldCompressData) LZF.uncompress(bytes)
|
||||
else bytes
|
||||
MessageSerializer.deserialize(MessageProtocol.parseFrom(messageBytes), None)
|
||||
}
|
||||
}
|
||||
|
||||
def createActorRefToUseForReplay(snapshotAsBytes: Option[Array[Byte]], actorAddress: String, newActorRef: LocalActorRef): ActorRef = {
|
||||
snapshotAsBytes match {
|
||||
|
||||
// we have a new actor ref - the snapshot
|
||||
case Some(bytes) ⇒
|
||||
// stop the new actor ref and use the snapshot instead
|
||||
//TODO: What if that actor already has been retrieved and is being used??
|
||||
//So do we have a race here?
|
||||
cluster.remoteService.unregister(actorAddress)
|
||||
|
||||
// deserialize the snapshot actor ref and register it as remote actor
|
||||
val uncompressedBytes =
|
||||
if (Cluster.shouldCompressData) LZF.uncompress(bytes)
|
||||
else bytes
|
||||
|
||||
val snapshotActorRef = fromBinary(uncompressedBytes, newActorRef.uuid).start()
|
||||
cluster.remoteService.register(actorAddress, snapshotActorRef)
|
||||
|
||||
// FIXME we should call 'stop()' here (to GC the actor), but can't since that will currently
|
||||
//shut down the TransactionLog for this UUID - since both this actor and the new snapshotActorRef
|
||||
//have the same UUID (which they should)
|
||||
//newActorRef.stop()
|
||||
|
||||
snapshotActorRef
|
||||
|
||||
// we have no snapshot - use the new actor ref
|
||||
case None ⇒
|
||||
newActorRef
|
||||
}
|
||||
}
|
||||
|
||||
try {
|
||||
if (message.hasActorAddress) {
|
||||
val actorAddress = message.getActorAddress
|
||||
cluster.serializerForActor(actorAddress) foreach { serializer ⇒
|
||||
cluster.use(actorAddress, serializer) foreach { newActorRef ⇒
|
||||
cluster.remoteService.register(actorAddress, newActorRef)
|
||||
|
||||
if (message.hasReplicateActorFromUuid) {
|
||||
// replication is used - fetch the messages and replay them
|
||||
val replicateFromUuid = uuidProtocolToUuid(message.getReplicateActorFromUuid)
|
||||
val deployment = Deployer.deploymentFor(actorAddress)
|
||||
val replicationScheme = DeploymentConfig.replicationSchemeFor(deployment).getOrElse(
|
||||
throw new IllegalStateException(
|
||||
"Actor [" + actorAddress + "] should have been configured as a replicated actor but could not find its ReplicationScheme"))
|
||||
val isWriteBehind = DeploymentConfig.isWriteBehindReplication(replicationScheme)
|
||||
|
||||
try {
|
||||
// get the transaction log for the actor UUID
|
||||
val readonlyTxLog = TransactionLog.logFor(replicateFromUuid.toString, isWriteBehind, replicationScheme)
|
||||
|
||||
// get the latest snapshot (Option[Array[Byte]]) and all the subsequent messages (Array[Byte])
|
||||
val (snapshotAsBytes, entriesAsBytes) = readonlyTxLog.latestSnapshotAndSubsequentEntries
|
||||
|
||||
// deserialize and restore actor snapshot. This call will automatically recreate a transaction log.
|
||||
val actorRef = createActorRefToUseForReplay(snapshotAsBytes, actorAddress, newActorRef)
|
||||
|
||||
// deserialize the messages
|
||||
val messages: Vector[AnyRef] = deserializeMessages(entriesAsBytes)
|
||||
|
||||
EventHandler.info(this, "Replaying [%s] messages to actor [%s]".format(messages.size, actorAddress))
|
||||
|
||||
// replay all messages
|
||||
messages foreach { message ⇒
|
||||
EventHandler.debug(this, "Replaying message [%s] to actor [%s]".format(message, actorAddress))
|
||||
|
||||
// FIXME how to handle '?' messages?
|
||||
// We can *not* replay them with the correct semantics. Should we:
|
||||
// 1. Ignore/drop them and log warning?
|
||||
// 2. Throw exception when about to log them?
|
||||
// 3. Other?
|
||||
actorRef ! message
|
||||
}
|
||||
|
||||
} catch {
|
||||
case e: Throwable ⇒
|
||||
EventHandler.error(e, this, e.toString)
|
||||
throw e
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
} else {
|
||||
EventHandler.error(this, "Actor 'address' is not defined, ignoring remote cluster daemon command [%s]".format(message))
|
||||
}
|
||||
self.reply(Success)
|
||||
} catch {
|
||||
case error ⇒
|
||||
self.reply(Failure(error))
|
||||
throw error
|
||||
}
|
||||
}
|
||||
|
||||
def handle_fun0_unit(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
|
||||
localActorOf(new Actor() {
|
||||
self.dispatcher = computeGridDispatcher
|
||||
|
||||
def receive = {
|
||||
case f: Function0[_] ⇒ try {
|
||||
f()
|
||||
} finally {
|
||||
self.stop()
|
||||
}
|
||||
}
|
||||
}).start ! payloadFor(message, classOf[Function0[Unit]])
|
||||
}
|
||||
|
||||
def handle_fun0_any(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
|
||||
localActorOf(new Actor() {
|
||||
self.dispatcher = computeGridDispatcher
|
||||
|
||||
def receive = {
|
||||
case f: Function0[_] ⇒ try {
|
||||
self.reply(f())
|
||||
} finally {
|
||||
self.stop()
|
||||
}
|
||||
}
|
||||
}).start forward payloadFor(message, classOf[Function0[Any]])
|
||||
}
|
||||
|
||||
def handle_fun1_arg_unit(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
|
||||
localActorOf(new Actor() {
|
||||
self.dispatcher = computeGridDispatcher
|
||||
|
||||
def receive = {
|
||||
case (fun: Function[_, _], param: Any) ⇒ try {
|
||||
fun.asInstanceOf[Any ⇒ Unit].apply(param)
|
||||
} finally {
|
||||
self.stop()
|
||||
}
|
||||
}
|
||||
}).start ! payloadFor(message, classOf[Tuple2[Function1[Any, Unit], Any]])
|
||||
}
|
||||
|
||||
def handle_fun1_arg_any(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
|
||||
localActorOf(new Actor() {
|
||||
self.dispatcher = computeGridDispatcher
|
||||
|
||||
def receive = {
|
||||
case (fun: Function[_, _], param: Any) ⇒ try {
|
||||
self.reply(fun.asInstanceOf[Any ⇒ Any](param))
|
||||
} finally {
|
||||
self.stop()
|
||||
}
|
||||
}
|
||||
}).start forward payloadFor(message, classOf[Tuple2[Function1[Any, Any], Any]])
|
||||
}
|
||||
|
||||
def handleFailover(message: ClusterProtocol.RemoteDaemonMessageProtocol) {
|
||||
val (from, to) = payloadFor(message, classOf[(InetSocketAddress, InetSocketAddress)])
|
||||
cluster.failOverClusterActorRefConnections(from, to)
|
||||
}
|
||||
|
||||
private def payloadFor[T](message: RemoteDaemonMessageProtocol, clazz: Class[T]): T = {
|
||||
Serialization.deserialize(message.getPayload.toByteArray, clazz, None) match {
|
||||
case Left(error) ⇒ throw error
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue