This commit is contained in:
Peter Veentjer 2011-08-06 23:08:01 +03:00
parent 02aeec6b57
commit aaec3aef77
13 changed files with 439 additions and 302 deletions

View file

@ -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&#233;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