diff --git a/akka-actor/src/main/java/akka/actor/Actors.java b/akka-actor/src/main/java/akka/actor/Actors.java new file mode 100644 index 0000000000..d19ee01785 --- /dev/null +++ b/akka-actor/src/main/java/akka/actor/Actors.java @@ -0,0 +1,72 @@ +package akka.actor; + +import akka.japi.Creator; +import akka.remoteinterface.RemoteSupport; + +/** + * JAVA API for + * - creating actors, + * - creating remote actors, + * - locating actors + */ +public class Actors { + /** + * + * @return The actor registry + */ + public static ActorRegistry registry() { + return Actor$.MODULE$.registry(); + } + + /** + * + * @return + * @throws UnsupportedOperationException If remoting isn't configured + * @throws ModuleNotAvailableException If the class for the remote support cannot be loaded + */ + public static RemoteSupport remote() { + return Actor$.MODULE$.remote(); + } + + /** + * NOTE: Use this convenience method with care, do NOT make it possible to get a reference to the + * UntypedActor instance directly, but only through its 'ActorRef' wrapper reference. + *
+ * Creates an ActorRef out of the Actor. Allows you to pass in the instance for the UntypedActor. + * Only use this method when you need to pass in constructor arguments into the 'UntypedActor'. + * + * You use it by implementing the UntypedActorFactory interface. + * Example in Java: + *
+ * ActorRef actor = Actors.actorOf(new UntypedActorFactory() {
+ * public UntypedActor create() {
+ * return new MyUntypedActor("service:name", 5);
+ * }
+ * });
+ * actor.start();
+ * actor.sendOneWay(message, context);
+ * actor.stop();
+ *
+ */
+ public static ActorRef actorOf(final Creator+ * ActorRef actor = Actors.actorOf(MyUntypedActor.class); + * actor.start(); + * actor.sendOneWay(message, context); + * actor.stop(); + *+ * You can create and start the actor in one statement like this: + *
+ * val actor = Actors.actorOf(MyActor.class).start(); + *+ */ + public static ActorRef actorOf(final Class extends Actor> type) { + return Actor$.MODULE$.actorOf(type); + } +} \ No newline at end of file diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 1d532b2623..e6f9cf6da8 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -15,19 +15,8 @@ import java.net.InetSocketAddress import scala.reflect.BeanProperty import akka.util. {ReflectiveAccess, Logging, Duration} -import akka.japi.Procedure - -/** - * Extend this abstract class to create a remote actor. - * - * Equivalent to invoking the
makeRemote(..) method in the body of the ActorJonas Bonér
- */
-abstract class RemoteActor(address: InetSocketAddress) extends Actor {
- def this(hostname: String, port: Int) = this(new InetSocketAddress(hostname, port))
- self.makeRemote(address)
-}
+import akka.remoteinterface.RemoteSupport
+import akka.japi. {Creator, Procedure}
/**
* Life-cycle messages for the Actors
@@ -92,7 +81,6 @@ case class UnhandledMessageException(msg: Any, ref: ActorRef) extends Exception
* @author Jonas Bonér
*/
object Actor extends Logging {
-
/**
* Add shutdown cleanups
*/
@@ -115,8 +103,18 @@ object Actor extends Logging {
hook
}
- val TIMEOUT = Duration(config.getInt("akka.actor.timeout", 5), TIME_UNIT).toMillis
- val SERIALIZE_MESSAGES = config.getBool("akka.actor.serialize-messages", false)
+ val registry = new ActorRegistry
+
+ lazy val remote: RemoteSupport = {
+ ReflectiveAccess.
+ Remote.
+ defaultRemoteSupport.
+ map(_()).
+ getOrElse(throw new UnsupportedOperationException("You need to have akka-remote on classpath"))
+ }
+
+ private[akka] val TIMEOUT = Duration(config.getInt("akka.actor.timeout", 5), TIME_UNIT).toMillis
+ private[akka] val SERIALIZE_MESSAGES = config.getBool("akka.actor.serialize-messages", false)
/**
* A Receive is a convenience type that defines actor message behavior currently modeled as
@@ -126,8 +124,10 @@ object Actor extends Logging {
private[actor] val actorRefInCreation = new scala.util.DynamicVariable[Option[ActorRef]](None)
- /**
- * Creates an ActorRef out of the Actor with type T.
+
+
+ /**
+ * Creates an ActorRef out of the Actor with type T.
*
* import Actor._
* val actor = actorOf[MyActor]
@@ -143,17 +143,17 @@ object Actor extends Logging {
def actorOf[T <: Actor : Manifest]: ActorRef = actorOf(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
/**
- * Creates an ActorRef out of the Actor with type T.
+ * Creates an ActorRef out of the Actor of the specified Class.
*
* import Actor._
- * val actor = actorOf[MyActor]
+ * val actor = actorOf(classOf[MyActor])
* actor.start
* actor ! message
* actor.stop
*
* You can create and start the actor in one statement like this:
*
- * val actor = actorOf[MyActor].start
+ * val actor = actorOf(classOf[MyActor]).start
*
*/
def actorOf(clazz: Class[_ <: Actor]): ActorRef = new LocalActorRef(() => {
@@ -164,7 +164,7 @@ object Actor extends Logging {
"\nMake sure Actor is NOT defined inside a class/trait," +
"\nif so put it outside the class/trait, f.e. in a companion object," +
"\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'."))
- })
+ }, None)
/**
* Creates an ActorRef out of the Actor. Allows you to pass in a factory function
@@ -184,7 +184,17 @@ object Actor extends Logging {
* val actor = actorOf(new MyActor).start
*
*/
- def actorOf(factory: => Actor): ActorRef = new LocalActorRef(() => factory)
+ def actorOf(factory: => Actor): ActorRef = new LocalActorRef(() => factory, None)
+
+ /**
+ * Creates an ActorRef out of the Actor. Allows you to pass in a factory (Creator
- * import Actor._
+ * import Actor.{spawn}
*
* spawn {
* ... // do stuff
@@ -210,7 +220,6 @@ object Actor extends Logging {
}
}).start ! Spawn
}
-
/**
* Implicitly converts the given Option[Any] to a AnyOptionAsTypedOption which offers the method as[T]
* to convert an Option[Any] to an Option[T].
diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala
index 0c911ff643..84769371ab 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala
@@ -77,8 +77,6 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
@volatile
protected[this] var _status: ActorRefInternals.StatusType = ActorRefInternals.UNSTARTED
@volatile
- protected[akka] var _homeAddress = new InetSocketAddress(RemoteServerModule.HOSTNAME, RemoteServerModule.PORT)
- @volatile
protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None
protected[akka] val guard = new ReentrantGuard
@@ -174,7 +172,17 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
def getDispatcher(): MessageDispatcher = dispatcher
/**
- * Holds the hot swapped partial function.
+ * Returns on which node this actor lives if None it lives in the local ActorRegistry
+ */
+ def homeAddress: Option[InetSocketAddress]
+
+ /**
+ * Java API
+ */
+ def getHomeAddress(): InetSocketAddress = homeAddress getOrElse null
+
+ /**
+ * Holds the hot swapped partial function.
*/
@volatile
protected[akka] var hotswap = Stack[PartialFunction[Any, Unit]]()
@@ -390,62 +398,6 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
*/
def dispatcher: MessageDispatcher
- /**
- * Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
- */
- def makeRemote(hostname: String, port: Int): Unit
-
- /**
- * Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
- */
- def makeRemote(address: InetSocketAddress): Unit
-
- /**
- * Returns the home address and port for this actor.
- */
- def homeAddress: InetSocketAddress = _homeAddress
-
- /**
- * Akka Java API
- * Returns the home address and port for this actor.
- */
- def getHomeAddress(): InetSocketAddress = homeAddress
-
- /**
- * Set the home address and port for this actor.
- */
- def homeAddress_=(hostnameAndPort: Tuple2[String, Int]): Unit =
- homeAddress_=(new InetSocketAddress(hostnameAndPort._1, hostnameAndPort._2))
-
- /**
- * Akka Java API
- * Set the home address and port for this actor.
- */
- def setHomeAddress(hostname: String, port: Int): Unit = homeAddress = (hostname, port)
-
- /**
- * Set the home address and port for this actor.
- */
- def homeAddress_=(address: InetSocketAddress): Unit
-
- /**
- * Akka Java API
- * Set the home address and port for this actor.
- */
- def setHomeAddress(address: InetSocketAddress): Unit = homeAddress = address
-
- /**
- * Returns the remote address for the actor, if any, else None.
- */
- def remoteAddress: Option[InetSocketAddress]
- protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit
-
- /**
- * Akka Java API
- * Gets the remote address for the actor, if any, else None.
- */
- def getRemoteAddress(): Option[InetSocketAddress] = remoteAddress
-
/**
* Starts up the actor and its message queue.
*/
@@ -499,7 +451,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
*
* To be invoked from within the actor itself.
*/
- def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef
/**
* Atomically create (from actor class), link and start an actor.
@@ -513,7 +465,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
*
* To be invoked from within the actor itself.
*/
- def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef
/**
* Returns the mailbox size.
@@ -598,11 +550,10 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal
* @author Jonas Bonér
*/
class LocalActorRef private[akka] (
- private[this] val actorFactory: () => Actor)
+ private[this] val actorFactory: () => Actor,
+ val homeAddress: Option[InetSocketAddress])
extends ActorRef with ScalaActorRef {
- @volatile
- private[akka] var _remoteAddress: Option[InetSocketAddress] = None // only mutable to maintain identity across nodes
@volatile
private[akka] lazy val _linkedActors = new ConcurrentHashMap[Uuid, ActorRef]
@volatile
@@ -622,20 +573,19 @@ class LocalActorRef private[akka] (
if (isRunning) initializeActorInstance
// used only for deserialization
- private[akka] def this(__uuid: Uuid,
+ private[akka] def this(
+ __uuid: Uuid,
__id: String,
- __hostname: String,
- __port: Int,
__timeout: Long,
__receiveTimeout: Option[Long],
__lifeCycle: LifeCycle,
__supervisor: Option[ActorRef],
__hotswap: Stack[PartialFunction[Any, Unit]],
- __factory: () => Actor) = {
- this(__factory)
+ __factory: () => Actor,
+ __homeAddress: Option[InetSocketAddress]) = {
+ this(__factory, __homeAddress)
_uuid = __uuid
id = __id
- homeAddress = (__hostname, __port)
timeout = __timeout
receiveTimeout = __receiveTimeout
lifeCycle = __lifeCycle
@@ -643,9 +593,13 @@ class LocalActorRef private[akka] (
hotswap = __hotswap
setActorSelfFields(actor,this)
start
- ActorRegistry.register(this)
}
+ /**
+ * Returns whether this actor ref is client-managed remote or not
+ */
+ private[akka] final def isClientManaged_? = homeAddress.isDefined && isRemotingEnabled
+
// ========= PUBLIC FUNCTIONS =========
/**
@@ -674,41 +628,6 @@ class LocalActorRef private[akka] (
*/
def dispatcher: MessageDispatcher = _dispatcher
- /**
- * Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
- */
- def makeRemote(hostname: String, port: Int): Unit = {
- ensureRemotingEnabled
- if (!isRunning || isBeingRestarted) makeRemote(new InetSocketAddress(hostname, port))
- else throw new ActorInitializationException(
- "Can't make a running actor remote. Make sure you call 'makeRemote' before 'start'.")
- }
-
- /**
- * Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
- */
- def makeRemote(address: InetSocketAddress): Unit = guard.withGuard {
- ensureRemotingEnabled
- if (!isRunning || isBeingRestarted) {
- _remoteAddress = Some(address)
- RemoteClientModule.register(address, uuid)
- homeAddress = (RemoteServerModule.HOSTNAME, RemoteServerModule.PORT)
- } else throw new ActorInitializationException(
- "Can't make a running actor remote. Make sure you call 'makeRemote' before 'start'.")
- }
-
- /**
- * Set the contact address for this actor. This is used for replying to messages
- * sent asynchronously when no reply channel exists.
- */
- def homeAddress_=(address: InetSocketAddress): Unit = _homeAddress = address
-
- /**
- * Returns the remote address for the actor, if any, else None.
- */
- def remoteAddress: Option[InetSocketAddress] = _remoteAddress
- protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = _remoteAddress = addr
-
/**
* Starts up the actor and its message queue.
*/
@@ -724,6 +643,9 @@ class LocalActorRef private[akka] (
if ((actorInstance ne null) && (actorInstance.get ne null))
initializeActorInstance
+ if (isClientManaged_?)
+ Actor.remote.registerClientManagedActor(homeAddress.get.getHostName,homeAddress.get.getPort, uuid)
+
checkReceiveTimeout //Schedule the initial Receive timeout
}
this
@@ -739,11 +661,11 @@ class LocalActorRef private[akka] (
dispatcher.detach(this)
_status = ActorRefInternals.SHUTDOWN
actor.postStop
- ActorRegistry.unregister(this)
+ Actor.registry.unregister(this)
if (isRemotingEnabled) {
- if (remoteAddress.isDefined)
- RemoteClientModule.unregister(remoteAddress.get, uuid)
- RemoteServerModule.unregister(this)
+ if (isClientManaged_?)
+ Actor.remote.registerClientManagedActor(homeAddress.get.getHostName,homeAddress.get.getPort, uuid)
+ Actor.remote.unregister(this)
}
setActorSelfFields(actorInstance.get,null)
} //else if (isBeingRestarted) throw new ActorKilledException("Actor [" + toString + "] is being restarted.")
@@ -795,11 +717,8 @@ class LocalActorRef private[akka] (
*
* To be invoked from within the actor itself.
*/
- def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = guard.withGuard {
- ensureRemotingEnabled
- actorRef.makeRemote(hostname, port)
- link(actorRef)
- actorRef.start
+ def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = {
+ //TODO: REVISIT: REMOVED
}
/**
@@ -816,12 +735,11 @@ class LocalActorRef private[akka] (
*
* To be invoked from within the actor itself.
*/
- def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = guard.withGuard {
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long = Actor.TIMEOUT): ActorRef = guard.withGuard {
ensureRemotingEnabled
- val actor = Actor.actorOf(clazz)
- actor.makeRemote(hostname, port)
- actor.start
- actor
+ val ref = Actor.remote.actorOf(clazz, hostname, port)
+ ref.timeout = timeout
+ ref.start
}
/**
@@ -841,14 +759,15 @@ class LocalActorRef private[akka] (
*
* To be invoked from within the actor itself.
*/
- def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = guard.withGuard {
- ensureRemotingEnabled
- val actor = Actor.actorOf(clazz)
- actor.makeRemote(hostname, port)
- link(actor)
- actor.start
- actor
- }
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long = Actor.TIMEOUT): ActorRef =
+ guard.withGuard {
+ ensureRemotingEnabled
+ val actor = Actor.remote.actorOf(clazz, hostname, port)
+ actor.timeout = timeout
+ link(actor)
+ actor.start
+ actor
+ }
/**
* Returns the mailbox.
@@ -877,32 +796,26 @@ class LocalActorRef private[akka] (
protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = _supervisor = sup
- protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = {
- if (remoteAddress.isDefined && isRemotingEnabled) {
- RemoteClientModule.send[Any](
- message, senderOption, None, remoteAddress.get, timeout, true, this, None, ActorType.ScalaActor)
- } else {
- val invocation = new MessageInvocation(this, message, senderOption, None)
- dispatcher dispatchMessage invocation
- }
- }
+ protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit =
+ if (isClientManaged_?) {
+ Actor.remote.send[Any](
+ message, senderOption, None, homeAddress.get, timeout, true, this, None, ActorType.ScalaActor, None)
+ } else
+ dispatcher dispatchMessage new MessageInvocation(this, message, senderOption, None)
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T](
message: Any,
timeout: Long,
senderOption: Option[ActorRef],
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
-
- if (remoteAddress.isDefined && isRemotingEnabled) {
- val future = RemoteClientModule.send[T](
- message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, ActorType.ScalaActor)
+ if (isClientManaged_?) {
+ val future = Actor.remote.send[T](
+ message, senderOption, senderFuture, homeAddress.get, timeout, false, this, None, ActorType.ScalaActor, None)
if (future.isDefined) future.get
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
} else {
val future = if (senderFuture.isDefined) senderFuture else Some(new DefaultCompletableFuture[T](timeout))
- val invocation = new MessageInvocation(
- this, message, senderOption, future.asInstanceOf[Some[CompletableFuture[Any]]])
- dispatcher dispatchMessage invocation
+ dispatcher dispatchMessage new MessageInvocation(this, message, senderOption, future.asInstanceOf[Some[CompletableFuture[Any]]])
future.get
}
}
@@ -1064,7 +977,8 @@ class LocalActorRef private[akka] (
protected[akka] def registerSupervisorAsRemoteActor: Option[Uuid] = guard.withGuard {
ensureRemotingEnabled
if (_supervisor.isDefined) {
- remoteAddress.foreach(address => RemoteClientModule.registerSupervisorForActor(address, this))
+ if (homeAddress.isDefined)
+ Actor.remote.registerSupervisorForActor(this)
Some(_supervisor.get.uuid)
} else None
}
@@ -1171,34 +1085,8 @@ class LocalActorRef private[akka] (
private def initializeActorInstance = {
actor.preStart // run actor preStart
Actor.log.slf4j.trace("[{}] has started", toString)
- ActorRegistry.register(this)
+ Actor.registry.register(this)
}
-
- /*
- private def serializeMessage(message: AnyRef): AnyRef = if (Actor.SERIALIZE_MESSAGES) {
- if (!message.isInstanceOf[String] &&
- !message.isInstanceOf[Byte] &&
- !message.isInstanceOf[Int] &&
- !message.isInstanceOf[Long] &&
- !message.isInstanceOf[Float] &&
- !message.isInstanceOf[Double] &&
- !message.isInstanceOf[Boolean] &&
- !message.isInstanceOf[Char] &&
- !message.isInstanceOf[Tuple2[_, _]] &&
- !message.isInstanceOf[Tuple3[_, _, _]] &&
- !message.isInstanceOf[Tuple4[_, _, _, _]] &&
- !message.isInstanceOf[Tuple5[_, _, _, _, _]] &&
- !message.isInstanceOf[Tuple6[_, _, _, _, _, _]] &&
- !message.isInstanceOf[Tuple7[_, _, _, _, _, _, _]] &&
- !message.isInstanceOf[Tuple8[_, _, _, _, _, _, _, _]] &&
- !message.getClass.isArray &&
- !message.isInstanceOf[List[_]] &&
- !message.isInstanceOf[scala.collection.immutable.Map[_, _]] &&
- !message.isInstanceOf[scala.collection.immutable.Set[_]]) {
- Serializer.Java.deepClone(message)
- } else message
- } else message
- */
}
/**
@@ -1228,31 +1116,33 @@ private[akka] case class RemoteActorRef private[akka] (
ensureRemotingEnabled
- _homeAddress = new InetSocketAddress(hostname, port)
- val remoteAddress: Option[InetSocketAddress] = Some(_homeAddress)
+ val homeAddress = Some(new InetSocketAddress(hostname, port))
+ //protected def clientManaged = classOrServiceName.isEmpty //If no class or service name, it's client managed
id = classOrServiceName
+ //id = classOrServiceName.getOrElse("uuid:" + uuid) //If we're a server-managed we want to have classOrServiceName as id, or else, we're a client-managed and we want to have our uuid as id
+
timeout = _timeout
start
def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit =
- RemoteClientModule.send[Any](
- message, senderOption, None, remoteAddress.get, timeout, true, this, None, actorType, loader)
+ Actor.remote.send[Any](message, senderOption, None, homeAddress.get, timeout, true, this, None, actorType, loader)
def postMessageToMailboxAndCreateFutureResultWithTimeout[T](
message: Any,
timeout: Long,
senderOption: Option[ActorRef],
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
- val future = RemoteClientModule.send[T](
- message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, actorType, loader)
+ val future = Actor.remote.send[T](message, senderOption, senderFuture, homeAddress.get, timeout, false, this, None, actorType, loader)
if (future.isDefined) future.get
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
}
def start: ActorRef = synchronized {
_status = ActorRefInternals.RUNNING
+ //if (clientManaged)
+ // Actor.remote.registerClientManagedActor(homeAddress.getHostName,homeAddress.getPort, uuid)
this
}
@@ -1260,6 +1150,8 @@ private[akka] case class RemoteActorRef private[akka] (
if (_status == ActorRefInternals.RUNNING) {
_status = ActorRefInternals.SHUTDOWN
postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
+ // if (clientManaged)
+ // Actor.remote.unregisterClientManagedActor(homeAddress.getHostName,homeAddress.getPort, uuid)
}
}
@@ -1269,17 +1161,14 @@ private[akka] case class RemoteActorRef private[akka] (
def actorClass: Class[_ <: Actor] = unsupported
def dispatcher_=(md: MessageDispatcher): Unit = unsupported
def dispatcher: MessageDispatcher = unsupported
- def makeRemote(hostname: String, port: Int): Unit = unsupported
- def makeRemote(address: InetSocketAddress): Unit = unsupported
- def homeAddress_=(address: InetSocketAddress): Unit = unsupported
def link(actorRef: ActorRef): Unit = unsupported
def unlink(actorRef: ActorRef): Unit = unsupported
def startLink(actorRef: ActorRef): Unit = unsupported
def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = unsupported
def spawn(clazz: Class[_ <: Actor]): ActorRef = unsupported
- def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef = unsupported
def spawnLink(clazz: Class[_ <: Actor]): ActorRef = unsupported
- def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long): ActorRef = unsupported
def supervisor: Option[ActorRef] = unsupported
def shutdownLinkedActors: Unit = unsupported
protected[akka] def mailbox: AnyRef = unsupported
@@ -1289,7 +1178,6 @@ private[akka] case class RemoteActorRef private[akka] (
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def linkedActors: JMap[Uuid, ActorRef] = unsupported
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
- protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported
protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = unsupported
protected[akka] def actorInstance: AtomicReference[Actor] = unsupported
private def unsupported = throw new UnsupportedOperationException("Not supported for RemoteActorRef")
@@ -1517,9 +1405,9 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
/**
* Atomically create (from actor class), start and make an actor remote.
*/
- def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef = {
+ def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int, timeout: Long): ActorRef = {
ensureRemotingEnabled
- spawnRemote(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]], hostname, port)
+ spawnRemote(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]], hostname, port, timeout)
}
/**
@@ -1531,9 +1419,9 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
/**
* Atomically create (from actor class), start, link and make an actor remote.
*/
- def spawnLinkRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef = {
+ def spawnLinkRemote[T <: Actor: Manifest](hostname: String, port: Int, timeout: Long): ActorRef = {
ensureRemotingEnabled
- spawnLinkRemote(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]], hostname, port)
+ spawnLinkRemote(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]], hostname, port, timeout)
}
}
diff --git a/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala b/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala
index 84ecf37f1c..18fa30d740 100644
--- a/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala
+++ b/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala
@@ -12,8 +12,7 @@ import java.util.{Set => JSet}
import annotation.tailrec
import akka.util.ReflectiveAccess._
-import akka.util.{ReadWriteGuard, Address, ListenerManagement}
-import java.net.InetSocketAddress
+import akka.util. {ReflectiveAccess, ReadWriteGuard, ListenerManagement}
/**
* Base trait for ActorRegistry events, allows listen to when an actor is added and removed from the ActorRegistry.
@@ -36,10 +35,11 @@ case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent
*
* @author Jonas Bonér
*/
-object ActorRegistry extends ListenerManagement {
+
+private[actor] final class ActorRegistry private[actor] () extends ListenerManagement {
+
private val actorsByUUID = new ConcurrentHashMap[Uuid, ActorRef]
private val actorsById = new Index[String,ActorRef]
- private val remoteActorSets = Map[Address, RemoteActorSet]()
private val guard = new ReadWriteGuard
/**
@@ -74,7 +74,7 @@ object ActorRegistry extends ListenerManagement {
}
/**
- * Finds all actors that are subtypes of the class passed in as the Manifest argument and supproting passed message.
+ * Finds all actors that are subtypes of the class passed in as the Manifest argument and supporting passed message.
*/
def actorsFor[T <: Actor](message: Any)(implicit manifest: Manifest[T] ): Array[ActorRef] =
filter(a => manifest.erasure.isAssignableFrom(a.actor.getClass) && a.isDefinedAt(message))
@@ -99,7 +99,7 @@ object ActorRegistry extends ListenerManagement {
actorsFor[T](manifest.erasure.asInstanceOf[Class[T]])
/**
- * Finds any actor that matches T.
+ * Finds any actor that matches T. Very expensive, traverses ALL alive actors.
*/
def actorFor[T <: Actor](implicit manifest: Manifest[T]): Option[ActorRef] =
find({ case a: ActorRef if manifest.erasure.isAssignableFrom(a.actor.getClass) => a })
@@ -223,16 +223,15 @@ object ActorRegistry extends ListenerManagement {
TypedActorModule.typedActorObjectInstance.get.proxyFor(actorRef)
}
-
/**
- * Registers an actor in the ActorRegistry.
+ * Registers an actor in the ActorRegistry.
*/
- private[akka] def register(actor: ActorRef) = {
- // ID
- actorsById.put(actor.id, actor)
+ private[akka] def register(actor: ActorRef) {
+ val id = actor.id
+ val uuid = actor.uuid
- // UUID
- actorsByUUID.put(actor.uuid, actor)
+ actorsById.put(id, actor)
+ actorsByUUID.put(uuid, actor)
// notify listeners
notifyListeners(ActorRegistered(actor))
@@ -241,10 +240,12 @@ object ActorRegistry extends ListenerManagement {
/**
* Unregisters an actor in the ActorRegistry.
*/
- private[akka] def unregister(actor: ActorRef) = {
- actorsByUUID remove actor.uuid
+ private[akka] def unregister(actor: ActorRef) {
+ val id = actor.id
+ val uuid = actor.uuid
- actorsById.remove(actor.id,actor)
+ actorsByUUID remove uuid
+ actorsById.remove(id,actor)
// notify listeners
notifyListeners(ActorUnregistered(actor))
@@ -264,41 +265,13 @@ object ActorRegistry extends ListenerManagement {
else actorRef.stop
}
} else foreach(_.stop)
+ if (Remote.isEnabled) {
+ Actor.remote.clear //TODO: REVISIT: Should this be here?
+ }
actorsByUUID.clear
actorsById.clear
log.slf4j.info("All actors have been shut down and unregistered from ActorRegistry")
}
-
- /**
- * Get the remote actors for the given server address. For internal use only.
- */
- private[akka] def actorsFor(remoteServerAddress: Address): RemoteActorSet = guard.withWriteGuard {
- remoteActorSets.getOrElseUpdate(remoteServerAddress, new RemoteActorSet)
- }
-
- private[akka] def registerActorByUuid(address: InetSocketAddress, uuid: String, actor: ActorRef) {
- actorsByUuid(Address(address.getHostName, address.getPort)).putIfAbsent(uuid, actor)
- }
-
- private[akka] def registerTypedActorByUuid(address: InetSocketAddress, uuid: String, typedActor: AnyRef) {
- typedActorsByUuid(Address(address.getHostName, address.getPort)).putIfAbsent(uuid, typedActor)
- }
-
- private[akka] def actors(address: Address) = actorsFor(address).actors
- private[akka] def actorsByUuid(address: Address) = actorsFor(address).actorsByUuid
- private[akka] def actorsFactories(address: Address) = actorsFor(address).actorsFactories
- private[akka] def typedActors(address: Address) = actorsFor(address).typedActors
- private[akka] def typedActorsByUuid(address: Address) = actorsFor(address).typedActorsByUuid
- private[akka] def typedActorsFactories(address: Address) = actorsFor(address).typedActorsFactories
-
- private[akka] class RemoteActorSet {
- private[ActorRegistry] val actors = new ConcurrentHashMap[String, ActorRef]
- private[ActorRegistry] val actorsByUuid = new ConcurrentHashMap[String, ActorRef]
- private[ActorRegistry] val actorsFactories = new ConcurrentHashMap[String, () => ActorRef]
- private[ActorRegistry] val typedActors = new ConcurrentHashMap[String, AnyRef]
- private[ActorRegistry] val typedActorsByUuid = new ConcurrentHashMap[String, AnyRef]
- private[ActorRegistry] val typedActorsFactories = new ConcurrentHashMap[String, () => AnyRef]
- }
}
/**
diff --git a/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala
index 2288a1cba9..6600f486a5 100644
--- a/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala
+++ b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala
@@ -62,6 +62,6 @@ trait BootableActorLoaderService extends Bootable with Logging {
abstract override def onUnload = {
super.onUnload
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
}
}
diff --git a/akka-actor/src/main/scala/akka/actor/Supervisor.scala b/akka-actor/src/main/scala/akka/actor/Supervisor.scala
index 6755ae78a4..efba95aa07 100644
--- a/akka-actor/src/main/scala/akka/actor/Supervisor.scala
+++ b/akka-actor/src/main/scala/akka/actor/Supervisor.scala
@@ -130,7 +130,7 @@ sealed class Supervisor(handler: FaultHandlingStrategy) {
case SupervisorConfig(_, servers) =>
servers.map(server =>
server match {
- case Supervise(actorRef, lifeCycle, remoteAddress) =>
+ case Supervise(actorRef, lifeCycle, registerAsRemoteService) =>
actorRef.start
val className = actorRef.actor.getClass.getName
val currentActors = {
@@ -141,10 +141,8 @@ sealed class Supervisor(handler: FaultHandlingStrategy) {
_childActors.put(className, actorRef :: currentActors)
actorRef.lifeCycle = lifeCycle
supervisor.link(actorRef)
- if (remoteAddress.isDefined) {
- val address = remoteAddress.get
- RemoteServerModule.registerActor(new InetSocketAddress(address.hostname, address.port), actorRef)
- }
+ if (registerAsRemoteService)
+ Actor.remote.register(actorRef)
case supervisorConfig @ SupervisorConfig(_, _) => // recursive supervisor configuration
val childSupervisor = Supervisor(supervisorConfig)
supervisor.link(childSupervisor.supervisor)
diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala
index ad3cc7fdf2..ac7daa2dc6 100644
--- a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala
+++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala
@@ -43,14 +43,14 @@ import scala.reflect.BeanProperty
*
* } else if (msg.equals("ForwardMessage")) {
* // Retreive an actor from the ActorRegistry by ID and get an ActorRef back
- * ActorRef actorRef = ActorRegistry.actorsFor("some-actor-id").head();
+ * ActorRef actorRef = Actor.registry.actorsFor("some-actor-id").head();
*
* } else throw new IllegalArgumentException("Unknown message: " + message);
* } else throw new IllegalArgumentException("Unknown message: " + message);
* }
*
* public static void main(String[] args) {
- * ActorRef actor = UntypedActor.actorOf(SampleUntypedActor.class);
+ * ActorRef actor = Actors.actorOf(SampleUntypedActor.class);
* actor.start();
* actor.sendOneWay("SendToSelf");
* actor.stop();
@@ -86,76 +86,8 @@ abstract class UntypedActor extends Actor {
}
/**
- * Factory closure for an UntypedActor, to be used with 'UntypedActor.actorOf(factory)'.
+ * Factory closure for an UntypedActor, to be used with 'Actors.actorOf(factory)'.
*
* @author Jonas Bonér
*/
-trait UntypedActorFactory extends Creator[Actor]
-
-/**
- * Extend this abstract class to create a remote UntypedActor.
- *
- * @author Jonas Bonér
- */
-abstract class RemoteUntypedActor(address: InetSocketAddress) extends UntypedActor {
- def this(hostname: String, port: Int) = this(new InetSocketAddress(hostname, port))
- self.makeRemote(address)
-}
-
-/**
- * Factory object for creating and managing 'UntypedActor's. Meant to be used from Java.
- *
- * Example on how to create an actor:
- *
- * ActorRef actor = UntypedActor.actorOf(MyUntypedActor.class);
- * actor.start();
- * actor.sendOneWay(message, context)
- * actor.stop();
- *
- * You can create and start the actor in one statement like this:
- *
- * ActorRef actor = UntypedActor.actorOf(MyUntypedActor.class).start();
- *
- *
- * @author Jonas Bonér
- */
-object UntypedActor {
-
- /**
- * Creates an ActorRef out of the Actor type represented by the class provided.
- * Example in Java:
- *
- * ActorRef actor = UntypedActor.actorOf(MyUntypedActor.class);
- * actor.start();
- * actor.sendOneWay(message, context);
- * actor.stop();
- *
- * You can create and start the actor in one statement like this:
- *
- * val actor = actorOf(classOf[MyActor]).start
- *
- */
- def actorOf[T <: Actor](clazz: Class[T]): ActorRef = Actor.actorOf(clazz)
-
- /**
- * NOTE: Use this convenience method with care, do NOT make it possible to get a reference to the
- * UntypedActor instance directly, but only through its 'ActorRef' wrapper reference.
- *
- * Creates an ActorRef out of the Actor. Allows you to pass in the instance for the UntypedActor.
- * Only use this method when you need to pass in constructor arguments into the 'UntypedActor'.
- *
- * You use it by implementing the UntypedActorFactory interface.
- * Example in Java:
- *
- * ActorRef actor = UntypedActor.actorOf(new UntypedActorFactory() {
- * public UntypedActor create() {
- * return new MyUntypedActor("service:name", 5);
- * }
- * });
- * actor.start();
- * actor.sendOneWay(message, context);
- * actor.stop();
- *
- */
- def actorOf(factory: UntypedActorFactory): ActorRef = Actor.actorOf(factory.create)
-}
+trait UntypedActorFactory extends Creator[Actor]
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/akka/config/SupervisionConfig.scala b/akka-actor/src/main/scala/akka/config/SupervisionConfig.scala
index d0a8e7b84c..9f63c64bc1 100644
--- a/akka-actor/src/main/scala/akka/config/SupervisionConfig.scala
+++ b/akka-actor/src/main/scala/akka/config/SupervisionConfig.scala
@@ -26,20 +26,16 @@ object Supervision {
def this(restartStrategy: FaultHandlingStrategy, worker: Array[Server]) = this(restartStrategy,worker.toList)
}
- class Supervise(val actorRef: ActorRef, val lifeCycle: LifeCycle, val remoteAddress: Option[RemoteAddress]) extends Server {
- //Java API
- def this(actorRef: ActorRef, lifeCycle: LifeCycle, remoteAddress: RemoteAddress) =
- this(actorRef, lifeCycle, Option(remoteAddress))
-
+ class Supervise(val actorRef: ActorRef, val lifeCycle: LifeCycle, val registerAsRemoteService: Boolean = false) extends Server {
//Java API
def this(actorRef: ActorRef, lifeCycle: LifeCycle) =
- this(actorRef, lifeCycle, None)
+ this(actorRef, lifeCycle, false)
}
object Supervise {
- def apply(actorRef: ActorRef, lifeCycle: LifeCycle, remoteAddress: RemoteAddress) = new Supervise(actorRef, lifeCycle, remoteAddress)
- def apply(actorRef: ActorRef, lifeCycle: LifeCycle) = new Supervise(actorRef, lifeCycle, None)
- def unapply(supervise: Supervise) = Some((supervise.actorRef, supervise.lifeCycle, supervise.remoteAddress))
+ def apply(actorRef: ActorRef, lifeCycle: LifeCycle, registerAsRemoteService: Boolean = false) = new Supervise(actorRef, lifeCycle, registerAsRemoteService)
+ def apply(actorRef: ActorRef, lifeCycle: LifeCycle) = new Supervise(actorRef, lifeCycle, false)
+ def unapply(supervise: Supervise) = Some((supervise.actorRef, supervise.lifeCycle, supervise.registerAsRemoteService))
}
object AllForOneStrategy {
diff --git a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala
index bc8ed1805a..bca7650a10 100644
--- a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala
+++ b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala
@@ -132,7 +132,7 @@ trait MessageDispatcher extends MailboxFactory with Logging {
val i = uuids.iterator
while(i.hasNext()) {
val uuid = i.next()
- ActorRegistry.actorFor(uuid) match {
+ Actor.registry.actorFor(uuid) match {
case Some(actor) => actor.stop
case None =>
log.slf4j.error("stopAllLinkedActors couldn't find linked actor: " + uuid)
diff --git a/akka-actor/src/main/scala/akka/remoteinterface/RemoteInterface.scala b/akka-actor/src/main/scala/akka/remoteinterface/RemoteInterface.scala
new file mode 100644
index 0000000000..f27ba25dc8
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/remoteinterface/RemoteInterface.scala
@@ -0,0 +1,430 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.remoteinterface
+
+import akka.japi.Creator
+import java.net.InetSocketAddress
+import akka.actor._
+import akka.util._
+import akka.dispatch.CompletableFuture
+import akka.config.Config.{config, TIME_UNIT}
+import java.util.concurrent.ConcurrentHashMap
+import akka.AkkaException
+import reflect.BeanProperty
+
+trait RemoteModule extends Logging {
+ val UUID_PREFIX = "uuid:"
+
+ def optimizeLocalScoped_?(): Boolean //Apply optimizations for remote operations in local scope
+ protected[akka] def notifyListeners(message: => Any): Unit
+
+
+ private[akka] def actors: ConcurrentHashMap[String, ActorRef]
+ private[akka] def actorsByUuid: ConcurrentHashMap[String, ActorRef]
+ private[akka] def actorsFactories: ConcurrentHashMap[String, () => ActorRef]
+ private[akka] def typedActors: ConcurrentHashMap[String, AnyRef]
+ private[akka] def typedActorsByUuid: ConcurrentHashMap[String, AnyRef]
+ private[akka] def typedActorsFactories: ConcurrentHashMap[String, () => AnyRef]
+
+
+ /** Lookup methods **/
+
+ private[akka] def findActorById(id: String) : ActorRef = actors.get(id)
+
+ private[akka] def findActorByUuid(uuid: String) : ActorRef = actorsByUuid.get(uuid)
+
+ private[akka] def findActorFactory(id: String) : () => ActorRef = actorsFactories.get(id)
+
+ private[akka] def findTypedActorById(id: String) : AnyRef = typedActors.get(id)
+
+ private[akka] def findTypedActorFactory(id: String) : () => AnyRef = typedActorsFactories.get(id)
+
+ private[akka] def findTypedActorByUuid(uuid: String) : AnyRef = typedActorsByUuid.get(uuid)
+
+ private[akka] def findActorByIdOrUuid(id: String, uuid: String) : ActorRef = {
+ var actorRefOrNull = if (id.startsWith(UUID_PREFIX)) findActorByUuid(id.substring(UUID_PREFIX.length))
+ else findActorById(id)
+ if (actorRefOrNull eq null) actorRefOrNull = findActorByUuid(uuid)
+ actorRefOrNull
+ }
+
+ private[akka] def findTypedActorByIdOrUuid(id: String, uuid: String) : AnyRef = {
+ var actorRefOrNull = if (id.startsWith(UUID_PREFIX)) findTypedActorByUuid(id.substring(UUID_PREFIX.length))
+ else findTypedActorById(id)
+ if (actorRefOrNull eq null) actorRefOrNull = findTypedActorByUuid(uuid)
+ actorRefOrNull
+ }
+}
+
+/**
+ * Life-cycle events for RemoteClient.
+ */
+sealed trait RemoteClientLifeCycleEvent //TODO: REVISIT: Document change from RemoteClient to RemoteClientModule + remoteAddress
+case class RemoteClientError(
+ @BeanProperty cause: Throwable,
+ @BeanProperty client: RemoteClientModule, remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
+case class RemoteClientDisconnected(
+ @BeanProperty client: RemoteClientModule, remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
+case class RemoteClientConnected(
+ @BeanProperty client: RemoteClientModule, remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
+case class RemoteClientStarted(
+ @BeanProperty client: RemoteClientModule, remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
+case class RemoteClientShutdown(
+ @BeanProperty client: RemoteClientModule, remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
+case class RemoteClientWriteFailed(
+ @BeanProperty request: AnyRef,
+ @BeanProperty cause: Throwable,
+ @BeanProperty client: RemoteClientModule, remoteAddress: InetSocketAddress) extends RemoteClientLifeCycleEvent
+
+
+/**
+ * Life-cycle events for RemoteServer.
+ */
+sealed trait RemoteServerLifeCycleEvent //TODO: REVISIT: Document change from RemoteServer to RemoteServerModule
+case class RemoteServerStarted(
+ @BeanProperty val server: RemoteServerModule) extends RemoteServerLifeCycleEvent
+case class RemoteServerShutdown(
+ @BeanProperty val server: RemoteServerModule) extends RemoteServerLifeCycleEvent
+case class RemoteServerError(
+ @BeanProperty val cause: Throwable,
+ @BeanProperty val server: RemoteServerModule) extends RemoteServerLifeCycleEvent
+case class RemoteServerClientConnected(
+ @BeanProperty val server: RemoteServerModule,
+ @BeanProperty val clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
+case class RemoteServerClientDisconnected(
+ @BeanProperty val server: RemoteServerModule,
+ @BeanProperty val clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
+case class RemoteServerClientClosed(
+ @BeanProperty val server: RemoteServerModule,
+ @BeanProperty val clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
+case class RemoteServerWriteFailed(
+ @BeanProperty request: AnyRef,
+ @BeanProperty cause: Throwable,
+ @BeanProperty client: RemoteServerModule, remoteAddress: InetSocketAddress) extends RemoteServerLifeCycleEvent
+
+/**
+ * Thrown for example when trying to send a message using a RemoteClient that is either not started or shut down.
+ */
+class RemoteClientException private[akka] (message: String,
+ @BeanProperty val client: RemoteClientModule,
+ val remoteAddress: InetSocketAddress) extends AkkaException(message)
+
+/**
+ * Returned when a remote exception cannot be instantiated or parsed
+ */
+case class UnparsableException private[akka] (originalClassName: String,
+ originalMessage: String) extends AkkaException(originalMessage)
+
+
+abstract class RemoteSupport extends ListenerManagement with RemoteServerModule with RemoteClientModule {
+ def shutdown {
+ this.shutdownClientModule
+ this.shutdownServerModule
+ clear
+ }
+
+
+ /**
+ * Creates a Client-managed ActorRef out of the Actor of the specified Class.
+ * If the supplied host and port is identical of the configured local node, it will be a local actor
+ *
+ * import Actor._
+ * val actor = actorOf(classOf[MyActor],"www.akka.io",2552)
+ * actor.start
+ * actor ! message
+ * actor.stop
+ *
+ * You can create and start the actor in one statement like this:
+ *
+ * val actor = actorOf(classOf[MyActor],"www.akka.io",2552).start
+ *
+ */
+ def actorOf(factory: => Actor, host: String, port: Int): ActorRef =
+ Actor.remote.clientManagedActorOf(() => factory, host, port)
+
+ /**
+ * Creates a Client-managed ActorRef out of the Actor of the specified Class.
+ * If the supplied host and port is identical of the configured local node, it will be a local actor
+ *
+ * import Actor._
+ * val actor = actorOf(classOf[MyActor],"www.akka.io",2552)
+ * actor.start
+ * actor ! message
+ * actor.stop
+ *
+ * You can create and start the actor in one statement like this:
+ *
+ * val actor = actorOf(classOf[MyActor],"www.akka.io",2552).start
+ *
+ */
+ def actorOf(clazz: Class[_ <: Actor], host: String, port: Int): ActorRef = {
+ import ReflectiveAccess.{ createInstance, noParams, noArgs }
+ clientManagedActorOf(() =>
+ createInstance[Actor](clazz.asInstanceOf[Class[_]], noParams, noArgs).getOrElse(
+ throw new ActorInitializationException(
+ "Could not instantiate Actor" +
+ "\nMake sure Actor is NOT defined inside a class/trait," +
+ "\nif so put it outside the class/trait, f.e. in a companion object," +
+ "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.")),
+ host, port)
+ }
+
+ /**
+ * Creates a Client-managed ActorRef out of the Actor of the specified Class.
+ * If the supplied host and port is identical of the configured local node, it will be a local actor
+ *
+ * import Actor._
+ * val actor = actorOf[MyActor]("www.akka.io",2552)
+ * actor.start
+ * actor ! message
+ * actor.stop
+ *
+ * You can create and start the actor in one statement like this:
+ *
+ * val actor = actorOf[MyActor]("www.akka.io",2552).start
+ *
+ */
+ def actorOf[T <: Actor : Manifest](host: String, port: Int): ActorRef = {
+ import ReflectiveAccess.{ createInstance, noParams, noArgs }
+ clientManagedActorOf(() =>
+ createInstance[Actor](manifest[T].erasure.asInstanceOf[Class[_]], noParams, noArgs).getOrElse(
+ throw new ActorInitializationException(
+ "Could not instantiate Actor" +
+ "\nMake sure Actor is NOT defined inside a class/trait," +
+ "\nif so put it outside the class/trait, f.e. in a companion object," +
+ "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.")),
+ host, port)
+ }
+
+ protected override def manageLifeCycleOfListeners = false
+ protected[akka] override def notifyListeners(message: => Any): Unit = super.notifyListeners(message)
+
+ private[akka] val actors = new ConcurrentHashMap[String, ActorRef]
+ private[akka] val actorsByUuid = new ConcurrentHashMap[String, ActorRef]
+ private[akka] val actorsFactories = new ConcurrentHashMap[String, () => ActorRef]
+ private[akka] val typedActors = new ConcurrentHashMap[String, AnyRef]
+ private[akka] val typedActorsByUuid = new ConcurrentHashMap[String, AnyRef]
+ private[akka] val typedActorsFactories = new ConcurrentHashMap[String, () => AnyRef]
+
+ def clear {
+ List(actors,actorsByUuid,actorsFactories,typedActors,typedActorsByUuid,typedActorsFactories) foreach (_.clear)
+ }
+}
+
+/**
+ * This is the interface for the RemoteServer functionality, it's used in Actor.remote
+ */
+trait RemoteServerModule extends RemoteModule {
+ protected val guard = new ReentrantGuard
+
+ /**
+ * Signals whether the server is up and running or not
+ */
+ def isRunning: Boolean
+
+ /**
+ * Gets the name of the server instance
+ */
+ def name: String
+
+ /**
+ * Gets the address of the server instance
+ */
+ def address: InetSocketAddress
+
+ /**
+ * Starts the server up
+ */
+ def start(host: String = ReflectiveAccess.Remote.configDefaultAddress.getHostName,
+ port: Int = ReflectiveAccess.Remote.configDefaultAddress.getPort,
+ loader: Option[ClassLoader] = None): RemoteServerModule
+
+ /**
+ * Shuts the server down
+ */
+ def shutdownServerModule: Unit
+
+ /**
+ * Register typed actor by interface name.
+ */
+ def registerTypedActor(intfClass: Class[_], typedActor: AnyRef) : Unit = registerTypedActor(intfClass.getName, typedActor)
+
+ /**
+ * Register remote typed actor by a specific id.
+ * @param id custom actor id
+ * @param typedActor typed actor to register
+ */
+ def registerTypedActor(id: String, typedActor: AnyRef): Unit
+
+ /**
+ * Register typed actor by interface name.
+ */
+ def registerTypedPerSessionActor(intfClass: Class[_], factory: => AnyRef) : Unit = registerTypedActor(intfClass.getName, factory)
+
+ /**
+ * Register typed actor by interface name.
+ * Java API
+ */
+ def registerTypedPerSessionActor(intfClass: Class[_], factory: Creator[AnyRef]) : Unit = registerTypedActor(intfClass.getName, factory)
+
+ /**
+ * Register remote typed actor by a specific id.
+ * @param id custom actor id
+ * @param typedActor typed actor to register
+ */
+ def registerTypedPerSessionActor(id: String, factory: => AnyRef): Unit
+
+ /**
+ * Register remote typed actor by a specific id.
+ * @param id custom actor id
+ * @param typedActor typed actor to register
+ * Java API
+ */
+ def registerTypedPerSessionActor(id: String, factory: Creator[AnyRef]): Unit = registerTypedPerSessionActor(id, factory.create)
+
+ /**
+ * Register Remote Actor by the Actor's 'id' field. It starts the Actor if it is not started already.
+ */
+ def register(actorRef: ActorRef): Unit = register(actorRef.id, actorRef)
+
+ /**
+ * Register Remote Actor by the Actor's uuid field. It starts the Actor if it is not started already.
+ */
+ def registerByUuid(actorRef: ActorRef): Unit
+
+ /**
+ * Register Remote Actor by a specific 'id' passed as argument.
+ *
+ * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
+ */
+ def register(id: String, actorRef: ActorRef): Unit
+
+ /**
+ * Register Remote Session Actor by a specific 'id' passed as argument.
+ *
+ * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
+ */
+ def registerPerSession(id: String, factory: => ActorRef): Unit
+
+ /**
+ * Register Remote Session Actor by a specific 'id' passed as argument.
+ *
+ * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
+ * Java API
+ */
+ def registerPerSession(id: String, factory: Creator[ActorRef]): Unit = registerPerSession(id, factory.create)
+
+ /**
+ * Unregister Remote Actor that is registered using its 'id' field (not custom ID).
+ */
+ def unregister(actorRef: ActorRef): Unit
+
+ /**
+ * Unregister Remote Actor by specific 'id'.
+ *
+ * NOTE: You need to call this method if you have registered an actor by a custom ID.
+ */
+ def unregister(id: String): Unit
+
+ /**
+ * Unregister Remote Actor by specific 'id'.
+ *
+ * NOTE: You need to call this method if you have registered an actor by a custom ID.
+ */
+ def unregisterPerSession(id: String): Unit
+
+ /**
+ * Unregister Remote Typed Actor by specific 'id'.
+ *
+ * NOTE: You need to call this method if you have registered an actor by a custom ID.
+ */
+ def unregisterTypedActor(id: String): Unit
+
+ /**
+ * Unregister Remote Typed Actor by specific 'id'.
+ *
+ * NOTE: You need to call this method if you have registered an actor by a custom ID.
+ */
+ def unregisterTypedPerSessionActor(id: String): Unit
+}
+
+trait RemoteClientModule extends RemoteModule { self: RemoteModule =>
+
+ def actorFor(classNameOrServiceId: String, hostname: String, port: Int): ActorRef =
+ actorFor(classNameOrServiceId, classNameOrServiceId, Actor.TIMEOUT, hostname, port, None)
+
+ def actorFor(classNameOrServiceId: String, hostname: String, port: Int, loader: ClassLoader): ActorRef =
+ actorFor(classNameOrServiceId, classNameOrServiceId, Actor.TIMEOUT, hostname, port, Some(loader))
+
+ def actorFor(serviceId: String, className: String, hostname: String, port: Int): ActorRef =
+ actorFor(serviceId, className, Actor.TIMEOUT, hostname, port, None)
+
+ def actorFor(serviceId: String, className: String, hostname: String, port: Int, loader: ClassLoader): ActorRef =
+ actorFor(serviceId, className, Actor.TIMEOUT, hostname, port, Some(loader))
+
+ def actorFor(classNameOrServiceId: String, timeout: Long, hostname: String, port: Int): ActorRef =
+ actorFor(classNameOrServiceId, classNameOrServiceId, timeout, hostname, port, None)
+
+ def actorFor(classNameOrServiceId: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): ActorRef =
+ actorFor(classNameOrServiceId, classNameOrServiceId, timeout, hostname, port, Some(loader))
+
+ def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int): ActorRef =
+ actorFor(serviceId, className, timeout, hostname, port, None)
+
+ def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, hostname: String, port: Int): T =
+ typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, Actor.TIMEOUT, hostname, port, None)
+
+ def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, timeout: Long, hostname: String, port: Int): T =
+ typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, timeout, hostname, port, None)
+
+ def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): T =
+ typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, timeout, hostname, port, Some(loader))
+
+ def typedActorFor[T](intfClass: Class[T], serviceId: String, implClassName: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): T =
+ typedActorFor(intfClass, serviceId, implClassName, timeout, hostname, port, Some(loader))
+
+ def clientManagedActorOf(factory: () => Actor, host: String, port: Int): ActorRef
+
+
+ /**
+ * Clean-up all open connections.
+ */
+ def shutdownClientModule: Unit
+
+ /**
+ * Shuts down a specific client connected to the supplied remote address returns true if successful
+ */
+ def shutdownClientConnection(address: InetSocketAddress): Boolean
+
+ /**
+ * Restarts a specific client connected to the supplied remote address, but only if the client is not shut down
+ */
+ def restartClientConnection(address: InetSocketAddress): Boolean
+
+ /** Methods that needs to be implemented by a transport **/
+
+ protected[akka] def typedActorFor[T](intfClass: Class[T], serviceId: String, implClassName: String, timeout: Long, host: String, port: Int, loader: Option[ClassLoader]): T
+
+ protected[akka] def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int, loader: Option[ClassLoader]): ActorRef
+
+ protected[akka] def send[T](message: Any,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[T]],
+ remoteAddress: InetSocketAddress,
+ timeout: Long,
+ isOneWay: Boolean,
+ actorRef: ActorRef,
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: ActorType,
+ loader: Option[ClassLoader]): Option[CompletableFuture[T]]
+
+ private[akka] def registerSupervisorForActor(actorRef: ActorRef): ActorRef
+
+ private[akka] def deregisterSupervisorForActor(actorRef: ActorRef): ActorRef
+
+ private[akka] def registerClientManagedActor(hostname: String, port: Int, uuid: Uuid): Unit
+
+ private[akka] def unregisterClientManagedActor(hostname: String, port: Int, uuid: Uuid): Unit
+}
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/akka/util/Address.scala b/akka-actor/src/main/scala/akka/util/Address.scala
index 07ffbec303..4e1749c560 100644
--- a/akka-actor/src/main/scala/akka/util/Address.scala
+++ b/akka-actor/src/main/scala/akka/util/Address.scala
@@ -3,12 +3,18 @@
*/
package akka.util
+import java.net.InetSocketAddress
+
object Address {
def apply(hostname: String, port: Int) = new Address(hostname, port)
+ def apply(inetAddress: InetSocketAddress): Address = inetAddress match {
+ case null => null
+ case inet => new Address(inet.getHostName, inet.getPort)
+ }
}
class Address(val hostname: String, val port: Int) {
- override def hashCode: Int = {
+ override val hashCode: Int = {
var result = HashCode.SEED
result = HashCode.hash(result, hostname)
result = HashCode.hash(result, port)
diff --git a/akka-actor/src/main/scala/akka/util/LockUtil.scala b/akka-actor/src/main/scala/akka/util/LockUtil.scala
index bccf42a121..ab383f5f85 100644
--- a/akka-actor/src/main/scala/akka/util/LockUtil.scala
+++ b/akka-actor/src/main/scala/akka/util/LockUtil.scala
@@ -37,8 +37,8 @@ class ReentrantGuard {
*/
class ReadWriteGuard {
private val rwl = new ReentrantReadWriteLock
- private val readLock = rwl.readLock
- private val writeLock = rwl.writeLock
+ val readLock = rwl.readLock
+ val writeLock = rwl.writeLock
def withWriteGuard[T](body: => T): T = {
writeLock.lock
diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala
index 4e4bcdb7e1..494a94e9a0 100644
--- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala
+++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala
@@ -4,12 +4,13 @@
package akka.util
-import akka.actor.{ActorRef, IllegalActorStateException, ActorType, Uuid}
import akka.dispatch.{Future, CompletableFuture, MessageInvocation}
import akka.config.{Config, ModuleNotAvailableException}
import akka.AkkaException
import java.net.InetSocketAddress
+import akka.remoteinterface.RemoteSupport
+import akka.actor._
/**
* Helper class for reflective access to different modules in order to allow optional loading of modules.
@@ -20,10 +21,10 @@ object ReflectiveAccess extends Logging {
val loader = getClass.getClassLoader
- lazy val isRemotingEnabled = RemoteClientModule.isEnabled
+ def isRemotingEnabled = Remote.isEnabled
lazy val isTypedActorEnabled = TypedActorModule.isEnabled
- def ensureRemotingEnabled = RemoteClientModule.ensureEnabled
+ def ensureRemotingEnabled = Remote.ensureEnabled
def ensureTypedActorEnabled = TypedActorModule.ensureEnabled
/**
@@ -31,112 +32,26 @@ object ReflectiveAccess extends Logging {
*
* @author Jonas Bonér
*/
- object RemoteClientModule {
+ object Remote {
+ val TRANSPORT = Config.config.getString("akka.remote.layer","akka.remote.netty.NettyRemoteSupport")
- type RemoteClient = {
- def send[T](
- message: Any,
- senderOption: Option[ActorRef],
- senderFuture: Option[CompletableFuture[_]],
- remoteAddress: InetSocketAddress,
- timeout: Long,
- isOneWay: Boolean,
- actorRef: ActorRef,
- typedActorInfo: Option[Tuple2[String, String]],
- actorType: ActorType): Option[CompletableFuture[T]]
- def registerSupervisorForActor(actorRef: ActorRef)
- }
+ private[akka] val configDefaultAddress =
+ new InetSocketAddress(Config.config.getString("akka.remote.server.hostname", "localhost"),
+ Config.config.getInt("akka.remote.server.port", 2552))
- type RemoteClientObject = {
- def register(hostname: String, port: Int, uuid: Uuid): Unit
- def unregister(hostname: String, port: Int, uuid: Uuid): Unit
- def clientFor(address: InetSocketAddress): RemoteClient
- def clientFor(hostname: String, port: Int, loader: Option[ClassLoader]): RemoteClient
- }
- lazy val isEnabled = remoteClientObjectInstance.isDefined
+ lazy val isEnabled = remoteSupportClass.isDefined
def ensureEnabled = if (!isEnabled) throw new ModuleNotAvailableException(
"Can't load the remoting module, make sure that akka-remote.jar is on the classpath")
- val remoteClientObjectInstance: Option[RemoteClientObject] =
- getObjectFor("akka.remote.RemoteClient$")
+ val remoteSupportClass: Option[Class[_ <: RemoteSupport]] = getClassFor(TRANSPORT)
- def register(address: InetSocketAddress, uuid: Uuid) = {
- ensureEnabled
- remoteClientObjectInstance.get.register(address.getHostName, address.getPort, uuid)
- }
-
- def unregister(address: InetSocketAddress, uuid: Uuid) = {
- ensureEnabled
- remoteClientObjectInstance.get.unregister(address.getHostName, address.getPort, uuid)
- }
-
- def registerSupervisorForActor(remoteAddress: InetSocketAddress, actorRef: ActorRef) = {
- ensureEnabled
- val remoteClient = remoteClientObjectInstance.get.clientFor(remoteAddress)
- remoteClient.registerSupervisorForActor(actorRef)
- }
-
- def clientFor(hostname: String, port: Int, loader: Option[ClassLoader]): RemoteClient = {
- ensureEnabled
- remoteClientObjectInstance.get.clientFor(hostname, port, loader)
- }
-
- def send[T](
- message: Any,
- senderOption: Option[ActorRef],
- senderFuture: Option[CompletableFuture[_]],
- remoteAddress: InetSocketAddress,
- timeout: Long,
- isOneWay: Boolean,
- actorRef: ActorRef,
- typedActorInfo: Option[Tuple2[String, String]],
- actorType: ActorType,
- loader: Option[ClassLoader] = None): Option[CompletableFuture[T]] = {
- ensureEnabled
- clientFor(remoteAddress.getHostName, remoteAddress.getPort, loader).send[T](
- message, senderOption, senderFuture, remoteAddress, timeout, isOneWay, actorRef, typedActorInfo, actorType)
- }
- }
-
- /**
- * Reflective access to the RemoteServer module.
- *
- * @author Jonas Bonér
- */
- object RemoteServerModule {
- val HOSTNAME = Config.config.getString("akka.remote.server.hostname", "localhost")
- val PORT = Config.config.getInt("akka.remote.server.port", 2552)
-
- type RemoteServerObject = {
- def registerActor(address: InetSocketAddress, actor: ActorRef): Unit
- def registerTypedActor(address: InetSocketAddress, name: String, typedActor: AnyRef): Unit
- }
-
- type RemoteNodeObject = {
- def unregister(actorRef: ActorRef): Unit
- }
-
- val remoteServerObjectInstance: Option[RemoteServerObject] =
- getObjectFor("akka.remote.RemoteServer$")
-
- val remoteNodeObjectInstance: Option[RemoteNodeObject] =
- getObjectFor("akka.remote.RemoteNode$")
-
- def registerActor(address: InetSocketAddress, actorRef: ActorRef) = {
- RemoteClientModule.ensureEnabled
- remoteServerObjectInstance.get.registerActor(address, actorRef)
- }
-
- def registerTypedActor(address: InetSocketAddress, implementationClassName: String, proxy: AnyRef) = {
- RemoteClientModule.ensureEnabled
- remoteServerObjectInstance.get.registerTypedActor(address, implementationClassName, proxy)
- }
-
- def unregister(actorRef: ActorRef) = {
- RemoteClientModule.ensureEnabled
- remoteNodeObjectInstance.get.unregister(actorRef)
+ protected[akka] val defaultRemoteSupport: Option[() => RemoteSupport] = remoteSupportClass map {
+ remoteClass => () => createInstance[RemoteSupport](remoteClass,Array[Class[_]](),Array[AnyRef]()).
+ getOrElse(throw new ModuleNotAvailableException("Can't instantiate "+
+ remoteClass.getName+
+ ", make sure that akka-remote.jar is on the classpath"))
}
}
diff --git a/akka-actor/src/test/java/akka/actor/JavaAPI.java b/akka-actor/src/test/java/akka/actor/JavaAPI.java
new file mode 100644
index 0000000000..61c829f540
--- /dev/null
+++ b/akka-actor/src/test/java/akka/actor/JavaAPI.java
@@ -0,0 +1,34 @@
+package akka.actor;
+
+import akka.japi.Creator;
+import org.junit.Test;
+import akka.actor.Actors;
+import akka.remoteinterface.RemoteSupport;
+import static org.junit.Assert.*;
+
+public class JavaAPI {
+
+ @Test void mustBeAbleToUseUntypedActor() {
+ final RemoteSupport remote = Actors.remote();
+ assertNotNull(remote);
+ }
+
+ @Test void mustInteractWithActorRegistry() {
+ final ActorRegistry registry = Actors.registry();
+ assertNotNull(registry);
+ }
+
+ @Test void mustBeAbleToCreateActorRefFromClass() {
+ ActorRef ref = Actors.actorOf(JavaAPITestActor.class);
+ assertNotNull(ref);
+ }
+
+ @Test void mustBeAbleToCreateActorRefFromFactory() {
+ ActorRef ref = Actors.actorOf(new Creator() {
+ public Actor create() {
+ return new JavaAPITestActor();
+ }
+ });
+ assertNotNull(ref);
+ }
+}
diff --git a/akka-actor/src/test/java/akka/actor/JavaAPITestActor.java b/akka-actor/src/test/java/akka/actor/JavaAPITestActor.java
new file mode 100644
index 0000000000..fe729970cc
--- /dev/null
+++ b/akka-actor/src/test/java/akka/actor/JavaAPITestActor.java
@@ -0,0 +1,7 @@
+package akka.actor;
+
+public class JavaAPITestActor extends UntypedActor {
+ public void onReceive(Object msg) {
+ getContext().replySafe("got it!");
+ }
+}
diff --git a/akka-actor/src/test/java/akka/config/SupervisionConfig.java b/akka-actor/src/test/java/akka/config/SupervisionConfig.java
index 271e6c490e..fd71c86bf1 100644
--- a/akka-actor/src/test/java/akka/config/SupervisionConfig.java
+++ b/akka-actor/src/test/java/akka/config/SupervisionConfig.java
@@ -13,7 +13,7 @@ public class SupervisionConfig {
public SupervisorConfig createSupervisorConfig(List toSupervise) {
ArrayList targets = new ArrayList(toSupervise.size());
for(ActorRef ref : toSupervise) {
- targets.add(new Supervise(ref, permanent(), new RemoteAddress("localhost",2552)));
+ targets.add(new Supervise(ref, permanent(), true));
}
return new SupervisorConfig(new AllForOneStrategy(new Class[] { Exception.class },50,1000), targets.toArray(new Server[0]));
diff --git a/akka-actor/src/test/scala/akka/dataflow/DataFlowSpec.scala b/akka-actor/src/test/scala/akka/dataflow/DataFlowSpec.scala
index ae01e84b19..f5a107f511 100644
--- a/akka-actor/src/test/scala/akka/dataflow/DataFlowSpec.scala
+++ b/akka-actor/src/test/scala/akka/dataflow/DataFlowSpec.scala
@@ -72,7 +72,7 @@ class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll {
/*
it("should be able to join streams") {
import DataFlow._
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
def ints(n: Int, max: Int, stream: DataFlowStream[Int]): Unit = if (n != max) {
stream <<< n
@@ -139,7 +139,7 @@ class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll {
/* it("should be able to conditionally set variables") {
import DataFlow._
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val latch = new CountDownLatch(1)
val x, y, z, v = new DataFlowVariable[Int]
diff --git a/akka-actor/src/test/scala/akka/misc/ActorRegistrySpec.scala b/akka-actor/src/test/scala/akka/misc/ActorRegistrySpec.scala
index f951281f2e..09a23dbc5c 100644
--- a/akka-actor/src/test/scala/akka/misc/ActorRegistrySpec.scala
+++ b/akka-actor/src/test/scala/akka/misc/ActorRegistrySpec.scala
@@ -33,10 +33,10 @@ class ActorRegistrySpec extends JUnitSuite {
import ActorRegistrySpec._
@Test def shouldGetActorByIdFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor = actorOf[TestActor]
actor.start
- val actors = ActorRegistry.actorsFor("MyID")
+ val actors = Actor.registry.actorsFor("MyID")
assert(actors.size === 1)
assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID")
@@ -44,21 +44,21 @@ class ActorRegistrySpec extends JUnitSuite {
}
@Test def shouldGetActorByUUIDFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor = actorOf[TestActor]
val uuid = actor.uuid
actor.start
- val actorOrNone = ActorRegistry.actorFor(uuid)
+ val actorOrNone = Actor.registry.actorFor(uuid)
assert(actorOrNone.isDefined)
assert(actorOrNone.get.uuid === uuid)
actor.stop
}
@Test def shouldGetActorByClassFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor = actorOf[TestActor]
actor.start
- val actors = ActorRegistry.actorsFor(classOf[TestActor])
+ val actors = Actor.registry.actorsFor(classOf[TestActor])
assert(actors.size === 1)
assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID")
@@ -66,10 +66,10 @@ class ActorRegistrySpec extends JUnitSuite {
}
@Test def shouldGetActorByManifestFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor = actorOf[TestActor]
actor.start
- val actors = ActorRegistry.actorsFor[TestActor]
+ val actors = Actor.registry.actorsFor[TestActor]
assert(actors.size === 1)
assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID")
@@ -77,10 +77,10 @@ class ActorRegistrySpec extends JUnitSuite {
}
@Test def shouldFindThingsFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor = actorOf[TestActor]
actor.start
- val found = ActorRegistry.find({ case a: ActorRef if a.actor.isInstanceOf[TestActor] => a })
+ val found = Actor.registry.find({ case a: ActorRef if a.actor.isInstanceOf[TestActor] => a })
assert(found.isDefined)
assert(found.get.actor.isInstanceOf[TestActor])
assert(found.get.id === "MyID")
@@ -88,12 +88,12 @@ class ActorRegistrySpec extends JUnitSuite {
}
@Test def shouldGetActorsByIdFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor1 = actorOf[TestActor]
actor1.start
val actor2 = actorOf[TestActor]
actor2.start
- val actors = ActorRegistry.actorsFor("MyID")
+ val actors = Actor.registry.actorsFor("MyID")
assert(actors.size === 2)
assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID")
@@ -104,12 +104,12 @@ class ActorRegistrySpec extends JUnitSuite {
}
@Test def shouldGetActorsByClassFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor1 = actorOf[TestActor]
actor1.start
val actor2 = actorOf[TestActor]
actor2.start
- val actors = ActorRegistry.actorsFor(classOf[TestActor])
+ val actors = Actor.registry.actorsFor(classOf[TestActor])
assert(actors.size === 2)
assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID")
@@ -120,12 +120,12 @@ class ActorRegistrySpec extends JUnitSuite {
}
@Test def shouldGetActorsByManifestFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor1 = actorOf[TestActor]
actor1.start
val actor2 = actorOf[TestActor]
actor2.start
- val actors = ActorRegistry.actorsFor[TestActor]
+ val actors = Actor.registry.actorsFor[TestActor]
assert(actors.size === 2)
assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID")
@@ -137,26 +137,26 @@ class ActorRegistrySpec extends JUnitSuite {
@Test def shouldGetActorsByMessageFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor1 = actorOf[TestActor]
actor1.start
val actor2 = actorOf[TestActor2]
actor2.start
- val actorsForAcotrTestActor = ActorRegistry.actorsFor[TestActor]
+ val actorsForAcotrTestActor = Actor.registry.actorsFor[TestActor]
assert(actorsForAcotrTestActor.size === 1)
- val actorsForAcotrTestActor2 = ActorRegistry.actorsFor[TestActor2]
+ val actorsForAcotrTestActor2 = Actor.registry.actorsFor[TestActor2]
assert(actorsForAcotrTestActor2.size === 1)
- val actorsForAcotr = ActorRegistry.actorsFor[Actor]
+ val actorsForAcotr = Actor.registry.actorsFor[Actor]
assert(actorsForAcotr.size === 2)
- val actorsForMessagePing2 = ActorRegistry.actorsFor[Actor]("ping2")
+ val actorsForMessagePing2 = Actor.registry.actorsFor[Actor]("ping2")
assert(actorsForMessagePing2.size === 1)
- val actorsForMessagePing = ActorRegistry.actorsFor[Actor]("ping")
+ val actorsForMessagePing = Actor.registry.actorsFor[Actor]("ping")
assert(actorsForMessagePing.size === 2)
actor1.stop
@@ -164,12 +164,12 @@ class ActorRegistrySpec extends JUnitSuite {
}
@Test def shouldGetAllActorsFromActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor1 = actorOf[TestActor]
actor1.start
val actor2 = actorOf[TestActor]
actor2.start
- val actors = ActorRegistry.actors
+ val actors = Actor.registry.actors
assert(actors.size === 2)
assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID")
@@ -180,43 +180,43 @@ class ActorRegistrySpec extends JUnitSuite {
}
@Test def shouldGetResponseByAllActorsInActorRegistryWhenInvokingForeach {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor1 = actorOf[TestActor]
actor1.start
val actor2 = actorOf[TestActor]
actor2.start
record = ""
- ActorRegistry.foreach(actor => actor !! "ping")
+ Actor.registry.foreach(actor => actor !! "ping")
assert(record === "pongpong")
actor1.stop
actor2.stop
}
@Test def shouldShutdownAllActorsInActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor1 = actorOf[TestActor]
actor1.start
val actor2 = actorOf[TestActor]
actor2.start
- ActorRegistry.shutdownAll
- assert(ActorRegistry.actors.size === 0)
+ Actor.registry.shutdownAll
+ assert(Actor.registry.actors.size === 0)
}
@Test def shouldRemoveUnregisterActorInActorRegistry {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val actor1 = actorOf[TestActor]
actor1.start
val actor2 = actorOf[TestActor]
actor2.start
- assert(ActorRegistry.actors.size === 2)
- ActorRegistry.unregister(actor1)
- assert(ActorRegistry.actors.size === 1)
- ActorRegistry.unregister(actor2)
- assert(ActorRegistry.actors.size === 0)
+ assert(Actor.registry.actors.size === 2)
+ Actor.registry.unregister(actor1)
+ assert(Actor.registry.actors.size === 1)
+ Actor.registry.unregister(actor2)
+ assert(Actor.registry.actors.size === 0)
}
@Test def shouldBeAbleToRegisterActorsConcurrently {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
def mkTestActors = for(i <- (1 to 10).toList;j <- 1 to 3000) yield actorOf( new Actor {
self.id = i.toString
@@ -244,7 +244,7 @@ class ActorRegistrySpec extends JUnitSuite {
for(i <- 1 to 10) {
val theId = i.toString
- val actors = ActorRegistry.actorsFor(theId).toSet
+ val actors = Actor.registry.actorsFor(theId).toSet
for(a <- actors if a.id == theId) assert(actors contains a)
assert(actors.size === 9000)
}
diff --git a/akka-actor/src/test/scala/akka/misc/SchedulerSpec.scala b/akka-actor/src/test/scala/akka/misc/SchedulerSpec.scala
index a4471503da..79b09d49d1 100644
--- a/akka-actor/src/test/scala/akka/misc/SchedulerSpec.scala
+++ b/akka-actor/src/test/scala/akka/misc/SchedulerSpec.scala
@@ -12,7 +12,7 @@ class SchedulerSpec extends JUnitSuite {
def withCleanEndState(action: => Unit) {
action
Scheduler.restart
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
}
@@ -62,10 +62,10 @@ class SchedulerSpec extends JUnitSuite {
val actor = actorOf(new Actor {
def receive = { case Ping => ticks.countDown }
}).start
- val numActors = ActorRegistry.actors.length
+ val numActors = Actor.registry.actors.length
(1 to 1000).foreach( _ => Scheduler.scheduleOnce(actor,Ping,1,TimeUnit.MILLISECONDS) )
assert(ticks.await(10,TimeUnit.SECONDS))
- assert(ActorRegistry.actors.length === numActors)
+ assert(Actor.registry.actors.length === numActors)
}
/**
diff --git a/akka-http/src/main/scala/akka/http/Mist.scala b/akka-http/src/main/scala/akka/http/Mist.scala
index 992e4695be..3c2ae06ced 100644
--- a/akka-http/src/main/scala/akka/http/Mist.scala
+++ b/akka-http/src/main/scala/akka/http/Mist.scala
@@ -70,7 +70,7 @@ trait Mist extends Logging {
/**
* The root endpoint actor
*/
- protected val _root = ActorRegistry.actorsFor(RootActorID).head
+ protected val _root = Actor.registry.actorsFor(RootActorID).head
/**
* Server-specific method factory
diff --git a/akka-http/src/main/scala/akka/security/Security.scala b/akka-http/src/main/scala/akka/security/Security.scala
index bd50b0a1c7..553984a22e 100644
--- a/akka-http/src/main/scala/akka/security/Security.scala
+++ b/akka-http/src/main/scala/akka/security/Security.scala
@@ -110,7 +110,7 @@ class AkkaSecurityFilterFactory extends ResourceFilterFactory with Logging {
* Currently we always take the first, since there usually should be at most one authentication actor, but a round-robin
* strategy could be implemented in the future
*/
- def authenticator: ActorRef = ActorRegistry.actorsFor(authenticatorFQN).head
+ def authenticator: ActorRef = Actor.registry.actorsFor(authenticatorFQN).head
def mkFilter(roles: Option[List[String]]): java.util.List[ResourceFilter] =
java.util.Collections.singletonList(new Filter(authenticator, roles))
diff --git a/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala b/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala
index 018a33dcd8..c49e505d5a 100644
--- a/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala
+++ b/akka-remote/src/main/scala/akka/remote/BootableRemoteActorService.scala
@@ -4,9 +4,9 @@
package akka.remote
-import akka.actor.BootableActorLoaderService
-import akka.util.{Bootable, Logging}
import akka.config.Config.config
+import akka.actor. {Actor, BootableActorLoaderService}
+import akka.util. {ReflectiveAccess, Bootable, Logging}
/**
* This bundle/service is responsible for booting up and shutting down the remote actors facility
@@ -17,16 +17,13 @@ trait BootableRemoteActorService extends Bootable with Logging {
self: BootableActorLoaderService =>
protected lazy val remoteServerThread = new Thread(new Runnable() {
- def run = {
- if (self.applicationLoader.isDefined) RemoteNode.start(self.applicationLoader.get)
- else RemoteNode.start
- }
+ def run = Actor.remote.start(loader = self.applicationLoader) //Use config host/port
}, "Akka Remote Service")
def startRemoteService = remoteServerThread.start
abstract override def onLoad = {
- if (RemoteServer.isRemotingEnabled) {
+ if (ReflectiveAccess.isRemotingEnabled && RemoteServerSettings.isRemotingEnabled) {
log.slf4j.info("Initializing Remote Actors Service...")
startRemoteService
log.slf4j.info("Remote Actors Service initialized")
@@ -36,7 +33,7 @@ trait BootableRemoteActorService extends Bootable with Logging {
abstract override def onUnload = {
log.slf4j.info("Shutting down Remote Actors Service")
- RemoteNode.shutdown
+ Actor.remote.shutdown
if (remoteServerThread.isAlive) remoteServerThread.join(1000)
log.slf4j.info("Remote Actors Service has been shut down")
super.onUnload
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteClient.scala b/akka-remote/src/main/scala/akka/remote/RemoteClient.scala
deleted file mode 100644
index 424a743282..0000000000
--- a/akka-remote/src/main/scala/akka/remote/RemoteClient.scala
+++ /dev/null
@@ -1,525 +0,0 @@
-/**
- * Copyright (C) 2009-2011 Scalable Solutions AB
- */
-
-package akka.remote
-
-import akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _}
-import akka.actor.{Exit, Actor, ActorRef, ActorType, RemoteActorRef, IllegalActorStateException}
-import akka.dispatch.{DefaultCompletableFuture, CompletableFuture}
-import akka.actor.{Uuid,newUuid,uuidFrom}
-import akka.config.Config._
-import akka.serialization.RemoteActorSerialization._
-import akka.AkkaException
-import Actor._
-
-import org.jboss.netty.channel._
-import group.DefaultChannelGroup
-import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
-import org.jboss.netty.bootstrap.ClientBootstrap
-import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender }
-import org.jboss.netty.handler.codec.compression.{ ZlibDecoder, ZlibEncoder }
-import org.jboss.netty.handler.codec.protobuf.{ ProtobufDecoder, ProtobufEncoder }
-import org.jboss.netty.handler.timeout.ReadTimeoutHandler
-import org.jboss.netty.util.{ TimerTask, Timeout, HashedWheelTimer }
-import org.jboss.netty.handler.ssl.SslHandler
-
-import java.net.{ SocketAddress, InetSocketAddress }
-import java.util.concurrent.{ TimeUnit, Executors, ConcurrentMap, ConcurrentHashMap, ConcurrentSkipListSet }
-import java.util.concurrent.atomic.{ AtomicLong, AtomicBoolean }
-
-import scala.collection.mutable.{ HashSet, HashMap }
-import scala.reflect.BeanProperty
-
-import akka.actor._
-import akka.util._
-
-
-/**
- * Life-cycle events for RemoteClient.
- */
-sealed trait RemoteClientLifeCycleEvent
-case class RemoteClientError(
- @BeanProperty val cause: Throwable,
- @BeanProperty val client: RemoteClient) extends RemoteClientLifeCycleEvent
-case class RemoteClientDisconnected(
- @BeanProperty val client: RemoteClient) extends RemoteClientLifeCycleEvent
-case class RemoteClientConnected(
- @BeanProperty val client: RemoteClient) extends RemoteClientLifeCycleEvent
-case class RemoteClientStarted(
- @BeanProperty val client: RemoteClient) extends RemoteClientLifeCycleEvent
-case class RemoteClientShutdown(
- @BeanProperty val client: RemoteClient) extends RemoteClientLifeCycleEvent
-
-/**
- * Thrown for example when trying to send a message using a RemoteClient that is either not started or shut down.
- */
-class RemoteClientException private[akka] (message: String, @BeanProperty val client: RemoteClient) extends AkkaException(message)
-
-case class UnparsableException private[akka] (originalClassName: String, originalMessage: String) extends AkkaException(originalMessage)
-
-/**
- * The RemoteClient object manages RemoteClient instances and gives you an API to lookup remote actor handles.
- *
- * @author Jonas Bonér
- */
-object RemoteClient extends Logging {
-
- val SECURE_COOKIE: Option[String] = {
- val cookie = config.getString("akka.remote.secure-cookie", "")
- if (cookie == "") None
- else Some(cookie)
- }
-
- val READ_TIMEOUT = Duration(config.getInt("akka.remote.client.read-timeout", 1), TIME_UNIT)
- val RECONNECT_DELAY = Duration(config.getInt("akka.remote.client.reconnect-delay", 5), TIME_UNIT)
- val MESSAGE_FRAME_SIZE = config.getInt("akka.remote.client.message-frame-size", 1048576)
-
- private val remoteClients = new HashMap[String, RemoteClient]
- private val remoteActors = new HashMap[Address, HashSet[Uuid]]
-
- def actorFor(classNameOrServiceId: String, hostname: String, port: Int): ActorRef =
- actorFor(classNameOrServiceId, classNameOrServiceId, Actor.TIMEOUT, hostname, port, None)
-
- def actorFor(classNameOrServiceId: String, hostname: String, port: Int, loader: ClassLoader): ActorRef =
- actorFor(classNameOrServiceId, classNameOrServiceId, Actor.TIMEOUT, hostname, port, Some(loader))
-
- def actorFor(serviceId: String, className: String, hostname: String, port: Int): ActorRef =
- actorFor(serviceId, className, Actor.TIMEOUT, hostname, port, None)
-
- def actorFor(serviceId: String, className: String, hostname: String, port: Int, loader: ClassLoader): ActorRef =
- actorFor(serviceId, className, Actor.TIMEOUT, hostname, port, Some(loader))
-
- def actorFor(classNameOrServiceId: String, timeout: Long, hostname: String, port: Int): ActorRef =
- actorFor(classNameOrServiceId, classNameOrServiceId, timeout, hostname, port, None)
-
- def actorFor(classNameOrServiceId: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): ActorRef =
- actorFor(classNameOrServiceId, classNameOrServiceId, timeout, hostname, port, Some(loader))
-
- def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int): ActorRef =
- RemoteActorRef(serviceId, className, hostname, port, timeout, None)
-
- def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, hostname: String, port: Int): T = {
- typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, Actor.TIMEOUT, hostname, port, None)
- }
-
- def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, timeout: Long, hostname: String, port: Int): T = {
- typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, timeout, hostname, port, None)
- }
-
- def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): T = {
- typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, timeout, hostname, port, Some(loader))
- }
-
- def typedActorFor[T](intfClass: Class[T], serviceId: String, implClassName: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): T = {
- typedActorFor(intfClass, serviceId, implClassName, timeout, hostname, port, Some(loader))
- }
-
- private[akka] def typedActorFor[T](intfClass: Class[T], serviceId: String, implClassName: String, timeout: Long, hostname: String, port: Int, loader: Option[ClassLoader]): T = {
- val actorRef = RemoteActorRef(serviceId, implClassName, hostname, port, timeout, loader, ActorType.TypedActor)
- TypedActor.createProxyForRemoteActorRef(intfClass, actorRef)
- }
-
- private[akka] def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): ActorRef =
- RemoteActorRef(serviceId, className, hostname, port, timeout, Some(loader))
-
- private[akka] def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int, loader: Option[ClassLoader]): ActorRef =
- RemoteActorRef(serviceId, className, hostname, port, timeout, loader)
-
- def clientFor(hostname: String, port: Int): RemoteClient =
- clientFor(new InetSocketAddress(hostname, port), None)
-
- def clientFor(hostname: String, port: Int, loader: ClassLoader): RemoteClient =
- clientFor(new InetSocketAddress(hostname, port), Some(loader))
-
- def clientFor(address: InetSocketAddress): RemoteClient =
- clientFor(address, None)
-
- def clientFor(address: InetSocketAddress, loader: ClassLoader): RemoteClient =
- clientFor(address, Some(loader))
-
- private[akka] def clientFor(hostname: String, port: Int, loader: Option[ClassLoader]): RemoteClient =
- clientFor(new InetSocketAddress(hostname, port), loader)
-
- private[akka] def clientFor(
- address: InetSocketAddress, loader: Option[ClassLoader]): RemoteClient = synchronized {
- val hostname = address.getHostName
- val port = address.getPort
- val hash = hostname + ':' + port
- loader.foreach(MessageSerializer.setClassLoader(_))
- if (remoteClients.contains(hash)) remoteClients(hash)
- else {
- val client = new RemoteClient(hostname, port, loader)
- client.connect
- remoteClients += hash -> client
- client
- }
- }
-
- def shutdownClientFor(address: InetSocketAddress) = synchronized {
- val hostname = address.getHostName
- val port = address.getPort
- val hash = hostname + ':' + port
- if (remoteClients.contains(hash)) {
- val client = remoteClients(hash)
- client.shutdown
- remoteClients -= hash
- }
- }
-
- /**
- * Clean-up all open connections.
- */
- def shutdownAll = synchronized {
- remoteClients.foreach({ case (addr, client) => client.shutdown })
- remoteClients.clear
- }
-
- def register(hostname: String, port: Int, uuid: Uuid) = synchronized {
- actorsFor(Address(hostname, port)) += uuid
- }
-
- private[akka] def unregister(hostname: String, port: Int, uuid: Uuid) = synchronized {
- val set = actorsFor(Address(hostname, port))
- set -= uuid
- if (set.isEmpty) shutdownClientFor(new InetSocketAddress(hostname, port))
- }
-
- private[akka] def actorsFor(remoteServerAddress: Address): HashSet[Uuid] = {
- val set = remoteActors.get(remoteServerAddress)
- if (set.isDefined && (set.get ne null)) set.get
- else {
- val remoteActorSet = new HashSet[Uuid]
- remoteActors.put(remoteServerAddress, remoteActorSet)
- remoteActorSet
- }
- }
-}
-
-/**
- * RemoteClient represents a connection to a RemoteServer. Is used to send messages to remote actors on the RemoteServer.
- *
- * @author Jonas Bonér
- */
-class RemoteClient private[akka] (
- val hostname: String, val port: Int, val loader: Option[ClassLoader] = None)
- extends Logging with ListenerManagement {
- val name = "RemoteClient@" + hostname + "::" + port
-
- //FIXME Should these be clear:ed on postStop?
- private val futures = new ConcurrentHashMap[Uuid, CompletableFuture[_]]
- private val supervisors = new ConcurrentHashMap[Uuid, ActorRef]
-
- private val remoteAddress = new InetSocketAddress(hostname, port)
-
- //FIXME rewrite to a wrapper object (minimize volatile access and maximize encapsulation)
- @volatile
- private var bootstrap: ClientBootstrap = _
- @volatile
- private[remote] var connection: ChannelFuture = _
- @volatile
- private[remote] var openChannels: DefaultChannelGroup = _
- @volatile
- private var timer: HashedWheelTimer = _
- private[remote] val runSwitch = new Switch()
- private[remote] val isAuthenticated = new AtomicBoolean(false)
-
- private[remote] def isRunning = runSwitch.isOn
-
- private val reconnectionTimeWindow = Duration(config.getInt(
- "akka.remote.client.reconnection-time-window", 600), TIME_UNIT).toMillis
- @volatile
- private var reconnectionTimeWindowStart = 0L
-
- def connect = runSwitch switchOn {
- openChannels = new DefaultChannelGroup(classOf[RemoteClient].getName)
- timer = new HashedWheelTimer
-
- bootstrap = new ClientBootstrap(new NioClientSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool))
- bootstrap.setPipelineFactory(new RemoteClientPipelineFactory(name, futures, supervisors, bootstrap, remoteAddress, timer, this))
- bootstrap.setOption("tcpNoDelay", true)
- bootstrap.setOption("keepAlive", true)
-
- log.slf4j.info("Starting remote client connection to [{}:{}]", hostname, port)
-
- // Wait until the connection attempt succeeds or fails.
- connection = bootstrap.connect(remoteAddress)
- val channel = connection.awaitUninterruptibly.getChannel
- openChannels.add(channel)
-
- if (!connection.isSuccess) {
- notifyListeners(RemoteClientError(connection.getCause, this))
- log.slf4j.error("Remote client connection to [{}:{}] has failed", hostname, port)
- log.slf4j.debug("Remote client connection failed", connection.getCause)
- }
- notifyListeners(RemoteClientStarted(this))
- }
-
- def shutdown = runSwitch switchOff {
- log.slf4j.info("Shutting down {}", name)
- notifyListeners(RemoteClientShutdown(this))
- timer.stop
- timer = null
- openChannels.close.awaitUninterruptibly
- openChannels = null
- bootstrap.releaseExternalResources
- bootstrap = null
- connection = null
- log.slf4j.info("{} has been shut down", name)
- }
-
- @deprecated("Use addListener instead")
- def registerListener(actorRef: ActorRef) = addListener(actorRef)
-
- @deprecated("Use removeListener instead")
- def deregisterListener(actorRef: ActorRef) = removeListener(actorRef)
-
- override def notifyListeners(message: => Any): Unit = super.notifyListeners(message)
-
- protected override def manageLifeCycleOfListeners = false
-
- def send[T](
- message: Any,
- senderOption: Option[ActorRef],
- senderFuture: Option[CompletableFuture[T]],
- remoteAddress: InetSocketAddress,
- timeout: Long,
- isOneWay: Boolean,
- actorRef: ActorRef,
- typedActorInfo: Option[Tuple2[String, String]],
- actorType: ActorType): Option[CompletableFuture[T]] = {
- val cookie = if (isAuthenticated.compareAndSet(false, true)) RemoteClient.SECURE_COOKIE
- else None
- send(createRemoteMessageProtocolBuilder(
- Some(actorRef),
- Left(actorRef.uuid),
- actorRef.id,
- actorRef.actorClassName,
- actorRef.timeout,
- Left(message),
- isOneWay,
- senderOption,
- typedActorInfo,
- actorType,
- cookie
- ).build, senderFuture)
- }
-
- def send[T](
- request: RemoteMessageProtocol,
- senderFuture: Option[CompletableFuture[T]]): Option[CompletableFuture[T]] = {
- 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)
- connection.getChannel.write(request)
- Some(futureResult)
- }
- } else {
- val exception = new RemoteClientException(
- "Remote client is not running, make sure you have invoked 'RemoteClient.connect' before using it.", this)
- notifyListeners(RemoteClientError(exception, this))
- throw exception
- }
- }
-
- private[akka] def registerSupervisorForActor(actorRef: ActorRef) =
- if (!actorRef.supervisor.isDefined) throw new IllegalActorStateException(
- "Can't register supervisor for " + actorRef + " since it is not under supervision")
- else supervisors.putIfAbsent(actorRef.supervisor.get.uuid, actorRef)
-
- private[akka] def deregisterSupervisorForActor(actorRef: ActorRef) =
- if (!actorRef.supervisor.isDefined) throw new IllegalActorStateException(
- "Can't unregister supervisor for " + actorRef + " since it is not under supervision")
- else supervisors.remove(actorRef.supervisor.get.uuid)
-
- private[akka] def isWithinReconnectionTimeWindow: Boolean = {
- if (reconnectionTimeWindowStart == 0L) {
- reconnectionTimeWindowStart = System.currentTimeMillis
- true
- } else {
- val timeLeft = reconnectionTimeWindow - (System.currentTimeMillis - reconnectionTimeWindowStart)
- if (timeLeft > 0) {
- log.slf4j.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft)
- true
- } else false
- }
- }
-
- private[akka] def resetReconnectionTimeWindow = reconnectionTimeWindowStart = 0L
-}
-
-/**
- * @author Jonas Bonér
- */
-class RemoteClientPipelineFactory(
- name: String,
- futures: ConcurrentMap[Uuid, CompletableFuture[_]],
- supervisors: ConcurrentMap[Uuid, ActorRef],
- bootstrap: ClientBootstrap,
- remoteAddress: SocketAddress,
- timer: HashedWheelTimer,
- client: RemoteClient) extends ChannelPipelineFactory {
-
- def getPipeline: ChannelPipeline = {
- def join(ch: ChannelHandler*) = Array[ChannelHandler](ch: _*)
-
- lazy val engine = {
- val e = RemoteServerSslContext.client.createSSLEngine()
- e.setEnabledCipherSuites(e.getSupportedCipherSuites) //TODO is this sensible?
- e.setUseClientMode(true)
- e
- }
-
- val ssl = if (RemoteServer.SECURE) join(new SslHandler(engine)) else join()
- val timeout = new ReadTimeoutHandler(timer, RemoteClient.READ_TIMEOUT.toMillis.toInt)
- val lenDec = new LengthFieldBasedFrameDecoder(RemoteClient.MESSAGE_FRAME_SIZE, 0, 4, 0, 4)
- val lenPrep = new LengthFieldPrepender(4)
- val protobufDec = new ProtobufDecoder(RemoteMessageProtocol.getDefaultInstance)
- val protobufEnc = new ProtobufEncoder
- val (enc, dec) = RemoteServer.COMPRESSION_SCHEME match {
- case "zlib" => (join(new ZlibEncoder(RemoteServer.ZLIB_COMPRESSION_LEVEL)), join(new ZlibDecoder))
- case _ => (join(), join())
- }
-
- val remoteClient = new RemoteClientHandler(name, futures, supervisors, bootstrap, remoteAddress, timer, client)
- val stages = ssl ++ join(timeout) ++ dec ++ join(lenDec, protobufDec) ++ enc ++ join(lenPrep, protobufEnc, remoteClient)
- new StaticChannelPipeline(stages: _*)
- }
-}
-
-/**
- * @author Jonas Bonér
- */
-@ChannelHandler.Sharable
-class RemoteClientHandler(
- val name: String,
- val futures: ConcurrentMap[Uuid, CompletableFuture[_]],
- val supervisors: ConcurrentMap[Uuid, ActorRef],
- val bootstrap: ClientBootstrap,
- val remoteAddress: SocketAddress,
- val timer: HashedWheelTimer,
- val client: RemoteClient)
- extends SimpleChannelUpstreamHandler with Logging {
-
- override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
- if (event.isInstanceOf[ChannelStateEvent] &&
- event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) {
- log.slf4j.debug(event.toString)
- }
- super.handleUpstream(ctx, event)
- }
-
- override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) {
- try {
- event.getMessage match {
- case reply: RemoteMessageProtocol =>
- val replyUuid = uuidFrom(reply.getActorInfo.getUuid.getHigh, reply.getActorInfo.getUuid.getLow)
- log.debug("Remote client received RemoteMessageProtocol[\n{}]",reply)
- 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 {
- val exception = parseException(reply, client.loader)
- 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, exception)
- }
-
- future.completeWithException(exception)
- }
-
- case message =>
- val exception = new RemoteClientException("Unknown message received in remote client handler: " + message, client)
- client.notifyListeners(RemoteClientError(exception, client))
- throw exception
- }
- } catch {
- case e: Exception =>
- client.notifyListeners(RemoteClientError(e, client))
- log.slf4j.error("Unexpected exception in remote client handler", e)
- throw e
- }
- }
-
- override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = client.runSwitch ifOn {
- if (client.isWithinReconnectionTimeWindow) {
- timer.newTimeout(new TimerTask() {
- def run(timeout: Timeout) = {
- client.openChannels.remove(event.getChannel)
- client.isAuthenticated.set(false)
- log.slf4j.debug("Remote client reconnecting to [{}]", remoteAddress)
- client.connection = bootstrap.connect(remoteAddress)
- client.connection.awaitUninterruptibly // Wait until the connection attempt succeeds or fails.
- if (!client.connection.isSuccess) {
- client.notifyListeners(RemoteClientError(client.connection.getCause, client))
- log.slf4j.error("Reconnection to [{}] has failed", remoteAddress)
- log.slf4j.debug("Reconnection failed", client.connection.getCause)
- }
- }
- }, RemoteClient.RECONNECT_DELAY.toMillis, TimeUnit.MILLISECONDS)
- } else spawn { client.shutdown }
- }
-
- override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
- def connect = {
- client.notifyListeners(RemoteClientConnected(client))
- log.slf4j.debug("Remote client connected to [{}]", ctx.getChannel.getRemoteAddress)
- client.resetReconnectionTimeWindow
- }
-
- if (RemoteServer.SECURE) {
- val sslHandler: SslHandler = ctx.getPipeline.get(classOf[SslHandler])
- sslHandler.handshake.addListener(new ChannelFutureListener {
- def operationComplete(future: ChannelFuture): Unit = {
- if (future.isSuccess) connect
- else throw new RemoteClientException("Could not establish SSL handshake", client)
- }
- })
- } else connect
- }
-
- override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
- client.notifyListeners(RemoteClientDisconnected(client))
- log.slf4j.debug("Remote client disconnected from [{}]", ctx.getChannel.getRemoteAddress)
- }
-
- override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
- client.notifyListeners(RemoteClientError(event.getCause, client))
- if (event.getCause ne null)
- log.slf4j.error("Unexpected exception from downstream in remote client", event.getCause)
- else
- log.slf4j.error("Unexpected exception from downstream in remote client: {}", event)
-
- event.getChannel.close
- }
-
- private def parseException(reply: RemoteMessageProtocol, loader: Option[ClassLoader]): Throwable = {
- val exception = reply.getException
- val classname = exception.getClassname
- try {
- val exceptionClass = if (loader.isDefined) loader.get.loadClass(classname)
- else Class.forName(classname)
- exceptionClass
- .getConstructor(Array[Class[_]](classOf[String]): _*)
- .newInstance(exception.getMessage).asInstanceOf[Throwable]
- } catch {
- case problem =>
- log.debug("Couldn't parse exception returned from RemoteServer",problem)
- log.warn("Couldn't create instance of {} with message {}, returning UnparsableException",classname, exception.getMessage)
- UnparsableException(classname, exception.getMessage)
- }
- }
-}
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteServer.scala b/akka-remote/src/main/scala/akka/remote/RemoteServer.scala
deleted file mode 100644
index 68052ce768..0000000000
--- a/akka-remote/src/main/scala/akka/remote/RemoteServer.scala
+++ /dev/null
@@ -1,981 +0,0 @@
-/**
- * Copyright (C) 2009-2011 Scalable Solutions AB
- */
-
-package akka.remote
-
-import java.lang.reflect.InvocationTargetException
-import java.net.InetSocketAddress
-import java.util.concurrent.{ConcurrentHashMap, Executors}
-import java.util.{Map => JMap}
-
-import akka.actor.Actor._
-import akka.actor.{Actor, TypedActor, ActorRef, IllegalActorStateException, RemoteActorSystemMessage, uuidFrom, Uuid, ActorRegistry, LifeCycleMessage, ActorType => AkkaActorType}
-import akka.util._
-import akka.remote.protocol.RemoteProtocol._
-import akka.remote.protocol.RemoteProtocol.ActorType._
-import akka.config.Config._
-import akka.config.ConfigurationException
-import akka.serialization.RemoteActorSerialization
-import akka.serialization.RemoteActorSerialization._
-
-import org.jboss.netty.bootstrap.ServerBootstrap
-import org.jboss.netty.channel._
-import org.jboss.netty.channel.group.{DefaultChannelGroup, ChannelGroup}
-import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
-import org.jboss.netty.handler.codec.frame.{LengthFieldBasedFrameDecoder, LengthFieldPrepender}
-import org.jboss.netty.handler.codec.protobuf.{ProtobufDecoder, ProtobufEncoder}
-import org.jboss.netty.handler.codec.compression.{ZlibEncoder, ZlibDecoder}
-import org.jboss.netty.handler.ssl.SslHandler
-
-import scala.collection.mutable.Map
-import scala.reflect.BeanProperty
-import akka.dispatch. {Future, DefaultCompletableFuture, CompletableFuture}
-import akka.japi.Creator
-
-/**
- * Use this object if you need a single remote server on a specific node.
- *
- *
- * // takes hostname and port from 'akka.conf'
- * RemoteNode.start
- *
- *
- *
- * RemoteNode.start(hostname, port)
- *
- *
- * You can specify the class loader to use to load the remote actors.
- *
- * RemoteNode.start(hostname, port, classLoader)
- *
- *
- * If you need to create more than one, then you can use the RemoteServer:
- *
- *
- * val server = new RemoteServer
- * server.start(hostname, port)
- *
- *
- * @author Jonas Bonér
- */
-object RemoteNode extends RemoteServer
-
-/**
- * For internal use only. Holds configuration variables, remote actors, remote typed actors and remote servers.
- *
- * @author Jonas Bonér
- */
-object RemoteServer {
- val isRemotingEnabled = config.getList("akka.enabled-modules").exists(_ == "remote")
-
- val UUID_PREFIX = "uuid:"
- val MESSAGE_FRAME_SIZE = config.getInt("akka.remote.server.message-frame-size", 1048576)
- val SECURE_COOKIE = config.getString("akka.remote.secure-cookie")
- val REQUIRE_COOKIE = {
- val requireCookie = config.getBool("akka.remote.server.require-cookie", true)
- if (isRemotingEnabled && requireCookie && RemoteServer.SECURE_COOKIE.isEmpty) throw new ConfigurationException(
- "Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'.")
- requireCookie
- }
-
- val UNTRUSTED_MODE = config.getBool("akka.remote.server.untrusted-mode", false)
- val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost")
- val PORT = config.getInt("akka.remote.server.port", 2552)
- val CONNECTION_TIMEOUT_MILLIS = Duration(config.getInt("akka.remote.server.connection-timeout", 1), TIME_UNIT)
- val COMPRESSION_SCHEME = config.getString("akka.remote.compression-scheme", "zlib")
- val ZLIB_COMPRESSION_LEVEL = {
- val level = config.getInt("akka.remote.zlib-compression-level", 6)
- if (level < 1 && level > 9) throw new IllegalArgumentException(
- "zlib compression level has to be within 1-9, with 1 being fastest and 9 being the most compressed")
- level
- }
-
- val SECURE = {
- /*if (config.getBool("akka.remote.ssl.service",false)) {
- val properties = List(
- ("key-store-type" , "keyStoreType"),
- ("key-store" , "keyStore"),
- ("key-store-pass" , "keyStorePassword"),
- ("trust-store-type", "trustStoreType"),
- ("trust-store" , "trustStore"),
- ("trust-store-pass", "trustStorePassword")
- ).map(x => ("akka.remote.ssl." + x._1, "javax.net.ssl." + x._2))
-
- // If property is not set, and we have a value from our akka.conf, use that value
- for {
- p <- properties if System.getProperty(p._2) eq null
- c <- config.getString(p._1)
- } System.setProperty(p._2, c)
-
- if (config.getBool("akka.remote.ssl.debug", false)) System.setProperty("javax.net.debug","ssl")
- true
- } else */false
- }
-
- private val guard = new ReadWriteGuard
- private val remoteServers = Map[Address, RemoteServer]()
-
- def serverFor(address: InetSocketAddress): Option[RemoteServer] =
- serverFor(address.getHostName, address.getPort)
-
- def serverFor(hostname: String, port: Int): Option[RemoteServer] = guard.withReadGuard {
- remoteServers.get(Address(hostname, port))
- }
-
- private[akka] def getOrCreateServer(address: InetSocketAddress): RemoteServer = guard.withWriteGuard {
- serverFor(address) match {
- case Some(server) => server
- case None => (new RemoteServer).start(address)
- }
- }
-
- private[akka] def register(hostname: String, port: Int, server: RemoteServer) = guard.withWriteGuard {
- remoteServers.put(Address(hostname, port), server)
- }
-
- private[akka] def unregister(hostname: String, port: Int) = guard.withWriteGuard {
- remoteServers.remove(Address(hostname, port))
- }
-
- /**
- * Used in ReflectiveAccess
- */
- private[akka] def registerActor(address: InetSocketAddress, actorRef: ActorRef) {
- serverFor(address) foreach { _.register(actorRef) }
- }
-
- /**
- * Used in Reflective
- */
- private[akka] def registerTypedActor(address: InetSocketAddress, implementationClassName: String, proxy: AnyRef) {
- serverFor(address) foreach { _.registerTypedActor(implementationClassName,proxy)}
- }
-}
-
-/**
- * Life-cycle events for RemoteServer.
- */
-sealed trait RemoteServerLifeCycleEvent
-case class RemoteServerStarted(
- @BeanProperty val server: RemoteServer) extends RemoteServerLifeCycleEvent
-case class RemoteServerShutdown(
- @BeanProperty val server: RemoteServer) extends RemoteServerLifeCycleEvent
-case class RemoteServerError(
- @BeanProperty val cause: Throwable,
- @BeanProperty val server: RemoteServer) extends RemoteServerLifeCycleEvent
-case class RemoteServerClientConnected(
- @BeanProperty val server: RemoteServer,
- @BeanProperty val clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
-case class RemoteServerClientDisconnected(
- @BeanProperty val server: RemoteServer,
- @BeanProperty val clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
-case class RemoteServerClientClosed(
- @BeanProperty val server: RemoteServer,
- @BeanProperty val clientAddress: Option[InetSocketAddress]) extends RemoteServerLifeCycleEvent
-
-/**
- * Use this class if you need a more than one remote server on a specific node.
- *
- *
- * val server = new RemoteServer
- * server.start
- *
- *
- * If you need to create more than one, then you can use the RemoteServer:
- *
- *
- * RemoteNode.start
- *
- *
- * @author Jonas Bonér
- */
-class RemoteServer extends Logging with ListenerManagement {
- import RemoteServer._
- def name = "RemoteServer@" + hostname + ":" + port
-
- private[akka] var address = Address(RemoteServer.HOSTNAME,RemoteServer.PORT)
-
- def hostname = address.hostname
- def port = address.port
-
- @volatile private var _isRunning = false
-
- private val factory = new NioServerSocketChannelFactory(
- Executors.newCachedThreadPool,
- Executors.newCachedThreadPool)
-
- private val bootstrap = new ServerBootstrap(factory)
-
- // group of open channels, used for clean-up
- private val openChannels: ChannelGroup = new DefaultChannelGroup("akka-remote-server")
-
- def isRunning = _isRunning
-
- def start: RemoteServer =
- start(hostname, port, None)
-
- def start(loader: ClassLoader): RemoteServer =
- start(hostname, port, Some(loader))
-
- def start(address: InetSocketAddress): RemoteServer =
- start(address.getHostName, address.getPort, None)
-
- def start(address: InetSocketAddress, loader: ClassLoader): RemoteServer =
- start(address.getHostName, address.getPort, Some(loader))
-
- def start(_hostname: String, _port: Int): RemoteServer =
- start(_hostname, _port, None)
-
- private def start(_hostname: String, _port: Int, loader: ClassLoader): RemoteServer =
- start(_hostname, _port, Some(loader))
-
- private def start(_hostname: String, _port: Int, loader: Option[ClassLoader]): RemoteServer = synchronized {
- try {
- if (!_isRunning) {
- address = Address(_hostname,_port)
- log.slf4j.info("Starting remote server at [{}:{}]", hostname, port)
- RemoteServer.register(hostname, port, this)
-
- val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, loader, this)
- bootstrap.setPipelineFactory(pipelineFactory)
- bootstrap.setOption("child.tcpNoDelay", true)
- bootstrap.setOption("child.keepAlive", true)
- bootstrap.setOption("child.reuseAddress", true)
- bootstrap.setOption("child.connectTimeoutMillis", RemoteServer.CONNECTION_TIMEOUT_MILLIS.toMillis)
-
- openChannels.add(bootstrap.bind(new InetSocketAddress(hostname, port)))
- _isRunning = true
- notifyListeners(RemoteServerStarted(this))
- }
- } catch {
- case e =>
- log.slf4j.error("Could not start up remote server", e)
- notifyListeners(RemoteServerError(e, this))
- }
- this
- }
-
- def shutdown = synchronized {
- if (_isRunning) {
- try {
- RemoteServer.unregister(hostname, port)
- openChannels.disconnect
- openChannels.close.awaitUninterruptibly
- bootstrap.releaseExternalResources
- notifyListeners(RemoteServerShutdown(this))
- } catch {
- case e: java.nio.channels.ClosedChannelException => {}
- case e => log.slf4j.warn("Could not close remote server channel in a graceful way")
- }
- }
- }
-
- /**
- * Register typed actor by interface name.
- */
- def registerTypedActor(intfClass: Class[_], typedActor: AnyRef) : Unit = registerTypedActor(intfClass.getName, typedActor)
-
- /**
- * Register remote typed actor by a specific id.
- * @param id custom actor id
- * @param typedActor typed actor to register
- */
- def registerTypedActor(id: String, typedActor: AnyRef): Unit = synchronized {
- log.slf4j.debug("Registering server side remote typed actor [{}] with id [{}]", typedActor.getClass.getName, id)
- if (id.startsWith(UUID_PREFIX)) registerTypedActor(id.substring(UUID_PREFIX.length), typedActor, typedActorsByUuid)
- else registerTypedActor(id, typedActor, typedActors)
- }
-
- /**
- * Register typed actor by interface name.
- */
- def registerTypedPerSessionActor(intfClass: Class[_], factory: => AnyRef) : Unit = registerTypedActor(intfClass.getName, factory)
-
- /**
- * Register typed actor by interface name.
- * Java API
- */
- def registerTypedPerSessionActor(intfClass: Class[_], factory: Creator[AnyRef]) : Unit = registerTypedActor(intfClass.getName, factory)
-
- /**
- * Register remote typed actor by a specific id.
- * @param id custom actor id
- * @param typedActor typed actor to register
- */
- def registerTypedPerSessionActor(id: String, factory: => AnyRef): Unit = synchronized {
- log.slf4j.debug("Registering server side typed remote session actor with id [{}]", id)
- registerTypedPerSessionActor(id, () => factory, typedActorsFactories)
- }
-
- /**
- * Register remote typed actor by a specific id.
- * @param id custom actor id
- * @param typedActor typed actor to register
- * Java API
- */
- def registerTypedPerSessionActor(id: String, factory: Creator[AnyRef]): Unit = synchronized {
- log.slf4j.debug("Registering server side typed remote session actor with id [{}]", id)
- registerTypedPerSessionActor(id, factory.create _, typedActorsFactories)
- }
-
- /**
- * Register Remote Actor by the Actor's 'id' field. It starts the Actor if it is not started already.
- */
- def register(actorRef: ActorRef): Unit = register(actorRef.id, actorRef)
-
- /**
- * Register Remote Actor by a specific 'id' passed as argument.
- *
- * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
- */
- def register(id: String, actorRef: ActorRef): Unit = synchronized {
- log.slf4j.debug("Registering server side remote actor [{}] with id [{}]", actorRef.actorClass.getName, id)
- if (id.startsWith(UUID_PREFIX)) register(id.substring(UUID_PREFIX.length), actorRef, actorsByUuid)
- else register(id, actorRef, actors)
- }
-
- /**
- * Register Remote Session Actor by a specific 'id' passed as argument.
- *
- * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
- */
- def registerPerSession(id: String, factory: => ActorRef): Unit = synchronized {
- log.slf4j.debug("Registering server side remote session actor with id [{}]", id)
- registerPerSession(id, () => factory, actorsFactories)
- }
-
- /**
- * Register Remote Session Actor by a specific 'id' passed as argument.
- *
- * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
- * Java API
- */
- def registerPerSession(id: String, factory: Creator[ActorRef]): Unit = synchronized {
- log.slf4j.debug("Registering server side remote session actor with id [{}]", id)
- registerPerSession(id, factory.create _, actorsFactories)
- }
-
- private def register[Key](id: Key, actorRef: ActorRef, registry: ConcurrentHashMap[Key, ActorRef]) {
- if (_isRunning) {
- registry.put(id, actorRef) //TODO change to putIfAbsent
- if (!actorRef.isRunning) actorRef.start
- }
- }
-
- private def registerPerSession[Key](id: Key, factory: () => ActorRef, registry: ConcurrentHashMap[Key,() => ActorRef]) {
- if (_isRunning)
- registry.put(id, factory) //TODO change to putIfAbsent
- }
-
- private def registerTypedActor[Key](id: Key, typedActor: AnyRef, registry: ConcurrentHashMap[Key, AnyRef]) {
- if (_isRunning)
- registry.put(id, typedActor) //TODO change to putIfAbsent
- }
-
- private def registerTypedPerSessionActor[Key](id: Key, factory: () => AnyRef, registry: ConcurrentHashMap[Key,() => AnyRef]) {
- if (_isRunning)
- registry.put(id, factory) //TODO change to putIfAbsent
- }
-
- /**
- * Unregister Remote Actor that is registered using its 'id' field (not custom ID).
- */
- def unregister(actorRef: ActorRef):Unit = synchronized {
- if (_isRunning) {
- log.slf4j.debug("Unregistering server side remote actor [{}] with id [{}:{}]", Array[AnyRef](actorRef.actorClass.getName, actorRef.id, actorRef.uuid))
- actors.remove(actorRef.id, actorRef)
- actorsByUuid.remove(actorRef.uuid, actorRef)
- }
- }
-
- /**
- * Unregister Remote Actor by specific 'id'.
- *
- * NOTE: You need to call this method if you have registered an actor by a custom ID.
- */
- def unregister(id: String):Unit = synchronized {
- if (_isRunning) {
- log.slf4j.info("Unregistering server side remote actor with id [{}]", id)
- if (id.startsWith(UUID_PREFIX)) actorsByUuid.remove(id.substring(UUID_PREFIX.length))
- else {
- val actorRef = actors get id
- actorsByUuid.remove(actorRef.uuid, actorRef)
- actors.remove(id,actorRef)
- }
- }
- }
-
- /**
- * Unregister Remote Actor by specific 'id'.
- *
- * NOTE: You need to call this method if you have registered an actor by a custom ID.
- */
- def unregisterPerSession(id: String):Unit = {
- if (_isRunning) {
- log.slf4j.info("Unregistering server side remote session actor with id [{}]", id)
- actorsFactories.remove(id)
- }
- }
-
- /**
- * Unregister Remote Typed Actor by specific 'id'.
- *
- * NOTE: You need to call this method if you have registered an actor by a custom ID.
- */
- def unregisterTypedActor(id: String):Unit = synchronized {
- if (_isRunning) {
- log.slf4j.info("Unregistering server side remote typed actor with id [{}]", id)
- if (id.startsWith(UUID_PREFIX)) typedActorsByUuid.remove(id.substring(UUID_PREFIX.length))
- else typedActors.remove(id)
- }
- }
-
- /**
- * Unregister Remote Typed Actor by specific 'id'.
- *
- * NOTE: You need to call this method if you have registered an actor by a custom ID.
- */
- def unregisterTypedPerSessionActor(id: String):Unit = {
- if (_isRunning) {
- typedActorsFactories.remove(id)
- }
- }
-
- protected override def manageLifeCycleOfListeners = false
-
- protected[akka] override def notifyListeners(message: => Any): Unit = super.notifyListeners(message)
-
-
- private[akka] def actors = ActorRegistry.actors(address)
- private[akka] def actorsByUuid = ActorRegistry.actorsByUuid(address)
- private[akka] def actorsFactories = ActorRegistry.actorsFactories(address)
- private[akka] def typedActors = ActorRegistry.typedActors(address)
- private[akka] def typedActorsByUuid = ActorRegistry.typedActorsByUuid(address)
- private[akka] def typedActorsFactories = ActorRegistry.typedActorsFactories(address)
-}
-
-object RemoteServerSslContext {
- import javax.net.ssl.SSLContext
-
- val (client, server) = {
- val protocol = "TLS"
- //val algorithm = Option(Security.getProperty("ssl.KeyManagerFactory.algorithm")).getOrElse("SunX509")
- //val store = KeyStore.getInstance("JKS")
- val s = SSLContext.getInstance(protocol)
- s.init(null, null, null)
- val c = SSLContext.getInstance(protocol)
- c.init(null, null, null)
- (c, s)
- }
-}
-
-/**
- * @author Jonas Bonér
- */
-class RemoteServerPipelineFactory(
- val name: String,
- val openChannels: ChannelGroup,
- val loader: Option[ClassLoader],
- val server: RemoteServer) extends ChannelPipelineFactory {
- import RemoteServer._
-
- def getPipeline: ChannelPipeline = {
- def join(ch: ChannelHandler*) = Array[ChannelHandler](ch:_*)
-
- lazy val engine = {
- val e = RemoteServerSslContext.server.createSSLEngine()
- e.setEnabledCipherSuites(e.getSupportedCipherSuites) //TODO is this sensible?
- e.setUseClientMode(false)
- e
- }
-
- val ssl = if(RemoteServer.SECURE) join(new SslHandler(engine)) else join()
- val lenDec = new LengthFieldBasedFrameDecoder(RemoteServer.MESSAGE_FRAME_SIZE, 0, 4, 0, 4)
- val lenPrep = new LengthFieldPrepender(4)
- val protobufDec = new ProtobufDecoder(RemoteMessageProtocol.getDefaultInstance)
- val protobufEnc = new ProtobufEncoder
- val (enc, dec) = RemoteServer.COMPRESSION_SCHEME match {
- case "zlib" => (join(new ZlibEncoder(RemoteServer.ZLIB_COMPRESSION_LEVEL)), join(new ZlibDecoder))
- case _ => (join(), join())
- }
-
- val remoteServer = new RemoteServerHandler(name, openChannels, loader, server)
- val stages = ssl ++ dec ++ join(lenDec, protobufDec) ++ enc ++ join(lenPrep, protobufEnc, remoteServer)
- new StaticChannelPipeline(stages: _*)
- }
-}
-
-/**
- * @author Jonas Bonér
- */
-@ChannelHandler.Sharable
-class RemoteServerHandler(
- val name: String,
- val openChannels: ChannelGroup,
- val applicationLoader: Option[ClassLoader],
- val server: RemoteServer) extends SimpleChannelUpstreamHandler with Logging {
- import RemoteServer._
-
- val AW_PROXY_PREFIX = "$$ProxiedByAW".intern
- val CHANNEL_INIT = "channel-init".intern
-
- val sessionActors = new ChannelLocal[ConcurrentHashMap[String, ActorRef]]()
- val typedSessionActors = new ChannelLocal[ConcurrentHashMap[String, AnyRef]]()
-
- applicationLoader.foreach(MessageSerializer.setClassLoader(_))
-
- /**
- * ChannelOpen overridden to store open channels for a clean postStop of a RemoteServer.
- * If a channel is closed before, it is automatically removed from the open channels group.
- */
- override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel)
-
- override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
- val clientAddress = getClientAddress(ctx)
- sessionActors.set(event.getChannel(), new ConcurrentHashMap[String, ActorRef]())
- typedSessionActors.set(event.getChannel(), new ConcurrentHashMap[String, AnyRef]())
- log.slf4j.debug("Remote client [{}] connected to [{}]", clientAddress, server.name)
- if (RemoteServer.SECURE) {
- val sslHandler: SslHandler = ctx.getPipeline.get(classOf[SslHandler])
- // Begin handshake.
- sslHandler.handshake().addListener(new ChannelFutureListener {
- def operationComplete(future: ChannelFuture): Unit = {
- if (future.isSuccess) {
- openChannels.add(future.getChannel)
- server.notifyListeners(RemoteServerClientConnected(server, clientAddress))
- } else future.getChannel.close
- }
- })
- } else server.notifyListeners(RemoteServerClientConnected(server, clientAddress))
- if (RemoteServer.REQUIRE_COOKIE) ctx.setAttachment(CHANNEL_INIT) // signal that this is channel initialization, which will need authentication
- }
-
- override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
- val clientAddress = getClientAddress(ctx)
- log.slf4j.debug("Remote client [{}] disconnected from [{}]", clientAddress, server.name)
- // stop all session actors
- val channelActors = sessionActors.remove(event.getChannel)
- if (channelActors ne null) {
- val channelActorsIterator = channelActors.elements
- while (channelActorsIterator.hasMoreElements) {
- channelActorsIterator.nextElement.stop
- }
- }
-
- val channelTypedActors = typedSessionActors.remove(event.getChannel)
- if (channelTypedActors ne null) {
- val channelTypedActorsIterator = channelTypedActors.elements
- while (channelTypedActorsIterator.hasMoreElements) {
- TypedActor.stop(channelTypedActorsIterator.nextElement)
- }
- }
- server.notifyListeners(RemoteServerClientDisconnected(server, clientAddress))
- }
-
- override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
- val clientAddress = getClientAddress(ctx)
- log.slf4j.debug("Remote client [{}] channel closed from [{}]", clientAddress, server.name)
- server.notifyListeners(RemoteServerClientClosed(server, clientAddress))
- }
-
- override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
- if (event.isInstanceOf[ChannelStateEvent] && event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) {
- log.slf4j.debug(event.toString)
- }
- super.handleUpstream(ctx, event)
- }
-
- override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = {
- val message = event.getMessage
- if (message eq null) throw new IllegalActorStateException("Message in remote MessageEvent is null: " + event)
- if (message.isInstanceOf[RemoteMessageProtocol]) {
- val requestProtocol = message.asInstanceOf[RemoteMessageProtocol]
- if (RemoteServer.REQUIRE_COOKIE) authenticateRemoteClient(requestProtocol, ctx)
- handleRemoteMessageProtocol(requestProtocol, event.getChannel)
- }
- }
-
- override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
- log.slf4j.error("Unexpected exception from remote downstream", event.getCause)
- event.getChannel.close
- server.notifyListeners(RemoteServerError(event.getCause, server))
- }
-
- private def getClientAddress(ctx: ChannelHandlerContext): Option[InetSocketAddress] = {
- val remoteAddress = ctx.getChannel.getRemoteAddress
- if (remoteAddress.isInstanceOf[InetSocketAddress]) Some(remoteAddress.asInstanceOf[InetSocketAddress])
- else None
- }
-
- private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = {
- log.slf4j.debug("Received RemoteMessageProtocol[\n{}]",request)
- request.getActorInfo.getActorType match {
- case SCALA_ACTOR => dispatchToActor(request, channel)
- case TYPED_ACTOR => dispatchToTypedActor(request, channel)
- case JAVA_ACTOR => throw new IllegalActorStateException("ActorType JAVA_ACTOR is currently not supported")
- case other => throw new IllegalActorStateException("Unknown ActorType [" + other + "]")
- }
- }
-
- private def dispatchToActor(request: RemoteMessageProtocol, channel: Channel) {
- val actorInfo = request.getActorInfo
- log.slf4j.debug("Dispatching to remote actor [{}:{}]", actorInfo.getTarget, actorInfo.getUuid)
-
- val actorRef =
- try {
- createActor(actorInfo, channel).start
- } catch {
- case e: SecurityException =>
- channel.write(createErrorReplyMessage(e, request, AkkaActorType.ScalaActor))
- server.notifyListeners(RemoteServerError(e, server))
- return
- }
-
- val message = MessageSerializer.deserialize(request.getMessage)
- val sender =
- if (request.hasSender) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(request.getSender, applicationLoader))
- else None
-
- message match { // first match on system messages
- case RemoteActorSystemMessage.Stop =>
- if (RemoteServer.UNTRUSTED_MODE) throw new SecurityException("Remote server is operating is untrusted mode, can not stop the actor")
- else actorRef.stop
- case _: LifeCycleMessage if (RemoteServer.UNTRUSTED_MODE) =>
- throw new SecurityException("Remote server is operating is untrusted mode, can not pass on a LifeCycleMessage to the remote actor")
-
- case _ => // then match on user defined messages
- if (request.getOneWay) actorRef.!(message)(sender)
- else actorRef.postMessageToMailboxAndCreateFutureResultWithTimeout(
- message,
- request.getActorInfo.getTimeout,
- None,
- Some(new DefaultCompletableFuture[AnyRef](request.getActorInfo.getTimeout).
- onComplete(f => {
- val result = f.result
- val exception = f.exception
-
- if (exception.isDefined) {
- log.slf4j.debug("Returning exception from actor invocation [{}]",exception.get)
- try {
- channel.write(createErrorReplyMessage(exception.get, request, AkkaActorType.ScalaActor))
- } catch {
- case e: Throwable => server.notifyListeners(RemoteServerError(e, server))
- }
- }
- else if (result.isDefined) {
- log.slf4j.debug("Returning result from actor invocation [{}]",result.get)
- val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder(
- Some(actorRef),
- Right(request.getUuid),
- actorInfo.getId,
- actorInfo.getTarget,
- actorInfo.getTimeout,
- Left(result.get),
- true,
- Some(actorRef),
- None,
- AkkaActorType.ScalaActor,
- None)
-
- // FIXME lift in the supervisor uuid management into toh createRemoteMessageProtocolBuilder method
- if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid)
-
- try {
- channel.write(messageBuilder.build)
- } catch {
- case e: Throwable => server.notifyListeners(RemoteServerError(e, server))
- }
- }
- }
- )
- ))
- }
- }
-
- private def dispatchToTypedActor(request: RemoteMessageProtocol, channel: Channel) = {
- val actorInfo = request.getActorInfo
- val typedActorInfo = actorInfo.getTypedActorInfo
- log.slf4j.debug("Dispatching to remote typed actor [{} :: {}]", typedActorInfo.getMethod, typedActorInfo.getInterface)
-
- val typedActor = createTypedActor(actorInfo, channel)
- val args = MessageSerializer.deserialize(request.getMessage).asInstanceOf[Array[AnyRef]].toList
- val argClasses = args.map(_.getClass)
-
- try {
- val messageReceiver = typedActor.getClass.getDeclaredMethod(typedActorInfo.getMethod, argClasses: _*)
- if (request.getOneWay) messageReceiver.invoke(typedActor, args: _*)
- else {
- //Sends the response
- def sendResponse(result: Either[Any,Throwable]): Unit = try {
- val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder(
- None,
- Right(request.getUuid),
- actorInfo.getId,
- actorInfo.getTarget,
- actorInfo.getTimeout,
- result,
- true,
- None,
- None,
- AkkaActorType.TypedActor,
- None)
- if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid)
- channel.write(messageBuilder.build)
- log.slf4j.debug("Returning result from remote typed actor invocation [{}]", result)
- } catch {
- case e: Throwable => server.notifyListeners(RemoteServerError(e, server))
- }
-
- messageReceiver.invoke(typedActor, args: _*) match {
- case f: Future[_] => //If it's a future, we can lift on that to defer the send to when the future is completed
- f.onComplete( future => {
- val result: Either[Any,Throwable] = if (future.exception.isDefined) Right(future.exception.get) else Left(future.result.get)
- sendResponse(result)
- })
- case other => sendResponse(Left(other))
- }
- }
- } catch {
- case e: InvocationTargetException =>
- channel.write(createErrorReplyMessage(e.getCause, request, AkkaActorType.TypedActor))
- server.notifyListeners(RemoteServerError(e, server))
- case e: Throwable =>
- channel.write(createErrorReplyMessage(e, request, AkkaActorType.TypedActor))
- server.notifyListeners(RemoteServerError(e, server))
- }
- }
-
- private def findActorById(id: String) : ActorRef = {
- server.actors.get(id)
- }
-
- private def findActorByUuid(uuid: String) : ActorRef = {
- server.actorsByUuid.get(uuid)
- }
-
- private def findActorFactory(id: String) : () => ActorRef = {
- server.actorsFactories.get(id)
- }
-
- private def findSessionActor(id: String, channel: Channel) : ActorRef = {
- val map = sessionActors.get(channel)
- if (map ne null) map.get(id)
- else null
- }
-
- private def findTypedActorById(id: String) : AnyRef = {
- server.typedActors.get(id)
- }
-
- private def findTypedActorFactory(id: String) : () => AnyRef = {
- server.typedActorsFactories.get(id)
- }
-
- private def findTypedSessionActor(id: String, channel: Channel) : AnyRef = {
- val map = typedSessionActors.get(channel)
- if (map ne null) map.get(id)
- else null
- }
-
- private def findTypedActorByUuid(uuid: String) : AnyRef = {
- server.typedActorsByUuid.get(uuid)
- }
-
- private def findActorByIdOrUuid(id: String, uuid: String) : ActorRef = {
- var actorRefOrNull = if (id.startsWith(UUID_PREFIX)) findActorByUuid(id.substring(UUID_PREFIX.length))
- else findActorById(id)
- if (actorRefOrNull eq null) actorRefOrNull = findActorByUuid(uuid)
- actorRefOrNull
- }
-
- private def findTypedActorByIdOrUuid(id: String, uuid: String) : AnyRef = {
- var actorRefOrNull = if (id.startsWith(UUID_PREFIX)) findTypedActorByUuid(id.substring(UUID_PREFIX.length))
- else findTypedActorById(id)
- if (actorRefOrNull eq null) actorRefOrNull = findTypedActorByUuid(uuid)
- actorRefOrNull
- }
-
- /**
- * gets the actor from the session, or creates one if there is a factory for it
- */
- private def createSessionActor(actorInfo: ActorInfoProtocol, channel: Channel): ActorRef = {
- val uuid = actorInfo.getUuid
- val id = actorInfo.getId
- val sessionActorRefOrNull = findSessionActor(id, channel)
- if (sessionActorRefOrNull ne null)
- sessionActorRefOrNull
- else
- {
- // we dont have it in the session either, see if we have a factory for it
- val actorFactoryOrNull = findActorFactory(id)
- if (actorFactoryOrNull ne null) {
- val actorRef = actorFactoryOrNull()
- actorRef.uuid = uuidFrom(uuid.getHigh,uuid.getLow)
- sessionActors.get(channel).put(id, actorRef)
- actorRef
- }
- else
- null
- }
- }
-
-
- private def createClientManagedActor(actorInfo: ActorInfoProtocol): ActorRef = {
- val uuid = actorInfo.getUuid
- val id = actorInfo.getId
- val timeout = actorInfo.getTimeout
- val name = actorInfo.getTarget
-
- try {
- 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)
- val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name)
- else Class.forName(name)
- val actorRef = Actor.actorOf(clazz.asInstanceOf[Class[_ <: Actor]])
- actorRef.uuid = uuidFrom(uuid.getHigh,uuid.getLow)
- actorRef.id = id
- actorRef.timeout = timeout
- actorRef.remoteAddress = None
- server.actorsByUuid.put(actorRef.uuid.toString, actorRef) // register by uuid
- actorRef
- } catch {
- case e =>
- log.slf4j.error("Could not create remote actor instance", e)
- server.notifyListeners(RemoteServerError(e, server))
- throw e
- }
-
- }
-
- /**
- * Creates a new instance of the actor with name, uuid and timeout specified as arguments.
- *
- * If actor already created then just return it from the registry.
- *
- * Does not start the actor.
- */
- private def createActor(actorInfo: ActorInfoProtocol, channel: Channel): ActorRef = {
- val uuid = actorInfo.getUuid
- val id = actorInfo.getId
-
- val actorRefOrNull = findActorByIdOrUuid(id, uuidFrom(uuid.getHigh,uuid.getLow).toString)
-
- if (actorRefOrNull ne null)
- actorRefOrNull
- else
- {
- // the actor has not been registered globally. See if we have it in the session
- val sessionActorRefOrNull = createSessionActor(actorInfo, channel)
- if (sessionActorRefOrNull ne null)
- sessionActorRefOrNull
- else // maybe it is a client managed actor
- createClientManagedActor(actorInfo)
- }
- }
-
- /**
- * gets the actor from the session, or creates one if there is a factory for it
- */
- private def createTypedSessionActor(actorInfo: ActorInfoProtocol, channel: Channel):AnyRef ={
- val id = actorInfo.getId
- val sessionActorRefOrNull = findTypedSessionActor(id, channel)
- if (sessionActorRefOrNull ne null)
- sessionActorRefOrNull
- else {
- val actorFactoryOrNull = findTypedActorFactory(id)
- if (actorFactoryOrNull ne null) {
- val newInstance = actorFactoryOrNull()
- typedSessionActors.get(channel).put(id, newInstance)
- newInstance
- }
- else
- null
- }
-
- }
-
- private def createClientManagedTypedActor(actorInfo: ActorInfoProtocol) = {
- val typedActorInfo = actorInfo.getTypedActorInfo
- val interfaceClassname = typedActorInfo.getInterface
- val targetClassname = actorInfo.getTarget
- val uuid = actorInfo.getUuid
-
- try {
- 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 typed actor:\n\t[{} :: {}]", interfaceClassname, targetClassname)
-
- val (interfaceClass, targetClass) =
- if (applicationLoader.isDefined) (applicationLoader.get.loadClass(interfaceClassname),
- applicationLoader.get.loadClass(targetClassname))
- else (Class.forName(interfaceClassname), Class.forName(targetClassname))
-
- val newInstance = TypedActor.newInstance(
- interfaceClass, targetClass.asInstanceOf[Class[_ <: TypedActor]], actorInfo.getTimeout).asInstanceOf[AnyRef]
- server.typedActors.put(uuidFrom(uuid.getHigh,uuid.getLow).toString, newInstance) // register by uuid
- newInstance
- } catch {
- case e =>
- log.slf4j.error("Could not create remote typed actor instance", e)
- server.notifyListeners(RemoteServerError(e, server))
- throw e
- }
- }
-
- private def createTypedActor(actorInfo: ActorInfoProtocol, channel: Channel): AnyRef = {
- val uuid = actorInfo.getUuid
- val id = actorInfo.getId
-
- val typedActorOrNull = findTypedActorByIdOrUuid(id, uuidFrom(uuid.getHigh,uuid.getLow).toString)
- if (typedActorOrNull ne null)
- typedActorOrNull
- else
- {
- // the actor has not been registered globally. See if we have it in the session
- val sessionActorRefOrNull = createTypedSessionActor(actorInfo, channel)
- if (sessionActorRefOrNull ne null)
- sessionActorRefOrNull
- else // maybe it is a client managed actor
- createClientManagedTypedActor(actorInfo)
- }
- }
-
- private def createErrorReplyMessage(exception: Throwable, request: RemoteMessageProtocol, actorType: AkkaActorType): RemoteMessageProtocol = {
- val actorInfo = request.getActorInfo
- log.slf4j.error("Could not invoke remote actor [{}]", actorInfo.getTarget)
- log.slf4j.debug("Could not invoke remote actor", exception)
- 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) = {
- val attachment = ctx.getAttachment
- if ((attachment ne null) &&
- attachment.isInstanceOf[String] &&
- attachment.asInstanceOf[String] == CHANNEL_INIT) { // is first time around, channel initialization
- ctx.setAttachment(null)
- val clientAddress = ctx.getChannel.getRemoteAddress.toString
- if (!request.hasCookie) throw new SecurityException(
- "The remote client [" + clientAddress + "] does not have a secure cookie.")
- if (!(request.getCookie == RemoteServer.SECURE_COOKIE.get)) throw new SecurityException(
- "The remote client [" + clientAddress + "] secure cookie is not the same as remote server secure cookie")
- log.slf4j.info("Remote client [{}] successfully authenticated using secure cookie", clientAddress)
- }
- }
-}
diff --git a/akka-remote/src/main/scala/akka/remote/RemoteShared.scala b/akka-remote/src/main/scala/akka/remote/RemoteShared.scala
new file mode 100644
index 0000000000..57a28f5c21
--- /dev/null
+++ b/akka-remote/src/main/scala/akka/remote/RemoteShared.scala
@@ -0,0 +1,66 @@
+/**
+ * Copyright (C) 2009-2011 Scalable Solutions AB
+ */
+
+package akka.remote
+
+import akka.util.Duration
+import akka.config.Config._
+import akka.config.ConfigurationException
+
+object RemoteClientSettings {
+ val SECURE_COOKIE: Option[String] = config.getString("akka.remote.secure-cookie", "") match {
+ case "" => None
+ case cookie => Some(cookie)
+ }
+ val RECONNECTION_TIME_WINDOW = Duration(config.getInt("akka.remote.client.reconnection-time-window", 600), TIME_UNIT).toMillis
+ val READ_TIMEOUT = Duration(config.getInt("akka.remote.client.read-timeout", 1), TIME_UNIT)
+ val RECONNECT_DELAY = Duration(config.getInt("akka.remote.client.reconnect-delay", 5), TIME_UNIT)
+ val MESSAGE_FRAME_SIZE = config.getInt("akka.remote.client.message-frame-size", 1048576)
+}
+
+object RemoteServerSettings {
+ val isRemotingEnabled = config.getList("akka.enabled-modules").exists(_ == "remote")
+ val MESSAGE_FRAME_SIZE = config.getInt("akka.remote.server.message-frame-size", 1048576)
+ val SECURE_COOKIE = config.getString("akka.remote.secure-cookie")
+ val REQUIRE_COOKIE = {
+ val requireCookie = config.getBool("akka.remote.server.require-cookie", true)
+ if (isRemotingEnabled && requireCookie && SECURE_COOKIE.isEmpty) throw new ConfigurationException(
+ "Configuration option 'akka.remote.server.require-cookie' is turned on but no secure cookie is defined in 'akka.remote.secure-cookie'.")
+ requireCookie
+ }
+
+ val UNTRUSTED_MODE = config.getBool("akka.remote.server.untrusted-mode", false)
+ val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost")
+ val PORT = config.getInt("akka.remote.server.port", 2552)
+ val CONNECTION_TIMEOUT_MILLIS = Duration(config.getInt("akka.remote.server.connection-timeout", 1), TIME_UNIT)
+ val COMPRESSION_SCHEME = config.getString("akka.remote.compression-scheme", "zlib")
+ val ZLIB_COMPRESSION_LEVEL = {
+ val level = config.getInt("akka.remote.zlib-compression-level", 6)
+ if (level < 1 && level > 9) throw new IllegalArgumentException(
+ "zlib compression level has to be within 1-9, with 1 being fastest and 9 being the most compressed")
+ level
+ }
+
+ val SECURE = {
+ /*if (config.getBool("akka.remote.ssl.service",false)) {
+ val properties = List(
+ ("key-store-type" , "keyStoreType"),
+ ("key-store" , "keyStore"),
+ ("key-store-pass" , "keyStorePassword"),
+ ("trust-store-type", "trustStoreType"),
+ ("trust-store" , "trustStore"),
+ ("trust-store-pass", "trustStorePassword")
+ ).map(x => ("akka.remote.ssl." + x._1, "javax.net.ssl." + x._2))
+
+ // If property is not set, and we have a value from our akka.conf, use that value
+ for {
+ p <- properties if System.getProperty(p._2) eq null
+ c <- config.getString(p._1)
+ } System.setProperty(p._2, c)
+
+ if (config.getBool("akka.remote.ssl.debug", false)) System.setProperty("javax.net.debug","ssl")
+ true
+ } else */false
+ }
+}
diff --git a/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala
new file mode 100644
index 0000000000..98bf270ab5
--- /dev/null
+++ b/akka-remote/src/main/scala/akka/remote/netty/NettyRemoteSupport.scala
@@ -0,0 +1,1220 @@
+/**
+ * Copyright (C) 2009-2011 Scalable Solutions AB
+ */
+
+package akka.remote.netty
+
+import akka.dispatch.{DefaultCompletableFuture, CompletableFuture, Future}
+import akka.remote.protocol.RemoteProtocol._
+import akka.remote.protocol.RemoteProtocol.ActorType._
+import akka.config.ConfigurationException
+import akka.serialization.RemoteActorSerialization
+import akka.serialization.RemoteActorSerialization._
+import akka.japi.Creator
+import akka.config.Config._
+import akka.remoteinterface._
+import akka.actor. {Index, ActorInitializationException, LocalActorRef, newUuid, ActorRegistry, Actor, RemoteActorRef, TypedActor, ActorRef, IllegalActorStateException, RemoteActorSystemMessage, uuidFrom, Uuid, Exit, LifeCycleMessage, ActorType => AkkaActorType}
+import akka.AkkaException
+import akka.actor.Actor._
+import akka.util._
+import akka.remote.{MessageSerializer, RemoteClientSettings, RemoteServerSettings}
+
+import org.jboss.netty.channel._
+import org.jboss.netty.channel.group.{DefaultChannelGroup,ChannelGroup}
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory
+import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory
+import org.jboss.netty.bootstrap.{ServerBootstrap,ClientBootstrap}
+import org.jboss.netty.handler.codec.frame.{ LengthFieldBasedFrameDecoder, LengthFieldPrepender }
+import org.jboss.netty.handler.codec.compression.{ ZlibDecoder, ZlibEncoder }
+import org.jboss.netty.handler.codec.protobuf.{ ProtobufDecoder, ProtobufEncoder }
+import org.jboss.netty.handler.timeout.ReadTimeoutHandler
+import org.jboss.netty.util.{ TimerTask, Timeout, HashedWheelTimer }
+import org.jboss.netty.handler.ssl.SslHandler
+
+import java.net.{ SocketAddress, InetSocketAddress }
+import java.util.concurrent.{ TimeUnit, Executors, ConcurrentMap, ConcurrentHashMap, ConcurrentSkipListSet }
+import scala.collection.mutable.{ HashMap }
+import scala.reflect.BeanProperty
+import java.lang.reflect.InvocationTargetException
+import java.util.concurrent.atomic. {AtomicReference, AtomicLong, AtomicBoolean}
+
+trait NettyRemoteClientModule extends RemoteClientModule { self: ListenerManagement with Logging =>
+ private val remoteClients = new HashMap[Address, RemoteClient]
+ private val remoteActors = new Index[Address, Uuid]
+ private val lock = new ReadWriteGuard
+
+ 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(serviceId, implClassName, hostname, port, timeout, loader, AkkaActorType.TypedActor))
+
+ protected[akka] def send[T](message: Any,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[T]],
+ remoteAddress: InetSocketAddress,
+ timeout: Long,
+ isOneWay: Boolean,
+ actorRef: ActorRef,
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: AkkaActorType,
+ loader: Option[ClassLoader]): Option[CompletableFuture[T]] =
+ withClientFor(remoteAddress, loader)(_.send[T](message, senderOption, senderFuture, remoteAddress, timeout, isOneWay, actorRef, typedActorInfo, actorType))
+
+ private[akka] def withClientFor[T](
+ address: InetSocketAddress, loader: Option[ClassLoader])(fun: RemoteClient => T): T = {
+ loader.foreach(MessageSerializer.setClassLoader(_))//TODO: REVISIT: THIS SMELLS FUNNY
+
+ val key = Address(address)
+ lock.readLock.lock
+ try {
+ val c = remoteClients.get(key) match {
+ case Some(client) => client
+ case None =>
+ lock.readLock.unlock
+ lock.writeLock.lock //Lock upgrade, not supported natively
+ try {
+ try {
+ remoteClients.get(key) match { //Recheck for addition, race between upgrades
+ case Some(client) => client //If already populated by other writer
+ case None => //Populate map
+ val client = new ActiveRemoteClient(this, address, loader, self.notifyListeners _)
+ client.connect()
+ remoteClients += key -> client
+ client
+ }
+ } finally { lock.readLock.lock } //downgrade
+ } finally { lock.writeLock.unlock }
+ }
+ fun(c)
+ } finally { lock.readLock.unlock }
+ }
+
+ def shutdownClientConnection(address: InetSocketAddress): Boolean = lock withWriteGuard {
+ remoteClients.remove(Address(address)) match {
+ case Some(client) => client.shutdown
+ case None => false
+ }
+ }
+
+ def restartClientConnection(address: InetSocketAddress): Boolean = lock withReadGuard {
+ remoteClients.get(Address(address)) match {
+ case Some(client) => client.connect(reconnectIfAlreadyConnected = true)
+ case None => false
+ }
+ }
+
+ private[akka] def registerSupervisorForActor(actorRef: ActorRef): ActorRef =
+ withClientFor(actorRef.homeAddress.get, None)(_.registerSupervisorForActor(actorRef))
+
+ private[akka] def deregisterSupervisorForActor(actorRef: ActorRef): ActorRef = lock withReadGuard {
+ remoteClients.get(Address(actorRef.homeAddress.get)) match {
+ case Some(client) => client.deregisterSupervisorForActor(actorRef)
+ case None => actorRef
+ }
+ }
+
+ /**
+ * Clean-up all open connections.
+ */
+ def shutdownClientModule = {
+ shutdownRemoteClients
+ //TODO: Should we empty our remoteActors too?
+ //remoteActors.clear
+ }
+
+ def shutdownRemoteClients = lock withWriteGuard {
+ remoteClients.foreach({ case (addr, client) => client.shutdown })
+ remoteClients.clear
+ }
+
+ def registerClientManagedActor(hostname: String, port: Int, uuid: Uuid) = {
+ remoteActors.put(Address(hostname, port), uuid)
+ }
+
+ private[akka] def unregisterClientManagedActor(hostname: String, port: Int, uuid: Uuid) = {
+ remoteActors.remove(Address(hostname,port), uuid)
+ //TODO: should the connection be closed when the last actor deregisters?
+ }
+}
+
+/**
+ * This is the abstract baseclass for netty remote clients,
+ * currently there's only an ActiveRemoteClient, but otehrs could be feasible, like a PassiveRemoteClient that
+ * reuses an already established connection.
+ */
+abstract class RemoteClient private[akka] (
+ val module: NettyRemoteClientModule,
+ val remoteAddress: InetSocketAddress) extends Logging {
+
+ val name = this.getClass.getSimpleName + "@" + remoteAddress.getHostName + "::" + remoteAddress.getPort
+
+ protected val futures = new ConcurrentHashMap[Uuid, CompletableFuture[_]]
+ protected val supervisors = new ConcurrentHashMap[Uuid, ActorRef]
+ private[remote] val runSwitch = new Switch()
+ private[remote] val isAuthenticated = new AtomicBoolean(false)
+
+ private[remote] def isRunning = runSwitch.isOn
+
+ protected def notifyListeners(msg: => Any); Unit
+ protected def currentChannel: Channel
+
+ def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean
+ def shutdown: Boolean
+
+ /**
+ * Converts the message to the wireprotocol and sends the message across the wire
+ */
+ def send[T](
+ message: Any,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[T]],
+ remoteAddress: InetSocketAddress,
+ timeout: Long,
+ isOneWay: Boolean,
+ actorRef: ActorRef,
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: AkkaActorType): Option[CompletableFuture[T]] = {
+ send(createRemoteMessageProtocolBuilder(
+ Some(actorRef),
+ Left(actorRef.uuid),
+ actorRef.id,
+ actorRef.actorClassName,
+ actorRef.timeout,
+ Left(message),
+ isOneWay,
+ senderOption,
+ typedActorInfo,
+ actorType,
+ if (isAuthenticated.compareAndSet(false, true)) RemoteClientSettings.SECURE_COOKIE else None
+ ).build, senderFuture)
+ }
+
+ /**
+ * Sends the message across the wire
+ */
+ def send[T](
+ request: RemoteMessageProtocol,
+ senderFuture: Option[CompletableFuture[T]]): Option[CompletableFuture[T]] = {
+ log.slf4j.debug("sending message: {} has future {}", request, senderFuture)
+ if (isRunning) {
+ if (request.getOneWay) {
+ currentChannel.write(request).addListener(new ChannelFutureListener {
+ def operationComplete(future: ChannelFuture) {
+ if (future.isCancelled) {
+ //We don't care about that right now
+ } else if (!future.isSuccess) {
+ notifyListeners(RemoteClientWriteFailed(request, future.getCause, module, remoteAddress))
+ }
+ }
+ })
+ None
+ } else {
+ val futureResult = if (senderFuture.isDefined) senderFuture.get
+ else new DefaultCompletableFuture[T](request.getActorInfo.getTimeout)
+
+ currentChannel.write(request).addListener(new ChannelFutureListener {
+ def operationComplete(future: ChannelFuture) {
+ if (future.isCancelled) {
+ //We don't care about that right now
+ } else if (!future.isSuccess) {
+ notifyListeners(RemoteClientWriteFailed(request, future.getCause, module, remoteAddress))
+ } else {
+ val futureUuid = uuidFrom(request.getUuid.getHigh, request.getUuid.getLow)
+ futures.put(futureUuid, futureResult)
+ }
+ }
+ })
+ Some(futureResult)
+ }
+ } else {
+ val exception = new RemoteClientException("Remote client is not running, make sure you have invoked 'RemoteClient.connect' before using it.", module, remoteAddress)
+ notifyListeners(RemoteClientError(exception, module, remoteAddress))
+ throw exception
+ }
+ }
+
+ private[akka] def registerSupervisorForActor(actorRef: ActorRef): ActorRef =
+ if (!actorRef.supervisor.isDefined) throw new IllegalActorStateException(
+ "Can't register supervisor for " + actorRef + " since it is not under supervision")
+ else supervisors.putIfAbsent(actorRef.supervisor.get.uuid, actorRef)
+
+ private[akka] def deregisterSupervisorForActor(actorRef: ActorRef): ActorRef =
+ if (!actorRef.supervisor.isDefined) throw new IllegalActorStateException(
+ "Can't unregister supervisor for " + actorRef + " since it is not under supervision")
+ else supervisors.remove(actorRef.supervisor.get.uuid)
+}
+
+/**
+ * RemoteClient represents a connection to an Akka node. Is used to send messages to remote actors on the node.
+ *
+ * @author Jonas Bonér
+ */
+class ActiveRemoteClient private[akka] (
+ module: NettyRemoteClientModule, remoteAddress: InetSocketAddress,
+ val loader: Option[ClassLoader] = None, notifyListenersFun: (=> Any) => Unit) extends RemoteClient(module, remoteAddress) {
+ import RemoteClientSettings._
+ //FIXME rewrite to a wrapper object (minimize volatile access and maximize encapsulation)
+ @volatile private var bootstrap: ClientBootstrap = _
+ @volatile private[remote] var connection: ChannelFuture = _
+ @volatile private[remote] var openChannels: DefaultChannelGroup = _
+ @volatile private var timer: HashedWheelTimer = _
+ @volatile private var reconnectionTimeWindowStart = 0L
+
+ def notifyListeners(msg: => Any): Unit = notifyListenersFun(msg)
+ def currentChannel = connection.getChannel
+
+ def connect(reconnectIfAlreadyConnected: Boolean = false): Boolean = {
+ runSwitch switchOn {
+ openChannels = new DefaultChannelGroup(classOf[RemoteClient].getName)
+ timer = new HashedWheelTimer
+
+ bootstrap = new ClientBootstrap(new NioClientSocketChannelFactory(Executors.newCachedThreadPool, Executors.newCachedThreadPool))
+ bootstrap.setPipelineFactory(new ActiveRemoteClientPipelineFactory(name, futures, supervisors, bootstrap, remoteAddress, timer, this))
+ bootstrap.setOption("tcpNoDelay", true)
+ bootstrap.setOption("keepAlive", true)
+
+ log.slf4j.info("Starting remote client connection to [{}]", remoteAddress)
+
+ // Wait until the connection attempt succeeds or fails.
+ connection = bootstrap.connect(remoteAddress)
+ openChannels.add(connection.awaitUninterruptibly.getChannel)
+
+ if (!connection.isSuccess) {
+ notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress))
+ log.slf4j.error("Remote client connection to [{}] has failed", remoteAddress)
+ log.slf4j.debug("Remote client connection failed", connection.getCause)
+ false
+ } else {
+ notifyListeners(RemoteClientStarted(module, remoteAddress))
+ true
+ }
+ } match {
+ case true => true
+ case false if reconnectIfAlreadyConnected =>
+ isAuthenticated.set(false)
+ log.slf4j.debug("Remote client reconnecting to [{}]", remoteAddress)
+ openChannels.remove(connection.getChannel)
+ connection.getChannel.close
+ connection = bootstrap.connect(remoteAddress)
+ openChannels.add(connection.awaitUninterruptibly.getChannel) // Wait until the connection attempt succeeds or fails.
+ if (!connection.isSuccess) {
+ notifyListeners(RemoteClientError(connection.getCause, module, remoteAddress))
+ log.slf4j.error("Reconnection to [{}] has failed", remoteAddress)
+ log.slf4j.debug("Reconnection failed", connection.getCause)
+ false
+ } else true
+ case false => false
+ }
+ }
+
+ def shutdown = runSwitch switchOff {
+ log.slf4j.info("Shutting down {}", name)
+ notifyListeners(RemoteClientShutdown(module, remoteAddress))
+ timer.stop
+ timer = null
+ openChannels.close.awaitUninterruptibly
+ openChannels = null
+ bootstrap.releaseExternalResources
+ bootstrap = null
+ connection = null
+ log.slf4j.info("{} has been shut down", name)
+ }
+
+ private[akka] def isWithinReconnectionTimeWindow: Boolean = {
+ if (reconnectionTimeWindowStart == 0L) {
+ reconnectionTimeWindowStart = System.currentTimeMillis
+ true
+ } else {
+ val timeLeft = RECONNECTION_TIME_WINDOW - (System.currentTimeMillis - reconnectionTimeWindowStart)
+ if (timeLeft > 0) {
+ log.slf4j.info("Will try to reconnect to remote server for another [{}] milliseconds", timeLeft)
+ true
+ } else false
+ }
+ }
+
+ private[akka] def resetReconnectionTimeWindow = reconnectionTimeWindowStart = 0L
+}
+
+/**
+ * @author Jonas Bonér
+ */
+class ActiveRemoteClientPipelineFactory(
+ name: String,
+ futures: ConcurrentMap[Uuid, CompletableFuture[_]],
+ supervisors: ConcurrentMap[Uuid, ActorRef],
+ bootstrap: ClientBootstrap,
+ remoteAddress: SocketAddress,
+ timer: HashedWheelTimer,
+ client: ActiveRemoteClient) extends ChannelPipelineFactory {
+
+ def getPipeline: ChannelPipeline = {
+ def join(ch: ChannelHandler*) = Array[ChannelHandler](ch: _*)
+
+ lazy val engine = {
+ val e = RemoteServerSslContext.client.createSSLEngine()
+ e.setEnabledCipherSuites(e.getSupportedCipherSuites) //TODO is this sensible?
+ e.setUseClientMode(true)
+ e
+ }
+
+ val ssl = if (RemoteServerSettings.SECURE) join(new SslHandler(engine)) else join()
+ val timeout = new ReadTimeoutHandler(timer, RemoteClientSettings.READ_TIMEOUT.toMillis.toInt)
+ val lenDec = new LengthFieldBasedFrameDecoder(RemoteClientSettings.MESSAGE_FRAME_SIZE, 0, 4, 0, 4)
+ val lenPrep = new LengthFieldPrepender(4)
+ val protobufDec = new ProtobufDecoder(RemoteMessageProtocol.getDefaultInstance)
+ val protobufEnc = new ProtobufEncoder
+ val (enc, dec) = RemoteServerSettings.COMPRESSION_SCHEME match {
+ case "zlib" => (join(new ZlibEncoder(RemoteServerSettings.ZLIB_COMPRESSION_LEVEL)), join(new ZlibDecoder))
+ case _ => (join(), join())
+ }
+
+ val remoteClient = new ActiveRemoteClientHandler(name, futures, supervisors, bootstrap, remoteAddress, timer, client)
+ val stages = ssl ++ join(timeout) ++ dec ++ join(lenDec, protobufDec) ++ enc ++ join(lenPrep, protobufEnc, remoteClient)
+ new StaticChannelPipeline(stages: _*)
+ }
+}
+
+/**
+ * @author Jonas Bonér
+ */
+@ChannelHandler.Sharable
+class ActiveRemoteClientHandler(
+ val name: String,
+ val futures: ConcurrentMap[Uuid, CompletableFuture[_]],
+ val supervisors: ConcurrentMap[Uuid, ActorRef],
+ val bootstrap: ClientBootstrap,
+ val remoteAddress: SocketAddress,
+ val timer: HashedWheelTimer,
+ val client: ActiveRemoteClient)
+ extends SimpleChannelUpstreamHandler with Logging {
+
+ override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
+ if (event.isInstanceOf[ChannelStateEvent] &&
+ event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) {
+ log.slf4j.debug(event.toString)
+ }
+ super.handleUpstream(ctx, event)
+ }
+
+ override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) {
+ try {
+ 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 {
+ val exception = parseException(reply, client.loader)
+
+ 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, exception)
+ }
+
+ future.completeWithException(exception)
+ }
+
+ case other =>
+ throw new RemoteClientException("Unknown message received in remote client handler: " + other, client.module, client.remoteAddress)
+ }
+ } catch {
+ case e: Exception =>
+ client.notifyListeners(RemoteClientError(e, client.module, client.remoteAddress))
+ log.slf4j.error("Unexpected exception in remote client handler", e)
+ throw e
+ }
+ }
+
+ override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = client.runSwitch ifOn {
+ if (client.isWithinReconnectionTimeWindow) {
+ timer.newTimeout(new TimerTask() {
+ def run(timeout: Timeout) = {
+ client.openChannels.remove(event.getChannel)
+ client.connect(reconnectIfAlreadyConnected = true)
+ }
+ }, RemoteClientSettings.RECONNECT_DELAY.toMillis, TimeUnit.MILLISECONDS)
+ } else spawn { client.shutdown }
+ }
+
+ override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
+ def connect = {
+ client.notifyListeners(RemoteClientConnected(client.module, client.remoteAddress))
+ log.slf4j.debug("Remote client connected to [{}]", ctx.getChannel.getRemoteAddress)
+ client.resetReconnectionTimeWindow
+ }
+
+ if (RemoteServerSettings.SECURE) {
+ val sslHandler: SslHandler = ctx.getPipeline.get(classOf[SslHandler])
+ sslHandler.handshake.addListener(new ChannelFutureListener {
+ def operationComplete(future: ChannelFuture): Unit = {
+ if (future.isSuccess) connect
+ else throw new RemoteClientException("Could not establish SSL handshake", client.module, client.remoteAddress)
+ }
+ })
+ } else connect
+ }
+
+ override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
+ client.notifyListeners(RemoteClientDisconnected(client.module, client.remoteAddress))
+ log.slf4j.debug("Remote client disconnected from [{}]", ctx.getChannel.getRemoteAddress)
+ }
+
+ override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
+ client.notifyListeners(RemoteClientError(event.getCause, client.module, client.remoteAddress))
+ if (event.getCause ne null)
+ log.slf4j.error("Unexpected exception from downstream in remote client", event.getCause)
+ else
+ log.slf4j.error("Unexpected exception from downstream in remote client: {}", event)
+
+ event.getChannel.close
+ }
+
+ private def parseException(reply: RemoteMessageProtocol, loader: Option[ClassLoader]): Throwable = {
+ val exception = reply.getException
+ val classname = exception.getClassname
+ try {
+ val exceptionClass = if (loader.isDefined) loader.get.loadClass(classname)
+ else Class.forName(classname)
+ exceptionClass
+ .getConstructor(Array[Class[_]](classOf[String]): _*)
+ .newInstance(exception.getMessage).asInstanceOf[Throwable]
+ } catch {
+ case problem =>
+ log.debug("Couldn't parse exception returned from RemoteServer",problem)
+ log.warn("Couldn't create instance of {} with message {}, returning UnparsableException",classname, exception.getMessage)
+ UnparsableException(classname, exception.getMessage)
+ }
+ }
+}
+
+/**
+ * Provides the implementation of the Netty remote support
+ */
+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, host: String, port: Int, loader: Option[ClassLoader]): ActorRef = {
+ if (optimizeLocalScoped_?) {
+ val home = this.address
+ if (host == home.getHostName && port == home.getPort) {//TODO: switch to InetSocketAddress.equals?
+ val localRef = findActorByIdOrUuid(serviceId,serviceId)
+ if (localRef ne null) return localRef //Code significantly simpler with the return statement
+ }
+ }
+
+ RemoteActorRef(serviceId, className, host, port, timeout, loader)
+ }
+
+ def clientManagedActorOf(factory: () => Actor, host: String, port: Int): ActorRef = {
+
+ if (optimizeLocalScoped_?) {
+ val home = this.address
+ if (host == home.getHostName && port == home.getPort)//TODO: switch to InetSocketAddress.equals?
+ return new LocalActorRef(factory, None) // Code is much simpler with return
+ }
+
+ val ref = new LocalActorRef(factory, Some(new InetSocketAddress(host, port)))
+ //ref.timeout = timeout //removed because setting default timeout should be done after construction
+ ref
+ }
+}
+
+class NettyRemoteServer(serverModule: NettyRemoteServerModule, val host: String, val port: Int, val loader: Option[ClassLoader]) {
+
+ val name = "NettyRemoteServer@" + host + ":" + port
+ val address = new InetSocketAddress(host,port)
+
+ private val factory = new NioServerSocketChannelFactory(Executors.newCachedThreadPool,Executors.newCachedThreadPool)
+
+ private val bootstrap = new ServerBootstrap(factory)
+
+ // group of open channels, used for clean-up
+ private val openChannels: ChannelGroup = new DefaultChannelGroup("akka-remote-server")
+
+ val pipelineFactory = new RemoteServerPipelineFactory(name, openChannels, loader, serverModule)
+ bootstrap.setPipelineFactory(pipelineFactory)
+ bootstrap.setOption("child.tcpNoDelay", true)
+ bootstrap.setOption("child.keepAlive", true)
+ bootstrap.setOption("child.reuseAddress", true)
+ bootstrap.setOption("child.connectTimeoutMillis", RemoteServerSettings.CONNECTION_TIMEOUT_MILLIS.toMillis)
+
+ openChannels.add(bootstrap.bind(address))
+ serverModule.notifyListeners(RemoteServerStarted(serverModule))
+
+ def shutdown {
+ try {
+ openChannels.disconnect
+ openChannels.close.awaitUninterruptibly
+ bootstrap.releaseExternalResources
+ serverModule.notifyListeners(RemoteServerShutdown(serverModule))
+ } catch {
+ case e: java.nio.channels.ClosedChannelException => {}
+ case e => serverModule.log.slf4j.warn("Could not close remote server channel in a graceful way")
+ }
+ }
+}
+
+trait NettyRemoteServerModule extends RemoteServerModule { self: RemoteModule =>
+ import RemoteServerSettings._
+
+ private[akka] val currentServer = new AtomicReference[Option[NettyRemoteServer]](None)
+ def address = currentServer.get match {
+ case Some(s) => s.address
+ case None => ReflectiveAccess.Remote.configDefaultAddress
+ }
+
+ def name = currentServer.get match {
+ case Some(s) => s.name
+ case None =>
+ val a = ReflectiveAccess.Remote.configDefaultAddress
+ "NettyRemoteServer@" + a.getHostName + ":" + a.getPort
+ }
+
+ private val _isRunning = new Switch(false)
+
+ def isRunning = _isRunning.isOn
+
+ def start(_hostname: String, _port: Int, loader: Option[ClassLoader] = None): RemoteServerModule = guard withGuard {
+ try {
+ _isRunning switchOn {
+ log.slf4j.debug("Starting up remote server on {}:{}",_hostname, _port)
+ currentServer.set(Some(new NettyRemoteServer(this, _hostname, _port, loader)))
+ }
+ } catch {
+ case e =>
+ log.slf4j.error("Could not start up remote server", e)
+ notifyListeners(RemoteServerError(e, this))
+ }
+ this
+ }
+
+ def shutdownServerModule = guard withGuard {
+ _isRunning switchOff {
+ currentServer.getAndSet(None) foreach {
+ instance =>
+ log.slf4j.debug("Shutting down remote server on {}:{}",instance.host, instance.port)
+ instance.shutdown
+ }
+ }
+ }
+
+ /**
+ * Register remote typed actor by a specific id.
+ * @param id custom actor id
+ * @param typedActor typed actor to register
+ */
+ def registerTypedActor(id: String, typedActor: AnyRef): Unit = guard withGuard {
+ log.slf4j.debug("Registering server side remote typed actor [{}] with id [{}]", typedActor.getClass.getName, id)
+ if (id.startsWith(UUID_PREFIX)) registerTypedActor(id.substring(UUID_PREFIX.length), typedActor, typedActorsByUuid)
+ else registerTypedActor(id, typedActor, typedActors)
+ }
+
+ /**
+ * Register remote typed actor by a specific id.
+ * @param id custom actor id
+ * @param typedActor typed actor to register
+ */
+ def registerTypedPerSessionActor(id: String, factory: => AnyRef): Unit = guard withGuard {
+ log.slf4j.debug("Registering server side typed remote session actor with id [{}]", id)
+ registerTypedPerSessionActor(id, () => factory, typedActorsFactories)
+ }
+
+ /**
+ * Register Remote Actor by a specific 'id' passed as argument.
+ *
+ * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
+ */
+ def register(id: String, actorRef: ActorRef): Unit = guard withGuard {
+ log.slf4j.debug("Registering server side remote actor [{}] with id [{}]", actorRef.actorClass.getName, id)
+ if (id.startsWith(UUID_PREFIX)) register(id.substring(UUID_PREFIX.length), actorRef, actorsByUuid)
+ else register(id, actorRef, actors)
+ }
+
+ 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)
+ }
+
+ private def register[Key](id: Key, actorRef: ActorRef, registry: ConcurrentHashMap[Key, ActorRef]) {
+ if (_isRunning.isOn) {
+ registry.put(id, actorRef) //TODO change to putIfAbsent
+ if (!actorRef.isRunning) actorRef.start
+ }
+ }
+
+ /**
+ * Register Remote Session Actor by a specific 'id' passed as argument.
+ *
+ * NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
+ */
+ def registerPerSession(id: String, factory: => ActorRef): Unit = synchronized {
+ log.slf4j.debug("Registering server side remote session actor with id [{}]", id)
+ registerPerSession(id, () => factory, actorsFactories)
+ }
+
+ private def registerPerSession[Key](id: Key, factory: () => ActorRef, registry: ConcurrentHashMap[Key,() => ActorRef]) {
+ if (_isRunning.isOn)
+ registry.put(id, factory) //TODO change to putIfAbsent
+ }
+
+ private def registerTypedActor[Key](id: Key, typedActor: AnyRef, registry: ConcurrentHashMap[Key, AnyRef]) {
+ if (_isRunning.isOn)
+ registry.put(id, typedActor) //TODO change to putIfAbsent
+ }
+
+ private def registerTypedPerSessionActor[Key](id: Key, factory: () => AnyRef, registry: ConcurrentHashMap[Key,() => AnyRef]) {
+ if (_isRunning.isOn)
+ registry.put(id, factory) //TODO change to putIfAbsent
+ }
+
+ /**
+ * Unregister Remote Actor that is registered using its 'id' field (not custom ID).
+ */
+ def unregister(actorRef: ActorRef): Unit = guard withGuard {
+ if (_isRunning.isOn) {
+ log.slf4j.debug("Unregistering server side remote actor [{}] with id [{}:{}]", Array[AnyRef](actorRef.actorClass.getName, actorRef.id, actorRef.uuid))
+ actors.remove(actorRef.id, actorRef)
+ actorsByUuid.remove(actorRef.uuid, actorRef)
+ }
+ }
+
+ /**
+ * Unregister Remote Actor by specific 'id'.
+ *
+ * NOTE: You need to call this method if you have registered an actor by a custom ID.
+ */
+ def unregister(id: String): Unit = guard withGuard {
+ if (_isRunning.isOn) {
+ log.slf4j.info("Unregistering server side remote actor with id [{}]", id)
+ if (id.startsWith(UUID_PREFIX)) actorsByUuid.remove(id.substring(UUID_PREFIX.length))
+ else {
+ val actorRef = actors get id
+ actorsByUuid.remove(actorRef.uuid, actorRef)
+ actors.remove(id,actorRef)
+ }
+ }
+ }
+
+ /**
+ * Unregister Remote Actor by specific 'id'.
+ *
+ * NOTE: You need to call this method if you have registered an actor by a custom ID.
+ */
+ def unregisterPerSession(id: String): Unit = {
+ if (_isRunning.isOn) {
+ log.slf4j.info("Unregistering server side remote session actor with id [{}]", id)
+ actorsFactories.remove(id)
+ }
+ }
+
+ /**
+ * Unregister Remote Typed Actor by specific 'id'.
+ *
+ * NOTE: You need to call this method if you have registered an actor by a custom ID.
+ */
+ def unregisterTypedActor(id: String):Unit = guard withGuard {
+ if (_isRunning.isOn) {
+ log.slf4j.info("Unregistering server side remote typed actor with id [{}]", id)
+ if (id.startsWith(UUID_PREFIX)) typedActorsByUuid.remove(id.substring(UUID_PREFIX.length))
+ else typedActors.remove(id)
+ }
+ }
+
+ /**
+ * Unregister Remote Typed Actor by specific 'id'.
+ *
+ * NOTE: You need to call this method if you have registered an actor by a custom ID.
+ */
+ def unregisterTypedPerSessionActor(id: String): Unit =
+ if (_isRunning.isOn) typedActorsFactories.remove(id)
+}
+
+object RemoteServerSslContext {
+ import javax.net.ssl.SSLContext
+
+ val (client, server) = {
+ val protocol = "TLS"
+ //val algorithm = Option(Security.getProperty("ssl.KeyManagerFactory.algorithm")).getOrElse("SunX509")
+ //val store = KeyStore.getInstance("JKS")
+ val s = SSLContext.getInstance(protocol)
+ s.init(null, null, null)
+ val c = SSLContext.getInstance(protocol)
+ c.init(null, null, null)
+ (c, s)
+ }
+}
+
+/**
+ * @author Jonas Bonér
+ */
+class RemoteServerPipelineFactory(
+ val name: String,
+ val openChannels: ChannelGroup,
+ val loader: Option[ClassLoader],
+ val server: NettyRemoteServerModule) extends ChannelPipelineFactory {
+ import RemoteServerSettings._
+
+ def getPipeline: ChannelPipeline = {
+ def join(ch: ChannelHandler*) = Array[ChannelHandler](ch:_*)
+
+ lazy val engine = {
+ val e = RemoteServerSslContext.server.createSSLEngine()
+ e.setEnabledCipherSuites(e.getSupportedCipherSuites) //TODO is this sensible?
+ e.setUseClientMode(false)
+ e
+ }
+
+ val ssl = if(SECURE) join(new SslHandler(engine)) else join()
+ val lenDec = new LengthFieldBasedFrameDecoder(MESSAGE_FRAME_SIZE, 0, 4, 0, 4)
+ val lenPrep = new LengthFieldPrepender(4)
+ val protobufDec = new ProtobufDecoder(RemoteMessageProtocol.getDefaultInstance)
+ val protobufEnc = new ProtobufEncoder
+ val (enc, dec) = COMPRESSION_SCHEME match {
+ case "zlib" => (join(new ZlibEncoder(ZLIB_COMPRESSION_LEVEL)), join(new ZlibDecoder))
+ case _ => (join(), join())
+ }
+
+ val remoteServer = new RemoteServerHandler(name, openChannels, loader, server)
+ val stages = ssl ++ dec ++ join(lenDec, protobufDec) ++ enc ++ join(lenPrep, protobufEnc, remoteServer)
+ new StaticChannelPipeline(stages: _*)
+ }
+}
+
+/**
+ * @author Jonas Bonér
+ */
+@ChannelHandler.Sharable
+class RemoteServerHandler(
+ val name: String,
+ val openChannels: ChannelGroup,
+ val applicationLoader: Option[ClassLoader],
+ val server: NettyRemoteServerModule) extends SimpleChannelUpstreamHandler with Logging {
+ import RemoteServerSettings._
+ val CHANNEL_INIT = "channel-init".intern
+
+ applicationLoader.foreach(MessageSerializer.setClassLoader(_)) //TODO: REVISIT: THIS FEELS A BIT DODGY
+
+ val sessionActors = new ChannelLocal[ConcurrentHashMap[String, ActorRef]]()
+ val typedSessionActors = new ChannelLocal[ConcurrentHashMap[String, AnyRef]]()
+
+ //Writes the specified message to the specified channel and propagates write errors to listeners
+ private def write(channel: Channel, message: AnyRef): Unit =
+ channel.write(message).addListener(
+ new ChannelFutureListener {
+ def operationComplete(future: ChannelFuture): Unit = {
+ if (future.isCancelled) {
+ //Not interesting at the moment
+ } else if (!future.isSuccess) {
+ val socketAddress = future.getChannel.getRemoteAddress match {
+ case i: InetSocketAddress => i
+ case _ => null
+ }
+ server.notifyListeners(RemoteServerWriteFailed(message, future.getCause, server, socketAddress))
+ }
+ }
+ })
+
+ /**
+ * ChannelOpen overridden to store open channels for a clean postStop of a node.
+ * If a channel is closed before, it is automatically removed from the open channels group.
+ */
+ override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel)
+
+ override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
+ val clientAddress = getClientAddress(ctx)
+ sessionActors.set(event.getChannel(), new ConcurrentHashMap[String, ActorRef]())
+ typedSessionActors.set(event.getChannel(), new ConcurrentHashMap[String, AnyRef]())
+ log.slf4j.debug("Remote client [{}] connected to [{}]", clientAddress, server.name)
+ if (SECURE) {
+ val sslHandler: SslHandler = ctx.getPipeline.get(classOf[SslHandler])
+ // Begin handshake.
+ sslHandler.handshake().addListener(new ChannelFutureListener {
+ def operationComplete(future: ChannelFuture): Unit = {
+ if (future.isSuccess) {
+ openChannels.add(future.getChannel)
+ server.notifyListeners(RemoteServerClientConnected(server, clientAddress))
+ } else future.getChannel.close
+ }
+ })
+ } else {
+ server.notifyListeners(RemoteServerClientConnected(server, clientAddress))
+ }
+ if (REQUIRE_COOKIE) ctx.setAttachment(CHANNEL_INIT) // signal that this is channel initialization, which will need authentication
+ }
+
+ override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
+ import scala.collection.JavaConversions.asScalaIterable
+ val clientAddress = getClientAddress(ctx)
+ log.slf4j.debug("Remote client [{}] disconnected from [{}]", clientAddress, server.name)
+
+ // stop all session actors
+ for (map <- Option(sessionActors.remove(event.getChannel));
+ actor <- asScalaIterable(map.values)) {
+ try { actor.stop } catch { case e: Exception => log.slf4j.warn("Couldn't stop {}",actor,e) }
+ }
+ // stop all typed session actors
+ for (map <- Option(typedSessionActors.remove(event.getChannel));
+ actor <- asScalaIterable(map.values)) {
+ try { TypedActor.stop(actor) } catch { case e: Exception => log.slf4j.warn("Couldn't stop {}",actor,e) }
+ }
+
+ server.notifyListeners(RemoteServerClientDisconnected(server, clientAddress))
+ }
+
+ override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
+ val clientAddress = getClientAddress(ctx)
+ log.slf4j.debug("Remote client [{}] channel closed from [{}]", clientAddress, server.name)
+ server.notifyListeners(RemoteServerClientClosed(server, clientAddress))
+ }
+
+ override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
+ if (event.isInstanceOf[ChannelStateEvent] && event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) {
+ log.slf4j.debug(event.toString)
+ }
+ super.handleUpstream(ctx, event)
+ }
+
+ override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = event.getMessage match {
+ case null => throw new IllegalActorStateException("Message in remote MessageEvent is null: " + event)
+ case requestProtocol: RemoteMessageProtocol =>
+ if (REQUIRE_COOKIE) authenticateRemoteClient(requestProtocol, ctx)
+ handleRemoteMessageProtocol(requestProtocol, event.getChannel)
+ case _ => //ignore
+ }
+
+ override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
+ log.slf4j.error("Unexpected exception from remote downstream", event.getCause)
+ event.getChannel.close
+ server.notifyListeners(RemoteServerError(event.getCause, server))
+ }
+
+ private def getClientAddress(ctx: ChannelHandlerContext): Option[InetSocketAddress] =
+ ctx.getChannel.getRemoteAddress match {
+ case inet: InetSocketAddress => Some(inet)
+ case _ => None
+ }
+
+ private def handleRemoteMessageProtocol(request: RemoteMessageProtocol, channel: Channel) = {
+ log.slf4j.debug("Received RemoteMessageProtocol[\n{}]",request)
+ request.getActorInfo.getActorType match {
+ case SCALA_ACTOR => dispatchToActor(request, channel)
+ case TYPED_ACTOR => dispatchToTypedActor(request, channel)
+ case JAVA_ACTOR => throw new IllegalActorStateException("ActorType JAVA_ACTOR is currently not supported")
+ case other => throw new IllegalActorStateException("Unknown ActorType [" + other + "]")
+ }
+ }
+
+ private def dispatchToActor(request: RemoteMessageProtocol, channel: Channel) {
+ val actorInfo = request.getActorInfo
+ log.slf4j.debug("Dispatching to remote actor [{}:{}]", actorInfo.getTarget, actorInfo.getUuid)
+
+ val actorRef =
+ try { createActor(actorInfo, channel).start } catch {
+ case e: SecurityException =>
+ write(channel, createErrorReplyMessage(e, request, AkkaActorType.ScalaActor))
+ server.notifyListeners(RemoteServerError(e, server))
+ return
+ }
+
+ val message = MessageSerializer.deserialize(request.getMessage)
+ val sender =
+ if (request.hasSender) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(request.getSender, applicationLoader))
+ else None
+
+ message match { // first match on system messages
+ case RemoteActorSystemMessage.Stop =>
+ if (UNTRUSTED_MODE) throw new SecurityException("Remote server is operating is untrusted mode, can not stop the actor")
+ else actorRef.stop
+ case _: LifeCycleMessage if (UNTRUSTED_MODE) =>
+ throw new SecurityException("Remote server is operating is untrusted mode, can not pass on a LifeCycleMessage to the remote actor")
+
+ case _ => // then match on user defined messages
+ if (request.getOneWay) actorRef.!(message)(sender)
+ else actorRef.postMessageToMailboxAndCreateFutureResultWithTimeout(
+ message,
+ request.getActorInfo.getTimeout,
+ 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.getClass)
+ write(channel, createErrorReplyMessage(exception.get, request, AkkaActorType.ScalaActor))
+ }
+ else if (result.isDefined) {
+ log.slf4j.debug("Returning result from actor invocation [{}]",result.get)
+ val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder(
+ Some(actorRef),
+ Right(request.getUuid),
+ actorInfo.getId,
+ actorInfo.getTarget,
+ actorInfo.getTimeout,
+ Left(result.get),
+ true,
+ Some(actorRef),
+ None,
+ AkkaActorType.ScalaActor,
+ None)
+
+ // FIXME lift in the supervisor uuid management into toh createRemoteMessageProtocolBuilder method
+ if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid)
+
+ write(channel, messageBuilder.build)
+ }
+ }
+ )
+ ))
+ }
+ }
+
+ private def dispatchToTypedActor(request: RemoteMessageProtocol, channel: Channel) = {
+ val actorInfo = request.getActorInfo
+ val typedActorInfo = actorInfo.getTypedActorInfo
+ log.slf4j.debug("Dispatching to remote typed actor [{} :: {}]", typedActorInfo.getMethod, typedActorInfo.getInterface)
+
+ val typedActor = createTypedActor(actorInfo, channel)
+ val args = MessageSerializer.deserialize(request.getMessage).asInstanceOf[Array[AnyRef]].toList
+ val argClasses = args.map(_.getClass)
+
+ try {
+ val messageReceiver = typedActor.getClass.getDeclaredMethod(typedActorInfo.getMethod, argClasses: _*)
+ if (request.getOneWay) messageReceiver.invoke(typedActor, args: _*)
+ else {
+ //Sends the response
+ def sendResponse(result: Either[Any,Throwable]): Unit = try {
+ val messageBuilder = RemoteActorSerialization.createRemoteMessageProtocolBuilder(
+ None,
+ Right(request.getUuid),
+ actorInfo.getId,
+ actorInfo.getTarget,
+ actorInfo.getTimeout,
+ result,
+ true,
+ None,
+ None,
+ AkkaActorType.TypedActor,
+ None)
+ if (request.hasSupervisorUuid) messageBuilder.setSupervisorUuid(request.getSupervisorUuid)
+
+ write(channel, messageBuilder.build)
+ log.slf4j.debug("Returning result from remote typed actor invocation [{}]", result)
+ } catch {
+ case e: Throwable => server.notifyListeners(RemoteServerError(e, server))
+ }
+
+ messageReceiver.invoke(typedActor, args: _*) match {
+ case f: Future[_] => //If it's a future, we can lift on that to defer the send to when the future is completed
+ f.onComplete( future => {
+ val result: Either[Any,Throwable] =
+ if (future.exception.isDefined) Right(future.exception.get) else Left(future.result.get)
+ sendResponse(result)
+ })
+ case other => sendResponse(Left(other))
+ }
+ }
+ } catch {
+ case e: InvocationTargetException =>
+ write(channel, createErrorReplyMessage(e.getCause, request, AkkaActorType.TypedActor))
+ server.notifyListeners(RemoteServerError(e, server))
+ case e: Throwable =>
+ write(channel, createErrorReplyMessage(e, request, AkkaActorType.TypedActor))
+ server.notifyListeners(RemoteServerError(e, server))
+ }
+ }
+
+ private def findSessionActor(id: String, channel: Channel) : ActorRef =
+ sessionActors.get(channel) match {
+ case null => null
+ case map => map get id
+ }
+
+ private def findTypedSessionActor(id: String, channel: Channel) : AnyRef =
+ typedSessionActors.get(channel) match {
+ case null => null
+ case map => map get id
+ }
+
+ /**
+ * gets the actor from the session, or creates one if there is a factory for it
+ */
+ private def createSessionActor(actorInfo: ActorInfoProtocol, channel: Channel): ActorRef = {
+ val uuid = actorInfo.getUuid
+ val id = actorInfo.getId
+
+ findSessionActor(id, channel) match {
+ case null => // we dont have it in the session either, see if we have a factory for it
+ server.findActorFactory(id) match {
+ case null => null
+ case factory =>
+ val actorRef = factory()
+ actorRef.uuid = parseUuid(uuid) //FIXME is this sensible?
+ sessionActors.get(channel).put(id, actorRef)
+ actorRef
+ }
+ case sessionActor => sessionActor
+ }
+ }
+
+
+ private def createClientManagedActor(actorInfo: ActorInfoProtocol): ActorRef = {
+ val uuid = actorInfo.getUuid
+ val id = actorInfo.getId
+ val timeout = actorInfo.getTimeout
+ val name = actorInfo.getTarget
+
+ try {
+ if (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 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]])
+ actorRef.uuid = parseUuid(uuid)
+ actorRef.id = id
+ actorRef.timeout = timeout
+ server.actorsByUuid.put(actorRef.uuid.toString, actorRef) // register by uuid
+ actorRef
+ } catch {
+ case e =>
+ log.slf4j.error("Could not create remote actor instance", e)
+ server.notifyListeners(RemoteServerError(e, server))
+ throw e
+ }
+
+ }
+
+ /**
+ * Creates a new instance of the actor with name, uuid and timeout specified as arguments.
+ *
+ * If actor already created then just return it from the registry.
+ *
+ * Does not start the actor.
+ */
+ private def createActor(actorInfo: ActorInfoProtocol, channel: Channel): ActorRef = {
+ val uuid = actorInfo.getUuid
+ val id = actorInfo.getId
+
+ server.findActorByIdOrUuid(id, parseUuid(uuid).toString) match {
+ case null => // the actor has not been registered globally. See if we have it in the session
+ createSessionActor(actorInfo, channel) match {
+ case null => createClientManagedActor(actorInfo) // maybe it is a client managed actor
+ case sessionActor => sessionActor
+ }
+ case actorRef => actorRef
+ }
+ }
+
+ /**
+ * gets the actor from the session, or creates one if there is a factory for it
+ */
+ private def createTypedSessionActor(actorInfo: ActorInfoProtocol, channel: Channel):AnyRef ={
+ val id = actorInfo.getId
+ findTypedSessionActor(id, channel) match {
+ case null =>
+ server.findTypedActorFactory(id) match {
+ case null => null
+ case factory =>
+ val newInstance = factory()
+ typedSessionActors.get(channel).put(id, newInstance)
+ newInstance
+ }
+ case sessionActor => sessionActor
+ }
+ }
+
+ private def createClientManagedTypedActor(actorInfo: ActorInfoProtocol) = {
+ val typedActorInfo = actorInfo.getTypedActorInfo
+ val interfaceClassname = typedActorInfo.getInterface
+ val targetClassname = actorInfo.getTarget
+ val uuid = actorInfo.getUuid
+
+ try {
+ if (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 typed actor:\n\t[{} :: {}]", interfaceClassname, targetClassname)
+
+ val (interfaceClass, targetClass) =
+ if (applicationLoader.isDefined) (applicationLoader.get.loadClass(interfaceClassname),
+ applicationLoader.get.loadClass(targetClassname))
+ else (Class.forName(interfaceClassname), Class.forName(targetClassname))
+
+ val newInstance = TypedActor.newInstance(
+ interfaceClass, targetClass.asInstanceOf[Class[_ <: TypedActor]], actorInfo.getTimeout).asInstanceOf[AnyRef]
+ server.typedActors.put(parseUuid(uuid).toString, newInstance) // register by uuid
+ newInstance
+ } catch {
+ case e =>
+ log.slf4j.error("Could not create remote typed actor instance", e)
+ server.notifyListeners(RemoteServerError(e, server))
+ throw e
+ }
+ }
+
+ private def createTypedActor(actorInfo: ActorInfoProtocol, channel: Channel): AnyRef = {
+ val uuid = actorInfo.getUuid
+
+ server.findTypedActorByIdOrUuid(actorInfo.getId, parseUuid(uuid).toString) match {
+ case null => // the actor has not been registered globally. See if we have it in the session
+ createTypedSessionActor(actorInfo, channel) match {
+ case null => createClientManagedTypedActor(actorInfo) //Maybe client managed actor?
+ case sessionActor => sessionActor
+ }
+ case typedActor => typedActor
+ }
+ }
+
+ private def createErrorReplyMessage(exception: Throwable, request: RemoteMessageProtocol, actorType: AkkaActorType): RemoteMessageProtocol = {
+ val actorInfo = request.getActorInfo
+ log.slf4j.error("Could not invoke remote actor [{}]", actorInfo.getTarget)
+ log.slf4j.debug("Could not invoke remote actor", exception)
+ 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) = {
+ val attachment = ctx.getAttachment
+ if ((attachment ne null) &&
+ attachment.isInstanceOf[String] &&
+ attachment.asInstanceOf[String] == CHANNEL_INIT) { // is first time around, channel initialization
+ ctx.setAttachment(null)
+ val clientAddress = ctx.getChannel.getRemoteAddress.toString
+ if (!request.hasCookie) throw new SecurityException(
+ "The remote client [" + clientAddress + "] does not have a secure cookie.")
+ if (!(request.getCookie == SECURE_COOKIE.get)) throw new SecurityException(
+ "The remote client [" + clientAddress + "] secure cookie is not the same as remote server secure cookie")
+ log.slf4j.info("Remote client [{}] successfully authenticated using secure cookie", clientAddress)
+ }
+ }
+
+ protected def parseUuid(protocol: UuidProtocol): Uuid = uuidFrom(protocol.getHigh,protocol.getLow)
+}
diff --git a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala
index 269c09a462..738e351840 100644
--- a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala
+++ b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala
@@ -5,7 +5,6 @@
package akka.serialization
import akka.dispatch.MessageInvocation
-import akka.remote.{RemoteServer, RemoteClient, MessageSerializer}
import akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _}
import ActorTypeProtocol._
@@ -16,6 +15,9 @@ import akka.actor._
import scala.collection.immutable.Stack
import com.google.protobuf.ByteString
+import akka.util.ReflectiveAccess
+import java.net.InetSocketAddress
+import akka.remote. {RemoteClientSettings, MessageSerializer}
/**
* Type class definition for Actor Serialization
@@ -88,6 +90,15 @@ object ActorSerialization {
def toBinaryJ[T <: Actor](a: ActorRef, format: Format[T], srlMailBox: Boolean = true): Array[Byte] =
toBinary(a, srlMailBox)(format)
+ private[akka] def toAddressProtocol(actorRef: ActorRef) = {
+ val address = actorRef.homeAddress.getOrElse(Actor.remote.address)
+ AddressProtocol.newBuilder
+ .setHostname(address.getHostName)
+ .setPort(address.getPort)
+ .build
+ }
+
+
private[akka] def toSerializedActorRefProtocol[T <: Actor](
actorRef: ActorRef, format: Format[T], serializeMailBox: Boolean = true): SerializedActorRefProtocol = {
val lifeCycleProtocol: Option[LifeCycleProtocol] = {
@@ -98,16 +109,11 @@ object ActorSerialization {
}
}
- val originalAddress = AddressProtocol.newBuilder
- .setHostname(actorRef.homeAddress.getHostName)
- .setPort(actorRef.homeAddress.getPort)
- .build
-
val builder = SerializedActorRefProtocol.newBuilder
.setUuid(UuidProtocol.newBuilder.setHigh(actorRef.uuid.getTime).setLow(actorRef.uuid.getClockSeqAndNode).build)
.setId(actorRef.id)
.setActorClassname(actorRef.actorClass.getName)
- .setOriginalAddress(originalAddress)
+ .setOriginalAddress(toAddressProtocol(actorRef))
.setTimeout(actorRef.timeout)
@@ -134,7 +140,7 @@ object ActorSerialization {
actorRef.getSender,
None,
ActorType.ScalaActor,
- RemoteClient.SECURE_COOKIE).build)
+ RemoteClientSettings.SECURE_COOKIE).build)
requestProtocols.foreach(rp => builder.addMessages(rp))
}
@@ -191,14 +197,13 @@ object ActorSerialization {
val ar = new LocalActorRef(
uuidFrom(protocol.getUuid.getHigh, protocol.getUuid.getLow),
protocol.getId,
- protocol.getOriginalAddress.getHostname,
- protocol.getOriginalAddress.getPort,
if (protocol.hasTimeout) protocol.getTimeout else Actor.TIMEOUT,
if (protocol.hasReceiveTimeout) Some(protocol.getReceiveTimeout) else None,
lifeCycle,
supervisor,
hotswap,
- factory)
+ factory,
+ None) //TODO: shouldn't originalAddress be optional?
val messages = protocol.getMessagesList.toArray.toList.asInstanceOf[List[RemoteMessageProtocol]]
messages.foreach(message => ar ! MessageSerializer.deserialize(message.getMessage))
@@ -227,13 +232,16 @@ object RemoteActorSerialization {
*/
private[akka] def fromProtobufToRemoteActorRef(protocol: RemoteActorRefProtocol, loader: Option[ClassLoader]): ActorRef = {
Actor.log.slf4j.debug("Deserializing RemoteActorRefProtocol to RemoteActorRef:\n {}", protocol)
- RemoteActorRef(
+ val ref = RemoteActorRef(
protocol.getClassOrServiceName,
protocol.getActorClassname,
protocol.getHomeAddress.getHostname,
protocol.getHomeAddress.getPort,
protocol.getTimeout,
loader)
+
+ Actor.log.slf4j.debug("Newly deserialized RemoteActorRef has uuid: {}", ref.uuid)
+ ref
}
/**
@@ -241,25 +249,22 @@ object RemoteActorSerialization {
*/
def toRemoteActorRefProtocol(ar: ActorRef): RemoteActorRefProtocol = {
import ar._
- val home = homeAddress
- val host = home.getHostName
- val port = home.getPort
- Actor.log.slf4j.debug("Register serialized Actor [{}] as remote @ [{}]",actorClassName, home)
- RemoteServer.getOrCreateServer(homeAddress)
- ActorRegistry.registerActorByUuid(homeAddress, uuid.toString, ar)
+ Actor.log.slf4j.debug("Register serialized Actor [{}] as remote @ [{}:{}]",actorClassName, ar.homeAddress)
+
+ Actor.remote.registerByUuid(ar)
RemoteActorRefProtocol.newBuilder
- .setClassOrServiceName(uuid.toString)
+ .setClassOrServiceName("uuid:"+uuid.toString)
.setActorClassname(actorClassName)
- .setHomeAddress(AddressProtocol.newBuilder.setHostname(host).setPort(port).build)
+ .setHomeAddress(ActorSerialization.toAddressProtocol(ar))
.setTimeout(timeout)
.build
}
def createRemoteMessageProtocolBuilder(
actorRef: Option[ActorRef],
- uuid: Either[Uuid, UuidProtocol],
+ replyUuid: Either[Uuid, UuidProtocol],
actorId: String,
actorClassName: String,
timeout: Long,
@@ -270,7 +275,7 @@ object RemoteActorSerialization {
actorType: ActorType,
secureCookie: Option[String]): RemoteMessageProtocol.Builder = {
- val uuidProtocol = uuid match {
+ val uuidProtocol = replyUuid match {
case Left(uid) => UuidProtocol.newBuilder.setHigh(uid.getTime).setLow(uid.getClockSeqAndNode).build
case Right(protocol) => protocol
}
@@ -308,10 +313,15 @@ object RemoteActorSerialization {
case Right(exception) =>
messageBuilder.setException(ExceptionProtocol.newBuilder
.setClassname(exception.getClass.getName)
- .setMessage(exception.getMessage)
+ .setMessage(empty(exception.getMessage))
.build)
}
+ def empty(s: String): String = s match {
+ case null => ""
+ case s => s
+ }
+
secureCookie.foreach(messageBuilder.setCookie(_))
actorRef.foreach { ref =>
@@ -324,11 +334,9 @@ object RemoteActorSerialization {
}
}
- senderOption.foreach { sender =>
- RemoteServer.getOrCreateServer(sender.homeAddress).register(sender.uuid.toString, sender)
- messageBuilder.setSender(toRemoteActorRefProtocol(sender))
+ if( senderOption.isDefined)
+ messageBuilder.setSender(toRemoteActorRefProtocol(senderOption.get))
- }
messageBuilder
}
}
diff --git a/akka-remote/src/test/scala/remote/AkkaRemoteTest.scala b/akka-remote/src/test/scala/remote/AkkaRemoteTest.scala
new file mode 100644
index 0000000000..be6234bf00
--- /dev/null
+++ b/akka-remote/src/test/scala/remote/AkkaRemoteTest.scala
@@ -0,0 +1,63 @@
+package akka.actor.remote
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
+import akka.remote.netty.NettyRemoteSupport
+import akka.actor. {Actor, ActorRegistry}
+import java.util.concurrent. {TimeUnit, CountDownLatch}
+
+object AkkaRemoteTest {
+ class ReplyHandlerActor(latch: CountDownLatch, expect: String) extends Actor {
+ def receive = {
+ case x: String if x == expect => latch.countDown
+ }
+ }
+}
+
+@RunWith(classOf[JUnitRunner])
+class AkkaRemoteTest extends
+ WordSpec with
+ MustMatchers with
+ BeforeAndAfterAll with
+ BeforeAndAfterEach {
+ import AkkaRemoteTest._
+
+ val remote = Actor.remote
+ val unit = TimeUnit.SECONDS
+
+ val host = "localhost"
+ val port = 25520
+
+ def OptimizeLocal = false
+
+ var optimizeLocal_? = remote.asInstanceOf[NettyRemoteSupport].optimizeLocalScoped_?
+
+ override def beforeAll {
+ if (!OptimizeLocal)
+ remote.asInstanceOf[NettyRemoteSupport].optimizeLocal.set(false) //Can't run the test if we're eliminating all remote calls
+ }
+
+ override def afterAll {
+ if (!OptimizeLocal)
+ remote.asInstanceOf[NettyRemoteSupport].optimizeLocal.set(optimizeLocal_?) //Reset optimizelocal after all tests
+ }
+
+ override def beforeEach {
+ remote.start(host,port)
+ Thread.sleep(2000)
+ super.beforeEach
+ }
+
+ override def afterEach() {
+ remote.shutdown
+ Actor.registry.shutdownAll
+ super.afterEach
+ }
+
+ /* Utilities */
+
+ def replyHandler(latch: CountDownLatch, expect: String) = Some(Actor.actorOf(new ReplyHandlerActor(latch, expect)).start)
+}
\ No newline at end of file
diff --git a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
index a5e4159366..36fd4ae586 100644
--- a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
@@ -1,183 +1,142 @@
package akka.actor.remote
import java.util.concurrent.{CountDownLatch, TimeUnit}
-import org.scalatest.junit.JUnitSuite
-import org.junit.{Test, Before, After}
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
-import akka.remote.{RemoteServer, RemoteClient}
import akka.dispatch.Dispatchers
-import akka.actor.{ActorRef, Actor}
-import Actor._
+import akka.actor.Actor._
+import akka.actor._
-object ClientInitiatedRemoteActorSpec {
- case class Send(actor: Actor)
+class ExpectedRemoteProblem(msg: String) extends RuntimeException(msg)
- object RemoteActorSpecActorUnidirectional {
- val latch = new CountDownLatch(1)
- }
- class RemoteActorSpecActorUnidirectional extends Actor {
- self.dispatcher = Dispatchers.newThreadBasedDispatcher(self)
+object RemoteActorSpecActorUnidirectional {
+ val latch = new CountDownLatch(1)
+}
+class RemoteActorSpecActorUnidirectional extends Actor {
+ self.dispatcher = Dispatchers.newThreadBasedDispatcher(self)
- def receive = {
- case "OneWay" =>
- RemoteActorSpecActorUnidirectional.latch.countDown
- }
- }
-
- class RemoteActorSpecActorBidirectional extends Actor {
- def receive = {
- case "Hello" =>
- self.reply("World")
- case "Failure" =>
- throw new RuntimeException("Expected exception; to test fault-tolerance")
- }
- }
-
- class SendOneWayAndReplyReceiverActor extends Actor {
- def receive = {
- case "Hello" =>
- self.reply("World")
- }
- }
-
- object SendOneWayAndReplySenderActor {
- val latch = new CountDownLatch(1)
- }
- class SendOneWayAndReplySenderActor extends Actor {
- var state: Option[AnyRef] = None
- var sendTo: ActorRef = _
- var latch: CountDownLatch = _
-
- def sendOff = sendTo ! "Hello"
-
- def receive = {
- case msg: AnyRef =>
- state = Some(msg)
- SendOneWayAndReplySenderActor.latch.countDown
- }
- }
-
- class MyActorCustomConstructor extends Actor {
- var prefix = "default-"
- var count = 0
- def receive = {
- case "incrPrefix" => count += 1; prefix = "" + count + "-"
- case msg: String => self.reply(prefix + msg)
- }
+ def receive = {
+ case "OneWay" =>
+ RemoteActorSpecActorUnidirectional.latch.countDown
}
}
-class ClientInitiatedRemoteActorSpec extends JUnitSuite {
- import ClientInitiatedRemoteActorSpec._
- akka.config.Config.config
-
- val HOSTNAME = "localhost"
- val PORT1 = 9990
- val PORT2 = 9991
- var s1: RemoteServer = null
-
- private val unit = TimeUnit.MILLISECONDS
-
- @Before
- def init() {
- s1 = new RemoteServer()
- s1.start(HOSTNAME, PORT1)
- Thread.sleep(1000)
- }
-
- @After
- def finished() {
- s1.shutdown
- val s2 = RemoteServer.serverFor(HOSTNAME, PORT2)
- if (s2.isDefined) s2.get.shutdown
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- }
-
- @Test
- def shouldSendOneWay = {
- val actor = actorOf[RemoteActorSpecActorUnidirectional]
- actor.makeRemote(HOSTNAME, PORT1)
- actor.start
- actor ! "OneWay"
- assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS))
- actor.stop
- }
-
-
- @Test
- def shouldSendOneWayAndReceiveReply = {
- val actor = actorOf[SendOneWayAndReplyReceiverActor]
- actor.makeRemote(HOSTNAME, PORT1)
- actor.start
- val sender = actorOf[SendOneWayAndReplySenderActor]
- sender.homeAddress = (HOSTNAME, PORT2)
- sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].sendTo = actor
- sender.start
- sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].sendOff
- assert(SendOneWayAndReplySenderActor.latch.await(3, TimeUnit.SECONDS))
- assert(sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].state.isDefined === true)
- assert("World" === sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].state.get.asInstanceOf[String])
- actor.stop
- sender.stop
- }
-
- @Test
- def shouldSendBangBangMessageAndReceiveReply = {
- val actor = actorOf[RemoteActorSpecActorBidirectional]
- actor.makeRemote(HOSTNAME, PORT1)
- actor.start
- val result = actor !! "Hello"
- assert("World" === result.get.asInstanceOf[String])
- actor.stop
- }
-
- @Test
- def shouldSendBangBangMessageAndReceiveReplyConcurrently = {
- val actors = (1 to 10).
- map(num => {
- val a = actorOf[RemoteActorSpecActorBidirectional]
- a.makeRemote(HOSTNAME, PORT1)
- a.start
- }).toList
- actors.map(_ !!! "Hello").foreach(future => assert("World" === future.await.result.asInstanceOf[Option[String]].get))
- actors.foreach(_.stop)
- }
-
- @Test
- def shouldRegisterActorByUuid {
- val actor1 = actorOf[MyActorCustomConstructor]
- actor1.makeRemote(HOSTNAME, PORT1)
- actor1.start
- actor1 ! "incrPrefix"
- assert((actor1 !! "test").get === "1-test")
- actor1 ! "incrPrefix"
- assert((actor1 !! "test").get === "2-test")
-
- val actor2 = actorOf[MyActorCustomConstructor]
- actor2.makeRemote(HOSTNAME, PORT1)
- actor2.start
-
- assert((actor2 !! "test").get === "default-test")
-
- 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
+class RemoteActorSpecActorBidirectional extends Actor {
+ def receive = {
+ case "Hello" =>
+ self.reply("World")
+ case "Failure" => throw new ExpectedRemoteProblem("expected")
}
}
+class SendOneWayAndReplyReceiverActor extends Actor {
+ def receive = {
+ case "Hello" =>
+ self.reply("World")
+ }
+}
+
+class CountDownActor(latch: CountDownLatch) extends Actor {
+ def receive = {
+ case "World" => latch.countDown
+ }
+}
+/*
+object SendOneWayAndReplySenderActor {
+ val latch = new CountDownLatch(1)
+}
+class SendOneWayAndReplySenderActor extends Actor {
+ var state: Option[AnyRef] = None
+ var sendTo: ActorRef = _
+ var latch: CountDownLatch = _
+
+ def sendOff = sendTo ! "Hello"
+
+ def receive = {
+ case msg: AnyRef =>
+ state = Some(msg)
+ SendOneWayAndReplySenderActor.latch.countDown
+ }
+}*/
+
+class MyActorCustomConstructor extends Actor {
+ var prefix = "default-"
+ var count = 0
+ def receive = {
+ case "incrPrefix" => count += 1; prefix = "" + count + "-"
+ case msg: String => self.reply(prefix + msg)
+ }
+}
+
+class ClientInitiatedRemoteActorSpec extends AkkaRemoteTest {
+ "ClientInitiatedRemoteActor" should {
+ "shouldSendOneWay" in {
+ val clientManaged = remote.actorOf[RemoteActorSpecActorUnidirectional](host,port).start
+ clientManaged must not be null
+ clientManaged.getClass must be (classOf[LocalActorRef])
+ clientManaged ! "OneWay"
+ RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS) must be (true)
+ clientManaged.stop
+ }
+
+ "shouldSendOneWayAndReceiveReply" in {
+ val latch = new CountDownLatch(1)
+ val actor = remote.actorOf[SendOneWayAndReplyReceiverActor](host,port).start
+ implicit val sender = Some(actorOf(new CountDownActor(latch)).start)
+
+ actor ! "Hello"
+
+ latch.await(3,TimeUnit.SECONDS) must be (true)
+ }
+
+ "shouldSendBangBangMessageAndReceiveReply" in {
+ val actor = remote.actorOf[RemoteActorSpecActorBidirectional](host,port).start
+ val result = actor !! "Hello"
+ "World" must equal (result.get.asInstanceOf[String])
+ actor.stop
+ }
+
+ "shouldSendBangBangMessageAndReceiveReplyConcurrently" in {
+ val actors = (1 to 10).map(num => { remote.actorOf[RemoteActorSpecActorBidirectional](host,port).start }).toList
+ actors.map(_ !!! "Hello") foreach { future =>
+ "World" must equal (future.await.result.asInstanceOf[Option[String]].get)
+ }
+ actors.foreach(_.stop)
+ }
+
+ "shouldRegisterActorByUuid" in {
+ val actor1 = remote.actorOf[MyActorCustomConstructor](host, port).start
+ val actor2 = remote.actorOf[MyActorCustomConstructor](host, port).start
+
+ actor1 ! "incrPrefix"
+
+ (actor1 !! "test").get must equal ("1-test")
+
+ actor1 ! "incrPrefix"
+
+ (actor1 !! "test").get must equal ("2-test")
+
+ (actor2 !! "test").get must equal ("default-test")
+
+ actor1.stop
+ actor2.stop
+ }
+
+ "shouldSendAndReceiveRemoteException" in {
+
+ val actor = remote.actorOf[RemoteActorSpecActorBidirectional](host, port).start
+ try {
+ implicit val timeout = 500000000L
+ val f = (actor !!! "Failure").await.resultOrException
+ fail("Shouldn't get here!!!")
+ } catch {
+ case e: ExpectedRemoteProblem =>
+ }
+ actor.stop
+ }
+ }
+}
diff --git a/akka-remote/src/test/scala/remote/OptimizedLocalScopedSpec.scala b/akka-remote/src/test/scala/remote/OptimizedLocalScopedSpec.scala
new file mode 100644
index 0000000000..f6e0c1806f
--- /dev/null
+++ b/akka-remote/src/test/scala/remote/OptimizedLocalScopedSpec.scala
@@ -0,0 +1,29 @@
+package akka.actor.remote
+
+import akka.actor.{Actor}
+
+object OptimizedLocalScopedSpec {
+ class TestActor extends Actor {
+ def receive = { case _ => }
+ }
+}
+
+class OptimizedLocalScopedSpec extends AkkaRemoteTest {
+ import OptimizedLocalScopedSpec._
+ override def OptimizeLocal = true
+
+ "An enabled optimized local scoped remote" should {
+ "Fetch local actor ref when scope is local" in {
+ val fooActor = Actor.actorOf[TestActor].start
+ remote.register("foo", fooActor)
+
+ remote.actorFor("foo", host, port) must be (fooActor)
+ }
+
+ "Create local actor when client-managed is hosted locally" in {
+ val localClientManaged = Actor.remote.actorOf[TestActor](host, port)
+ localClientManaged.homeAddress must be (None)
+ }
+
+ }
+}
\ No newline at end of file
diff --git a/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala b/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala
index 54bc33975c..f97ea75841 100644
--- a/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala
+++ b/akka-remote/src/test/scala/remote/RemoteSupervisorSpec.scala
@@ -7,10 +7,10 @@ package akka.actor.remote
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, BlockingQueue}
import akka.serialization.BinaryString
import akka.config.Supervision._
-import akka.remote.{RemoteServer, RemoteClient}
import akka.OneWay
-import org.scalatest.junit.JUnitSuite
-import org.junit.{Test, Before, After}
+import org.scalatest._
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
import akka.actor.{SupervisorFactory, Supervisor, ActorRef, Actor}
import Actor._
@@ -70,17 +70,7 @@ object Log {
}
}
-object RemoteSupervisorSpec {
- val HOSTNAME = "localhost"
- val PORT = 9988
- var server: RemoteServer = null
-}
-
-/**
- * @author Jonas Bonér
- */
-class RemoteSupervisorSpec extends JUnitSuite {
- import RemoteSupervisorSpec._
+class RemoteSupervisorSpec extends AkkaRemoteTest {
var pingpong1: ActorRef = _
var pingpong2: ActorRef = _
@@ -88,133 +78,251 @@ class RemoteSupervisorSpec extends JUnitSuite {
import Log._
- @Before
- def init {
- server = new RemoteServer()
- server.start(HOSTNAME, PORT)
- Thread.sleep(1000)
- }
+ "Remote supervision" should {
- @After
- def finished {
- try {
- server.shutdown
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- } catch {
- case e => ()
+ "start server" in {
+ Log.messageLog.clear
+ val sup = getSingleActorAllForOneSupervisor
+
+ (pingpong1 !! BinaryString("Ping")) must equal (Some("pong"))
+ }
+
+ "StartServerForNestedSupervisorHierarchy" in {
+ clearMessageLogs
+ val sup = getNestedSupervisorsAllForOneConf
+ sup.start
+
+ (pingpong1 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+ }
+
+ "killSingleActorOneForOne" in {
+ clearMessageLogs
+ val sup = getSingleActorOneForOneSupervisor
+
+ (pingpong1 !!! (BinaryString("Die"), 5000)).await.exception.isDefined must be (true)
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ }
+
+ "callKillCallSingleActorOneForOne" in {
+ clearMessageLogs
+ val sup = getSingleActorOneForOneSupervisor
+
+ (pingpong1 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+
+ (pingpong1 !!! (BinaryString("Die"), 5000)).await.exception.isDefined must be (true)
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ (pingpong1 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ }
+
+ "KillSingleActorAllForOne" in {
+ clearMessageLogs
+ val sup = getSingleActorAllForOneSupervisor
+
+ (pingpong1 !!! (BinaryString("Die"), 5000)).await.exception.isDefined must be (true)
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ }
+
+ "CallKillCallSingleActorAllForOne" in {
+ clearMessageLogs
+ val sup = getSingleActorAllForOneSupervisor
+
+ (pingpong1 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+
+ (pingpong1 !!! (BinaryString("Die"), 5000)).await.exception.isDefined must be (true)
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+
+ (pingpong1 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ }
+
+ "KillMultipleActorsOneForOne1" in {
+ clearMessageLogs
+ val sup = getMultipleActorsOneForOneConf
+
+ (pingpong1 !!! (BinaryString("Die"), 5000)).await.exception.isDefined must be (true)
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ }
+
+ "KillCallMultipleActorsOneForOne" in {
+ clearMessageLogs
+ val sup = getMultipleActorsOneForOneConf
+
+ (pingpong1 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+ (pingpong2 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+ (pingpong3 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+
+ (pingpong2 !!! (BinaryString("Die"), 5000)).await.exception.isDefined must be (true)
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+
+ (pingpong1 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+ (pingpong2 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+ (pingpong3 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ }
+
+ "KillMultipleActorsAllForOne" in {
+ clearMessageLogs
+ val sup = getMultipleActorsAllForOneConf
+
+ (pingpong2 !!! (BinaryString("Die"), 5000)).await.exception.isDefined must be (true)
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ }
+
+ "CallKillCallMultipleActorsAllForOne" in {
+ clearMessageLogs
+ val sup = getMultipleActorsAllForOneConf
+
+ pingpong1 !! (BinaryString("Ping"), 5000) must equal (Some("pong"))
+ pingpong2 !! (BinaryString("Ping"), 5000) must equal (Some("pong"))
+ pingpong3 !! (BinaryString("Ping"), 5000) must equal (Some("pong"))
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+
+ (pingpong2 !!! (BinaryString("Die"), 5000)).await.exception.isDefined must be (true)
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+
+ pingpong1 !! (BinaryString("Ping"), 5000) must equal (Some("pong"))
+ pingpong2 !! (BinaryString("Ping"), 5000) must equal (Some("pong"))
+ pingpong3 !! (BinaryString("Ping"), 5000) must equal (Some("pong"))
+
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("ping")
}
}
- @Test def shouldStartServer = {
- Log.messageLog.clear
- val sup = getSingleActorAllForOneSupervisor
+ def getSingleActorAllForOneSupervisor: Supervisor = {
- expect("pong") {
- (pingpong1 !! BinaryString("Ping")).getOrElse("nil")
- }
- }
- @Test def shouldStartServerForNestedSupervisorHierarchy = {
- clearMessageLogs
- val sup = getNestedSupervisorsAllForOneConf
- sup.start
+ // Create an abstract SupervisorContainer that works for all implementations
+ // of the different Actors (Services).
+ //
+ // Then create a concrete container in which we mix in support for the specific
+ // implementation of the Actors we want to use.
- expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
+ pingpong1 = remote.actorOf[RemotePingPong1Actor](host,port).start
+
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ AllForOneStrategy(List(classOf[Exception]), 3, 100),
+ Supervise(
+ pingpong1,
+ Permanent)
+ :: Nil))
+
+ factory.newInstance
}
- @Test def shouldKillSingleActorOneForOne = {
- clearMessageLogs
- val sup = getSingleActorOneForOneSupervisor
+ def getSingleActorOneForOneSupervisor: Supervisor = {
+ pingpong1 = remote.actorOf[RemotePingPong1Actor](host,port).start
- intercept[RuntimeException] {
- pingpong1 !! (BinaryString("Die"), 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ OneForOneStrategy(List(classOf[Exception]), 3, 100),
+ Supervise(
+ pingpong1,
+ Permanent)
+ :: Nil))
+ factory.newInstance
}
- @Test def shouldCallKillCallSingleActorOneForOne = {
- clearMessageLogs
- val sup = getSingleActorOneForOneSupervisor
+ def getMultipleActorsAllForOneConf: Supervisor = {
+ pingpong1 = remote.actorOf[RemotePingPong1Actor](host,port).start
+ pingpong2 = remote.actorOf[RemotePingPong2Actor](host,port).start
+ pingpong3 = remote.actorOf[RemotePingPong3Actor](host,port).start
- expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- intercept[RuntimeException] {
- pingpong1 !! (BinaryString("Die"), 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ AllForOneStrategy(List(classOf[Exception]), 3, 100),
+ Supervise(
+ pingpong1,
+ Permanent)
+ ::
+ Supervise(
+ pingpong2,
+ Permanent)
+ ::
+ Supervise(
+ pingpong3,
+ Permanent)
+ :: Nil))
+ factory.newInstance
}
- @Test def shouldKillSingleActorAllForOne = {
- clearMessageLogs
- val sup = getSingleActorAllForOneSupervisor
+ def getMultipleActorsOneForOneConf: Supervisor = {
+ pingpong1 = remote.actorOf[RemotePingPong1Actor](host,port).start
+ pingpong2 = remote.actorOf[RemotePingPong2Actor](host,port).start
+ pingpong3 = remote.actorOf[RemotePingPong3Actor](host,port).start
- intercept[RuntimeException] {
- pingpong1 !! (BinaryString("Die"), 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ OneForOneStrategy(List(classOf[Exception]), 3, 100),
+ Supervise(
+ pingpong1,
+ Permanent)
+ ::
+ Supervise(
+ pingpong2,
+ Permanent)
+ ::
+ Supervise(
+ pingpong3,
+ Permanent)
+ :: Nil))
+ factory.newInstance
}
- @Test def shouldCallKillCallSingleActorAllForOne = {
- clearMessageLogs
- val sup = getSingleActorAllForOneSupervisor
+ def getNestedSupervisorsAllForOneConf: Supervisor = {
+ pingpong1 = remote.actorOf[RemotePingPong1Actor](host,port).start
+ pingpong2 = remote.actorOf[RemotePingPong2Actor](host,port).start
+ pingpong3 = remote.actorOf[RemotePingPong3Actor](host,port).start
- expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- intercept[RuntimeException] {
- pingpong1 !! (BinaryString("Die"), 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldKillMultipleActorsOneForOne1 = {
- clearMessageLogs
- val sup = getMultipleActorsOneForOneConf
-
- intercept[RuntimeException] {
- pingpong1 !! (BinaryString("Die"), 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ AllForOneStrategy(List(classOf[Exception]), 3, 100),
+ Supervise(
+ pingpong1,
+ Permanent)
+ ::
+ SupervisorConfig(
+ AllForOneStrategy(List(classOf[Exception]), 3, 100),
+ Supervise(
+ pingpong2,
+ Permanent)
+ ::
+ Supervise(
+ pingpong3,
+ Permanent)
+ :: Nil)
+ :: Nil))
+ factory.newInstance
}
/*
@@ -232,140 +340,6 @@ class RemoteSupervisorSpec extends JUnitSuite {
}
}
*/
- @Test def shouldKillCallMultipleActorsOneForOne = {
- clearMessageLogs
- val sup = getMultipleActorsOneForOneConf
-
- expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- intercept[RuntimeException] {
- pingpong2 !! (BinaryString("Die"), 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldKillMultipleActorsAllForOne = {
- clearMessageLogs
- val sup = getMultipleActorsAllForOneConf
-
- intercept[RuntimeException] {
- pingpong2 !! (BinaryString("Die"), 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
-
- @Test def shouldCallKillCallMultipleActorsAllForOne = {
- clearMessageLogs
- val sup = getMultipleActorsAllForOneConf
-
- expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- intercept[RuntimeException] {
- pingpong2 !! (BinaryString("Die"), 5000)
- }
-
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("Expected exception; to test fault-tolerance") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong2 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("pong") {
- (pingpong3 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
- }
-
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- expect("ping") {
- messageLog.poll(5, TimeUnit.SECONDS)
- }
- }
/*
@@ -407,15 +381,15 @@ class RemoteSupervisorSpec extends JUnitSuite {
expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
+ (pingpong1 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
}
expect("pong") {
- (pingpong2 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
+ (pingpong2 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
}
expect("pong") {
- (pingpong3 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
+ (pingpong3 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
}
expect("ping") {
@@ -441,15 +415,15 @@ class RemoteSupervisorSpec extends JUnitSuite {
messageLog.poll(5, TimeUnit.SECONDS)
}
expect("pong") {
- (pingpong1 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
+ (pingpong1 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
}
expect("pong") {
- (pingpong2 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
+ (pingpong2 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
}
expect("pong") {
- (pingpong3 !! (BinaryString("Ping"), 5000)).getOrElse("nil")
+ (pingpong3 !! (BinaryString("Ping"), 5000)) must equal (Some("pong"))
}
expect("ping") {
@@ -463,136 +437,5 @@ class RemoteSupervisorSpec extends JUnitSuite {
}
}
*/
- // =============================================
- // Creat some supervisors with different configurations
- def getSingleActorAllForOneSupervisor: Supervisor = {
-
- // Create an abstract SupervisorContainer that works for all implementations
- // of the different Actors (Services).
- //
- // Then create a concrete container in which we mix in support for the specific
- // implementation of the Actors we want to use.
-
- pingpong1 = actorOf[RemotePingPong1Actor]
- pingpong1.makeRemote(HOSTNAME, PORT)
- pingpong1.start
-
- val factory = SupervisorFactory(
- SupervisorConfig(
- AllForOneStrategy(List(classOf[Exception]), 3, 100),
- Supervise(
- pingpong1,
- Permanent)
- :: Nil))
-
- factory.newInstance
- }
-
- def getSingleActorOneForOneSupervisor: Supervisor = {
- pingpong1 = actorOf[RemotePingPong1Actor]
- pingpong1.makeRemote(HOSTNAME, PORT)
- pingpong1.start
-
- val factory = SupervisorFactory(
- SupervisorConfig(
- OneForOneStrategy(List(classOf[Exception]), 3, 100),
- Supervise(
- pingpong1,
- Permanent)
- :: Nil))
- factory.newInstance
- }
-
- def getMultipleActorsAllForOneConf: Supervisor = {
- pingpong1 = actorOf[RemotePingPong1Actor]
- pingpong1.makeRemote(HOSTNAME, PORT)
- pingpong1.start
- pingpong2 = actorOf[RemotePingPong2Actor]
- pingpong2.makeRemote(HOSTNAME, PORT)
- pingpong2.start
- pingpong3 = actorOf[RemotePingPong3Actor]
- pingpong3.makeRemote(HOSTNAME, PORT)
- pingpong3.start
-
- val factory = SupervisorFactory(
- SupervisorConfig(
- AllForOneStrategy(List(classOf[Exception]), 3, 100),
- Supervise(
- pingpong1,
- Permanent)
- ::
- Supervise(
- pingpong2,
- Permanent)
- ::
- Supervise(
- pingpong3,
- Permanent)
- :: Nil))
- factory.newInstance
- }
-
- def getMultipleActorsOneForOneConf: Supervisor = {
- pingpong1 = actorOf[RemotePingPong1Actor]
- pingpong1.makeRemote(HOSTNAME, PORT)
- pingpong1 = actorOf[RemotePingPong1Actor]
- pingpong1.makeRemote(HOSTNAME, PORT)
- pingpong1.start
- pingpong2 = actorOf[RemotePingPong2Actor]
- pingpong2.makeRemote(HOSTNAME, PORT)
- pingpong2.start
- pingpong3 = actorOf[RemotePingPong3Actor]
- pingpong3.makeRemote(HOSTNAME, PORT)
- pingpong3.start
-
- val factory = SupervisorFactory(
- SupervisorConfig(
- OneForOneStrategy(List(classOf[Exception]), 3, 100),
- Supervise(
- pingpong1,
- Permanent)
- ::
- Supervise(
- pingpong2,
- Permanent)
- ::
- Supervise(
- pingpong3,
- Permanent)
- :: Nil))
- factory.newInstance
- }
-
- def getNestedSupervisorsAllForOneConf: Supervisor = {
- pingpong1 = actorOf[RemotePingPong1Actor]
- pingpong1.makeRemote(HOSTNAME, PORT)
- pingpong1.start
- pingpong2 = actorOf[RemotePingPong2Actor]
- pingpong2.makeRemote(HOSTNAME, PORT)
- pingpong2.start
- pingpong3 = actorOf[RemotePingPong3Actor]
- pingpong3.makeRemote(HOSTNAME, PORT)
- pingpong3.start
-
- val factory = SupervisorFactory(
- SupervisorConfig(
- AllForOneStrategy(List(classOf[Exception]), 3, 100),
- Supervise(
- pingpong1,
- Permanent)
- ::
- SupervisorConfig(
- AllForOneStrategy(List(classOf[Exception]), 3, 100),
- Supervise(
- pingpong2,
- Permanent)
- ::
- Supervise(
- pingpong3,
- Permanent)
- :: Nil)
- :: Nil))
- factory.newInstance
- }
}
diff --git a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala
index c036e19333..b72c49c204 100644
--- a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala
@@ -4,23 +4,11 @@
package akka.actor.remote
-import org.scalatest.matchers.ShouldMatchers
-import org.scalatest.junit.JUnitRunner
-import org.junit.runner.RunWith
-
import akka.config.Supervision._
import akka.actor._
-import akka.remote.{RemoteServer, RemoteClient}
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, BlockingQueue}
-import org.scalatest.{BeforeAndAfterEach, Spec, Assertions, BeforeAndAfterAll}
-import akka.config.{Config, TypedActorConfigurator, RemoteAddress}
-
-object RemoteTypedActorSpec {
- val HOSTNAME = "localhost"
- val PORT = 9988
- var server: RemoteServer = null
-}
+import akka.config. {RemoteAddress, Config, TypedActorConfigurator}
object RemoteTypedActorLog {
val messageLog: BlockingQueue[String] = new LinkedBlockingQueue[String]
@@ -32,21 +20,16 @@ object RemoteTypedActorLog {
}
}
-@RunWith(classOf[JUnitRunner])
-class RemoteTypedActorSpec extends
- Spec with
- ShouldMatchers with
- BeforeAndAfterEach with BeforeAndAfterAll {
+class RemoteTypedActorSpec extends AkkaRemoteTest {
import RemoteTypedActorLog._
- import RemoteTypedActorSpec._
- private val conf = new TypedActorConfigurator
+ private var conf: TypedActorConfigurator = _
- override def beforeAll = {
- server = new RemoteServer()
- server.start("localhost", 9995)
+ override def beforeEach {
+ super.beforeEach
Config.config
+ conf = new TypedActorConfigurator
conf.configure(
new AllForOneStrategy(List(classOf[Exception]), 3, 5000),
List(
@@ -55,74 +38,57 @@ class RemoteTypedActorSpec extends
classOf[RemoteTypedActorOneImpl],
Permanent,
10000,
- new RemoteAddress("localhost", 9995)),
+ RemoteAddress(host,port)),
new SuperviseTypedActor(
classOf[RemoteTypedActorTwo],
classOf[RemoteTypedActorTwoImpl],
Permanent,
10000,
- new RemoteAddress("localhost", 9995))
+ RemoteAddress(host,port))
).toArray).supervise
+ }
+
+ override def afterEach {
+ clearMessageLogs
+ conf.stop
+ super.afterEach
Thread.sleep(1000)
}
- override def afterAll = {
- conf.stop
- try {
- server.shutdown
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- } catch {
- case e => ()
- }
- ActorRegistry.shutdownAll
- }
+ "Remote Typed Actor " should {
- override def afterEach() {
- server.typedActors.clear
- }
-
- describe("Remote Typed Actor ") {
-
- it("should receive one-way message") {
- clearMessageLogs
+ /*"receives one-way message" in {
val ta = conf.getInstance(classOf[RemoteTypedActorOne])
- expect("oneway") {
- ta.oneWay
- oneWayLog.poll(5, TimeUnit.SECONDS)
- }
+ ta.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS) must equal ("oneway")
}
- it("should respond to request-reply message") {
- clearMessageLogs
+ "responds to request-reply message" in {
+ val ta = conf.getInstance(classOf[RemoteTypedActorOne])
+ ta.requestReply("ping") must equal ("pong")
+ } */
+
+ "be restarted on failure" in {
val ta = conf.getInstance(classOf[RemoteTypedActorOne])
- expect("pong") {
- ta.requestReply("ping")
- }
- }
-
- it("should be restarted on failure") {
- clearMessageLogs
- val ta = conf.getInstance(classOf[RemoteTypedActorOne])
-
- intercept[RuntimeException] {
+ try {
ta.requestReply("die")
- }
- messageLog.poll(5, TimeUnit.SECONDS) should equal ("Expected exception; to test fault-tolerance")
+ fail("Shouldn't get here")
+ } catch { case re: RuntimeException if re.getMessage == "Expected exception; to test fault-tolerance" => }
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
}
- it("should restart linked friends on failure") {
- clearMessageLogs
+ /* "restarts linked friends on failure" in {
val ta1 = conf.getInstance(classOf[RemoteTypedActorOne])
val ta2 = conf.getInstance(classOf[RemoteTypedActorTwo])
- intercept[RuntimeException] {
+ try {
ta1.requestReply("die")
- }
- 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")
- }
+ fail("Shouldn't get here")
+ } catch { case re: RuntimeException if re.getMessage == "Expected exception; to test fault-tolerance" => }
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ messageLog.poll(5, TimeUnit.SECONDS) must equal ("Expected exception; to test fault-tolerance")
+ }*/
}
}
diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala
index 73031ded1c..a8990e2b88 100644
--- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala
+++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSample.scala
@@ -1,7 +1,6 @@
package akka.actor.remote
-import akka.actor.Actor
-import akka.remote.{RemoteClient, RemoteNode}
+import akka.actor.{Actor, ActorRegistry}
import akka.util.Logging
import Actor._
@@ -36,22 +35,17 @@ class HelloWorldActor extends Actor {
object ServerInitiatedRemoteActorServer {
- def run = {
- RemoteNode.start("localhost", 2552)
- RemoteNode.register("hello-service", actorOf[HelloWorldActor])
+ def main(args: Array[String]) = {
+ Actor.remote.start("localhost", 2552)
+ Actor.remote.register("hello-service", actorOf[HelloWorldActor])
}
-
- def main(args: Array[String]) = run
}
object ServerInitiatedRemoteActorClient extends Logging {
-
- def run = {
- val actor = RemoteClient.actorFor("hello-service", "localhost", 2552)
+ def main(args: Array[String]) = {
+ val actor = Actor.remote.actorFor("hello-service", "localhost", 2552)
val result = actor !! "Hello"
log.slf4j.info("Result from Remote Actor: {}", result)
}
-
- def main(args: Array[String]) = run
}
diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
index 14387e7909..88a5ec8ec3 100644
--- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
@@ -1,29 +1,30 @@
package akka.actor.remote
import java.util.concurrent.{CountDownLatch, TimeUnit}
-import org.scalatest.junit.JUnitSuite
-import org.junit.{Test, Before, After}
-import akka.util._
-import akka.remote.{RemoteServer, RemoteClient}
import akka.actor.Actor._
import akka.actor.{ActorRegistry, ActorRef, Actor}
object ServerInitiatedRemoteActorSpec {
- val HOSTNAME = "localhost"
- val PORT = 9990
- var server: RemoteServer = null
-
case class Send(actor: ActorRef)
- object RemoteActorSpecActorUnidirectional {
- val latch = new CountDownLatch(1)
- }
class RemoteActorSpecActorUnidirectional extends Actor {
-
def receive = {
- case "OneWay" =>
- RemoteActorSpecActorUnidirectional.latch.countDown
+ case "Ping" => self.reply_?("Pong")
+ }
+ }
+
+ class Decrementer extends Actor {
+ def receive = {
+ case "done" => self.reply_?(false)
+ case i: Int if i > 0 =>
+ self.reply_?(i - 1)
+ case i: Int =>
+ self.reply_?(0)
+ this become {
+ case "done" => self.reply_?(true)
+ case _ => //Do Nothing
+ }
}
}
@@ -37,184 +38,167 @@ object ServerInitiatedRemoteActorSpec {
}
}
- object RemoteActorSpecActorAsyncSender {
- val latch = new CountDownLatch(1)
- }
- class RemoteActorSpecActorAsyncSender extends Actor {
-
+ class RemoteActorSpecActorAsyncSender(latch: CountDownLatch) extends Actor {
def receive = {
case Send(actor: ActorRef) =>
actor ! "Hello"
- case "World" =>
- RemoteActorSpecActorAsyncSender.latch.countDown
+ case "World" => latch.countDown
}
}
}
-class ServerInitiatedRemoteActorSpec extends JUnitSuite {
+class ServerInitiatedRemoteActorSpec extends AkkaRemoteTest {
import ServerInitiatedRemoteActorSpec._
- private val unit = TimeUnit.MILLISECONDS
- @Before
- def init {
- server = new RemoteServer()
+ "Server-managed remote actors" should {
+ "sendWithBang" in {
+ val latch = new CountDownLatch(1)
+ implicit val sender = replyHandler(latch, "Pong")
+ remote.register(actorOf[RemoteActorSpecActorUnidirectional])
+ val actor = remote.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional",5000L,host, port)
- server.start(HOSTNAME, PORT)
-
- server.register(actorOf[RemoteActorSpecActorUnidirectional])
- server.register(actorOf[RemoteActorSpecActorBidirectional])
- server.register(actorOf[RemoteActorSpecActorAsyncSender])
-
- Thread.sleep(1000)
- }
-
- // make sure the servers postStop cleanly after the test has finished
- @After
- def finished {
- try {
- server.shutdown
- val s2 = RemoteServer.serverFor(HOSTNAME, PORT + 1)
- if (s2.isDefined) s2.get.shutdown
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- } catch {
- case e => ()
- }
- }
-
- @Test
- def shouldSendWithBang {
- val actor = RemoteClient.actorFor(
- "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional",
- 5000L,
- HOSTNAME, PORT)
- val result = actor ! "OneWay"
- assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS))
- actor.stop
- }
-
- @Test
- def shouldSendWithBangBangAndGetReply {
- val actor = RemoteClient.actorFor(
- "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional",
- 5000L,
- HOSTNAME, PORT)
- val result = actor !! "Hello"
- assert("World" === result.get.asInstanceOf[String])
- actor.stop
- }
-
- @Test
- def shouldSendWithBangAndGetReplyThroughSenderRef {
- implicit val timeout = 500000000L
- val actor = RemoteClient.actorFor(
- "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional",
- timeout,
- HOSTNAME, PORT)
- val sender = actorOf[RemoteActorSpecActorAsyncSender]
- sender.homeAddress = (HOSTNAME, PORT + 1)
- sender.start
- sender ! Send(actor)
- assert(RemoteActorSpecActorAsyncSender.latch.await(1, TimeUnit.SECONDS))
- actor.stop
- }
-
- @Test
- def shouldSendWithBangBangAndReplyWithException {
- implicit val timeout = 500000000L
- val actor = RemoteClient.actorFor(
- "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional",
- timeout,
- HOSTNAME, PORT)
- 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 reflectiveAccessShouldNotCreateNewRemoteServerObject {
- val server1 = new RemoteServer()
- server1.start("localhost", 9990)
-
- var found = RemoteServer.serverFor("localhost", 9990)
- assert(found.isDefined, "sever not found")
-
- val a = actorOf( new Actor { def receive = { case _ => } } ).start
-
- found = RemoteServer.serverFor("localhost", 9990)
- assert(found.isDefined, "sever not found after creating an actor")
+ actor ! "Ping"
+ latch.await(1, TimeUnit.SECONDS) must be (true)
}
+ "sendWithBangBangAndGetReply" in {
+ remote.register(actorOf[RemoteActorSpecActorBidirectional])
+ val actor = remote.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional", 5000L,host, port)
+ (actor !! "Hello").as[String].get must equal ("World")
+ }
- @Test
- def shouldNotRecreateRegisteredActor {
- server.register(actorOf[RemoteActorSpecActorUnidirectional])
- val actor = RemoteClient.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", HOSTNAME, PORT)
- val numberOfActorsInRegistry = ActorRegistry.actors.length
- actor ! "OneWay"
- assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS))
- assert(numberOfActorsInRegistry === ActorRegistry.actors.length)
- actor.stop
+ "sendWithBangAndGetReplyThroughSenderRef" in {
+ remote.register(actorOf[RemoteActorSpecActorBidirectional])
+ implicit val timeout = 500000000L
+ val actor = remote.actorFor(
+ "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional", timeout,host, port)
+ val latch = new CountDownLatch(1)
+ val sender = actorOf( new RemoteActorSpecActorAsyncSender(latch) ).start
+ sender ! Send(actor)
+ latch.await(1, TimeUnit.SECONDS) must be (true)
+ }
+
+ "sendWithBangBangAndReplyWithException" in {
+ remote.register(actorOf[RemoteActorSpecActorBidirectional])
+ implicit val timeout = 500000000L
+ val actor = remote.actorFor(
+ "akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorBidirectional", timeout, host, port)
+ try {
+ actor !! "Failure"
+ fail("Should have thrown an exception")
+ } catch {
+ case e => e.getMessage must equal ("Expected exception; to test fault-tolerance")
+ }
+ }
+
+ "notRecreateRegisteredActor" in {
+ val latch = new CountDownLatch(1)
+ implicit val sender = replyHandler(latch, "Pong")
+ remote.register(actorOf[RemoteActorSpecActorUnidirectional])
+ val actor = remote.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", host, port)
+ val numberOfActorsInRegistry = Actor.registry.actors.length
+ actor ! "Ping"
+ latch.await(1, TimeUnit.SECONDS) must be (true)
+ numberOfActorsInRegistry must equal (Actor.registry.actors.length)
+ }
+
+ "UseServiceNameAsIdForRemoteActorRef" in {
+ val latch = new CountDownLatch(3)
+ implicit val sender = replyHandler(latch, "Pong")
+ remote.register(actorOf[RemoteActorSpecActorUnidirectional])
+ remote.register("my-service", actorOf[RemoteActorSpecActorUnidirectional])
+ val actor1 = remote.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", host, port)
+ val actor2 = remote.actorFor("my-service", host, port)
+ val actor3 = remote.actorFor("my-service", host, port)
+
+ actor1 ! "Ping"
+ actor2 ! "Ping"
+ actor3 ! "Ping"
+
+ latch.await(1, TimeUnit.SECONDS) must be (true)
+ actor1.uuid must not equal actor2.uuid
+ actor1.uuid must not equal actor3.uuid
+ actor1.id must not equal actor2.id
+ actor2.id must equal (actor3.id)
+ }
+
+ "shouldFindActorByUuid" in {
+ val latch = new CountDownLatch(2)
+ implicit val sender = replyHandler(latch, "Pong")
+ val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
+ val actor2 = actorOf[RemoteActorSpecActorUnidirectional]
+ remote.register("uuid:" + actor1.uuid, actor1)
+ remote.register("my-service", actor2)
+
+ val ref1 = remote.actorFor("uuid:" + actor1.uuid, host, port)
+ val ref2 = remote.actorFor("my-service", host, port)
+
+ ref1 ! "Ping"
+ ref2 ! "Ping"
+ latch.await(1, TimeUnit.SECONDS) must be (true)
+ }
+
+ "shouldRegisterAndUnregister" in {
+ val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
+
+ remote.register("my-service-1", actor1)
+ remote.actors.get("my-service-1") must not be null
+
+ remote.unregister("my-service-1")
+ remote.actors.get("my-service-1") must be (null)
+ }
+
+ "shouldRegisterAndUnregisterByUuid" in {
+ val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
+ val uuid = "uuid:" + actor1.uuid
+
+ remote.register(uuid, actor1)
+ remote.actorsByUuid.get(actor1.uuid.toString) must not be null
+
+ remote.unregister(uuid)
+ remote.actorsByUuid.get(actor1.uuid) must be (null)
+ }
+
+ "shouldHandleOneWayReplyThroughPassiveRemoteClient" in {
+ val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
+ remote.register("foo", actor1)
+ val latch = new CountDownLatch(1)
+ val actor2 = actorOf(new Actor { def receive = { case "Pong" => latch.countDown } }).start
+
+ val remoteActor = remote.actorFor("foo", host, port)
+ remoteActor.!("Ping")(Some(actor2))
+ latch.await(3,TimeUnit.SECONDS) must be (true)
+ }
+
+ "should be able to remotely communicate between 2 server-managed actors" in {
+ val localFoo = actorOf[Decrementer]
+ val localBar = actorOf[Decrementer]
+ remote.register("foo", localFoo)
+ remote.register("bar", localBar)
+
+ val remoteFoo = remote.actorFor("foo", host, port)
+ val remoteBar = remote.actorFor("bar", host, port)
+
+ //Seed the start
+ remoteFoo.!(10)(Some(remoteBar))
+
+ val latch = new CountDownLatch(100)
+
+ def testDone() = (remoteFoo !! "done").as[Boolean].getOrElse(false) &&
+ (remoteBar !! "done").as[Boolean].getOrElse(false)
+
+ while(!testDone()) {
+ if (latch.await(200, TimeUnit.MILLISECONDS))
+ error("Test didn't complete within 100 cycles")
+ else
+ latch.countDown
+ }
+
+ val decrementers = Actor.registry.actorsFor[Decrementer]
+ decrementers must have size(2) //No new are allowed to have been created
+ decrementers.find( _ eq localFoo) must equal (Some(localFoo))
+ decrementers.find( _ eq localBar) must equal (Some(localBar))
+ }
}
-
- @Test
- def shouldUseServiceNameAsIdForRemoteActorRef {
- server.register(actorOf[RemoteActorSpecActorUnidirectional])
- server.register("my-service", actorOf[RemoteActorSpecActorUnidirectional])
- val actor1 = RemoteClient.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", HOSTNAME, PORT)
- val actor2 = RemoteClient.actorFor("my-service", HOSTNAME, PORT)
- val actor3 = RemoteClient.actorFor("my-service", HOSTNAME, PORT)
-
- actor1 ! "OneWay"
- actor2 ! "OneWay"
- actor3 ! "OneWay"
-
- assert(actor1.uuid != actor2.uuid)
- assert(actor1.uuid != actor3.uuid)
- assert(actor1.id != actor2.id)
- assert(actor2.id == actor3.id)
- }
-
- @Test
- def shouldFindActorByUuid {
- val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
- val actor2 = actorOf[RemoteActorSpecActorUnidirectional]
- server.register("uuid:" + actor1.uuid, actor1)
- server.register("my-service", actor2)
-
- val ref1 = RemoteClient.actorFor("uuid:" + actor1.uuid, HOSTNAME, PORT)
- val ref2 = RemoteClient.actorFor("my-service", HOSTNAME, PORT)
-
- ref1 ! "OneWay"
- assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS))
- ref1.stop
- ref2 ! "OneWay"
- ref2.stop
-
- }
-
- @Test
- def shouldRegisterAndUnregister {
- val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
- server.register("my-service-1", actor1)
- assert(server.actors.get("my-service-1") ne null, "actor registered")
- server.unregister("my-service-1")
- assert(server.actors.get("my-service-1") eq null, "actor unregistered")
- }
-
- @Test
- def shouldRegisterAndUnregisterByUuid {
- val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
- server.register("uuid:" + actor1.uuid, actor1)
- assert(server.actorsByUuid.get(actor1.uuid.toString) ne null, "actor registered")
- server.unregister("uuid:" + actor1.uuid)
- assert(server.actorsByUuid.get(actor1.uuid) eq null, "actor unregistered")
- }
-
}
diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteSessionActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteSessionActorSpec.scala
index ada24ea9e7..99c6f77ea3 100644
--- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteSessionActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteSessionActorSpec.scala
@@ -4,159 +4,90 @@
package akka.actor.remote
-import org.scalatest._
-import org.scalatest.matchers.ShouldMatchers
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.junit.JUnitRunner
-import org.junit.runner.RunWith
-
-import java.util.concurrent.TimeUnit
-
-import akka.remote.{RemoteServer, RemoteClient}
import akka.actor._
import akka.actor.Actor._
-import RemoteTypedActorLog._
+import java.util.concurrent. {ConcurrentSkipListSet, TimeUnit}
+import akka.remote.netty.NettyRemoteSupport
object ServerInitiatedRemoteSessionActorSpec {
- val HOSTNAME = "localhost"
- val PORT = 9990
- var server: RemoteServer = null
case class Login(user:String)
case class GetUser()
case class DoSomethingFunny()
- var instantiatedSessionActors= Set[ActorRef]()
+ val instantiatedSessionActors = new ConcurrentSkipListSet[ActorRef]()
class RemoteStatefullSessionActorSpec extends Actor {
- var user : String= "anonymous"
-
- override def preStart = {
- instantiatedSessionActors += self
- }
-
- override def postStop = {
- instantiatedSessionActors -= self
- }
+ override def preStart = instantiatedSessionActors.add(self)
+ override def postStop = instantiatedSessionActors.remove(self)
+ var user: String = "anonymous"
def receive = {
- case Login(user) =>
- this.user = user
- case GetUser() =>
- self.reply(this.user)
- case DoSomethingFunny() =>
- throw new Exception("Bad boy")
+ case Login(user) => this.user = user
+ case GetUser() => self.reply(this.user)
+ case DoSomethingFunny() => throw new Exception("Bad boy")
}
}
}
-@RunWith(classOf[JUnitRunner])
-class ServerInitiatedRemoteSessionActorSpec extends
- FlatSpec with
- ShouldMatchers with
- BeforeAndAfterEach {
+class ServerInitiatedRemoteSessionActorSpec extends AkkaRemoteTest {
import ServerInitiatedRemoteSessionActorSpec._
- private val unit = TimeUnit.MILLISECONDS
+ "A remote session Actor" should {
+ "create a new session actor per connection" in {
+ remote.registerPerSession("untyped-session-actor-service", actorOf[RemoteStatefullSessionActorSpec])
+ val session1 = remote.actorFor("untyped-session-actor-service", 5000L, host, port)
- override def beforeEach = {
- server = new RemoteServer()
- server.start(HOSTNAME, PORT)
+ val default1 = session1 !! GetUser()
+ default1.as[String] must equal (Some("anonymous"))
- server.registerPerSession("untyped-session-actor-service", actorOf[RemoteStatefullSessionActorSpec])
+ session1 ! Login("session[1]")
+ val result1 = session1 !! GetUser()
+ result1.as[String] must equal (Some("session[1]"))
- Thread.sleep(1000)
- }
+ remote.shutdownClientModule
- // make sure the servers shutdown cleanly after the test has finished
- override def afterEach = {
- try {
- server.shutdown
- RemoteClient.shutdownAll
+ val session2 = remote.actorFor("untyped-session-actor-service", 5000L, host, port)
+
+ // since this is a new session, the server should reset the state
+ val default2 = session2 !! GetUser()
+ default2.as[String] must equal (Some("anonymous"))
+ }
+
+ "stop the actor when the client disconnects" in {
+ instantiatedSessionActors.clear
+ remote.registerPerSession("untyped-session-actor-service", actorOf[RemoteStatefullSessionActorSpec])
+ val session1 = remote.actorFor("untyped-session-actor-service", 5000L, host, port)
+
+ val default1 = session1 !! GetUser()
+ default1.as[String] must equal (Some("anonymous"))
+
+ instantiatedSessionActors must have size (1)
+ remote.shutdownClientModule
+ instantiatedSessionActors must have size (0)
+ }
+
+ "stop the actor when there is an error" in {
+ instantiatedSessionActors.clear
+ remote.registerPerSession("untyped-session-actor-service", actorOf[RemoteStatefullSessionActorSpec])
+ val session1 = remote.actorFor("untyped-session-actor-service", 5000L, host, port)
+
+ session1 ! DoSomethingFunny()
+ session1.stop()
Thread.sleep(1000)
- } catch {
- case e => ()
+
+ instantiatedSessionActors must have size (0)
+ }
+
+ "be able to unregister" in {
+ remote.registerPerSession("my-service-1", actorOf[RemoteStatefullSessionActorSpec])
+ remote.asInstanceOf[NettyRemoteSupport].actorsFactories.get("my-service-1") must not be (null)
+ remote.unregisterPerSession("my-service-1")
+ remote.asInstanceOf[NettyRemoteSupport].actorsFactories.get("my-service-1") must be (null)
}
}
-
- "A remote session Actor" should "create a new session actor per connection" in {
- clearMessageLogs
-
- val session1 = RemoteClient.actorFor(
- "untyped-session-actor-service",
- 5000L,
- HOSTNAME, PORT)
-
- val default1 = session1 !! GetUser()
- default1.get.asInstanceOf[String] should equal ("anonymous")
- session1 ! Login("session[1]")
- val result1 = session1 !! GetUser()
- result1.get.asInstanceOf[String] should equal ("session[1]")
-
- session1.stop()
-
- RemoteClient.shutdownAll
-
- //RemoteClient.clientFor(HOSTNAME, PORT).connect
-
- val session2 = RemoteClient.actorFor(
- "untyped-session-actor-service",
- 5000L,
- HOSTNAME, PORT)
-
- // since this is a new session, the server should reset the state
- val default2 = session2 !! GetUser()
- default2.get.asInstanceOf[String] should equal ("anonymous")
-
- session2.stop()
-
- }
-
- it should "stop the actor when the client disconnects" in {
-
- val session1 = RemoteClient.actorFor(
- "untyped-session-actor-service",
- 5000L,
- HOSTNAME, PORT)
-
-
- val default1 = session1 !! GetUser()
- default1.get.asInstanceOf[String] should equal ("anonymous")
-
- instantiatedSessionActors should have size (1)
-
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- instantiatedSessionActors should have size (0)
-
- }
-
- it should "stop the actor when there is an error" in {
-
- val session1 = RemoteClient.actorFor(
- "untyped-session-actor-service",
- 5000L,
- HOSTNAME, PORT)
-
-
- session1 ! DoSomethingFunny()
- session1.stop()
-
- Thread.sleep(1000)
-
- instantiatedSessionActors should have size (0)
- }
-
-
- it should "be able to unregister" in {
- server.registerPerSession("my-service-1", actorOf[RemoteStatefullSessionActorSpec])
- server.actorsFactories.get("my-service-1") should not be (null)
- server.unregisterPerSession("my-service-1")
- server.actorsFactories.get("my-service-1") should be (null)
- }
-
}
diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala
index 71de2a58fd..48520e2b34 100644
--- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala
@@ -4,133 +4,95 @@
package akka.actor.remote
-import org.scalatest.Spec
-import org.scalatest.matchers.ShouldMatchers
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.junit.JUnitRunner
-import org.junit.runner.RunWith
-
import java.util.concurrent.TimeUnit
-import akka.remote.{RemoteServer, RemoteClient}
import akka.actor._
import RemoteTypedActorLog._
-object ServerInitiatedRemoteTypedActorSpec {
- val HOSTNAME = "localhost"
- val PORT = 9990
- var server: RemoteServer = null
-}
-
-@RunWith(classOf[JUnitRunner])
-class ServerInitiatedRemoteTypedActorSpec extends
- Spec with
- ShouldMatchers with
- BeforeAndAfterAll {
- import ServerInitiatedRemoteTypedActorSpec._
-
- private val unit = TimeUnit.MILLISECONDS
-
-
- override def beforeAll = {
- server = new RemoteServer()
- server.start(HOSTNAME, PORT)
+class ServerInitiatedRemoteTypedActorSpec extends AkkaRemoteTest {
+ override def beforeEach = {
+ super.beforeEach
val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
- server.registerTypedActor("typed-actor-service", typedActor)
-
- Thread.sleep(1000)
+ remote.registerTypedActor("typed-actor-service", typedActor)
}
- // make sure the servers shutdown cleanly after the test has finished
- override def afterAll = {
- try {
- server.shutdown
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- } catch {
- case e => ()
- }
+ override def afterEach {
+ super.afterEach
+ clearMessageLogs
}
- describe("Server managed remote typed Actor ") {
+ def createRemoteActorRef = remote.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, host, port)
- it("should receive one-way message") {
- clearMessageLogs
- val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT)
- expect("oneway") {
- actor.oneWay
- oneWayLog.poll(5, TimeUnit.SECONDS)
- }
+ "Server managed remote typed Actor " should {
+
+ "receive one-way message" in {
+ val actor = createRemoteActorRef
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS) must equal ("oneway")
}
- it("should respond to request-reply message") {
- clearMessageLogs
- val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT)
- expect("pong") {
- actor.requestReply("ping")
- }
+ "should respond to request-reply message" in {
+ val actor = createRemoteActorRef
+ actor.requestReply("ping") must equal ("pong")
}
- it("should not recreate registered actors") {
- val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT)
- val numberOfActorsInRegistry = ActorRegistry.actors.length
- expect("oneway") {
- actor.oneWay
- oneWayLog.poll(5, TimeUnit.SECONDS)
- }
- assert(numberOfActorsInRegistry === ActorRegistry.actors.length)
+ "should not recreate registered actors" in {
+ val actor = createRemoteActorRef
+ val numberOfActorsInRegistry = Actor.registry.actors.length
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS) must equal ("oneway")
+ numberOfActorsInRegistry must be (Actor.registry.actors.length)
}
- it("should support multiple variants to get the actor from client side") {
- var actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT)
- expect("oneway") {
- actor.oneWay
- oneWayLog.poll(5, TimeUnit.SECONDS)
- }
- actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", HOSTNAME, PORT)
- expect("oneway") {
- actor.oneWay
- oneWayLog.poll(5, TimeUnit.SECONDS)
- }
- actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT, this.getClass().getClassLoader)
- expect("oneway") {
- actor.oneWay
- oneWayLog.poll(5, TimeUnit.SECONDS)
- }
+ "should support multiple variants to get the actor from client side" in {
+ var actor = createRemoteActorRef
+
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS) must equal ("oneway")
+
+ actor = remote.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", host, port)
+
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS) must equal ("oneway")
+
+ actor = remote.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, host, port, this.getClass().getClassLoader)
+
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS) must equal ("oneway")
}
- it("should register and unregister typed actors") {
+ "should register and unregister typed actors" in {
val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
- server.registerTypedActor("my-test-service", typedActor)
- assert(server.typedActors.get("my-test-service") ne null, "typed actor registered")
- server.unregisterTypedActor("my-test-service")
- assert(server.typedActors.get("my-test-service") eq null, "typed actor unregistered")
+ remote.registerTypedActor("my-test-service", typedActor)
+ remote.typedActors.get("my-test-service") must not be (null)
+ remote.unregisterTypedActor("my-test-service")
+ remote.typedActors.get("my-test-service") must be (null)
}
- it("should register and unregister typed actors by uuid") {
+ "should register and unregister typed actors by uuid" in {
val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
val init = AspectInitRegistry.initFor(typedActor)
val uuid = "uuid:" + init.actorRef.uuid
- server.registerTypedActor(uuid, typedActor)
- assert(server.typedActorsByUuid.get(init.actorRef.uuid.toString) ne null, "typed actor registered")
- server.unregisterTypedActor(uuid)
- assert(server.typedActorsByUuid.get(init.actorRef.uuid.toString) eq null, "typed actor unregistered")
+
+ remote.registerTypedActor(uuid, typedActor)
+ remote.typedActorsByUuid.get(init.actorRef.uuid.toString) must not be (null)
+
+ remote.unregisterTypedActor(uuid)
+ remote.typedActorsByUuid.get(init.actorRef.uuid.toString) must be (null)
}
- it("should find typed actors by uuid") {
+ "should find typed actors by uuid" in {
val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
val init = AspectInitRegistry.initFor(typedActor)
val uuid = "uuid:" + init.actorRef.uuid
- server.registerTypedActor(uuid, typedActor)
- assert(server.typedActorsByUuid.get(init.actorRef.uuid.toString) ne null, "typed actor registered")
- val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], uuid, HOSTNAME, PORT)
- expect("oneway") {
- actor.oneWay
- oneWayLog.poll(5, TimeUnit.SECONDS)
- }
+ remote.registerTypedActor(uuid, typedActor)
+ remote.typedActorsByUuid.get(init.actorRef.uuid.toString) must not be (null)
+ val actor = remote.typedActorFor(classOf[RemoteTypedActorOne], uuid, host, port)
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS) must equal ("oneway")
}
}
}
diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedSessionActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedSessionActorSpec.scala
index a5a635fa29..e357127641 100644
--- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedSessionActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedSessionActorSpec.scala
@@ -4,105 +4,72 @@
package akka.actor.remote
-import org.scalatest._
-import org.scalatest.matchers.ShouldMatchers
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.junit.JUnitRunner
-import org.junit.runner.RunWith
-
-import java.util.concurrent.TimeUnit
-
-import akka.remote.{RemoteServer, RemoteClient}
import akka.actor._
import RemoteTypedActorLog._
-object ServerInitiatedRemoteTypedSessionActorSpec {
- val HOSTNAME = "localhost"
- val PORT = 9990
- var server: RemoteServer = null
-}
-
-@RunWith(classOf[JUnitRunner])
-class ServerInitiatedRemoteTypedSessionActorSpec extends
- FlatSpec with
- ShouldMatchers with
- BeforeAndAfterEach {
- import ServerInitiatedRemoteTypedActorSpec._
-
- private val unit = TimeUnit.MILLISECONDS
+class ServerInitiatedRemoteTypedSessionActorSpec extends AkkaRemoteTest {
override def beforeEach = {
- server = new RemoteServer()
- server.start(HOSTNAME, PORT)
+ super.beforeEach
- server.registerTypedPerSessionActor("typed-session-actor-service",
+ remote.registerTypedPerSessionActor("typed-session-actor-service",
TypedActor.newInstance(classOf[RemoteTypedSessionActor], classOf[RemoteTypedSessionActorImpl], 1000))
-
- Thread.sleep(1000)
}
// make sure the servers shutdown cleanly after the test has finished
override def afterEach = {
- try {
- server.shutdown
- RemoteClient.shutdownAll
+ super.afterEach
+ clearMessageLogs
+ }
+
+ "A remote session Actor" should {
+ "create a new session actor per connection" in {
+
+ val session1 = remote.typedActorFor(classOf[RemoteTypedSessionActor], "typed-session-actor-service", 5000L, host, port)
+
+ session1.getUser() must equal ("anonymous")
+ session1.login("session[1]")
+ session1.getUser() must equal ("session[1]")
+
+ remote.shutdownClientModule
+
+ val session2 = remote.typedActorFor(classOf[RemoteTypedSessionActor], "typed-session-actor-service", 5000L, host, port)
+
+ session2.getUser() must equal ("anonymous")
+
+ }
+
+ "stop the actor when the client disconnects" in {
+ val session1 = remote.typedActorFor(classOf[RemoteTypedSessionActor], "typed-session-actor-service", 5000L, host, port)
+
+ session1.getUser() must equal ("anonymous")
+
+ RemoteTypedSessionActorImpl.getInstances() must have size (1)
+ remote.shutdownClientModule
Thread.sleep(1000)
- } catch {
- case e => ()
+ RemoteTypedSessionActorImpl.getInstances() must have size (0)
+
+ }
+
+ "stop the actor when there is an error" in {
+ val session1 = remote.typedActorFor(classOf[RemoteTypedSessionActor], "typed-session-actor-service", 5000L, host, port)
+
+ session1.doSomethingFunny()
+
+ remote.shutdownClientModule
+ Thread.sleep(1000)
+ RemoteTypedSessionActorImpl.getInstances() must have size (0)
+ }
+
+
+ "be able to unregister" in {
+ remote.registerTypedPerSessionActor("my-service-1",TypedActor.newInstance(classOf[RemoteTypedSessionActor], classOf[RemoteTypedSessionActorImpl], 1000))
+
+ remote.typedActorsFactories.get("my-service-1") must not be (null)
+ remote.unregisterTypedPerSessionActor("my-service-1")
+ remote.typedActorsFactories.get("my-service-1") must be (null)
}
}
-
- "A remote session Actor" should "create a new session actor per connection" in {
- clearMessageLogs
-
- val session1 = RemoteClient.typedActorFor(classOf[RemoteTypedSessionActor], "typed-session-actor-service", 5000L, HOSTNAME, PORT)
-
- session1.getUser() should equal ("anonymous")
- session1.login("session[1]")
- session1.getUser() should equal ("session[1]")
-
- RemoteClient.shutdownAll
-
- val session2 = RemoteClient.typedActorFor(classOf[RemoteTypedSessionActor], "typed-session-actor-service", 5000L, HOSTNAME, PORT)
-
- session2.getUser() should equal ("anonymous")
-
- }
-
- it should "stop the actor when the client disconnects" in {
-
- val session1 = RemoteClient.typedActorFor(classOf[RemoteTypedSessionActor], "typed-session-actor-service", 5000L, HOSTNAME, PORT)
-
- session1.getUser() should equal ("anonymous")
-
- RemoteTypedSessionActorImpl.getInstances() should have size (1)
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- RemoteTypedSessionActorImpl.getInstances() should have size (0)
-
- }
-
- it should "stop the actor when there is an error" in {
-
- val session1 = RemoteClient.typedActorFor(classOf[RemoteTypedSessionActor], "typed-session-actor-service", 5000L, HOSTNAME, PORT)
-
- session1.doSomethingFunny()
-
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- RemoteTypedSessionActorImpl.getInstances() should have size (0)
-
- }
-
-
- it should "be able to unregister" in {
- server.registerTypedPerSessionActor("my-service-1",TypedActor.newInstance(classOf[RemoteTypedSessionActor], classOf[RemoteTypedSessionActorImpl], 1000))
-
- server.typedActorsFactories.get("my-service-1") should not be (null)
- server.unregisterTypedPerSessionActor("my-service-1")
- server.typedActorsFactories.get("my-service-1") should be (null)
- }
-
}
diff --git a/akka-remote/src/test/scala/remote/ShutdownSpec.scala b/akka-remote/src/test/scala/remote/ShutdownSpec.scala
deleted file mode 100644
index 3e11ac1c5f..0000000000
--- a/akka-remote/src/test/scala/remote/ShutdownSpec.scala
+++ /dev/null
@@ -1,39 +0,0 @@
-package akka.remote
-
-import akka.actor.Actor
-
-import Actor._
-
-object ActorShutdownRunner {
- def main(args: Array[String]) {
- class MyActor extends Actor {
- def receive = {
- case "test" => println("received test")
- case m@_ => println("received unknown message " + m)
- }
- }
-
- val myActor = actorOf[MyActor]
- myActor.start
- myActor ! "test"
- myActor.stop
- }
-}
-
-
-// case 2
-
-object RemoteServerAndClusterShutdownRunner {
- def main(args: Array[String]) {
- val s1 = new RemoteServer
- val s2 = new RemoteServer
- val s3 = new RemoteServer
- s1.start("localhost", 2552)
- s2.start("localhost", 9998)
- s3.start("localhost", 9997)
- Thread.sleep(5000)
- s1.shutdown
- s2.shutdown
- s3.shutdown
- }
-}
diff --git a/akka-remote/src/test/scala/remote/UnOptimizedLocalScopedSpec.scala b/akka-remote/src/test/scala/remote/UnOptimizedLocalScopedSpec.scala
new file mode 100644
index 0000000000..001a66eae0
--- /dev/null
+++ b/akka-remote/src/test/scala/remote/UnOptimizedLocalScopedSpec.scala
@@ -0,0 +1,28 @@
+package akka.actor.remote
+import akka.actor. {ActorRegistry, Actor}
+
+object UnOptimizedLocalScopedSpec {
+ class TestActor extends Actor {
+ def receive = { case _ => }
+ }
+}
+
+class UnOptimizedLocalScopedSpec extends AkkaRemoteTest {
+ import UnOptimizedLocalScopedSpec._
+ override def OptimizeLocal = false
+
+ "An enabled optimized local scoped remote" should {
+ "Fetch remote actor ref when scope is local" in {
+ val fooActor = Actor.actorOf[TestActor].start
+ remote.register("foo", fooActor)
+
+ remote.actorFor("foo", host, port) must not be (fooActor)
+ }
+
+ "Create remote actor when client-managed is hosted locally" in {
+ val localClientManaged = Actor.remote.actorOf[TestActor](host, port)
+ localClientManaged.homeAddress must not be (None)
+ }
+
+ }
+}
\ No newline at end of file
diff --git a/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala b/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala
index c6d8baed55..2ae30c3b64 100644
--- a/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala
+++ b/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala
@@ -1,13 +1,9 @@
package akka.actor.serialization
-import java.util.concurrent.TimeUnit
-import org.scalatest.junit.JUnitSuite
-import org.junit.{Test, Before, After}
-
-import akka.remote.{RemoteServer, RemoteClient}
import akka.actor.{ProtobufProtocol, Actor}
import ProtobufProtocol.ProtobufPOJO
import Actor._
+import akka.actor.remote.AkkaRemoteTest
/* ---------------------------
Uses this Protobuf message:
@@ -20,11 +16,6 @@ message ProtobufPOJO {
--------------------------- */
object ProtobufActorMessageSerializationSpec {
- val unit = TimeUnit.MILLISECONDS
- val HOSTNAME = "localhost"
- val PORT = 9990
- var server: RemoteServer = null
-
class RemoteActorSpecActorBidirectional extends Actor {
def receive = {
case pojo: ProtobufPOJO =>
@@ -36,35 +27,16 @@ object ProtobufActorMessageSerializationSpec {
}
}
-class ProtobufActorMessageSerializationSpec extends JUnitSuite {
+class ProtobufActorMessageSerializationSpec extends AkkaRemoteTest {
import ProtobufActorMessageSerializationSpec._
- @Before
- def init() {
- server = new RemoteServer
- server.start(HOSTNAME, PORT)
- server.register("RemoteActorSpecActorBidirectional", actorOf[RemoteActorSpecActorBidirectional])
- Thread.sleep(1000)
- }
-
- // make sure the servers postStop cleanly after the test has finished
- @After
- def finished() {
- server.shutdown
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- }
-
- @Test
- def shouldSendReplyAsync {
- val actor = RemoteClient.actorFor("RemoteActorSpecActorBidirectional", 5000L, HOSTNAME, PORT)
- val result = actor !! ProtobufPOJO.newBuilder
- .setId(11)
- .setStatus(true)
- .setName("Coltrane")
- .build
- assert(12L === result.get.asInstanceOf[Long])
- actor.stop
+ "A ProtobufMessage" should {
+ "SendReplyAsync" in {
+ remote.register("RemoteActorSpecActorBidirectional",actorOf[RemoteActorSpecActorBidirectional])
+ val actor = remote.actorFor("RemoteActorSpecActorBidirectional", 5000L, host, port)
+ val result = actor !! ProtobufPOJO.newBuilder.setId(11).setStatus(true).setName("Coltrane").build
+ result.as[Long] must equal (Some(12))
+ }
}
}
diff --git a/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala b/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala
index 7026d1cfc0..02b29e6de1 100644
--- a/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala
+++ b/akka-remote/src/test/scala/serialization/ScalaJSONSerializerSpec.scala
@@ -7,6 +7,7 @@ import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
import akka.serialization.Serializer.ScalaJSON
+//TODO: FIXME WHY IS THIS COMMENTED OUT?
/*
object Protocols {
import sjson.json.DefaultProtocol._
diff --git a/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala b/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala
index ae5ef6d786..78ba95d60a 100644
--- a/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala
+++ b/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala
@@ -1,48 +1,31 @@
/**
* Copyright (C) 2009-2011 Scalable Solutions AB
*/
-package akka.actor.serialization
-import org.scalatest.Spec
-import org.scalatest.matchers.ShouldMatchers
-import org.scalatest.BeforeAndAfterAll
-import org.scalatest.junit.JUnitRunner
-import org.junit.runner.RunWith
+
+package akka.actor.serialization
import akka.serialization._
import akka.actor._
import TypedActorSerialization._
import Actor._
-import akka.remote.{RemoteClient, RemoteServer}
import akka.actor.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional
+import akka.actor.remote.AkkaRemoteTest
-@RunWith(classOf[JUnitRunner])
-class TypedActorSerializationSpec extends
- Spec with
- ShouldMatchers with
- BeforeAndAfterAll {
-
- var server1: RemoteServer = null
+class TypedActorSerializationSpec extends AkkaRemoteTest {
var typedActor: MyTypedActor = null
override def beforeAll = {
- server1 = new RemoteServer().start("localhost", 9991)
+ super.beforeAll
typedActor = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyTypedActorImpl], 1000)
- server1.registerTypedActor("typed-actor-service", typedActor)
- Thread.sleep(1000)
+ remote.registerTypedActor("typed-actor-service", typedActor)
}
// make sure the servers shutdown cleanly after the test has finished
override def afterAll = {
- try {
- TypedActor.stop(typedActor)
- server1.shutdown
- RemoteClient.shutdownAll
- Thread.sleep(1000)
- } catch {
- case e => ()
- }
+ TypedActor.stop(typedActor)
+ super.afterAll
}
object MyTypedStatelessActorFormat extends StatelessActorFormat[MyStatelessTypedActorImpl]
@@ -69,48 +52,48 @@ class TypedActorSerializationSpec extends
}
- describe("Serializable typed actor") {
+ "Serializable typed actor" should {
- it("should be able to serialize and de-serialize a stateless typed actor") {
+ "should be able to serialize and de-serialize a stateless typed actor" in {
val typedActor1 = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyStatelessTypedActorImpl], 1000)
- typedActor1.requestReply("hello") should equal("world")
- typedActor1.requestReply("hello") should equal("world")
+ typedActor1.requestReply("hello") must equal("world")
+ typedActor1.requestReply("hello") must equal("world")
val bytes = toBinaryJ(typedActor1, MyTypedStatelessActorFormat)
val typedActor2: MyTypedActor = fromBinaryJ(bytes, MyTypedStatelessActorFormat)
- typedActor2.requestReply("hello") should equal("world")
+ typedActor2.requestReply("hello") must equal("world")
}
- it("should be able to serialize and de-serialize a stateful typed actor") {
+ "should be able to serialize and de-serialize a stateful typed actor" in {
val typedActor1 = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyTypedActorImpl], 1000)
- typedActor1.requestReply("hello") should equal("world 1")
- typedActor1.requestReply("scala") should equal("hello scala 2")
+ typedActor1.requestReply("hello") must equal("world 1")
+ typedActor1.requestReply("scala") must equal("hello scala 2")
val f = new MyTypedActorFormat
val bytes = toBinaryJ(typedActor1, f)
val typedActor2: MyTypedActor = fromBinaryJ(bytes, f)
- typedActor2.requestReply("hello") should equal("world 3")
+ typedActor2.requestReply("hello") must equal("world 3")
}
- it("should be able to serialize and de-serialize a stateful typed actor with compound state") {
+ "should be able to serialize and de-serialize a stateful typed actor with compound state" in {
val typedActor1 = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyTypedActorWithDualCounter], 1000)
- typedActor1.requestReply("hello") should equal("world 1 1")
- typedActor1.requestReply("hello") should equal("world 2 2")
+ typedActor1.requestReply("hello") must equal("world 1 1")
+ typedActor1.requestReply("hello") must equal("world 2 2")
val f = new MyTypedActorWithDualCounterFormat
val bytes = toBinaryJ(typedActor1, f)
val typedActor2: MyTypedActor = fromBinaryJ(bytes, f)
- typedActor2.requestReply("hello") should equal("world 3 3")
+ typedActor2.requestReply("hello") must equal("world 3 3")
}
- it("should be able to serialize a local yped actor ref to a remote typed actor ref proxy") {
+ "should be able to serialize a local yped actor ref to a remote typed actor ref proxy" in {
val typedActor1 = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyStatelessTypedActorImpl], 1000)
- typedActor1.requestReply("hello") should equal("world")
- typedActor1.requestReply("hello") should equal("world")
+ typedActor1.requestReply("hello") must equal("world")
+ typedActor1.requestReply("hello") must equal("world")
val bytes = RemoteTypedActorSerialization.toBinary(typedActor1)
val typedActor2: MyTypedActor = RemoteTypedActorSerialization.fromBinaryToRemoteTypedActorRef(bytes)
- typedActor1.requestReply("hello") should equal("world")
+ typedActor1.requestReply("hello") must equal("world")
}
}
}
diff --git a/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala b/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala
index e844ff0104..b9752d2d55 100644
--- a/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala
+++ b/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala
@@ -43,7 +43,7 @@ class UntypedActorSerializationSpec extends
describe("Serializable untyped actor") {
it("should be able to serialize and de-serialize a stateful untyped actor") {
- val actor1 = UntypedActor.actorOf[MyUntypedActor](classOf[MyUntypedActor]).start
+ val actor1 = Actors.actorOf(classOf[MyUntypedActor]).start
actor1.sendRequestReply("hello") should equal("world 1")
actor1.sendRequestReply("debasish") should equal("hello debasish 2")
diff --git a/akka-remote/src/test/scala/ticket/Ticket434Spec.scala b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala
index 35501e5047..971f200340 100644
--- a/akka-remote/src/test/scala/ticket/Ticket434Spec.scala
+++ b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala
@@ -3,49 +3,39 @@
*/
package akka.actor.ticket
-import org.scalatest.Spec
-import org.scalatest.matchers.ShouldMatchers
+
import akka.actor.Actor._
import akka.actor.{Uuid,newUuid,uuidFrom}
import akka.actor.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional
-import java.util.concurrent.TimeUnit
-import akka.remote.{RemoteClient, RemoteServer}
import akka.remote.protocol.RemoteProtocol._
+import akka.actor.remote.AkkaRemoteTest
+import java.util.concurrent.CountDownLatch
+class Ticket434Spec extends AkkaRemoteTest {
+ "A server managed remote actor" should {
+ "can use a custom service name containing ':'" in {
+ val latch = new CountDownLatch(1)
+ implicit val sender = replyHandler(latch,"Pong")
+ remote.register("my:service", actorOf[RemoteActorSpecActorUnidirectional])
-class Ticket434Spec extends Spec with ShouldMatchers {
+ val actor = remote.actorFor("my:service", 5000L, host, port)
- val HOSTNAME = "localhost"
- val PORT = 9991
+ actor ! "Ping"
- describe("A server managed remote actor") {
- it("can use a custom service name containing ':'") {
- val server = new RemoteServer().start(HOSTNAME, PORT)
- server.register("my:service", actorOf[RemoteActorSpecActorUnidirectional])
-
- val actor = RemoteClient.actorFor("my:service", 5000L, HOSTNAME, PORT)
- actor ! "OneWay"
-
- assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS))
- actor.stop
-
- server.shutdown
- RemoteClient.shutdownAll
+ latch.await(1, unit) must be (true)
}
- }
- describe("The ActorInfoProtocol") {
- it("should be possible to set the acor id and uuuid") {
+ "should be possible to set the acor id and uuuid" in {
val uuid = newUuid
- val actorInfoBuilder = ActorInfoProtocol.newBuilder
+ val actorInfo = ActorInfoProtocol.newBuilder
.setUuid(UuidProtocol.newBuilder.setHigh(uuid.getTime).setLow(uuid.getClockSeqAndNode).build)
.setId("some-id")
.setTarget("actorClassName")
.setTimeout(5000L)
- .setActorType(ActorType.SCALA_ACTOR)
- val actorInfo = actorInfoBuilder.build
- assert(uuidFrom(actorInfo.getUuid.getHigh,actorInfo.getUuid.getLow) === uuid)
- assert(actorInfo.getId === "some-id")
+ .setActorType(ActorType.SCALA_ACTOR).build
+
+ uuidFrom(actorInfo.getUuid.getHigh,actorInfo.getUuid.getLow) must equal (uuid)
+ actorInfo.getId must equal ("some-id")
}
}
}
diff --git a/akka-remote/src/test/scala/ticket/Ticket506Spec.scala b/akka-remote/src/test/scala/ticket/Ticket506Spec.scala
index e82d4c5efb..cd58c4a6bd 100644
--- a/akka-remote/src/test/scala/ticket/Ticket506Spec.scala
+++ b/akka-remote/src/test/scala/ticket/Ticket506Spec.scala
@@ -1,61 +1,43 @@
package ticket
-import org.scalatest.Spec
-import org.scalatest.matchers.ShouldMatchers
-
-import akka.remote.{RemoteClient, RemoteNode, RemoteServer}
import akka.actor.{Actor, ActorRef}
import akka.serialization.RemoteActorSerialization
import akka.actor.Actor.actorOf
import java.util.concurrent.{CountDownLatch, TimeUnit}
-
-object State {
- val latch = new CountDownLatch(1)
-}
+import akka.actor.remote.AkkaRemoteTest
case class RecvActorRef(bytes:Array[Byte])
-class ActorRefService extends Actor {
+class ActorRefService(latch: CountDownLatch) extends Actor {
import self._
def receive:Receive = {
case RecvActorRef(bytes) =>
val ref = RemoteActorSerialization.fromBinaryToRemoteActorRef(bytes)
ref ! "hello"
- case "hello" =>
- State.latch.countDown
+ case "hello" => latch.countDown
}
}
-class Ticket506Spec extends Spec with ShouldMatchers {
- val hostname:String = "localhost"
- val port:Int = 9440
+class Ticket506Spec extends AkkaRemoteTest {
+ "a RemoteActorRef serialized" should {
+ "should be remotely usable" in {
- describe("a RemoteActorRef serialized") {
- it("should be remotely usable") {
- val s1,s2 = new RemoteServer
- s1.start(hostname, port)
- s2.start(hostname, port + 1)
+ val latch = new CountDownLatch(1)
+ val a1 = actorOf( new ActorRefService(null))
+ val a2 = actorOf( new ActorRefService(latch))
- val a1,a2 = actorOf[ActorRefService]
- a1.homeAddress = (hostname, port)
- a2.homeAddress = (hostname, port+1)
-
- s1.register("service", a1)
- s2.register("service", a2)
+ remote.register("service1", a1)
+ remote.register("service2", a2)
// connect to the first server/service
- val c1 = RemoteClient.actorFor("service", hostname, port)
+ val c1 = remote.actorFor("service1", host, port)
val bytes = RemoteActorSerialization.toRemoteActorRefProtocol(a2).toByteArray
c1 ! RecvActorRef(bytes)
- State.latch.await(1000, TimeUnit.MILLISECONDS) should be(true)
-
- RemoteClient.shutdownAll
- s1.shutdown
- s2.shutdown
+ latch.await(1, unit) must be(true)
}
}
}
diff --git a/akka-remote/src/test/scala/ticket/Ticket519Spec.scala b/akka-remote/src/test/scala/ticket/Ticket519Spec.scala
index 20e8bda00a..6edec702b5 100644
--- a/akka-remote/src/test/scala/ticket/Ticket519Spec.scala
+++ b/akka-remote/src/test/scala/ticket/Ticket519Spec.scala
@@ -3,28 +3,17 @@
*/
package akka.actor.ticket
-import org.scalatest.Spec
-import org.scalatest.matchers.ShouldMatchers
-import akka.remote.{RemoteClient, RemoteServer}
import akka.actor._
+import akka.actor.remote.AkkaRemoteTest
-class Ticket519Spec extends Spec with ShouldMatchers {
-
- val HOSTNAME = "localhost"
- val PORT = 6666
-
- describe("A remote TypedActor") {
- it("should handle remote future replies") {
- import akka.remote._
-
- val server = { val s = new RemoteServer; s.start(HOSTNAME,PORT); s}
- val actor = TypedActor.newRemoteInstance(classOf[SamplePojo], classOf[SamplePojoImpl],7000,HOSTNAME,PORT)
+class Ticket519Spec extends AkkaRemoteTest {
+ "A remote TypedActor" should {
+ "should handle remote future replies" in {
+ val actor = TypedActor.newRemoteInstance(classOf[SamplePojo], classOf[SamplePojoImpl],7000,host,port)
val r = actor.someFutureString
- r.await.result.get should equal ("foo")
- TypedActor.stop(actor)
- server.shutdown
+ r.await.result.get must equal ("foo")
}
}
}
diff --git a/akka-samples/akka-sample-remote/src/main/scala/ClientManagedRemoteActorSample.scala b/akka-samples/akka-sample-remote/src/main/scala/ClientManagedRemoteActorSample.scala
index d5a01ea272..f0adb42899 100644
--- a/akka-samples/akka-sample-remote/src/main/scala/ClientManagedRemoteActorSample.scala
+++ b/akka-samples/akka-sample-remote/src/main/scala/ClientManagedRemoteActorSample.scala
@@ -5,11 +5,11 @@
package sample.remote
import akka.actor.Actor._
-import akka.actor.RemoteActor
-import akka.remote.RemoteNode
import akka.util.Logging
+import akka.actor. {ActorRegistry, Actor}
+import Actor.remote
-class RemoteHelloWorldActor extends RemoteActor("localhost", 2552) {
+class RemoteHelloWorldActor extends Actor {
def receive = {
case "Hello" =>
log.slf4j.info("Received 'Hello'")
@@ -19,7 +19,7 @@ class RemoteHelloWorldActor extends RemoteActor("localhost", 2552) {
object ClientManagedRemoteActorServer extends Logging {
def run = {
- RemoteNode.start("localhost", 2552)
+ remote.start("localhost", 2552)
log.slf4j.info("Remote node started")
}
@@ -29,7 +29,7 @@ object ClientManagedRemoteActorServer extends Logging {
object ClientManagedRemoteActorClient extends Logging {
def run = {
- val actor = actorOf[RemoteHelloWorldActor].start
+ val actor = remote.actorOf[RemoteHelloWorldActor]("localhost",2552).start
log.slf4j.info("Remote actor created, moved to the server")
log.slf4j.info("Sending 'Hello' to remote actor")
val result = actor !! "Hello"
diff --git a/akka-samples/akka-sample-remote/src/main/scala/ServerManagedRemoteActorSample.scala b/akka-samples/akka-sample-remote/src/main/scala/ServerManagedRemoteActorSample.scala
index 0e599fe3c7..96e8d1debf 100644
--- a/akka-samples/akka-sample-remote/src/main/scala/ServerManagedRemoteActorSample.scala
+++ b/akka-samples/akka-sample-remote/src/main/scala/ServerManagedRemoteActorSample.scala
@@ -4,10 +4,9 @@
package sample.remote
-import akka.actor.Actor
import akka.actor.Actor._
-import akka.remote.{RemoteClient, RemoteNode}
import akka.util.Logging
+import akka.actor. {ActorRegistry, Actor}
class HelloWorldActor extends Actor {
def receive = {
@@ -20,9 +19,9 @@ class HelloWorldActor extends Actor {
object ServerManagedRemoteActorServer extends Logging {
def run = {
- RemoteNode.start("localhost", 2552)
+ Actor.remote.start("localhost", 2552)
log.slf4j.info("Remote node started")
- RemoteNode.register("hello-service", actorOf[HelloWorldActor])
+ Actor.remote.register("hello-service", actorOf[HelloWorldActor])
log.slf4j.info("Remote actor registered and started")
}
@@ -32,7 +31,7 @@ object ServerManagedRemoteActorServer extends Logging {
object ServerManagedRemoteActorClient extends Logging {
def run = {
- val actor = RemoteClient.actorFor("hello-service", "localhost", 2552)
+ val actor = Actor.remote.actorFor("hello-service", "localhost", 2552)
log.slf4j.info("Remote client created")
log.slf4j.info("Sending 'Hello' to remote actor")
val result = actor !! "Hello"
diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java
index 649ef42d3b..658ec71ff4 100644
--- a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java
+++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java
@@ -2,7 +2,7 @@ package akka.transactor.example;
import akka.transactor.Coordinated;
import akka.transactor.Atomically;
-import akka.actor.ActorRef;
+import akka.actor.Actors;
import akka.actor.UntypedActor;
import akka.stm.Ref;
diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java
index d3a2a14107..127a911677 100644
--- a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java
+++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java
@@ -2,7 +2,7 @@ package akka.transactor.example;
import akka.transactor.Coordinated;
import akka.actor.ActorRef;
-import akka.actor.UntypedActor;
+import akka.actor.Actors;
import akka.dispatch.Future;
import akka.dispatch.Futures;
@@ -12,8 +12,8 @@ public class UntypedCoordinatedExample {
System.out.println("Untyped transactor example");
System.out.println();
- ActorRef counter1 = UntypedActor.actorOf(UntypedCoordinatedCounter.class).start();
- ActorRef counter2 = UntypedActor.actorOf(UntypedCoordinatedCounter.class).start();
+ ActorRef counter1 = Actors.actorOf(UntypedCoordinatedCounter.class).start();
+ ActorRef counter2 = Actors.actorOf(UntypedCoordinatedCounter.class).start();
counter1.sendOneWay(new Coordinated(new Increment(counter2)));
diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java b/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java
index fdef74ca60..d2f83ff25e 100644
--- a/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java
+++ b/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java
@@ -1,7 +1,7 @@
package akka.transactor.example;
import akka.actor.ActorRef;
-import akka.actor.UntypedActor;
+import akka.actor.Actors;
import akka.dispatch.Future;
import akka.dispatch.Futures;
@@ -11,8 +11,8 @@ public class UntypedTransactorExample {
System.out.println("Untyped transactor example");
System.out.println();
- ActorRef counter1 = UntypedActor.actorOf(UntypedCounter.class).start();
- ActorRef counter2 = UntypedActor.actorOf(UntypedCounter.class).start();
+ ActorRef counter1 = Actors.actorOf(UntypedCounter.class).start();
+ ActorRef counter2 = Actors.actorOf(UntypedCounter.class).start();
counter1.sendOneWay(new Increment(counter2));
diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java
index 9e36409728..6f09a10173 100644
--- a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java
+++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java
@@ -3,6 +3,7 @@ package akka.transactor.test;
import akka.transactor.Coordinated;
import akka.transactor.Atomically;
import akka.actor.ActorRef;
+import akka.actor.Actors;
import akka.actor.UntypedActor;
import akka.stm.*;
import akka.util.FiniteDuration;
diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java
index 35635ceb4f..a9eabe7be1 100644
--- a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java
+++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java
@@ -5,6 +5,7 @@ import org.junit.Test;
import org.junit.Before;
import akka.transactor.Coordinated;
+import akka.actor.Actors;
import akka.actor.ActorRef;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
@@ -28,7 +29,7 @@ public class UntypedCoordinatedIncrementTest {
counters = new ArrayList();
for (int i = 1; i <= numCounters; i++) {
final String name = "counter" + i;
- ActorRef counter = UntypedActor.actorOf(new UntypedActorFactory() {
+ ActorRef counter = Actors.actorOf(new UntypedActorFactory() {
public UntypedActor create() {
return new UntypedCoordinatedCounter(name);
}
@@ -36,7 +37,7 @@ public class UntypedCoordinatedIncrementTest {
counter.start();
counters.add(counter);
}
- failer = UntypedActor.actorOf(UntypedFailer.class);
+ failer = Actors.actorOf(UntypedFailer.class);
failer.start();
}
diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java
index e378b1c598..9610aa2ebe 100644
--- a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java
+++ b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java
@@ -5,6 +5,7 @@ import org.junit.Test;
import org.junit.Before;
import akka.actor.ActorRef;
+import akka.actor.Actors;
import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory;
import akka.dispatch.Future;
@@ -27,7 +28,7 @@ public class UntypedTransactorTest {
counters = new ArrayList();
for (int i = 1; i <= numCounters; i++) {
final String name = "counter" + i;
- ActorRef counter = UntypedActor.actorOf(new UntypedActorFactory() {
+ ActorRef counter = Actors.actorOf(new UntypedActorFactory() {
public UntypedActor create() {
return new UntypedCounter(name);
}
@@ -35,7 +36,7 @@ public class UntypedTransactorTest {
counter.start();
counters.add(counter);
}
- failer = UntypedActor.actorOf(UntypedFailer.class);
+ failer = Actors.actorOf(UntypedFailer.class);
failer.start();
}
diff --git a/akka-typed-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/akka/actor/TypedActor.scala
index d0f09b3130..45962ed7fa 100644
--- a/akka-typed-actor/src/main/scala/akka/actor/TypedActor.scala
+++ b/akka-typed-actor/src/main/scala/akka/actor/TypedActor.scala
@@ -348,7 +348,7 @@ final class TypedActorContext(private[akka] val actorRef: ActorRef) {
/**
* Returns the home address and port for this actor.
*/
- def homeAddress: InetSocketAddress = actorRef.homeAddress
+ def homeAddress: InetSocketAddress = actorRef.homeAddress.getOrElse(null)//TODO: REVISIT: Sensible to return null?
}
object TypedActorConfiguration {
@@ -529,21 +529,29 @@ object TypedActor extends Logging {
* @param factory factory method that constructs the typed actor
* @paramm config configuration object fo the typed actor
*/
- def newInstance[T](intfClass: Class[T], factory: => AnyRef, config: TypedActorConfiguration): T = {
- val actorRef = actorOf(newTypedActor(factory))
- newInstance(intfClass, actorRef, config)
+ def newInstance[T](intfClass: Class[T], factory: => AnyRef, config: TypedActorConfiguration): T =
+ newInstance(intfClass, createActorRef(newTypedActor(factory),config), config)
+
+ /**
+ * Creates an ActorRef, can be local only or client-managed-remote
+ */
+ private[akka] def createActorRef(typedActor: => TypedActor, config: TypedActorConfiguration): ActorRef = {
+ config match {
+ case null => actorOf(typedActor)
+ case c: TypedActorConfiguration if (c._host.isDefined) =>
+ Actor.remote.actorOf(typedActor, c._host.get.getHostName, c._host.get.getPort)
+ case _ => actorOf(typedActor)
+ }
}
/**
- * Factory method for typed actor.
+ * Factory method for typed actor.
* @param intfClass interface the typed actor implements
* @param targetClass implementation class of the typed actor
* @paramm config configuration object fo the typed actor
*/
- def newInstance[T](intfClass: Class[T], targetClass: Class[_], config: TypedActorConfiguration): T = {
- val actorRef = actorOf(newTypedActor(targetClass))
- newInstance(intfClass, actorRef, config)
- }
+ def newInstance[T](intfClass: Class[T], targetClass: Class[_], config: TypedActorConfiguration): T =
+ newInstance(intfClass, createActorRef(newTypedActor(targetClass),config), config)
private[akka] def newInstance[T](intfClass: Class[T], actorRef: ActorRef): T = {
if (!actorRef.actorInstance.get.isInstanceOf[TypedActor]) throw new IllegalArgumentException("ActorRef is not a ref to a typed actor")
@@ -563,10 +571,11 @@ object TypedActor extends Logging {
typedActor.initialize(proxy)
if (config._messageDispatcher.isDefined) actorRef.dispatcher = config._messageDispatcher.get
if (config._threadBasedDispatcher.isDefined) actorRef.dispatcher = Dispatchers.newThreadBasedDispatcher(actorRef)
- if (config._host.isDefined) actorRef.makeRemote(config._host.get)
if (config._id.isDefined) actorRef.id = config._id.get
+
actorRef.timeout = config.timeout
- AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, actorRef.remoteAddress, actorRef.timeout))
+ //log.slf4j.debug("config._host for {} is {} but homeAddress is {} and on ref {}",Array[AnyRef](intfClass, config._host, typedActor.context.homeAddress,actorRef.homeAddress))
+ AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, actorRef.homeAddress, actorRef.timeout))
actorRef.start
proxy.asInstanceOf[T]
}
@@ -634,7 +643,7 @@ object TypedActor extends Logging {
val jProxy = JProxy.newProxyInstance(intfClass.getClassLoader(), interfaces, handler)
val awProxy = Proxy.newInstance(interfaces, Array(jProxy, jProxy), true, false)
- AspectInitRegistry.register(awProxy, AspectInit(intfClass, null, actorRef, None, 5000L))
+ AspectInitRegistry.register(awProxy, AspectInit(intfClass, null, actorRef, actorRef.homeAddress, 5000L))
awProxy.asInstanceOf[T]
}
@@ -667,9 +676,10 @@ object TypedActor extends Logging {
* Get the underlying typed actor for the given Typed Actor.
*/
def actorFor(proxy: AnyRef): Option[ActorRef] =
- ActorRegistry
- .actorsFor(classOf[TypedActor])
- .find(a => a.actor.asInstanceOf[TypedActor].proxy == proxy)
+ Actor.registry find {
+ case a if a.actor.isInstanceOf[TypedActor] && a.actor.asInstanceOf[TypedActor].proxy == proxy =>
+ a
+ }
/**
* Get the typed actor proxy for the given Typed Actor.
@@ -803,7 +813,7 @@ private[akka] sealed class ServerManagedTypedActorAspect extends ActorAspect {
override def initialize(joinPoint: JoinPoint): Unit = {
super.initialize(joinPoint)
- remoteAddress = actorRef.remoteAddress
+ //remoteAddress = actorRef.remoteAddress //TODO: REVISIT: Fix Server managed Typed Actor
}
}
@@ -897,11 +907,12 @@ private[akka] abstract class ActorAspect {
val (message: Array[AnyRef], isEscaped) = escapeArguments(methodRtti.getParameterValues)
- val future = RemoteClientModule.send[AnyRef](
+ val future = Actor.remote.send[AnyRef](
message, None, None, remoteAddress.get,
timeout, isOneWay, actorRef,
Some((interfaceClass.getName, methodRtti.getMethod.getName)),
- ActorType.TypedActor)
+ ActorType.TypedActor,
+ None) //TODO: REVISIT: Use another classloader?
if (isOneWay) null // for void methods
else if (TypedActor.returnsFuture_?(methodRtti)) future.get
diff --git a/akka-typed-actor/src/main/scala/akka/config/TypedActorGuiceConfigurator.scala b/akka-typed-actor/src/main/scala/akka/config/TypedActorGuiceConfigurator.scala
index 74c1e8f25b..81bde84281 100644
--- a/akka-typed-actor/src/main/scala/akka/config/TypedActorGuiceConfigurator.scala
+++ b/akka-typed-actor/src/main/scala/akka/config/TypedActorGuiceConfigurator.scala
@@ -108,22 +108,21 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa
val implementationClass = component.target
val timeout = component.timeout
- val actorRef = Actor.actorOf(TypedActor.newTypedActor(implementationClass))
+ val (remoteAddress,actorRef) =
+ component.remoteAddress match {
+ case Some(a) =>
+ (Some(new InetSocketAddress(a.hostname, a.port)),
+ Actor.remote.actorOf(TypedActor.newTypedActor(implementationClass), a.hostname, a.port))
+ case None =>
+ (None, Actor.actorOf(TypedActor.newTypedActor(implementationClass)))
+ }
+
actorRef.timeout = timeout
if (component.dispatcher.isDefined) actorRef.dispatcher = component.dispatcher.get
val typedActor = actorRef.actorInstance.get.asInstanceOf[TypedActor]
val proxy = Proxy.newInstance(Array(interfaceClass), Array(typedActor), true, false)
- val remoteAddress =
- if (component.remoteAddress.isDefined)
- Some(new InetSocketAddress(component.remoteAddress.get.hostname, component.remoteAddress.get.port))
- else None
-
- remoteAddress.foreach { address =>
- actorRef.makeRemote(remoteAddress.get)
- }
-
AspectInitRegistry.register(
proxy,
AspectInit(interfaceClass, typedActor, actorRef, remoteAddress, timeout))
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
index a8dd7a75ad..9790c3657e 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
@@ -112,11 +112,11 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft
}
}
- it("should postStop non-supervised, annotated typed actor on ActorRegistry.shutdownAll") {
+ it("should postStop non-supervised, annotated typed actor on Actor.registry.shutdownAll") {
val obj = TypedActor.newInstance(classOf[SamplePojoAnnotated])
assert(AspectInitRegistry.initFor(obj) ne null)
assert("hello akka" === obj.greet("akka"))
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
assert(AspectInitRegistry.initFor(obj) eq null)
assert(!obj.pre)
assert(!obj.post)
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorRegistrySpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorRegistrySpec.scala
index cbbb6c169d..0a031026ef 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorRegistrySpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorRegistrySpec.scala
@@ -17,59 +17,59 @@ class TypedActorRegistrySpec extends WordSpec with MustMatchers {
"Typed Actor" should {
"be able to be retreived from the registry by class" in {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val my = TypedActor.newInstance[My](classOf[My], classOf[MyImpl], 3000)
- val actors = ActorRegistry.typedActorsFor(classOf[My])
+ val actors = Actor.registry.typedActorsFor(classOf[My])
actors.length must be (1)
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
}
"be able to be retreived from the registry by manifest" in {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val my = TypedActor.newInstance[My](classOf[My], classOf[MyImpl], 3000)
- val option = ActorRegistry.typedActorFor[My]
+ val option = Actor.registry.typedActorFor[My]
option must not be (null)
option.isDefined must be (true)
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
}
"be able to be retreived from the registry by class two times" in {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val my = TypedActor.newInstance[My](classOf[My], classOf[MyImpl], 3000)
- val actors1 = ActorRegistry.typedActorsFor(classOf[My])
+ val actors1 = Actor.registry.typedActorsFor(classOf[My])
actors1.length must be (1)
- val actors2 = ActorRegistry.typedActorsFor(classOf[My])
+ val actors2 = Actor.registry.typedActorsFor(classOf[My])
actors2.length must be (1)
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
}
"be able to be retreived from the registry by manifest two times" in {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val my = TypedActor.newInstance[My](classOf[My], classOf[MyImpl], 3000)
- val option1 = ActorRegistry.typedActorFor[My]
+ val option1 = Actor.registry.typedActorFor[My]
option1 must not be (null)
option1.isDefined must be (true)
- val option2 = ActorRegistry.typedActorFor[My]
+ val option2 = Actor.registry.typedActorFor[My]
option2 must not be (null)
option2.isDefined must be (true)
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
}
"be able to be retreived from the registry by manifest two times (even when created in supervisor)" in {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
val manager = new TypedActorConfigurator
manager.configure(
OneForOneStrategy(classOf[Exception] :: Nil, 3, 1000),
Array(new SuperviseTypedActor(classOf[My], classOf[MyImpl], Permanent, 6000))
).supervise
- val option1 = ActorRegistry.typedActorFor[My]
+ val option1 = Actor.registry.typedActorFor[My]
option1 must not be (null)
option1.isDefined must be (true)
- val option2 = ActorRegistry.typedActorFor[My]
+ val option2 = Actor.registry.typedActorFor[My]
option2 must not be (null)
option2.isDefined must be (true)
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
}
}
}
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala
index 12e9691de7..863a1bb064 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorSpec.scala
@@ -68,7 +68,7 @@ class TypedActorSpec extends
}
override def afterEach() {
- ActorRegistry.shutdownAll
+ Actor.registry.shutdownAll
}
describe("TypedActor") {
@@ -115,67 +115,67 @@ class TypedActorSpec extends
it("should support finding a typed actor by uuid ") {
val typedActorRef = TypedActor.actorFor(simplePojo).get
val uuid = typedActorRef.uuid
- assert(ActorRegistry.typedActorFor(newUuid()) === None)
- assert(ActorRegistry.typedActorFor(uuid).isDefined)
- assert(ActorRegistry.typedActorFor(uuid).get === simplePojo)
+ assert(Actor.registry.typedActorFor(newUuid()) === None)
+ assert(Actor.registry.typedActorFor(uuid).isDefined)
+ assert(Actor.registry.typedActorFor(uuid).get === simplePojo)
}
it("should support finding typed actors by id ") {
- val typedActors = ActorRegistry.typedActorsFor("my-custom-id")
+ val typedActors = Actor.registry.typedActorsFor("my-custom-id")
assert(typedActors.length === 1)
assert(typedActors.contains(pojo))
// creating untyped actor with same custom id
val actorRef = Actor.actorOf[MyActor].start
- val typedActors2 = ActorRegistry.typedActorsFor("my-custom-id")
+ val typedActors2 = Actor.registry.typedActorsFor("my-custom-id")
assert(typedActors2.length === 1)
assert(typedActors2.contains(pojo))
actorRef.stop
}
it("should support to filter typed actors") {
- val actors = ActorRegistry.filterTypedActors(ta => ta.isInstanceOf[MyTypedActor])
+ val actors = Actor.registry.filterTypedActors(ta => ta.isInstanceOf[MyTypedActor])
assert(actors.length === 1)
assert(actors.contains(pojo))
}
it("should support to find typed actors by class") {
- val actors = ActorRegistry.typedActorsFor(classOf[MyTypedActorImpl])
+ val actors = Actor.registry.typedActorsFor(classOf[MyTypedActorImpl])
assert(actors.length === 1)
assert(actors.contains(pojo))
- assert(ActorRegistry.typedActorsFor(classOf[MyActor]).isEmpty)
+ assert(Actor.registry.typedActorsFor(classOf[MyActor]).isEmpty)
}
it("should support to get all typed actors") {
- val actors = ActorRegistry.typedActors
+ val actors = Actor.registry.typedActors
assert(actors.length === 2)
assert(actors.contains(pojo))
assert(actors.contains(simplePojo))
}
it("should support to find typed actors by manifest") {
- val actors = ActorRegistry.typedActorsFor[MyTypedActorImpl]
+ val actors = Actor.registry.typedActorsFor[MyTypedActorImpl]
assert(actors.length === 1)
assert(actors.contains(pojo))
- assert(ActorRegistry.typedActorsFor[MyActor].isEmpty)
+ assert(Actor.registry.typedActorsFor[MyActor].isEmpty)
}
it("should support foreach for typed actors") {
val actorRef = Actor.actorOf[MyActor].start
- assert(ActorRegistry.actors.size === 3)
- assert(ActorRegistry.typedActors.size === 2)
- ActorRegistry.foreachTypedActor(TypedActor.stop(_))
- assert(ActorRegistry.actors.size === 1)
- assert(ActorRegistry.typedActors.size === 0)
+ assert(Actor.registry.actors.size === 3)
+ assert(Actor.registry.typedActors.size === 2)
+ Actor.registry.foreachTypedActor(TypedActor.stop(_))
+ assert(Actor.registry.actors.size === 1)
+ assert(Actor.registry.typedActors.size === 0)
}
it("should shutdown all typed and untyped actors") {
val actorRef = Actor.actorOf[MyActor].start
- assert(ActorRegistry.actors.size === 3)
- assert(ActorRegistry.typedActors.size === 2)
- ActorRegistry.shutdownAll()
- assert(ActorRegistry.actors.size === 0)
- assert(ActorRegistry.typedActors.size === 0)
+ assert(Actor.registry.actors.size === 3)
+ assert(Actor.registry.typedActors.size === 2)
+ Actor.registry.shutdownAll()
+ assert(Actor.registry.actors.size === 0)
+ assert(Actor.registry.typedActors.size === 0)
}
}
}