restructure remoting
- remove Remote, incorporate its few fields into RemoteActorRefProvider - rename RemoteSupport to RemoteTransport to sync up with conf and the way we talk about it - remove LocalAddress/RemoteAddress etc. and just have a final case class Address(protocol, system, host, port) - split netty settings out or RemoteSettings into NettySettings - split out from NettyRemoteSupport.scala: Server.scala, Client.scala, Settings.scala plus a few fixes, including using the contextClassLoader when loading the provider for ActorSystemImpl
This commit is contained in:
parent
6db3e59ce1
commit
edceda8edf
34 changed files with 1415 additions and 1520 deletions
|
|
@ -16,6 +16,10 @@ import akka.event.EventStream
|
|||
import akka.dispatch.Promise
|
||||
import akka.config.ConfigurationException
|
||||
import java.util.concurrent.{ TimeoutException }
|
||||
import com.typesafe.config.Config
|
||||
import akka.util.ReflectiveAccess
|
||||
import akka.serialization.Serialization
|
||||
import akka.serialization.SerializationExtension
|
||||
|
||||
/**
|
||||
* Remote ActorRefProvider. Starts up actor on remote node and creates a RemoteActorRef representing it.
|
||||
|
|
@ -37,22 +41,63 @@ class RemoteActorRefProvider(
|
|||
|
||||
val deployer = new RemoteDeployer(settings)
|
||||
|
||||
val remote = new Remote(settings, remoteSettings)
|
||||
implicit val transports = remote.transports
|
||||
val transport: RemoteTransport = {
|
||||
val fqn = remoteSettings.RemoteTransport
|
||||
// TODO check if this classloader is the right one
|
||||
ReflectiveAccess.createInstance[RemoteTransport](
|
||||
fqn,
|
||||
Seq(classOf[RemoteSettings] -> remoteSettings),
|
||||
getClass.getClassLoader) match {
|
||||
case Left(problem) ⇒ throw new RemoteTransportException("Could not load remote transport layer " + fqn, problem)
|
||||
case Right(remote) ⇒ remote
|
||||
}
|
||||
}
|
||||
|
||||
val log = Logging(eventStream, "RemoteActorRefProvider(" + remote.remoteAddress + ")")
|
||||
val log = Logging(eventStream, "RemoteActorRefProvider(" + transport.address + ")")
|
||||
|
||||
val rootPath: ActorPath = RootActorPath(remote.remoteAddress)
|
||||
val rootPath: ActorPath = RootActorPath(transport.address)
|
||||
|
||||
private val local = new LocalActorRefProvider(systemName, settings, eventStream, scheduler, _deadLetters, rootPath, deployer)
|
||||
|
||||
val failureDetector = new AccrualFailureDetector(remoteSettings.FailureDetectorThreshold, remoteSettings.FailureDetectorMaxSampleSize)
|
||||
|
||||
@volatile
|
||||
private var _serialization: Serialization = _
|
||||
def serialization = _serialization
|
||||
|
||||
@volatile
|
||||
private var _remoteDaemon: InternalActorRef = _
|
||||
def remoteDaemon = _remoteDaemon
|
||||
|
||||
@volatile
|
||||
private var _networkEventStream: NetworkEventStream = _
|
||||
def networkEventStream = _networkEventStream
|
||||
|
||||
val deathWatch = new RemoteDeathWatch(local.deathWatch, this)
|
||||
|
||||
def init(system: ActorSystemImpl) {
|
||||
local.init(system)
|
||||
remote.init(system, this)
|
||||
local.registerExtraNames(Map(("remote", remote.remoteDaemon)))
|
||||
terminationFuture.onComplete(_ ⇒ remote.transport.shutdown())
|
||||
|
||||
_remoteDaemon = new RemoteSystemDaemon(system, transport.address, rootPath / "remote", rootGuardian, log)
|
||||
_serialization = SerializationExtension(system)
|
||||
|
||||
transport.start(system, this)
|
||||
|
||||
val remoteClientLifeCycleHandler = system.systemActorOf(Props(new Actor {
|
||||
def receive = {
|
||||
case RemoteClientError(cause, remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case RemoteClientDisconnected(remote, address) ⇒ remote.shutdownClientConnection(address)
|
||||
case _ ⇒ //ignore other
|
||||
}
|
||||
}), "RemoteClientLifeCycleListener")
|
||||
|
||||
_networkEventStream = new NetworkEventStream(system)
|
||||
|
||||
system.eventStream.subscribe(networkEventStream.sender, classOf[RemoteLifeCycleEvent])
|
||||
system.eventStream.subscribe(remoteClientLifeCycleHandler, classOf[RemoteLifeCycleEvent])
|
||||
|
||||
local.registerExtraNames(Map(("remote", remoteDaemon)))
|
||||
terminationFuture.onComplete(_ ⇒ transport.shutdown())
|
||||
}
|
||||
|
||||
def actorOf(system: ActorSystemImpl, props: Props, supervisor: InternalActorRef, path: ActorPath, systemService: Boolean, deploy: Option[Deploy]): InternalActorRef = {
|
||||
|
|
@ -102,18 +147,12 @@ class RemoteActorRefProvider(
|
|||
})
|
||||
|
||||
deployment match {
|
||||
case Some(Deploy(_, _, _, RemoteScope(address))) ⇒
|
||||
// FIXME RK this should be done within the deployer, i.e. the whole parsing business
|
||||
address.parse(remote.transports) match {
|
||||
case Left(x) ⇒
|
||||
throw new ConfigurationException("cannot parse remote address: " + x)
|
||||
case Right(addr) ⇒
|
||||
if (addr == rootPath.address) local.actorOf(system, props, supervisor, path, false, deployment)
|
||||
else {
|
||||
val rpath = RootActorPath(addr) / "remote" / rootPath.address.hostPort / path.elements
|
||||
useActorOnNode(rpath, props.creator, supervisor)
|
||||
new RemoteActorRef(this, remote.transport, rpath, supervisor, None)
|
||||
}
|
||||
case Some(Deploy(_, _, _, RemoteScope(addr))) ⇒
|
||||
if (addr == rootPath.address) local.actorOf(system, props, supervisor, path, false, deployment)
|
||||
else {
|
||||
val rpath = RootActorPath(addr) / "remote" / rootPath.address.hostPort / path.elements
|
||||
useActorOnNode(rpath, props.creator, supervisor)
|
||||
new RemoteActorRef(this, transport, rpath, supervisor, None)
|
||||
}
|
||||
|
||||
case _ ⇒ local.actorOf(system, props, supervisor, path, systemService, deployment)
|
||||
|
|
@ -123,14 +162,13 @@ class RemoteActorRefProvider(
|
|||
|
||||
def actorFor(path: ActorPath): InternalActorRef = path.root match {
|
||||
case `rootPath` ⇒ actorFor(rootGuardian, path.elements)
|
||||
case RootActorPath(_: RemoteSystemAddress[_], _) ⇒ new RemoteActorRef(this, remote.transport, path, Nobody, None)
|
||||
case _ ⇒ local.actorFor(path)
|
||||
case _ ⇒ new RemoteActorRef(this, transport, path, Nobody, None)
|
||||
}
|
||||
|
||||
def actorFor(ref: InternalActorRef, path: String): InternalActorRef = path match {
|
||||
case ParsedActorPath(address, elems) ⇒
|
||||
case ActorPathExtractor(address, elems) ⇒
|
||||
if (address == rootPath.address) actorFor(rootGuardian, elems)
|
||||
else new RemoteActorRef(this, remote.transport, new RootActorPath(address) / elems, Nobody, None)
|
||||
else new RemoteActorRef(this, transport, new RootActorPath(address) / elems, Nobody, None)
|
||||
case _ ⇒ local.actorFor(ref, path)
|
||||
}
|
||||
|
||||
|
|
@ -159,7 +197,7 @@ trait RemoteRef extends ActorRefScope {
|
|||
*/
|
||||
private[akka] class RemoteActorRef private[akka] (
|
||||
provider: RemoteActorRefProvider,
|
||||
remote: RemoteSupport[ParsedTransportAddress],
|
||||
remote: RemoteTransport,
|
||||
val path: ActorPath,
|
||||
val getParent: InternalActorRef,
|
||||
loader: Option[ClassLoader])
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue