Get remoting working under the remote actor ref provider
This commit is contained in:
parent
e94860b57b
commit
d9e0088f3c
39 changed files with 309 additions and 460 deletions
|
|
@ -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é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é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")
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue