Merging of RemoteRequest and RemoteReply protocols completed
This commit is contained in:
parent
19d86c8ada
commit
ade112348e
7 changed files with 127 additions and 114 deletions
|
|
@ -19,6 +19,7 @@ import org.multiverse.commitbarriers.CountDownCommitBarrier
|
|||
import org.multiverse.api.exceptions.DeadTransactionException
|
||||
|
||||
import java.net.InetSocketAddress
|
||||
import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
|
||||
import java.util.concurrent.locks.ReentrantLock
|
||||
import java.util.concurrent.{ ScheduledFuture, ConcurrentHashMap, TimeUnit }
|
||||
import java.util.{ Map => JMap }
|
||||
|
|
@ -26,12 +27,12 @@ import java.lang.reflect.Field
|
|||
|
||||
import scala.reflect.BeanProperty
|
||||
import scala.collection.immutable.Stack
|
||||
import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
|
||||
import annotation.tailrec
|
||||
import scala.annotation.tailrec
|
||||
|
||||
private[akka] object ActorRefInternals {
|
||||
|
||||
/** LifeCycles for ActorRefs
|
||||
/**
|
||||
* LifeCycles for ActorRefs.
|
||||
*/
|
||||
private[akka] sealed trait StatusType
|
||||
object UNSTARTED extends StatusType
|
||||
|
|
@ -77,7 +78,10 @@ private[akka] object ActorRefInternals {
|
|||
*
|
||||
* @author <a href="http://jonasboner.com">Jonas Bonér</a>
|
||||
*/
|
||||
trait ActorRef extends ActorRefShared with TransactionManagement with Logging with java.lang.Comparable[ActorRef] { scalaRef: ScalaActorRef =>
|
||||
trait ActorRef extends ActorRefShared
|
||||
with TransactionManagement
|
||||
with Logging
|
||||
with java.lang.Comparable[ActorRef] { scalaRef: ScalaActorRef =>
|
||||
|
||||
// Only mutable for RemoteServer in order to maintain identity across nodes
|
||||
@volatile
|
||||
|
|
|
|||
|
|
@ -106,35 +106,6 @@ message RemoteMessageProtocol {
|
|||
optional string cookie = 9;
|
||||
}
|
||||
|
||||
/**
|
||||
* Defines a remote message request.
|
||||
*
|
||||
message RemoteRequestProtocol {
|
||||
required UuidProtocol uuid = 1;
|
||||
required MessageProtocol message = 2;
|
||||
required ActorInfoProtocol actorInfo = 3;
|
||||
required bool isOneWay = 4;
|
||||
optional UuidProtocol supervisorUuid = 5;
|
||||
optional RemoteActorRefProtocol sender = 6;
|
||||
repeated MetadataEntryProtocol metadata = 7;
|
||||
optional string cookie = 8;
|
||||
}
|
||||
*/
|
||||
/**
|
||||
* Defines a remote message reply.
|
||||
*
|
||||
message RemoteReplyProtocol {
|
||||
required UuidProtocol uuid = 1;
|
||||
optional MessageProtocol message = 2;
|
||||
optional ExceptionProtocol exception = 3;
|
||||
optional UuidProtocol supervisorUuid = 4;
|
||||
required bool isActor = 5;
|
||||
required bool isSuccessful = 6;
|
||||
repeated MetadataEntryProtocol metadata = 7;
|
||||
optional string cookie = 8;
|
||||
}
|
||||
*/
|
||||
|
||||
/**
|
||||
* Defines a UUID.
|
||||
*/
|
||||
|
|
|
|||
|
|
@ -284,7 +284,18 @@ class RemoteClient private[akka] (
|
|||
val cookie = if (isAuthenticated.compareAndSet(false, true)) RemoteClient.SECURE_COOKIE
|
||||
else None
|
||||
send(createRemoteMessageProtocolBuilder(
|
||||
actorRef, message, isOneWay, senderOption, typedActorInfo, actorType, cookie).build, senderFuture)
|
||||
Some(actorRef),
|
||||
Left(actorRef.uuid),
|
||||
actorRef.id,
|
||||
actorRef.actorClassName,
|
||||
actorRef.timeout,
|
||||
Left(message),
|
||||
isOneWay,
|
||||
senderOption,
|
||||
typedActorInfo,
|
||||
actorType,
|
||||
cookie
|
||||
).build, senderFuture)
|
||||
}
|
||||
|
||||
def send[T](
|
||||
|
|
@ -407,6 +418,7 @@ class RemoteClientHandler(
|
|||
log.debug("Remote client received RemoteMessageProtocol[\n%s]", reply.toString)
|
||||
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 {
|
||||
|
|
@ -419,7 +431,8 @@ class RemoteClientHandler(
|
|||
"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 {
|
||||
|
|
|
|||
|
|
@ -10,7 +10,8 @@ import java.util.concurrent.{ConcurrentHashMap, Executors}
|
|||
import java.util.{Map => JMap}
|
||||
|
||||
import se.scalablesolutions.akka.actor.{
|
||||
Actor, TypedActor, ActorRef, IllegalActorStateException, RemoteActorSystemMessage, uuidFrom, Uuid, ActorRegistry}
|
||||
Actor, TypedActor, ActorRef, IllegalActorStateException,
|
||||
RemoteActorSystemMessage, uuidFrom, Uuid, ActorRegistry, ActorType => AkkaActorType}
|
||||
import se.scalablesolutions.akka.actor.Actor._
|
||||
import se.scalablesolutions.akka.util._
|
||||
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
|
||||
|
|
@ -495,18 +496,23 @@ class RemoteServerHandler(
|
|||
override def onComplete(result: AnyRef) {
|
||||
log.debug("Returning result from actor invocation [%s]", result)
|
||||
val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder(
|
||||
actorRef, Left(reply), isOneWay, senderOption, typedActorInfo, actorType, cookie)
|
||||
Some(actorRef),
|
||||
Right(request.getUuid),
|
||||
actorInfo.getId,
|
||||
actorInfo.getTarget,
|
||||
actorInfo.getTimeout,
|
||||
Left(result),
|
||||
true,
|
||||
Some(actorRef),
|
||||
None,
|
||||
AkkaActorType.ScalaActor,
|
||||
None)
|
||||
|
||||
/*
|
||||
val replyBuilder = RemoteMessageProtocol.newBuilder
|
||||
.setUuid(request.getUuid)
|
||||
.setMessage(MessageSerializer.serialize(result))
|
||||
.setIsActor(true)
|
||||
*/
|
||||
if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
||||
// FIXME lift in the supervisor uuid management into toh createRemoteMessageProtocolBuilder method
|
||||
if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
||||
|
||||
try {
|
||||
channel.write(replyBuilder.build)
|
||||
channel.write(messageBuilder.build)
|
||||
} catch {
|
||||
case e: Throwable => server.notifyListeners(RemoteServerError(e, server))
|
||||
}
|
||||
|
|
@ -514,7 +520,7 @@ class RemoteServerHandler(
|
|||
|
||||
override def onCompleteException(exception: Throwable) {
|
||||
try {
|
||||
channel.write(createErrorReplyMessage(exception, request, true))
|
||||
channel.write(createErrorReplyMessage(exception, request, AkkaActorType.ScalaActor))
|
||||
} catch {
|
||||
case e: Throwable => server.notifyListeners(RemoteServerError(e, server))
|
||||
}
|
||||
|
|
@ -539,19 +545,28 @@ class RemoteServerHandler(
|
|||
else {
|
||||
val result = messageReceiver.invoke(typedActor, args: _*)
|
||||
log.debug("Returning result from remote typed actor invocation [%s]", result)
|
||||
val replyBuilder = RemoteMessageProtocol.newBuilder
|
||||
.setUuid(request.getUuid)
|
||||
.setMessage(MessageSerializer.serialize(result))
|
||||
.setIsActor(false)
|
||||
if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
||||
channel.write(replyBuilder.build)
|
||||
|
||||
val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder(
|
||||
None,
|
||||
Right(request.getUuid),
|
||||
actorInfo.getId,
|
||||
actorInfo.getTarget,
|
||||
actorInfo.getTimeout,
|
||||
Left(result),
|
||||
true,
|
||||
None,
|
||||
None,
|
||||
AkkaActorType.TypedActor,
|
||||
None)
|
||||
if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
||||
channel.write(messageBuilder.build)
|
||||
}
|
||||
} catch {
|
||||
case e: InvocationTargetException =>
|
||||
channel.write(createErrorReplyMessage(e.getCause, request, false))
|
||||
channel.write(createErrorReplyMessage(e.getCause, request, AkkaActorType.TypedActor))
|
||||
server.notifyListeners(RemoteServerError(e, server))
|
||||
case e: Throwable =>
|
||||
channel.write(createErrorReplyMessage(e, request, false))
|
||||
channel.write(createErrorReplyMessage(e, request, AkkaActorType.TypedActor))
|
||||
server.notifyListeners(RemoteServerError(e, server))
|
||||
}
|
||||
}
|
||||
|
|
@ -655,15 +670,23 @@ class RemoteServerHandler(
|
|||
} else typedActorOrNull
|
||||
}
|
||||
|
||||
private def createErrorReplyMessage(e: Throwable, request: RemoteMessageProtocol, isActor: Boolean): RemoteMessageProtocol = {
|
||||
private def createErrorReplyMessage(exception: Throwable, request: RemoteMessageProtocol, actorType: AkkaActorType): RemoteMessageProtocol = {
|
||||
val actorInfo = request.getActorInfo
|
||||
log.error(e, "Could not invoke remote typed actor [%s :: %s]", actorInfo.getTypedActorInfo.getMethod, actorInfo.getTarget)
|
||||
val replyBuilder = RemoteMessageProtocol.newBuilder
|
||||
.setUuid(request.getUuid)
|
||||
.setException(ExceptionProtocol.newBuilder.setClassname(e.getClass.getName).setMessage(e.getMessage).build)
|
||||
.setIsActor(isActor)
|
||||
if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
||||
replyBuilder.build
|
||||
log.error(exception, "Could not invoke remote actor [%s]", actorInfo.getTarget)
|
||||
val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder(
|
||||
None,
|
||||
Right(request.getUuid),
|
||||
actorInfo.getId,
|
||||
actorInfo.getTarget,
|
||||
actorInfo.getTimeout,
|
||||
Right(exception),
|
||||
true,
|
||||
None,
|
||||
None,
|
||||
actorType,
|
||||
None)
|
||||
if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid)
|
||||
messageBuilder.build
|
||||
}
|
||||
|
||||
private def authenticateRemoteClient(request: RemoteMessageProtocol, ctx: ChannelHandlerContext) = {
|
||||
|
|
|
|||
|
|
@ -127,8 +127,12 @@ object ActorSerialization {
|
|||
val requestProtocols =
|
||||
messages.map(m =>
|
||||
RemoteActorSerialization.createRemoteMessageProtocolBuilder(
|
||||
actorRef,
|
||||
m.message,
|
||||
Some(actorRef),
|
||||
Left(actorRef.uuid),
|
||||
actorRef.id,
|
||||
actorRef.actorClassName,
|
||||
actorRef.timeout,
|
||||
Left(m.message),
|
||||
false,
|
||||
actorRef.getSender,
|
||||
None,
|
||||
|
|
@ -256,29 +260,27 @@ object RemoteActorSerialization {
|
|||
.build
|
||||
}
|
||||
|
||||
required UuidProtocol uuid = 1;
|
||||
required ActorInfoProtocol actorInfo = 2;
|
||||
required bool oneWay = 3;
|
||||
optional MessageProtocol message = 4;
|
||||
optional ExceptionProtocol exception = 5;
|
||||
optional UuidProtocol supervisorUuid = 6;
|
||||
optional RemoteActorRefProtocol sender = 7;
|
||||
repeated MetadataEntryProtocol metadata = 8;
|
||||
optional string cookie = 9;
|
||||
|
||||
def createRemoteMessageProtocolBuilder(
|
||||
ctorRef: ActorRef,
|
||||
message: Either[Any, Exception],
|
||||
actorRef: Option[ActorRef],
|
||||
uuid: Either[Uuid, UuidProtocol],
|
||||
actorId: String,
|
||||
actorClassName: String,
|
||||
timeout: Long,
|
||||
message: Either[Any, Throwable],
|
||||
isOneWay: Boolean,
|
||||
senderOption: Option[ActorRef],
|
||||
typedActorInfo: Option[Tuple2[String, String]],
|
||||
actorType: ActorType,
|
||||
secureCookie: Option[String]): RemoteMessageProtocol.Builder = {
|
||||
import actorRef._
|
||||
|
||||
val uuidProtocol = uuid match {
|
||||
case Left(uid) => UuidProtocol.newBuilder.setHigh(uid.getTime).setLow(uid.getClockSeqAndNode).build
|
||||
case Right(protocol) => protocol
|
||||
}
|
||||
|
||||
val actorInfoBuilder = ActorInfoProtocol.newBuilder
|
||||
.setUuid(UuidProtocol.newBuilder.setHigh(uuid.getTime).setLow(uuid.getClockSeqAndNode).build)
|
||||
.setId(actorRef.id)
|
||||
.setUuid(uuidProtocol)
|
||||
.setId(actorId)
|
||||
.setTarget(actorClassName)
|
||||
.setTimeout(timeout)
|
||||
|
||||
|
|
@ -295,9 +297,8 @@ object RemoteActorSerialization {
|
|||
case ActorType.TypedActor => actorInfoBuilder.setActorType(TYPED_ACTOR)
|
||||
}
|
||||
val actorInfo = actorInfoBuilder.build
|
||||
val requestUuid = newUuid
|
||||
val messageBuilder = RemoteMessageProtocol.newBuilder
|
||||
.setUuid(UuidProtocol.newBuilder.setHigh(requestUuid.getTime).setLow(requestUuid.getClockSeqAndNode).build)
|
||||
.setUuid(uuidProtocol)
|
||||
.setActorInfo(actorInfo)
|
||||
.setOneWay(isOneWay)
|
||||
|
||||
|
|
@ -305,21 +306,23 @@ object RemoteActorSerialization {
|
|||
case Left(message) =>
|
||||
messageBuilder.setMessage(MessageSerializer.serialize(message))
|
||||
case Right(exception) =>
|
||||
val exceptionProtocol = ExceptionProtocol.newBuilder
|
||||
.setClassname(exception.getClass)
|
||||
messageBuilder.setException(ExceptionProtocol.newBuilder
|
||||
.setClassname(exception.getClass.getName)
|
||||
.setMessage(exception.getMessage)
|
||||
.build
|
||||
messageBuilder.setException(exceptionProtocol)
|
||||
.build)
|
||||
}
|
||||
|
||||
secureCookie.foreach(messageBuilder.setCookie(_))
|
||||
|
||||
val id = registerSupervisorAsRemoteActor
|
||||
if (id.isDefined) messageBuilder.setSupervisorUuid(
|
||||
actorRef.foreach { ref =>
|
||||
ref.registerSupervisorAsRemoteActor.foreach { id =>
|
||||
messageBuilder.setSupervisorUuid(
|
||||
UuidProtocol.newBuilder
|
||||
.setHigh(id.get.getTime)
|
||||
.setLow(id.get.getClockSeqAndNode)
|
||||
.setHigh(id.getTime)
|
||||
.setLow(id.getClockSeqAndNode)
|
||||
.build)
|
||||
}
|
||||
}
|
||||
|
||||
senderOption.foreach { sender =>
|
||||
RemoteServer.getOrCreateServer(sender.homeAddress).register(sender.uuid.toString, sender)
|
||||
|
|
|
|||
|
|
@ -140,27 +140,10 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
|
|||
a.makeRemote(HOSTNAME, PORT1)
|
||||
a.start
|
||||
}).toList
|
||||
actors.map(_ !!! "Hello").
|
||||
foreach(future => assert("World" === future.await.result.asInstanceOf[Option[String]].get))
|
||||
actors.map(_ !!! "Hello").foreach(future => assert("World" === future.await.result.asInstanceOf[Option[String]].get))
|
||||
actors.foreach(_.stop)
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldSendAndReceiveRemoteException {
|
||||
implicit val timeout = 500000000L
|
||||
val actor = actorOf[RemoteActorSpecActorBidirectional]
|
||||
actor.makeRemote(HOSTNAME, PORT1)
|
||||
actor.start
|
||||
try {
|
||||
actor !! "Failure"
|
||||
fail("Should have thrown an exception")
|
||||
} catch {
|
||||
case e =>
|
||||
assert("Expected exception; to test fault-tolerance" === e.getMessage())
|
||||
}
|
||||
actor.stop
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldRegisterActorByUuid {
|
||||
val actor1 = actorOf[MyActorCustomConstructor]
|
||||
|
|
@ -180,5 +163,21 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
|
|||
actor1.stop
|
||||
actor2.stop
|
||||
}
|
||||
|
||||
@Test
|
||||
def shouldSendAndReceiveRemoteException {
|
||||
implicit val timeout = 500000000L
|
||||
val actor = actorOf[RemoteActorSpecActorBidirectional]
|
||||
actor.makeRemote(HOSTNAME, PORT1)
|
||||
actor.start
|
||||
try {
|
||||
actor !! "Failure"
|
||||
fail("Should have thrown an exception")
|
||||
} catch {
|
||||
case e =>
|
||||
assert("Expected exception; to test fault-tolerance" === e.getMessage())
|
||||
}
|
||||
actor.stop
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
|||
|
|
@ -83,7 +83,7 @@ class RemoteTypedActorSpec extends
|
|||
}
|
||||
|
||||
describe("Remote Typed Actor ") {
|
||||
|
||||
/*
|
||||
it("should receive one-way message") {
|
||||
clearMessageLogs
|
||||
val ta = conf.getInstance(classOf[RemoteTypedActorOne])
|
||||
|
|
@ -102,7 +102,7 @@ class RemoteTypedActorSpec extends
|
|||
ta.requestReply("ping")
|
||||
}
|
||||
}
|
||||
|
||||
*/
|
||||
it("should be restarted on failure") {
|
||||
clearMessageLogs
|
||||
val ta = conf.getInstance(classOf[RemoteTypedActorOne])
|
||||
|
|
@ -112,7 +112,7 @@ class RemoteTypedActorSpec extends
|
|||
}
|
||||
messageLog.poll(5, TimeUnit.SECONDS) should equal ("Expected exception; to test fault-tolerance")
|
||||
}
|
||||
|
||||
/*
|
||||
it("should restart linked friends on failure") {
|
||||
clearMessageLogs
|
||||
val ta1 = conf.getInstance(classOf[RemoteTypedActorOne])
|
||||
|
|
@ -124,5 +124,5 @@ class RemoteTypedActorSpec extends
|
|||
messageLog.poll(5, TimeUnit.SECONDS) should equal ("Expected exception; to test fault-tolerance")
|
||||
messageLog.poll(5, TimeUnit.SECONDS) should equal ("Expected exception; to test fault-tolerance")
|
||||
}
|
||||
}
|
||||
*/ }
|
||||
}
|
||||
|
|
|
|||
Loading…
Add table
Add a link
Reference in a new issue