Reliable remote supervision and death watch, see #2993

* RemoteWatcher that monitors node failures, with heartbeats
  and failure detector
* Move RemoteDeploymentWatcher from CARP to RARP
* ClusterRemoteWatcher that handles cluster nodes
* Update documentation
* UID in Heartbeat msg to be able to quarantine,
  actual implementation of quarantining will be implemented
  in ticket 2594
This commit is contained in:
Patrik Nordwall 2013-04-15 09:26:51 +02:00
parent d230e80eeb
commit 4606612bd1
22 changed files with 1699 additions and 187 deletions

View file

@ -14,6 +14,8 @@ import scala.util.control.NonFatal
import akka.actor.SystemGuardian.{ TerminationHookDone, TerminationHook, RegisterTerminationHook }
import scala.util.control.Exception.Catcher
import scala.concurrent.{ ExecutionContext, Future }
import com.typesafe.config.Config
import akka.ConfigurationException
/**
* INTERNAL API
@ -133,21 +135,22 @@ private[akka] class RemoteActorRefProvider(
override def tempPath(): ActorPath = local.tempPath()
override def tempContainer: VirtualPathContainer = local.tempContainer
@volatile
private var _internals: Internals = _
@volatile private var _internals: Internals = _
def transport: RemoteTransport = _internals.transport
def serialization: Serialization = _internals.serialization
def remoteDaemon: InternalActorRef = _internals.remoteDaemon
// This actor ensures the ordering of shutdown between remoteDaemon and the transport
@volatile
private var remotingTerminator: ActorRef = _
@volatile private var remotingTerminator: ActorRef = _
@volatile private var remoteWatcher: ActorRef = _
@volatile private var remoteDeploymentWatcher: ActorRef = _
def init(system: ActorSystemImpl): Unit = {
local.init(system)
remotingTerminator = system.systemActorOf(Props(new RemotingTerminator(local.systemGuardian)), "remoting-terminator")
remotingTerminator = system.systemActorOf(Props(classOf[RemotingTerminator], local.systemGuardian), "remoting-terminator")
val internals = Internals(
remoteDaemon = {
@ -172,8 +175,37 @@ private[akka] class RemoteActorRefProvider(
// this enables reception of remote requests
transport.start()
remoteWatcher = createRemoteWatcher(system)
remoteDeploymentWatcher = createRemoteDeploymentWatcher(system)
}
protected def createRemoteWatcher(system: ActorSystemImpl): ActorRef = {
import remoteSettings._
val failureDetector = createRemoteWatcherFailureDetector(system)
system.systemActorOf(RemoteWatcher.props(
failureDetector,
heartbeatInterval = WatchHeartBeatInterval,
unreachableReaperInterval = WatchUnreachableReaperInterval,
heartbeatExpectedResponseAfter = WatchHeartbeatExpectedResponseAfter,
numberOfEndHeartbeatRequests = WatchNumberOfEndHeartbeatRequests), "remote-watcher")
}
protected def createRemoteWatcherFailureDetector(system: ExtendedActorSystem): FailureDetectorRegistry[Address] = {
def createFailureDetector(): FailureDetector = {
import remoteSettings.{ WatchFailureDetectorImplementationClass fqcn }
system.dynamicAccess.createInstanceFor[FailureDetector](
fqcn, List(classOf[Config] -> remoteSettings.WatchFailureDetectorConfig)).recover({
case e throw new ConfigurationException(
s"Could not create custom remote watcher failure detector [$fqcn] due to: ${e.toString}", e)
}).get
}
new DefaultFailureDetectorRegistry(() createFailureDetector())
}
protected def createRemoteDeploymentWatcher(system: ActorSystemImpl): ActorRef =
system.systemActorOf(Props[RemoteDeploymentWatcher], "remote-deployment-watcher")
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath,
systemService: Boolean, deploy: Option[Deploy], lookupDeploy: Boolean, async: Boolean): InternalActorRef = {
if (systemService) local.actorOf(system, props, supervisor, path, systemService, deploy, lookupDeploy, async)
@ -352,6 +384,8 @@ private[akka] class RemoteActorRefProvider(
// before someone can send messages to it
resolveActorRef(RootActorPath(ref.path.address) / "remote") !
DaemonMsgCreate(props, deploy, ref.path.toSerializationFormat, supervisor)
remoteDeploymentWatcher ! RemoteDeploymentWatcher.WatchRemote(ref, supervisor)
}
def getExternalAddressFor(addr: Address): Option[Address] = {
@ -367,6 +401,22 @@ private[akka] class RemoteActorRefProvider(
private def hasAddress(address: Address): Boolean =
address == local.rootPath.address || address == rootPath.address || transport.addresses(address)
def quarantine(address: Address, uid: Int): Unit = {
// FIXME send to EndpointManager
}
/**
* INTERNAL API
*/
private[akka] def afterSendSystemMessage(message: SystemMessage): Unit =
message match {
// Sending to local remoteWatcher relies strong delivery guarantees of local send, i.e.
// default dispatcher must not be changed to an implementation that defeats that
case Watch(watchee, watcher) remoteWatcher ! RemoteWatcher.WatchRemote(watchee, watcher)
case Unwatch(watchee, watcher) remoteWatcher ! RemoteWatcher.UnwatchRemote(watchee, watcher)
case _
}
}
private[akka] trait RemoteRef extends ActorRefScope {
@ -406,7 +456,11 @@ private[akka] class RemoteActorRef private[akka] (
remote.system.eventStream.publish(Error(e, path.toString, getClass, "swallowing exception during message send"))
}
def sendSystemMessage(message: SystemMessage): Unit = try remote.send(message, None, this) catch handleException
def sendSystemMessage(message: SystemMessage): Unit =
try {
remote.send(message, None, this)
provider.afterSendSystemMessage(message)
} catch handleException
override def !(message: Any)(implicit sender: ActorRef = Actor.noSender): Unit = {
if (message == null) throw new InvalidMessageException("Message is null")