Added support for server-initiated remote actors with clients getting a dummy handle to the remote actor

This commit is contained in:
Jonas Bonér 2010-02-16 15:39:09 +01:00
parent 8fb281f4b0
commit 41766bef22
6 changed files with 321 additions and 80 deletions

View file

@ -24,7 +24,56 @@ import java.net.{SocketAddress, InetSocketAddress}
import java.util.concurrent.{TimeUnit, Executors, ConcurrentMap, ConcurrentHashMap}
import java.util.concurrent.atomic.AtomicLong
import scala. collection.mutable.{HashSet, HashMap}
import scala.collection.mutable.{HashSet, HashMap}
/*
class RemoteActorHandle(id: String, className: String, timeout: Long, hostname: String, port: Int) extends Actor {
start
val remoteClient = RemoteClient.clientFor(hostname, port)
override def postMessageToMailbox(message: Any, sender: Option[Actor]): Unit = {
val requestBuilder = RemoteRequest.newBuilder
.setId(RemoteRequestIdFactory.nextId)
.setTarget(className)
.setTimeout(timeout)
.setUuid(id)
.setIsActor(true)
.setIsOneWay(true)
.setIsEscaped(false)
if (sender.isDefined) {
val s = sender.get
requestBuilder.setSourceTarget(s.getClass.getName)
requestBuilder.setSourceUuid(s.uuid)
val (host, port) = s._replyToAddress.map(a => (a.getHostName, a.getPort)).getOrElse((Actor.HOSTNAME, Actor.PORT))
requestBuilder.setSourceHostname(host)
requestBuilder.setSourcePort(port)
}
RemoteProtocolBuilder.setMessage(message, requestBuilder)
remoteClient.send(requestBuilder.build, None)
}
override def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any,
timeout: Long,
senderFuture: Option[CompletableFutureResult]): CompletableFutureResult = {
val requestBuilder = RemoteRequest.newBuilder
.setId(RemoteRequestIdFactory.nextId)
.setTarget(className)
.setTimeout(timeout)
.setUuid(id)
.setIsActor(true)
.setIsOneWay(false)
.setIsEscaped(false)
RemoteProtocolBuilder.setMessage(message, requestBuilder)
val future = remoteClient.send(requestBuilder.build, senderFuture)
if (future.isDefined) future.get
else throw new IllegalStateException("Expected a future from remote call to actor " + toString)
}
def receive = { case _ => {} }
}
*/
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
@ -32,6 +81,7 @@ import scala. collection.mutable.{HashSet, HashMap}
object RemoteRequestIdFactory {
private val nodeId = UUID.newUuid
private val id = new AtomicLong
def nextId: Long = id.getAndIncrement + nodeId
}
@ -45,6 +95,65 @@ object RemoteClient extends Logging {
private val remoteClients = new HashMap[String, RemoteClient]
private val remoteActors = new HashMap[RemoteServer.Address, HashSet[String]]
// FIXME: simplify overloaded methods when we have Scala 2.8
/*
def actorFor(className: String, hostname: String, port: Int): Actor =
actorFor(className, className, 5000, hostname, port)
def actorFor(actorId: String, className: String, hostname: String, port: Int): Actor =
actorFor(actorId, className, 5000, hostname, port)
*/
def actorFor(className: String, timeout: Long, hostname: String, port: Int): Actor =
actorFor(className, className, timeout, hostname, port)
def actorFor(actorId: String, className: String, timeout: Long, hostname: String, port: Int): Actor = {
new Actor {
start
val remoteClient = RemoteClient.clientFor(hostname, port)
override def postMessageToMailbox(message: Any, sender: Option[Actor]): Unit = {
val requestBuilder = RemoteRequest.newBuilder
.setId(RemoteRequestIdFactory.nextId)
.setTarget(className)
.setTimeout(timeout)
.setUuid(actorId)
.setIsActor(true)
.setIsOneWay(true)
.setIsEscaped(false)
if (sender.isDefined) {
val s = sender.get
requestBuilder.setSourceTarget(s.getClass.getName)
requestBuilder.setSourceUuid(s.uuid)
val (host, port) = s._replyToAddress.map(a => (a.getHostName, a.getPort)).getOrElse((Actor.HOSTNAME, Actor.PORT))
requestBuilder.setSourceHostname(host)
requestBuilder.setSourcePort(port)
}
RemoteProtocolBuilder.setMessage(message, requestBuilder)
remoteClient.send(requestBuilder.build, None)
}
override def postMessageToMailboxAndCreateFutureResultWithTimeout(
message: Any,
timeout: Long,
senderFuture: Option[CompletableFutureResult]): CompletableFutureResult = {
val requestBuilder = RemoteRequest.newBuilder
.setId(RemoteRequestIdFactory.nextId)
.setTarget(className)
.setTimeout(timeout)
.setUuid(actorId)
.setIsActor(true)
.setIsOneWay(false)
.setIsEscaped(false)
RemoteProtocolBuilder.setMessage(message, requestBuilder)
val future = remoteClient.send(requestBuilder.build, senderFuture)
if (future.isDefined) future.get
else throw new IllegalStateException("Expected a future from remote call to actor " + toString)
}
def receive = {case _ => {}}
}
}
def clientFor(hostname: String, port: Int): RemoteClient = clientFor(new InetSocketAddress(hostname, port))
def clientFor(address: InetSocketAddress): RemoteClient = synchronized {
@ -155,7 +264,7 @@ class RemoteClient(hostname: String, port: Int) extends Logging {
} else {
futures.synchronized {
val futureResult = if (senderFuture.isDefined) senderFuture.get
else new DefaultCompletableFutureResult(request.getTimeout)
else new DefaultCompletableFutureResult(request.getTimeout)
futures.put(request.getId, futureResult)
connection.getChannel.write(request)
Some(futureResult)
@ -185,21 +294,21 @@ class RemoteClientPipelineFactory(name: String,
timer: HashedWheelTimer,
client: RemoteClient) extends ChannelPipelineFactory {
def getPipeline: ChannelPipeline = {
val timeout = new ReadTimeoutHandler(timer, RemoteClient.READ_TIMEOUT)
val lenDec = new LengthFieldBasedFrameDecoder(1048576, 0, 4, 0, 4)
val lenPrep = new LengthFieldPrepender(4)
val protobufDec = new ProtobufDecoder(RemoteReply.getDefaultInstance)
val protobufEnc = new ProtobufEncoder
val timeout = new ReadTimeoutHandler(timer, RemoteClient.READ_TIMEOUT)
val lenDec = new LengthFieldBasedFrameDecoder(1048576, 0, 4, 0, 4)
val lenPrep = new LengthFieldPrepender(4)
val protobufDec = new ProtobufDecoder(RemoteReply.getDefaultInstance)
val protobufEnc = new ProtobufEncoder
val zipCodec = RemoteServer.COMPRESSION_SCHEME match {
case "zlib" => Some(Codec(new ZlibEncoder(RemoteServer.ZLIB_COMPRESSION_LEVEL), new ZlibDecoder))
case "zlib" => Some(Codec(new ZlibEncoder(RemoteServer.ZLIB_COMPRESSION_LEVEL), new ZlibDecoder))
//case "lzf" => Some(Codec(new LzfEncoder, new LzfDecoder))
case _ => None
}
val remoteClient = new RemoteClientHandler(name, futures, supervisors, bootstrap, remoteAddress, timer, client)
val stages: Array[ChannelHandler] =
zipCodec.map(codec => Array(timeout, codec.decoder, lenDec, protobufDec, codec.encoder, lenPrep, protobufEnc, remoteClient))
.getOrElse(Array(timeout, lenDec, protobufDec, lenPrep, protobufEnc, remoteClient))
zipCodec.map(codec => Array(timeout, codec.decoder, lenDec, protobufDec, codec.encoder, lenPrep, protobufEnc, remoteClient))
.getOrElse(Array(timeout, lenDec, protobufDec, lenPrep, protobufEnc, remoteClient))
new StaticChannelPipeline(stages: _*)
}
}
@ -207,7 +316,7 @@ class RemoteClientPipelineFactory(name: String,
/**
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
@ChannelPipelineCoverage { val value = "all" }
@ChannelPipelineCoverage {val value = "all"}
class RemoteClientHandler(val name: String,
val futures: ConcurrentMap[Long, CompletableFutureResult],
val supervisors: ConcurrentMap[String, Actor],
@ -215,7 +324,7 @@ class RemoteClientHandler(val name: String,
val remoteAddress: SocketAddress,
val timer: HashedWheelTimer,
val client: RemoteClient)
extends SimpleChannelUpstreamHandler with Logging {
extends SimpleChannelUpstreamHandler with Logging {
import Actor.Sender.Self
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
@ -284,7 +393,7 @@ class RemoteClientHandler(val name: String,
val exceptionType = Class.forName(exception.substring(0, exception.indexOf('$')))
val exceptionMessage = exception.substring(exception.indexOf('$') + 1, exception.length)
exceptionType
.getConstructor(Array[Class[_]](classOf[String]): _*)
.newInstance(exceptionMessage).asInstanceOf[Throwable]
.getConstructor(Array[Class[_]](classOf[String]): _*)
.newInstance(exceptionMessage).asInstanceOf[Throwable]
}
}