Get remoting working under the remote actor ref provider

This commit is contained in:
Peter Vlugter 2011-10-13 17:42:26 +02:00
parent e94860b57b
commit d9e0088f3c
39 changed files with 309 additions and 460 deletions

View file

@ -8,6 +8,7 @@ import akka.actor._
import akka.routing._
import akka.actor.Actor._
import akka.actor.Status._
import akka.dispatch._
import akka.event.EventHandler
import akka.util.duration._
import akka.config.ConfigurationException
@ -25,31 +26,36 @@ import akka.AkkaApplication
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) extends ActorRefProvider {
class RemoteActorRefProvider(val app: AkkaApplication) extends ActorRefProvider {
import java.util.concurrent.ConcurrentHashMap
import akka.dispatch.Promise
implicit def _app = app
val local = new LocalActorRefProvider(app)
val remote = new Remote(app)
private val actors = new ConcurrentHashMap[String, Promise[Option[ActorRef]]]
private val actors = new ConcurrentHashMap[String, Promise[ActorRef]]
private val remoteDaemonConnectionManager = new RemoteConnectionManager(
app,
remote = remote,
failureDetector = new BannagePeriodFailureDetector(60 seconds)) // FIXME make timeout configurable
def actorOf(props: Props, address: String): Option[ActorRef] = {
def defaultDispatcher = app.dispatcher
def defaultTimeout = app.AkkaConfig.ActorTimeout
def actorOf(props: Props, address: String): ActorRef = actorOf(props, address, false)
def actorOf(props: Props, address: String, systemService: Boolean): ActorRef = {
Address.validate(address)
val newFuture = Promise[Option[ActorRef]](5000) // FIXME is this proper timeout?
val newFuture = Promise[ActorRef](5000)(defaultDispatcher) // FIXME is this proper timeout?
val oldFuture = actors.putIfAbsent(address, newFuture)
if (oldFuture eq null) { // we won the race -- create the actor and resolve the future
val deploymentConfig = app.deployer.deploymentConfig
val actor = try {
val actor: ActorRef = try {
app.deployer.lookupDeploymentFor(address) match {
case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, deploymentConfig.RemoteScope(remoteAddresses)))
case Some(DeploymentConfig.Deploy(_, _, routerType, nrOfInstances, failureDetectorType, DeploymentConfig.RemoteScope(remoteAddresses)))
val failureDetector = DeploymentConfig.failureDetectorTypeFor(failureDetectorType) match {
case FailureDetectorType.NoOp new NoOpFailureDetector
@ -67,7 +73,10 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
if (isReplicaNode) {
// we are on one of the replica node for this remote actor
Some(new LocalActorRef(app, props, address, false)) // create a local actor
val localProps =
if (props.dispatcher == Props.defaultDispatcher) props.copy(dispatcher = app.dispatcher)
else props
new LocalActorRef(app, localProps, address, false)
} else {
// we are on the single "reference" node uses the remote actors on the replica nodes
@ -94,7 +103,7 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
if (remoteAddresses.size < 1) throw new ConfigurationException(
"Actor [%s] configured with ScatterGather router must have at least 1 remote node configured. Found [%s]"
.format(address, remoteAddresses.mkString(", ")))
() new ScatterGatherFirstCompletedRouter
() new ScatterGatherFirstCompletedRouter()(defaultDispatcher, defaultTimeout)
case RouterType.LeastCPU sys.error("Router LeastCPU not supported yet")
case RouterType.LeastRAM sys.error("Router LeastRAM not supported yet")
@ -105,19 +114,17 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
var connections = Map.empty[InetSocketAddress, ActorRef]
remoteAddresses foreach { remoteAddress: DeploymentConfig.RemoteAddress
val inetSocketAddress = new InetSocketAddress(remoteAddress.hostname, remoteAddress.port)
connections += (inetSocketAddress -> RemoteActorRef(app.remote, inetSocketAddress, address, None))
connections += (inetSocketAddress -> RemoteActorRef(remote.server, inetSocketAddress, address, None))
}
val connectionManager = new RemoteConnectionManager(app, remote, connections, failureDetector)
connections.keys foreach { useActorOnNode(_, address, props.creator) }
Some(app.createActor(RoutedProps(
routerFactory = routerFactory,
connectionManager = connectionManager)))
actorOf(RoutedProps(routerFactory = routerFactory, connectionManager = connectionManager), address)
}
case deploy None // non-remote actor
case deploy local.actorOf(props, address, systemService)
}
} catch {
case e: Exception
@ -131,21 +138,21 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
actor
} else { // we lost the race -- wait for future to complete
oldFuture.await.resultOrException.getOrElse(None)
oldFuture.await.resultOrException.get
}
}
/**
* Copied from LocalActorRefProvider...
*/
def actorOf(props: RoutedProps, address: String): Option[ActorRef] = {
def actorOf(props: RoutedProps, address: String): ActorRef = {
if (props.connectionManager.size == 0) throw new ConfigurationException("RoutedProps used for creating actor [" + address + "] has zero connections configured; can't create a router")
Some(new RoutedActorRef(props, address))
new RoutedActorRef(props, address)
}
def actorFor(address: String): Option[ActorRef] = actors.get(address) match {
case null None
case future future.await.resultOrException.getOrElse(None)
case future Some(future.get)
}
/**
@ -153,6 +160,12 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
*/
private[akka] def evict(address: String): Boolean = actors.remove(address) ne null
private[akka] def deserialize(actor: SerializedActorRef): Option[ActorRef] = {
local.actorFor(actor.address) orElse {
Some(RemoteActorRef(remote.server, new InetSocketAddress(actor.hostname, actor.port), actor.address, None))
}
}
/**
* Using (checking out) actor on a specific node.
*/
@ -213,3 +226,66 @@ class RemoteActorRefProvider(val app: AkkaApplication, val remote: Remote) exten
}
}
}
/**
* Remote ActorRef that is used when referencing the Actor on a different node than its "home" node.
* This reference is network-aware (remembers its origin) and immutable.
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
private[akka] case class RemoteActorRef private[akka] (
val remote: RemoteSupport,
val remoteAddress: InetSocketAddress,
val address: String,
loader: Option[ClassLoader])
extends ActorRef with ScalaActorRef {
@volatile
private var running: Boolean = true
def isShutdown: Boolean = !running
def postMessageToMailbox(message: Any, channel: UntypedChannel) {
val chSender = if (channel.isInstanceOf[ActorRef]) Some(channel.asInstanceOf[ActorRef]) else None
remote.send[Any](message, chSender, None, remoteAddress, true, this, loader)
}
def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any,
timeout: Timeout,
channel: UntypedChannel): Future[Any] = {
val chSender = if (channel.isInstanceOf[ActorRef]) Some(channel.asInstanceOf[ActorRef]) else None
val chFuture = if (channel.isInstanceOf[Promise[_]]) Some(channel.asInstanceOf[Promise[Any]]) else None
val future = remote.send[Any](message, chSender, chFuture, remoteAddress, false, this, loader)
if (future.isDefined) ActorPromise(future.get)(timeout)
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
}
def suspend(): Unit = unsupported
def resume(): Unit = unsupported
def stop() { //FIXME send the cause as well!
synchronized {
if (running) {
running = false
postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
}
}
}
@throws(classOf[java.io.ObjectStreamException])
private def writeReplace(): AnyRef = {
SerializedActorRef(uuid, address, remoteAddress.getAddress.getHostAddress, remoteAddress.getPort)
}
def link(actorRef: ActorRef): ActorRef = unsupported
def unlink(actorRef: ActorRef): ActorRef = unsupported
protected[akka] def restart(cause: Throwable): Unit = unsupported
private def unsupported = throw new UnsupportedOperationException("Not supported for RemoteActorRef")
}