Removing ActorRegistry object, UntypedActor object, introducing akka.actor.Actors for the Java API

This commit is contained in:
Viktor Klang 2011-01-04 13:24:28 +01:00
parent 435de7b689
commit dbe6f203b3
38 changed files with 320 additions and 331 deletions

View file

@ -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.
* <p/>
* 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'.
* <p/>
* You use it by implementing the UntypedActorFactory interface.
* Example in Java:
* <pre>
* ActorRef actor = Actors.actorOf(new UntypedActorFactory() {
* public UntypedActor create() {
* return new MyUntypedActor("service:name", 5);
* }
* });
* actor.start();
* actor.sendOneWay(message, context);
* actor.stop();
* </pre>
*/
public static ActorRef actorOf(final Creator<Actor> factory) {
return Actor$.MODULE$.actorOf(factory);
}
/**
* Creates an ActorRef out of the Actor type represented by the class provided.
* Example in Java:
* <pre>
* ActorRef actor = Actors.actorOf(MyUntypedActor.class);
* actor.start();
* actor.sendOneWay(message, context);
* actor.stop();
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = Actors.actorOf(MyActor.class).start();
* </pre>
*/
public static ActorRef actorOf(final Class<? extends Actor> type) {
return Actor$.MODULE$.actorOf(type);
}
}

View file

@ -15,7 +15,8 @@ import java.net.InetSocketAddress
import scala.reflect.BeanProperty
import akka.util. {ReflectiveAccess, Logging, Duration}
import akka.japi.Procedure
import akka.remoteinterface.RemoteSupport
import akka.japi. {Creator, Procedure}
/**
* Life-cycle messages for the Actors
@ -80,7 +81,6 @@ case class UnhandledMessageException(msg: Any, ref: ActorRef) extends Exception
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object Actor extends Logging {
/**
* Add shutdown cleanups
*/
@ -103,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
@ -114,6 +124,8 @@ 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.
* <pre>
@ -128,8 +140,7 @@ object Actor extends Logging {
* val actor = actorOf[MyActor].start
* </pre>
*/
@deprecated("Use ActorRegistry.actorOf instead")
def actorOf[T <: Actor : Manifest]: ActorRef = ActorRegistry.actorOf[T]
def actorOf[T <: Actor : Manifest]: ActorRef = actorOf(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
/**
* Creates an ActorRef out of the Actor of the specified Class.
@ -145,8 +156,15 @@ object Actor extends Logging {
* val actor = actorOf(classOf[MyActor]).start
* </pre>
*/
@deprecated("Use ActorRegistry.actorOf instead")
def actorOf(clazz: Class[_ <: Actor]): ActorRef = ActorRegistry.actorOf(clazz)
def actorOf(clazz: Class[_ <: Actor]): ActorRef = new LocalActorRef(() => {
import ReflectiveAccess.{ createInstance, noParams, noArgs }
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)'."))
}, None)
/**
* Creates an ActorRef out of the Actor. Allows you to pass in a factory function
@ -166,28 +184,42 @@ object Actor extends Logging {
* val actor = actorOf(new MyActor).start
* </pre>
*/
@deprecated("Use ActorRegistry.actorOf instead")
def actorOf(factory: => Actor): ActorRef = ActorRegistry.actorOf(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<Actor>)
* that creates the Actor. Please note that this function can be invoked multiple
* times if for example the Actor is supervised and needs to be restarted.
* <p/>
* This function should <b>NOT</b> be used for remote actors.
* JAVA API
*/
def actorOf(creator: Creator[Actor]): ActorRef = new LocalActorRef(() => creator.create, None)
/**
* Use to spawn out a block of code in an event-driven actor. Will shut actor down when
* the block has been executed.
* <p/>
* NOTE: If used from within an Actor then has to be qualified with 'ActorRegistry.spawn' since
* NOTE: If used from within an Actor then has to be qualified with 'Actor.spawn' since
* there is a method 'spawn[ActorType]' in the Actor trait already.
* Example:
* <pre>
* import ActorRegistry.{spawn}
* import Actor.{spawn}
*
* spawn {
* ... // do stuff
* }
* </pre>
*/
@deprecated("Use ActorRegistry.spawn instead")
def spawn(body: => Unit)(implicit dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher): Unit =
ActorRegistry.spawn(body)
def spawn(body: => Unit)(implicit dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher): Unit = {
case object Spawn
actorOf(new Actor() {
self.dispatcher = dispatcher
def receive = {
case Spawn => try { body } finally { self.stop }
}
}).start ! Spawn
}
/**
* Implicitly converts the given Option[Any] to a AnyOptionAsTypedOption which offers the method <code>as[T]</code>
* to convert an Option[Any] to an Option[T].

View file

@ -644,7 +644,7 @@ class LocalActorRef private[akka] (
initializeActorInstance
if (isClientManaged_?)
ActorRegistry.remote.registerClientManagedActor(homeAddress.get.getHostName,homeAddress.get.getPort, uuid)
Actor.remote.registerClientManagedActor(homeAddress.get.getHostName,homeAddress.get.getPort, uuid)
checkReceiveTimeout //Schedule the initial Receive timeout
}
@ -661,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 (isClientManaged_?)
ActorRegistry.remote.registerClientManagedActor(homeAddress.get.getHostName,homeAddress.get.getPort, uuid)
ActorRegistry.remote.unregister(this)
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.")
@ -737,7 +737,7 @@ class LocalActorRef private[akka] (
*/
def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long = Actor.TIMEOUT): ActorRef = guard.withGuard {
ensureRemotingEnabled
val ref = ActorRegistry.remote.actorOf(clazz, hostname, port)
val ref = Actor.remote.actorOf(clazz, hostname, port)
ref.timeout = timeout
ref.start
}
@ -762,7 +762,7 @@ class LocalActorRef private[akka] (
def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long = Actor.TIMEOUT): ActorRef =
guard.withGuard {
ensureRemotingEnabled
val actor = ActorRegistry.remote.actorOf(clazz, hostname, port)
val actor = Actor.remote.actorOf(clazz, hostname, port)
actor.timeout = timeout
link(actor)
actor.start
@ -798,7 +798,7 @@ class LocalActorRef private[akka] (
protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit =
if (isClientManaged_?) {
ActorRegistry.remote.send[Any](
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)
@ -809,7 +809,7 @@ class LocalActorRef private[akka] (
senderOption: Option[ActorRef],
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
if (isClientManaged_?) {
val future = ActorRegistry.remote.send[T](
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)
@ -978,7 +978,7 @@ class LocalActorRef private[akka] (
ensureRemotingEnabled
if (_supervisor.isDefined) {
if (homeAddress.isDefined)
ActorRegistry.remote.registerSupervisorForActor(this)
Actor.remote.registerSupervisorForActor(this)
Some(_supervisor.get.uuid)
} else None
}
@ -1085,7 +1085,7 @@ 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)
}
}
@ -1127,14 +1127,14 @@ private[akka] case class RemoteActorRef private[akka] (
start
def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit =
ActorRegistry.remote.send[Any](message, senderOption, None, homeAddress.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 = ActorRegistry.remote.send[T](message, senderOption, senderFuture, homeAddress.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)
}
@ -1142,7 +1142,7 @@ private[akka] case class RemoteActorRef private[akka] (
def start: ActorRef = synchronized {
_status = ActorRefInternals.RUNNING
//if (clientManaged)
// ActorRegistry.remote.registerClientManagedActor(homeAddress.getHostName,homeAddress.getPort, uuid)
// Actor.remote.registerClientManagedActor(homeAddress.getHostName,homeAddress.getPort, uuid)
this
}
@ -1151,7 +1151,7 @@ private[akka] case class RemoteActorRef private[akka] (
_status = ActorRefInternals.SHUTDOWN
postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
// if (clientManaged)
// ActorRegistry.remote.unregisterClientManagedActor(homeAddress.getHostName,homeAddress.getPort, uuid)
// Actor.remote.unregisterClientManagedActor(homeAddress.getHostName,homeAddress.getPort, uuid)
}
}

View file

@ -12,10 +12,7 @@ import java.util.{Set => JSet}
import annotation.tailrec
import akka.util.ReflectiveAccess._
import java.net.InetSocketAddress
import akka.util. {ReflectiveAccess, ReadWriteGuard, Address, ListenerManagement}
import akka.dispatch. {MessageDispatcher, Dispatchers}
import akka.remoteinterface. {RemoteSupport, RemoteServerModule}
import akka.util. {ReflectiveAccess, ReadWriteGuard, ListenerManagement}
/**
* Base trait for ActorRegistry events, allows listen to when an actor is added and removed from the ActorRegistry.
@ -39,9 +36,7 @@ case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object ActorRegistry extends ListenerManagement {
protected def remoteBootstrap = ReflectiveAccess.Remote.defaultRemoteSupport
private[actor] final class ActorRegistry private[actor] () extends ListenerManagement {
private val actorsByUUID = new ConcurrentHashMap[Uuid, ActorRef]
private val actorsById = new Index[String,ActorRef]
@ -228,103 +223,6 @@ object ActorRegistry extends ListenerManagement {
TypedActorModule.typedActorObjectInstance.get.proxyFor(actorRef)
}
/**
* Handy access to the RemoteServer module
*/
lazy val remote: RemoteSupport = remoteBootstrap.map(_()).getOrElse(throw new UnsupportedOperationException("You need to have akka-remote on classpath"))
/**
* Current home address of this ActorRegistry
*/
def homeAddress(): InetSocketAddress = if (isRemotingEnabled) remote.address else Remote.configDefaultAddress
/**
* Creates an ActorRef out of the Actor with type T.
* <pre>
* import Actor._
* val actor = actorOf[MyActor]
* actor.start
* actor ! message
* actor.stop
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf[MyActor].start
* </pre>
*/
def actorOf[T <: Actor : Manifest]: ActorRef = actorOf(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
/**
* Creates an ActorRef out of the Actor of the specified Class.
* <pre>
* import Actor._
* val actor = actorOf(classOf[MyActor])
* actor.start
* actor ! message
* actor.stop
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf(classOf[MyActor]).start
* </pre>
*/
def actorOf(clazz: Class[_ <: Actor]): ActorRef = new LocalActorRef(() => {
import ReflectiveAccess.{ createInstance, noParams, noArgs }
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)'."))
}, None)
/**
* Creates an ActorRef out of the Actor. Allows you to pass in a factory function
* that creates the Actor. Please note that this function can be invoked multiple
* times if for example the Actor is supervised and needs to be restarted.
* <p/>
* This function should <b>NOT</b> be used for remote actors.
* <pre>
* import Actor._
* val actor = actorOf(new MyActor)
* actor.start
* actor ! message
* actor.stop
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf(new MyActor).start
* </pre>
*/
def actorOf(factory: => Actor): ActorRef = new LocalActorRef(() => factory, None)
/**
* Use to spawn out a block of code in an event-driven actor. Will shut actor down when
* the block has been executed.
* <p/>
* NOTE: If used from within an Actor then has to be qualified with 'ActorRegistry.spawn' since
* there is a method 'spawn[ActorType]' in the Actor trait already.
* Example:
* <pre>
* import ActorRegistry.{spawn}
*
* spawn {
* ... // do stuff
* }
* </pre>
*/
def spawn(body: => Unit)(implicit dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher): Unit = {
case object Spawn
actorOf(new Actor() {
self.dispatcher = dispatcher
def receive = {
case Spawn => try { body } finally { self.stop }
}
}).start ! Spawn
}
/**
* Registers an actor in the ActorRegistry.
*/
@ -368,7 +266,7 @@ object ActorRegistry extends ListenerManagement {
}
} else foreach(_.stop)
if (Remote.isEnabled) {
remote.clear //TODO: REVISIT: Should this be here?
Actor.remote.clear //TODO: REVISIT: Should this be here?
}
actorsByUUID.clear
actorsById.clear

View file

@ -62,6 +62,6 @@ trait BootableActorLoaderService extends Bootable with Logging {
abstract override def onUnload = {
super.onUnload
ActorRegistry.shutdownAll
Actor.registry.shutdownAll
}
}

View file

@ -142,7 +142,7 @@ sealed class Supervisor(handler: FaultHandlingStrategy) {
actorRef.lifeCycle = lifeCycle
supervisor.link(actorRef)
if (registerAsRemoteService)
ActorRegistry.remote.register(actorRef)
Actor.remote.register(actorRef)
case supervisorConfig @ SupervisorConfig(_, _) => // recursive supervisor configuration
val childSupervisor = Supervisor(supervisorConfig)
supervisor.link(childSupervisor.supervisor)

View file

@ -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,66 +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 <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
trait UntypedActorFactory extends Creator[Actor]
/**
* Factory object for creating and managing 'UntypedActor's. Meant to be used from Java.
* <p/>
* Example on how to create an actor:
* <pre>
* ActorRef actor = UntypedActor.actorOf(MyUntypedActor.class);
* actor.start();
* actor.sendOneWay(message, context)
* actor.stop();
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* ActorRef actor = UntypedActor.actorOf(MyUntypedActor.class).start();
* </pre>
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object UntypedActor {
/**
* Creates an ActorRef out of the Actor type represented by the class provided.
* Example in Java:
* <pre>
* ActorRef actor = UntypedActor.actorOf(MyUntypedActor.class);
* actor.start();
* actor.sendOneWay(message, context);
* actor.stop();
* </pre>
* You can create and start the actor in one statement like this:
* <pre>
* val actor = actorOf(classOf[MyActor]).start
* </pre>
*/
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.
* <p/>
* 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'.
* <p/>
* You use it by implementing the UntypedActorFactory interface.
* Example in Java:
* <pre>
* ActorRef actor = UntypedActor.actorOf(new UntypedActorFactory() {
* public UntypedActor create() {
* return new MyUntypedActor("service:name", 5);
* }
* });
* actor.start();
* actor.sendOneWay(message, context);
* actor.stop();
* </pre>
*/
def actorOf(factory: UntypedActorFactory): ActorRef = Actor.actorOf(factory.create)
}

View file

@ -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)

View file

@ -142,7 +142,7 @@ abstract class RemoteSupport extends ListenerManagement with RemoteServerModule
* </pre>
*/
def actorOf(factory: => Actor, host: String, port: Int): ActorRef =
ActorRegistry.remote.clientManagedActorOf(() => factory, host, port)
Actor.remote.clientManagedActorOf(() => factory, host, port)
/**
* Creates a Client-managed ActorRef out of the Actor of the specified Class.
@ -214,7 +214,7 @@ abstract class RemoteSupport extends ListenerManagement with RemoteServerModule
}
/**
* This is the interface for the RemoteServer functionality, it's used in ActorRegistry.remote
* This is the interface for the RemoteServer functionality, it's used in Actor.remote
*/
trait RemoteServerModule extends RemoteModule {
protected val guard = new ReentrantGuard

View file

@ -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<Actor>() {
public Actor create() {
return new JavaAPITestActor();
}
});
assertNotNull(ref);
}
}

View file

@ -0,0 +1,7 @@
package akka.actor;
public class JavaAPITestActor extends UntypedActor {
public void onReceive(Object msg) {
getContext().replySafe("got it!");
}
}

View file

@ -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]

View file

@ -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)
}

View file

@ -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)
}
/**

View file

@ -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

View file

@ -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))

View file

@ -5,7 +5,7 @@
package akka.remote
import akka.config.Config.config
import akka.actor. {ActorRegistry, BootableActorLoaderService}
import akka.actor. {Actor, BootableActorLoaderService}
import akka.util. {ReflectiveAccess, Bootable, Logging}
/**
@ -17,7 +17,7 @@ trait BootableRemoteActorService extends Bootable with Logging {
self: BootableActorLoaderService =>
protected lazy val remoteServerThread = new Thread(new Runnable() {
def run = ActorRegistry.remote.start(loader = self.applicationLoader) //Use config host/port
def run = Actor.remote.start(loader = self.applicationLoader) //Use config host/port
}, "Akka Remote Service")
def startRemoteService = remoteServerThread.start
@ -33,7 +33,7 @@ trait BootableRemoteActorService extends Bootable with Logging {
abstract override def onUnload = {
log.slf4j.info("Shutting down Remote Actors Service")
ActorRegistry.remote.shutdown
Actor.remote.shutdown
if (remoteServerThread.isAlive) remoteServerThread.join(1000)
log.slf4j.info("Remote Actors Service has been shut down")
super.onUnload

View file

@ -91,7 +91,7 @@ object ActorSerialization {
toBinary(a, srlMailBox)(format)
private[akka] def toAddressProtocol(actorRef: ActorRef) = {
val address = actorRef.homeAddress.getOrElse(ActorRegistry.remote.address)
val address = actorRef.homeAddress.getOrElse(Actor.remote.address)
AddressProtocol.newBuilder
.setHostname(address.getHostName)
.setPort(address.getPort)
@ -252,7 +252,7 @@ object RemoteActorSerialization {
Actor.log.slf4j.debug("Register serialized Actor [{}] as remote @ [{}:{}]",actorClassName, ar.homeAddress)
ActorRegistry.remote.registerByUuid(ar)
Actor.remote.registerByUuid(ar)
RemoteActorRefProtocol.newBuilder
.setClassOrServiceName("uuid:"+uuid.toString)

View file

@ -25,7 +25,7 @@ class AkkaRemoteTest extends
BeforeAndAfterEach {
import AkkaRemoteTest._
val remote = ActorRegistry.remote
val remote = Actor.remote
val unit = TimeUnit.SECONDS
val host = "localhost"
@ -53,7 +53,7 @@ class AkkaRemoteTest extends
override def afterEach() {
remote.shutdown
ActorRegistry.shutdownAll
Actor.registry.shutdownAll
super.afterEach
}

View file

@ -1,6 +1,6 @@
package akka.actor.remote
import akka.actor. {ActorRegistry, Actor}
import akka.actor.{Actor}
object OptimizedLocalScopedSpec {
class TestActor extends Actor {
@ -14,14 +14,14 @@ class OptimizedLocalScopedSpec extends AkkaRemoteTest {
"An enabled optimized local scoped remote" should {
"Fetch local actor ref when scope is local" in {
val fooActor = ActorRegistry.actorOf[TestActor].start
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 = ActorRegistry.remote.actorOf[TestActor](host, port)
val localClientManaged = Actor.remote.actorOf[TestActor](host, port)
localClientManaged.homeAddress must be (None)
}

View file

@ -36,14 +36,14 @@ class HelloWorldActor extends Actor {
object ServerInitiatedRemoteActorServer {
def main(args: Array[String]) = {
ActorRegistry.remote.start("localhost", 2552)
ActorRegistry.remote.register("hello-service", actorOf[HelloWorldActor])
Actor.remote.start("localhost", 2552)
Actor.remote.register("hello-service", actorOf[HelloWorldActor])
}
}
object ServerInitiatedRemoteActorClient extends Logging {
def main(args: Array[String]) = {
val actor = ActorRegistry.remote.actorFor("hello-service", "localhost", 2552)
val actor = Actor.remote.actorFor("hello-service", "localhost", 2552)
val result = actor !! "Hello"
log.slf4j.info("Result from Remote Actor: {}", result)
}

View file

@ -96,10 +96,10 @@ class ServerInitiatedRemoteActorSpec extends AkkaRemoteTest {
implicit val sender = replyHandler(latch, "Pong")
remote.register(actorOf[RemoteActorSpecActorUnidirectional])
val actor = remote.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", host, port)
val numberOfActorsInRegistry = ActorRegistry.actors.length
val numberOfActorsInRegistry = Actor.registry.actors.length
actor ! "Ping"
latch.await(1, TimeUnit.SECONDS) must be (true)
numberOfActorsInRegistry must equal (ActorRegistry.actors.length)
numberOfActorsInRegistry must equal (Actor.registry.actors.length)
}
"UseServiceNameAsIdForRemoteActorRef" in {
@ -194,7 +194,7 @@ class ServerInitiatedRemoteActorSpec extends AkkaRemoteTest {
latch.countDown
}
val decrementers = ActorRegistry.actorsFor[Decrementer]
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))

View file

@ -39,10 +39,10 @@ class ServerInitiatedRemoteTypedActorSpec extends AkkaRemoteTest {
"should not recreate registered actors" in {
val actor = createRemoteActorRef
val numberOfActorsInRegistry = ActorRegistry.actors.length
val numberOfActorsInRegistry = Actor.registry.actors.length
actor.oneWay
oneWayLog.poll(5, TimeUnit.SECONDS) must equal ("oneway")
numberOfActorsInRegistry must be (ActorRegistry.actors.length)
numberOfActorsInRegistry must be (Actor.registry.actors.length)
}
"should support multiple variants to get the actor from client side" in {

View file

@ -13,14 +13,14 @@ class UnOptimizedLocalScopedSpec extends AkkaRemoteTest {
"An enabled optimized local scoped remote" should {
"Fetch remote actor ref when scope is local" in {
val fooActor = ActorRegistry.actorOf[TestActor].start
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 = ActorRegistry.remote.actorOf[TestActor](host, port)
val localClientManaged = Actor.remote.actorOf[TestActor](host, port)
localClientManaged.homeAddress must not be (None)
}

View file

@ -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")

View file

@ -7,7 +7,7 @@ package sample.remote
import akka.actor.Actor._
import akka.util.Logging
import akka.actor. {ActorRegistry, Actor}
import ActorRegistry.remote
import Actor.remote
class RemoteHelloWorldActor extends Actor {
def receive = {

View file

@ -19,9 +19,9 @@ class HelloWorldActor extends Actor {
object ServerManagedRemoteActorServer extends Logging {
def run = {
ActorRegistry.remote.start("localhost", 2552)
Actor.remote.start("localhost", 2552)
log.slf4j.info("Remote node started")
ActorRegistry.remote.register("hello-service", actorOf[HelloWorldActor])
Actor.remote.register("hello-service", actorOf[HelloWorldActor])
log.slf4j.info("Remote actor registered and started")
}
@ -31,7 +31,7 @@ object ServerManagedRemoteActorServer extends Logging {
object ServerManagedRemoteActorClient extends Logging {
def run = {
val actor = ActorRegistry.remote.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"

View file

@ -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;

View file

@ -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)));

View file

@ -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));

View file

@ -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.Duration;

View file

@ -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<ActorRef>();
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();
}

View file

@ -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<ActorRef>();
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();
}

View file

@ -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.getOrElse(ActorRegistry.homeAddress)
def homeAddress: InetSocketAddress = actorRef.homeAddress.getOrElse(null)//TODO: REVISIT: Sensible to return null?
}
object TypedActorConfiguration {
@ -539,7 +539,7 @@ object TypedActor extends Logging {
config match {
case null => actorOf(typedActor)
case c: TypedActorConfiguration if (c._host.isDefined) =>
ActorRegistry.remote.actorOf(typedActor, c._host.get.getHostName, c._host.get.getPort)
Actor.remote.actorOf(typedActor, c._host.get.getHostName, c._host.get.getPort)
case _ => actorOf(typedActor)
}
}
@ -676,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 classOf[TypedActor].isAssignableFrom(a.actor.getClass) && a.actor.asInstanceOf[TypedActor].proxy == proxy =>
a
}
/**
* Get the typed actor proxy for the given Typed Actor.
@ -906,7 +907,7 @@ private[akka] abstract class ActorAspect {
val (message: Array[AnyRef], isEscaped) = escapeArguments(methodRtti.getParameterValues)
val future = ActorRegistry.remote.send[AnyRef](
val future = Actor.remote.send[AnyRef](
message, None, None, remoteAddress.get,
timeout, isOneWay, actorRef,
Some((interfaceClass.getName, methodRtti.getMethod.getName)),

View file

@ -112,7 +112,7 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa
component.remoteAddress match {
case Some(a) =>
(Some(new InetSocketAddress(a.hostname, a.port)),
ActorRegistry.remote.actorOf(TypedActor.newTypedActor(implementationClass), a.hostname, a.port))
Actor.remote.actorOf(TypedActor.newTypedActor(implementationClass), a.hostname, a.port))
case None =>
(None, Actor.actorOf(TypedActor.newTypedActor(implementationClass)))
}

View file

@ -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)

View file

@ -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
}
}
}

View file

@ -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)
}
}
}