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:
parent
d230e80eeb
commit
4606612bd1
22 changed files with 1699 additions and 187 deletions
|
|
@ -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")
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue