Fixing a lot of stuff and starting to port unit tests
This commit is contained in:
parent
5f651c73ba
commit
8becbad787
8 changed files with 414 additions and 448 deletions
|
|
@ -11,8 +11,8 @@ import akka.remote.protocol.RemoteProtocol.ActorType._
|
|||
import akka.config.ConfigurationException
|
||||
import akka.serialization.RemoteActorSerialization
|
||||
import akka.japi.Creator
|
||||
import akka.actor.{ActorRegistryInstance, Actor, RemoteActorRef, TypedActor, ActorRef, IllegalActorStateException,
|
||||
RemoteActorSystemMessage, uuidFrom, Uuid, Exit, ActorRegistry, LifeCycleMessage, ActorType => AkkaActorType}
|
||||
import akka.actor.{newUuid,ActorRegistry, Actor, RemoteActorRef, TypedActor, ActorRef, IllegalActorStateException,
|
||||
RemoteActorSystemMessage, uuidFrom, Uuid, Exit, LifeCycleMessage, ActorType => AkkaActorType}
|
||||
import akka.remoteinterface. {RemoteSupport, RemoteModule, RemoteServerModule, RemoteClientModule}
|
||||
import akka.config.Config._
|
||||
import akka.serialization.RemoteActorSerialization._
|
||||
|
|
@ -73,7 +73,7 @@ trait NettyRemoteClientModule extends RemoteClientModule { self: ListenerManagem
|
|||
private val remoteActors = new HashMap[Address, HashSet[Uuid]]
|
||||
|
||||
protected[akka] def typedActorFor[T](intfClass: Class[T], serviceId: String, implClassName: String, timeout: Long, hostname: String, port: Int, loader: Option[ClassLoader]): T =
|
||||
TypedActor.createProxyForRemoteActorRef(intfClass, RemoteActorRef(registry, serviceId, implClassName, hostname, port, timeout, false, loader, AkkaActorType.TypedActor))
|
||||
TypedActor.createProxyForRemoteActorRef(intfClass, RemoteActorRef(Some(serviceId), implClassName, hostname, port, timeout, loader, AkkaActorType.TypedActor))
|
||||
|
||||
protected[akka] def send[T](message: Any,
|
||||
senderOption: Option[ActorRef],
|
||||
|
|
@ -241,8 +241,6 @@ class RemoteClient private[akka] (
|
|||
actorRef: ActorRef,
|
||||
typedActorInfo: Option[Tuple2[String, String]],
|
||||
actorType: AkkaActorType): Option[CompletableFuture[T]] = {
|
||||
val cookie = if (isAuthenticated.compareAndSet(false, true)) RemoteClient.SECURE_COOKIE
|
||||
else None
|
||||
send(createRemoteMessageProtocolBuilder(
|
||||
Some(actorRef),
|
||||
Left(actorRef.uuid),
|
||||
|
|
@ -254,21 +252,24 @@ class RemoteClient private[akka] (
|
|||
senderOption,
|
||||
typedActorInfo,
|
||||
actorType,
|
||||
cookie
|
||||
if (isAuthenticated.compareAndSet(false, true)) RemoteClient.SECURE_COOKIE else None
|
||||
).build, senderFuture)
|
||||
}
|
||||
|
||||
def send[T](
|
||||
request: RemoteMessageProtocol,
|
||||
senderFuture: Option[CompletableFuture[T]]): Option[CompletableFuture[T]] = {
|
||||
log.slf4j.debug("sending message: {} is running {} has future {}", Array[AnyRef](request, isRunning.asInstanceOf[AnyRef], senderFuture))
|
||||
if (isRunning) {
|
||||
if (request.getOneWay) {
|
||||
connection.getChannel.write(request)
|
||||
None
|
||||
} else {
|
||||
val futureResult = if (senderFuture.isDefined) senderFuture.get
|
||||
else new DefaultCompletableFuture[T](request.getActorInfo.getTimeout)
|
||||
futures.put(uuidFrom(request.getUuid.getHigh, request.getUuid.getLow), futureResult)
|
||||
else new DefaultCompletableFuture[T](request.getActorInfo.getTimeout)
|
||||
val futureUuid = uuidFrom(request.getUuid.getHigh, request.getUuid.getLow)
|
||||
futures.put(futureUuid, futureResult)
|
||||
log.slf4j.debug("Stashing away future for {}",futureUuid)
|
||||
connection.getChannel.write(request)
|
||||
Some(futureResult)
|
||||
}
|
||||
|
|
@ -369,34 +370,32 @@ class RemoteClientHandler(
|
|||
|
||||
override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) {
|
||||
try {
|
||||
val result = event.getMessage
|
||||
if (result.isInstanceOf[RemoteMessageProtocol]) {
|
||||
val reply = result.asInstanceOf[RemoteMessageProtocol]
|
||||
val replyUuid = uuidFrom(reply.getUuid.getHigh, reply.getUuid.getLow)
|
||||
log.debug("Remote client received RemoteMessageProtocol[\n{}]",reply)
|
||||
val future = futures.get(replyUuid).asInstanceOf[CompletableFuture[Any]]
|
||||
if (reply.hasMessage) {
|
||||
if (future eq null) throw new IllegalActorStateException("Future mapped to UUID " + replyUuid + " does not exist")
|
||||
val message = MessageSerializer.deserialize(reply.getMessage)
|
||||
future.completeWithResult(message)
|
||||
} else {
|
||||
if (reply.hasSupervisorUuid()) {
|
||||
val supervisorUuid = uuidFrom(reply.getSupervisorUuid.getHigh, reply.getSupervisorUuid.getLow)
|
||||
if (!supervisors.containsKey(supervisorUuid)) throw new IllegalActorStateException(
|
||||
"Expected a registered supervisor for UUID [" + supervisorUuid + "] but none was found")
|
||||
val supervisedActor = supervisors.get(supervisorUuid)
|
||||
if (!supervisedActor.supervisor.isDefined) throw new IllegalActorStateException(
|
||||
"Can't handle restart for remote actor " + supervisedActor + " since its supervisor has been removed")
|
||||
else supervisedActor.supervisor.get ! Exit(supervisedActor, parseException(reply, client.loader))
|
||||
event.getMessage match {
|
||||
case reply: RemoteMessageProtocol =>
|
||||
val replyUuid = uuidFrom(reply.getActorInfo.getUuid.getHigh, reply.getActorInfo.getUuid.getLow)
|
||||
log.slf4j.debug("Remote client received RemoteMessageProtocol[\n{}]",reply)
|
||||
log.slf4j.debug("Trying to map back to future: {}",replyUuid)
|
||||
val future = futures.remove(replyUuid).asInstanceOf[CompletableFuture[Any]]
|
||||
if (reply.hasMessage) {
|
||||
if (future eq null) throw new IllegalActorStateException("Future mapped to UUID " + replyUuid + " does not exist")
|
||||
val message = MessageSerializer.deserialize(reply.getMessage)
|
||||
future.completeWithResult(message)
|
||||
} else {
|
||||
if (reply.hasSupervisorUuid()) {
|
||||
val supervisorUuid = uuidFrom(reply.getSupervisorUuid.getHigh, reply.getSupervisorUuid.getLow)
|
||||
if (!supervisors.containsKey(supervisorUuid)) throw new IllegalActorStateException(
|
||||
"Expected a registered supervisor for UUID [" + supervisorUuid + "] but none was found")
|
||||
val supervisedActor = supervisors.get(supervisorUuid)
|
||||
if (!supervisedActor.supervisor.isDefined) throw new IllegalActorStateException(
|
||||
"Can't handle restart for remote actor " + supervisedActor + " since its supervisor has been removed")
|
||||
else supervisedActor.supervisor.get ! Exit(supervisedActor, parseException(reply, client.loader))
|
||||
}
|
||||
|
||||
future.completeWithException(parseException(reply, client.loader))
|
||||
}
|
||||
val exception = parseException(reply, client.loader)
|
||||
future.completeWithException(exception)
|
||||
}
|
||||
futures remove replyUuid
|
||||
} else {
|
||||
val exception = new RemoteClientException("Unknown message received in remote client handler: " + result, client)
|
||||
client.notifyListeners(RemoteClientError(exception, client))
|
||||
throw exception
|
||||
|
||||
case other =>
|
||||
throw new RemoteClientException("Unknown message received in remote client handler: " + other, client)
|
||||
}
|
||||
} catch {
|
||||
case e: Exception =>
|
||||
|
|
@ -547,8 +546,11 @@ case class RemoteServerClientClosed(
|
|||
/**
|
||||
* Provides the implementation of the Netty remote support
|
||||
*/
|
||||
class NettyRemoteSupport(val registry: ActorRegistryInstance)
|
||||
extends RemoteSupport with NettyRemoteServerModule with NettyRemoteClientModule {
|
||||
class NettyRemoteSupport extends RemoteSupport with NettyRemoteServerModule with NettyRemoteClientModule {
|
||||
//Needed for remote testing and switching on/off under run
|
||||
private[akka] val optimizeLocal = new AtomicBoolean(true)
|
||||
|
||||
def optimizeLocalScoped_?() = optimizeLocal.get
|
||||
|
||||
protected[akka] def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int, loader: Option[ClassLoader]): ActorRef = {
|
||||
//TODO: REVISIT: Possible to optimize server-managed actors in local scope?
|
||||
|
|
@ -562,7 +564,7 @@ class NettyRemoteSupport(val registry: ActorRegistryInstance)
|
|||
// case _ =>
|
||||
// RemoteActorRef(registry, serviceId, className, hostname, port, timeout, false, loader)
|
||||
//}
|
||||
RemoteActorRef(registry, serviceId, className, hostname, port, timeout, false, loader)
|
||||
RemoteActorRef(Some(serviceId), className, hostname, port, timeout, loader)
|
||||
}
|
||||
|
||||
def clientManagedActorOf(clazz: Class[_ <: Actor], host: String, port: Int, timeout: Long): ActorRef = {
|
||||
|
|
@ -571,13 +573,11 @@ class NettyRemoteSupport(val registry: ActorRegistryInstance)
|
|||
|
||||
(host,port) match {
|
||||
case (Host, Port) if optimizeLocalScoped_? =>
|
||||
registry.actorOf(clazz) //Local
|
||||
ActorRegistry.actorOf(clazz) //Local
|
||||
case _ =>
|
||||
new RemoteActorRef(registry,clazz.getName,clazz.getName,host,port,timeout,true /*Client managed*/, None)
|
||||
new RemoteActorRef(None,clazz.getName,host,port,timeout,None)
|
||||
}
|
||||
}
|
||||
|
||||
val optimizeLocalScoped_? = true
|
||||
}
|
||||
|
||||
trait NettyRemoteServerModule extends RemoteServerModule { self: RemoteModule =>
|
||||
|
|
@ -671,6 +671,7 @@ trait NettyRemoteServerModule extends RemoteServerModule { self: RemoteModule =>
|
|||
}
|
||||
|
||||
def registerByUuid(actorRef: ActorRef): Unit = guard withGuard {
|
||||
log.slf4j.debug("Registering remote actor {} to it's uuid {}", actorRef, actorRef.uuid)
|
||||
register(actorRef.uuid.toString, actorRef, actorsByUuid)
|
||||
}
|
||||
|
||||
|
|
@ -766,13 +767,6 @@ trait NettyRemoteServerModule extends RemoteServerModule { self: RemoteModule =>
|
|||
*/
|
||||
def unregisterTypedPerSessionActor(id: String): Unit =
|
||||
if (_isRunning.isOn) typedActorsFactories.remove(id)
|
||||
|
||||
private[akka] def actors = registry.actors(address)
|
||||
private[akka] def actorsByUuid = registry.actorsByUuid(address)
|
||||
private[akka] def actorsFactories = registry.actorsFactories(address)
|
||||
private[akka] def typedActors = registry.typedActors(address)
|
||||
private[akka] def typedActorsByUuid = registry.typedActorsByUuid(address)
|
||||
private[akka] def typedActorsFactories = registry.typedActorsFactories(address)
|
||||
}
|
||||
|
||||
object RemoteServerSslContext {
|
||||
|
|
@ -970,15 +964,18 @@ class RemoteServerHandler(
|
|||
None,
|
||||
Some(new DefaultCompletableFuture[AnyRef](request.getActorInfo.getTimeout).
|
||||
onComplete(f => {
|
||||
log.slf4j.debug("Future was completed, now flushing to remote!")
|
||||
val result = f.result
|
||||
val exception = f.exception
|
||||
|
||||
if (exception.isDefined) {
|
||||
log.slf4j.debug("Returning exception from actor invocation [{}]",exception.get)
|
||||
log.slf4j.debug("Returning exception from actor invocation [{}]",exception.get.getClass)
|
||||
try {
|
||||
channel.write(createErrorReplyMessage(exception.get, request, AkkaActorType.ScalaActor))
|
||||
} catch {
|
||||
case e: Throwable => server.notifyListeners(RemoteServerError(e, server))
|
||||
case e: Throwable =>
|
||||
log.slf4j.debug("An error occurred in sending the reply",e)
|
||||
server.notifyListeners(RemoteServerError(e, server))
|
||||
}
|
||||
}
|
||||
else if (result.isDefined) {
|
||||
|
|
@ -1069,6 +1066,7 @@ class RemoteServerHandler(
|
|||
}
|
||||
|
||||
private def findActorByUuid(uuid: String) : ActorRef = {
|
||||
log.slf4j.debug("Trying to find actor for uuid '{}' inside {}",uuid,server.actorsByUuid)
|
||||
server.actorsByUuid.get(uuid)
|
||||
}
|
||||
|
||||
|
|
@ -1149,10 +1147,10 @@ class RemoteServerHandler(
|
|||
if (RemoteServer.UNTRUSTED_MODE) throw new SecurityException(
|
||||
"Remote server is operating is untrusted mode, can not create remote actors on behalf of the remote client")
|
||||
|
||||
log.slf4j.info("Creating a new remote actor [{}:{}]", name, uuid)
|
||||
log.slf4j.info("Creating a new client-managed remote actor [{}:{}]", name, uuid)
|
||||
val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name)
|
||||
else Class.forName(name)
|
||||
val actorRef = Actor.actorOf(clazz.asInstanceOf[Class[_ <: Actor]])
|
||||
val actorRef = ActorRegistry.actorOf(clazz.asInstanceOf[Class[_ <: Actor]])
|
||||
actorRef.uuid = uuidFrom(uuid.getHigh,uuid.getLow)
|
||||
actorRef.id = id
|
||||
actorRef.timeout = timeout
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue