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:
Roland 2012-01-20 14:29:50 +01:00
parent 6db3e59ce1
commit edceda8edf
34 changed files with 1415 additions and 1520 deletions

View file

@ -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])