make remote supervision and path continuation work
- add supervisor to remote USE message - make remoteDaemon a VirtualPathContainer like LocalActorRefProvider.tempContainer (i.e. synchonous with CHM-based child lookup), scrap remoteDaemonSupervisor and rename remoteDaemon to “/remote” to match the plans in the docs - comment out the remote deployment configuration section, to be done when Henrik is finished with RoutedActorRef work - for now only “remote.nodes = ["sys@host:port"]” is looked at, i.e. if at least one is present, the first one is used to determine where to deploy the currently created child (routers will do the scaling-out component) [rest is commented out] - multi-jvm tests not yet re-enabled (need to be adapted), but all other tests are GREEN (at least on my machine)
This commit is contained in:
parent
fac840adfc
commit
e5bd8b5f88
10 changed files with 391 additions and 270 deletions
|
|
@ -48,12 +48,9 @@ class RemoteActorRefProvider(
|
|||
def nodename = remoteSettings.NodeName
|
||||
def clustername = remoteSettings.ClusterName
|
||||
|
||||
private val actors = new ConcurrentHashMap[String, AnyRef]
|
||||
|
||||
val rootPath: ActorPath = RootActorPath(RemoteAddress(systemName, remoteSettings.serverSettings.Hostname, remoteSettings.serverSettings.Port))
|
||||
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, _deadLetters, rootPath)
|
||||
private var serialization: Serialization = _
|
||||
private var remoteDaemonConnectionManager: RemoteConnectionManager = _
|
||||
|
||||
private var _remote: Remote = _
|
||||
def remote = _remote
|
||||
|
|
@ -62,7 +59,7 @@ class RemoteActorRefProvider(
|
|||
local.init(system)
|
||||
serialization = SerializationExtension(system)
|
||||
_remote = new Remote(system, nodename, remoteSettings)
|
||||
remoteDaemonConnectionManager = new RemoteConnectionManager(system, remote)
|
||||
local.registerExtraNames(Map(("remote", remote.remoteDaemon)))
|
||||
terminationFuture.onComplete(_ ⇒ remote.server.shutdown())
|
||||
}
|
||||
|
||||
|
|
@ -77,90 +74,80 @@ class RemoteActorRefProvider(
|
|||
if (systemService) local.actorOf(system, props, supervisor, name, systemService)
|
||||
else {
|
||||
val path = supervisor.path / name
|
||||
val newFuture = Promise[ActorRef](system.settings.ActorTimeout)(dispatcher)
|
||||
|
||||
actors.putIfAbsent(path.toString, newFuture) match { // we won the race -- create the actor and resolve the future
|
||||
case null ⇒
|
||||
val actor: InternalActorRef = try {
|
||||
deployer.lookupDeploymentFor(path.toString) match {
|
||||
case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, RemoteDeploymentConfig.RemoteScope(remoteAddresses))) ⇒
|
||||
deployer.lookupDeploymentFor(path.elements.mkString("/", "/", "")) match {
|
||||
case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, RemoteDeploymentConfig.RemoteScope(remoteAddresses))) ⇒
|
||||
|
||||
def isReplicaNode: Boolean = remoteAddresses exists { _ == remote.remoteAddress }
|
||||
|
||||
//system.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(system.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
|
||||
|
||||
if (isReplicaNode) {
|
||||
// we are on one of the replica node for this remote actor
|
||||
local.actorOf(system, props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create (is this fixed now?)
|
||||
} else {
|
||||
|
||||
implicit val dispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher
|
||||
implicit val timeout = system.settings.ActorTimeout
|
||||
|
||||
// we are on the single "reference" node uses the remote actors on the replica nodes
|
||||
val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match {
|
||||
case RouterType.Direct ⇒
|
||||
if (remoteAddresses.size != 1) throw new ConfigurationException(
|
||||
"Actor [%s] configured with Direct router must have exactly 1 remote node configured. Found [%s]"
|
||||
.format(name, remoteAddresses.mkString(", ")))
|
||||
() ⇒ new DirectRouter
|
||||
|
||||
case RouterType.Broadcast ⇒
|
||||
if (remoteAddresses.size != 1) throw new ConfigurationException(
|
||||
"Actor [%s] configured with Broadcast router must have exactly 1 remote node configured. Found [%s]"
|
||||
.format(name, remoteAddresses.mkString(", ")))
|
||||
() ⇒ new BroadcastRouter
|
||||
|
||||
case RouterType.Random ⇒
|
||||
if (remoteAddresses.size < 1) throw new ConfigurationException(
|
||||
"Actor [%s] configured with Random router must have at least 1 remote node configured. Found [%s]"
|
||||
.format(name, remoteAddresses.mkString(", ")))
|
||||
() ⇒ new RandomRouter
|
||||
|
||||
case RouterType.RoundRobin ⇒
|
||||
if (remoteAddresses.size < 1) throw new ConfigurationException(
|
||||
"Actor [%s] configured with RoundRobin router must have at least 1 remote node configured. Found [%s]"
|
||||
.format(name, remoteAddresses.mkString(", ")))
|
||||
() ⇒ new RoundRobinRouter
|
||||
|
||||
case RouterType.ScatterGather ⇒
|
||||
if (remoteAddresses.size < 1) throw new ConfigurationException(
|
||||
"Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]"
|
||||
.format(name, remoteAddresses.mkString(", ")))
|
||||
() ⇒ new ScatterGatherFirstCompletedRouter()(dispatcher, defaultTimeout)
|
||||
|
||||
case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet")
|
||||
case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet")
|
||||
case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet")
|
||||
case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass)
|
||||
}
|
||||
|
||||
val connections = (Map.empty[RemoteAddress, ActorRef] /: remoteAddresses) { (conns, a) ⇒
|
||||
conns + (a -> new RemoteActorRef(this, remote.server, path, None)) // FIXME RK correct path must be put in here
|
||||
}
|
||||
|
||||
val connectionManager = new RemoteConnectionManager(system, remote, connections)
|
||||
|
||||
connections.keys foreach { useActorOnNode(system, _, path.toString, props.creator) }
|
||||
|
||||
actorOf(system, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
|
||||
}
|
||||
|
||||
case deploy ⇒ local.actorOf(system, props, supervisor, name, systemService)
|
||||
}
|
||||
} catch {
|
||||
case e: Exception ⇒
|
||||
newFuture completeWithException e // so the other threads gets notified of error
|
||||
throw e
|
||||
// FIXME RK deployer shall only concern itself with placement of actors on remote nodes
|
||||
val address = remoteAddresses.head
|
||||
if (address == rootPath.address) local.actorOf(system, props, supervisor, name, true) // FIXME RK make non-system
|
||||
else {
|
||||
val rpath = RootActorPath(address) / "remote" / rootPath.address.hostPort / path.elements
|
||||
useActorOnNode(rpath, props.creator, supervisor)
|
||||
new RemoteActorRef(this, remote.server, rpath, supervisor, None)
|
||||
}
|
||||
|
||||
// actor foreach system.registry.register // only for ActorRegistry backward compat, will be removed later
|
||||
|
||||
newFuture completeWithResult actor
|
||||
actors.replace(path.toString, newFuture, actor)
|
||||
actor
|
||||
case actor: InternalActorRef ⇒ actor
|
||||
case future: Future[_] ⇒ future.get.asInstanceOf[InternalActorRef]
|
||||
// def isReplicaNode: Boolean = remoteAddresses exists { _ == remote.remoteAddress }
|
||||
//
|
||||
// //system.eventHandler.debug(this, "%s: Deploy Remote Actor with address [%s] connected to [%s]: isReplica(%s)".format(system.defaultAddress, address, remoteAddresses.mkString, isReplicaNode))
|
||||
//
|
||||
// if (isReplicaNode) {
|
||||
// // we are on one of the replica node for this remote actor
|
||||
// local.actorOf(system, props, supervisor, name, true) //FIXME systemService = true here to bypass Deploy, should be fixed when create-or-get is replaced by get-or-create (is this fixed now?)
|
||||
// } else {
|
||||
//
|
||||
// implicit val dispatcher = if (props.dispatcher == Props.defaultDispatcher) system.dispatcher else props.dispatcher
|
||||
// implicit val timeout = system.settings.ActorTimeout
|
||||
//
|
||||
// // we are on the single "reference" node uses the remote actors on the replica nodes
|
||||
// val routerFactory: () ⇒ Router = DeploymentConfig.routerTypeFor(routerType) match {
|
||||
// case RouterType.Direct ⇒
|
||||
// if (remoteAddresses.size != 1) throw new ConfigurationException(
|
||||
// "Actor [%s] configured with Direct router must have exactly 1 remote node configured. Found [%s]"
|
||||
// .format(name, remoteAddresses.mkString(", ")))
|
||||
// () ⇒ new DirectRouter
|
||||
//
|
||||
// case RouterType.Broadcast ⇒
|
||||
// if (remoteAddresses.size != 1) throw new ConfigurationException(
|
||||
// "Actor [%s] configured with Broadcast router must have exactly 1 remote node configured. Found [%s]"
|
||||
// .format(name, remoteAddresses.mkString(", ")))
|
||||
// () ⇒ new BroadcastRouter
|
||||
//
|
||||
// case RouterType.Random ⇒
|
||||
// if (remoteAddresses.size < 1) throw new ConfigurationException(
|
||||
// "Actor [%s] configured with Random router must have at least 1 remote node configured. Found [%s]"
|
||||
// .format(name, remoteAddresses.mkString(", ")))
|
||||
// () ⇒ new RandomRouter
|
||||
//
|
||||
// case RouterType.RoundRobin ⇒
|
||||
// if (remoteAddresses.size < 1) throw new ConfigurationException(
|
||||
// "Actor [%s] configured with RoundRobin router must have at least 1 remote node configured. Found [%s]"
|
||||
// .format(name, remoteAddresses.mkString(", ")))
|
||||
// () ⇒ new RoundRobinRouter
|
||||
//
|
||||
// case RouterType.ScatterGather ⇒
|
||||
// if (remoteAddresses.size < 1) throw new ConfigurationException(
|
||||
// "Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]"
|
||||
// .format(name, remoteAddresses.mkString(", ")))
|
||||
// () ⇒ new ScatterGatherFirstCompletedRouter()(dispatcher, defaultTimeout)
|
||||
//
|
||||
// case RouterType.LeastCPU ⇒ sys.error("Router LeastCPU not supported yet")
|
||||
// case RouterType.LeastRAM ⇒ sys.error("Router LeastRAM not supported yet")
|
||||
// case RouterType.LeastMessages ⇒ sys.error("Router LeastMessages not supported yet")
|
||||
// case RouterType.Custom(implClass) ⇒ () ⇒ Routing.createCustomRouter(implClass)
|
||||
// }
|
||||
//
|
||||
// val connections = (Map.empty[RemoteAddress, ActorRef] /: remoteAddresses) { (conns, a) ⇒
|
||||
// conns + (a -> new RemoteActorRef(this, remote.server, path, None)) // FIXME RK correct path must be put in here
|
||||
// }
|
||||
//
|
||||
// val connectionManager = new RemoteConnectionManager(system, remote, connections)
|
||||
//
|
||||
// connections.keys foreach { useActorOnNode(system, _, path.toString, props.creator) }
|
||||
//
|
||||
// actorOf(system, RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), supervisor, name)
|
||||
// }
|
||||
case deploy ⇒ local.actorOf(system, props, supervisor, name, systemService)
|
||||
}
|
||||
}
|
||||
|
||||
|
|
@ -175,14 +162,14 @@ class RemoteActorRefProvider(
|
|||
|
||||
def actorFor(path: ActorPath): InternalActorRef = path.root match {
|
||||
case `rootPath` ⇒ actorFor(rootGuardian, path.elements)
|
||||
case RootActorPath(_: RemoteAddress, _) ⇒ new RemoteActorRef(this, remote.server, path, None)
|
||||
case RootActorPath(_: RemoteAddress, _) ⇒ new RemoteActorRef(this, remote.server, path, Nobody, None)
|
||||
case _ ⇒ local.actorFor(path)
|
||||
}
|
||||
|
||||
def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match {
|
||||
case RemoteActorPath(address, elems) ⇒
|
||||
if (address == rootPath.address) actorFor(rootGuardian, elems)
|
||||
else new RemoteActorRef(this, remote.server, new RootActorPath(address) / elems, None)
|
||||
else new RemoteActorRef(this, remote.server, new RootActorPath(address) / elems, Nobody, None)
|
||||
case _ ⇒ local.actorFor(ref, path)
|
||||
}
|
||||
|
||||
|
|
@ -192,16 +179,11 @@ class RemoteActorRefProvider(
|
|||
val optimizeLocal = new AtomicBoolean(true)
|
||||
def optimizeLocalScoped_?() = optimizeLocal.get
|
||||
|
||||
/**
|
||||
* Returns true if the actor was in the provider's cache and evicted successfully, else false.
|
||||
*/
|
||||
private[akka] def evict(path: ActorPath): Boolean = actors.remove(path) ne null
|
||||
|
||||
/**
|
||||
* Using (checking out) actor on a specific node.
|
||||
*/
|
||||
def useActorOnNode(system: ActorSystem, remoteAddress: RemoteAddress, actorPath: String, actorFactory: () ⇒ Actor) {
|
||||
log.debug("[{}] Instantiating Actor [{}] on node [{}]", rootPath, actorPath, remoteAddress)
|
||||
def useActorOnNode(path: ActorPath, actorFactory: () ⇒ Actor, supervisor: ActorRef) {
|
||||
log.debug("[{}] Instantiating Remote Actor [{}]", rootPath, path)
|
||||
|
||||
val actorFactoryBytes =
|
||||
serialization.serialize(actorFactory) match {
|
||||
|
|
@ -211,16 +193,13 @@ class RemoteActorRefProvider(
|
|||
|
||||
val command = RemoteSystemDaemonMessageProtocol.newBuilder
|
||||
.setMessageType(USE)
|
||||
.setActorPath(actorPath)
|
||||
.setActorPath(path.toString)
|
||||
.setPayload(ByteString.copyFrom(actorFactoryBytes))
|
||||
.setSupervisor(supervisor.path.toString)
|
||||
.build()
|
||||
|
||||
val connectionFactory = () ⇒ actorFor(RootActorPath(remoteAddress) / remote.remoteDaemon.path.elements)
|
||||
|
||||
// try to get the connection for the remote address, if not already there then create it
|
||||
val connection = remoteDaemonConnectionManager.putIfAbsent(remoteAddress, connectionFactory)
|
||||
|
||||
sendCommandToRemoteNode(connection, command, withACK = true) // ensure we get an ACK on the USE command
|
||||
// we don’t wait for the ACK, because the remote end will process this command before any other message to the new actor
|
||||
actorFor(RootActorPath(path.address) / "remote") ! command
|
||||
}
|
||||
|
||||
private def sendCommandToRemoteNode(connection: ActorRef, command: RemoteSystemDaemonMessageProtocol, withACK: Boolean) {
|
||||
|
|
@ -265,12 +244,13 @@ private[akka] class RemoteActorRef private[akka] (
|
|||
provider: ActorRefProvider,
|
||||
remote: RemoteSupport,
|
||||
val path: ActorPath,
|
||||
val getParent: InternalActorRef,
|
||||
loader: Option[ClassLoader])
|
||||
extends InternalActorRef {
|
||||
|
||||
// FIXME RK
|
||||
def getParent = Nobody
|
||||
def getChild(name: Iterator[String]) = Nobody
|
||||
def getChild(name: Iterator[String]): InternalActorRef = {
|
||||
new RemoteActorRef(provider, remote, path / name.toStream, Nobody, loader)
|
||||
}
|
||||
|
||||
@volatile
|
||||
private var running: Boolean = true
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue