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 scala.reflect.BeanProperty
import akka.util. {ReflectiveAccess, Logging, Duration} 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 * 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> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
object Actor extends Logging { object Actor extends Logging {
/** /**
* Add shutdown cleanups * Add shutdown cleanups
*/ */
@ -103,8 +103,18 @@ object Actor extends Logging {
hook hook
} }
val TIMEOUT = Duration(config.getInt("akka.actor.timeout", 5), TIME_UNIT).toMillis val registry = new ActorRegistry
val SERIALIZE_MESSAGES = config.getBool("akka.actor.serialize-messages", false)
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 * 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) 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.
* <pre> * <pre>
@ -128,8 +140,7 @@ object Actor extends Logging {
* val actor = actorOf[MyActor].start * val actor = actorOf[MyActor].start
* </pre> * </pre>
*/ */
@deprecated("Use ActorRegistry.actorOf instead") def actorOf[T <: Actor : Manifest]: ActorRef = actorOf(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
def actorOf[T <: Actor : Manifest]: ActorRef = ActorRegistry.actorOf[T]
/** /**
* Creates an ActorRef out of the Actor of the specified Class. * 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 * val actor = actorOf(classOf[MyActor]).start
* </pre> * </pre>
*/ */
@deprecated("Use ActorRegistry.actorOf instead") def actorOf(clazz: Class[_ <: Actor]): ActorRef = new LocalActorRef(() => {
def actorOf(clazz: Class[_ <: Actor]): ActorRef = ActorRegistry.actorOf(clazz) 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 * 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 * val actor = actorOf(new MyActor).start
* </pre> * </pre>
*/ */
@deprecated("Use ActorRegistry.actorOf instead") def actorOf(factory: => Actor): ActorRef = new LocalActorRef(() => factory, None)
def actorOf(factory: => Actor): ActorRef = ActorRegistry.actorOf(factory)
/**
* 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 * Use to spawn out a block of code in an event-driven actor. Will shut actor down when
* the block has been executed. * the block has been executed.
* <p/> * <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. * there is a method 'spawn[ActorType]' in the Actor trait already.
* Example: * Example:
* <pre> * <pre>
* import ActorRegistry.{spawn} * import Actor.{spawn}
* *
* spawn { * spawn {
* ... // do stuff * ... // do stuff
* } * }
* </pre> * </pre>
*/ */
@deprecated("Use ActorRegistry.spawn instead") def spawn(body: => Unit)(implicit dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher): Unit = {
def spawn(body: => Unit)(implicit dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher): Unit = case object Spawn
ActorRegistry.spawn(body) 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> * 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]. * to convert an Option[Any] to an Option[T].

View file

@ -644,7 +644,7 @@ class LocalActorRef private[akka] (
initializeActorInstance initializeActorInstance
if (isClientManaged_?) 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 checkReceiveTimeout //Schedule the initial Receive timeout
} }
@ -661,11 +661,11 @@ class LocalActorRef private[akka] (
dispatcher.detach(this) dispatcher.detach(this)
_status = ActorRefInternals.SHUTDOWN _status = ActorRefInternals.SHUTDOWN
actor.postStop actor.postStop
ActorRegistry.unregister(this) Actor.registry.unregister(this)
if (isRemotingEnabled) { if (isRemotingEnabled) {
if (isClientManaged_?) if (isClientManaged_?)
ActorRegistry.remote.registerClientManagedActor(homeAddress.get.getHostName,homeAddress.get.getPort, uuid) Actor.remote.registerClientManagedActor(homeAddress.get.getHostName,homeAddress.get.getPort, uuid)
ActorRegistry.remote.unregister(this) Actor.remote.unregister(this)
} }
setActorSelfFields(actorInstance.get,null) setActorSelfFields(actorInstance.get,null)
} //else if (isBeingRestarted) throw new ActorKilledException("Actor [" + toString + "] is being restarted.") } //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 { def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long = Actor.TIMEOUT): ActorRef = guard.withGuard {
ensureRemotingEnabled ensureRemotingEnabled
val ref = ActorRegistry.remote.actorOf(clazz, hostname, port) val ref = Actor.remote.actorOf(clazz, hostname, port)
ref.timeout = timeout ref.timeout = timeout
ref.start ref.start
} }
@ -762,7 +762,7 @@ class LocalActorRef private[akka] (
def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long = Actor.TIMEOUT): ActorRef = def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int, timeout: Long = Actor.TIMEOUT): ActorRef =
guard.withGuard { guard.withGuard {
ensureRemotingEnabled ensureRemotingEnabled
val actor = ActorRegistry.remote.actorOf(clazz, hostname, port) val actor = Actor.remote.actorOf(clazz, hostname, port)
actor.timeout = timeout actor.timeout = timeout
link(actor) link(actor)
actor.start actor.start
@ -798,7 +798,7 @@ class LocalActorRef private[akka] (
protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit =
if (isClientManaged_?) { if (isClientManaged_?) {
ActorRegistry.remote.send[Any]( Actor.remote.send[Any](
message, senderOption, None, homeAddress.get, timeout, true, this, None, ActorType.ScalaActor, None) message, senderOption, None, homeAddress.get, timeout, true, this, None, ActorType.ScalaActor, None)
} else } else
dispatcher dispatchMessage new MessageInvocation(this, message, senderOption, None) dispatcher dispatchMessage new MessageInvocation(this, message, senderOption, None)
@ -809,7 +809,7 @@ class LocalActorRef private[akka] (
senderOption: Option[ActorRef], senderOption: Option[ActorRef],
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = { senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
if (isClientManaged_?) { 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) message, senderOption, senderFuture, homeAddress.get, timeout, false, this, None, ActorType.ScalaActor, None)
if (future.isDefined) future.get if (future.isDefined) future.get
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString) else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
@ -978,7 +978,7 @@ class LocalActorRef private[akka] (
ensureRemotingEnabled ensureRemotingEnabled
if (_supervisor.isDefined) { if (_supervisor.isDefined) {
if (homeAddress.isDefined) if (homeAddress.isDefined)
ActorRegistry.remote.registerSupervisorForActor(this) Actor.remote.registerSupervisorForActor(this)
Some(_supervisor.get.uuid) Some(_supervisor.get.uuid)
} else None } else None
} }
@ -1085,7 +1085,7 @@ class LocalActorRef private[akka] (
private def initializeActorInstance = { private def initializeActorInstance = {
actor.preStart // run actor preStart actor.preStart // run actor preStart
Actor.log.slf4j.trace("[{}] has started", toString) 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 start
def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = 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]( def postMessageToMailboxAndCreateFutureResultWithTimeout[T](
message: Any, message: Any,
timeout: Long, timeout: Long,
senderOption: Option[ActorRef], senderOption: Option[ActorRef],
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = { 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 if (future.isDefined) future.get
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString) 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 { def start: ActorRef = synchronized {
_status = ActorRefInternals.RUNNING _status = ActorRefInternals.RUNNING
//if (clientManaged) //if (clientManaged)
// ActorRegistry.remote.registerClientManagedActor(homeAddress.getHostName,homeAddress.getPort, uuid) // Actor.remote.registerClientManagedActor(homeAddress.getHostName,homeAddress.getPort, uuid)
this this
} }
@ -1151,7 +1151,7 @@ private[akka] case class RemoteActorRef private[akka] (
_status = ActorRefInternals.SHUTDOWN _status = ActorRefInternals.SHUTDOWN
postMessageToMailbox(RemoteActorSystemMessage.Stop, None) postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
// if (clientManaged) // 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 annotation.tailrec
import akka.util.ReflectiveAccess._ import akka.util.ReflectiveAccess._
import java.net.InetSocketAddress import akka.util. {ReflectiveAccess, ReadWriteGuard, ListenerManagement}
import akka.util. {ReflectiveAccess, ReadWriteGuard, Address, ListenerManagement}
import akka.dispatch. {MessageDispatcher, Dispatchers}
import akka.remoteinterface. {RemoteSupport, RemoteServerModule}
/** /**
* Base trait for ActorRegistry events, allows listen to when an actor is added and removed from the ActorRegistry. * 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> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
object ActorRegistry extends ListenerManagement { private[actor] final class ActorRegistry private[actor] () extends ListenerManagement {
protected def remoteBootstrap = ReflectiveAccess.Remote.defaultRemoteSupport
private val actorsByUUID = new ConcurrentHashMap[Uuid, ActorRef] private val actorsByUUID = new ConcurrentHashMap[Uuid, ActorRef]
private val actorsById = new Index[String,ActorRef] private val actorsById = new Index[String,ActorRef]
@ -228,103 +223,6 @@ object ActorRegistry extends ListenerManagement {
TypedActorModule.typedActorObjectInstance.get.proxyFor(actorRef) 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. * Registers an actor in the ActorRegistry.
*/ */
@ -368,7 +266,7 @@ object ActorRegistry extends ListenerManagement {
} }
} else foreach(_.stop) } else foreach(_.stop)
if (Remote.isEnabled) { if (Remote.isEnabled) {
remote.clear //TODO: REVISIT: Should this be here? Actor.remote.clear //TODO: REVISIT: Should this be here?
} }
actorsByUUID.clear actorsByUUID.clear
actorsById.clear actorsById.clear

View file

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

View file

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

View file

@ -43,14 +43,14 @@ import scala.reflect.BeanProperty
* *
* } else if (msg.equals("ForwardMessage")) { * } else if (msg.equals("ForwardMessage")) {
* // Retreive an actor from the ActorRegistry by ID and get an ActorRef back * // 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);
* } else throw new IllegalArgumentException("Unknown message: " + message); * } else throw new IllegalArgumentException("Unknown message: " + message);
* } * }
* *
* public static void main(String[] args) { * public static void main(String[] args) {
* ActorRef actor = UntypedActor.actorOf(SampleUntypedActor.class); * ActorRef actor = Actors.actorOf(SampleUntypedActor.class);
* actor.start(); * actor.start();
* actor.sendOneWay("SendToSelf"); * actor.sendOneWay("SendToSelf");
* actor.stop(); * 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> * @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/ */
trait UntypedActorFactory extends Creator[Actor] 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 val i = uuids.iterator
while(i.hasNext()) { while(i.hasNext()) {
val uuid = i.next() val uuid = i.next()
ActorRegistry.actorFor(uuid) match { Actor.registry.actorFor(uuid) match {
case Some(actor) => actor.stop case Some(actor) => actor.stop
case None => case None =>
log.slf4j.error("stopAllLinkedActors couldn't find linked actor: " + uuid) log.slf4j.error("stopAllLinkedActors couldn't find linked actor: " + uuid)

View file

@ -142,7 +142,7 @@ abstract class RemoteSupport extends ListenerManagement with RemoteServerModule
* </pre> * </pre>
*/ */
def actorOf(factory: => Actor, host: String, port: Int): ActorRef = 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. * 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 { trait RemoteServerModule extends RemoteModule {
protected val guard = new ReentrantGuard 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") { it("should be able to join streams") {
import DataFlow._ import DataFlow._
ActorRegistry.shutdownAll Actor.registry.shutdownAll
def ints(n: Int, max: Int, stream: DataFlowStream[Int]): Unit = if (n != max) { def ints(n: Int, max: Int, stream: DataFlowStream[Int]): Unit = if (n != max) {
stream <<< n stream <<< n
@ -139,7 +139,7 @@ class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll {
/* it("should be able to conditionally set variables") { /* it("should be able to conditionally set variables") {
import DataFlow._ import DataFlow._
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val latch = new CountDownLatch(1) val latch = new CountDownLatch(1)
val x, y, z, v = new DataFlowVariable[Int] val x, y, z, v = new DataFlowVariable[Int]

View file

@ -33,10 +33,10 @@ class ActorRegistrySpec extends JUnitSuite {
import ActorRegistrySpec._ import ActorRegistrySpec._
@Test def shouldGetActorByIdFromActorRegistry { @Test def shouldGetActorByIdFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor = actorOf[TestActor] val actor = actorOf[TestActor]
actor.start actor.start
val actors = ActorRegistry.actorsFor("MyID") val actors = Actor.registry.actorsFor("MyID")
assert(actors.size === 1) assert(actors.size === 1)
assert(actors.head.actor.isInstanceOf[TestActor]) assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID") assert(actors.head.id === "MyID")
@ -44,21 +44,21 @@ class ActorRegistrySpec extends JUnitSuite {
} }
@Test def shouldGetActorByUUIDFromActorRegistry { @Test def shouldGetActorByUUIDFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor = actorOf[TestActor] val actor = actorOf[TestActor]
val uuid = actor.uuid val uuid = actor.uuid
actor.start actor.start
val actorOrNone = ActorRegistry.actorFor(uuid) val actorOrNone = Actor.registry.actorFor(uuid)
assert(actorOrNone.isDefined) assert(actorOrNone.isDefined)
assert(actorOrNone.get.uuid === uuid) assert(actorOrNone.get.uuid === uuid)
actor.stop actor.stop
} }
@Test def shouldGetActorByClassFromActorRegistry { @Test def shouldGetActorByClassFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor = actorOf[TestActor] val actor = actorOf[TestActor]
actor.start actor.start
val actors = ActorRegistry.actorsFor(classOf[TestActor]) val actors = Actor.registry.actorsFor(classOf[TestActor])
assert(actors.size === 1) assert(actors.size === 1)
assert(actors.head.actor.isInstanceOf[TestActor]) assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID") assert(actors.head.id === "MyID")
@ -66,10 +66,10 @@ class ActorRegistrySpec extends JUnitSuite {
} }
@Test def shouldGetActorByManifestFromActorRegistry { @Test def shouldGetActorByManifestFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor = actorOf[TestActor] val actor = actorOf[TestActor]
actor.start actor.start
val actors = ActorRegistry.actorsFor[TestActor] val actors = Actor.registry.actorsFor[TestActor]
assert(actors.size === 1) assert(actors.size === 1)
assert(actors.head.actor.isInstanceOf[TestActor]) assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID") assert(actors.head.id === "MyID")
@ -77,10 +77,10 @@ class ActorRegistrySpec extends JUnitSuite {
} }
@Test def shouldFindThingsFromActorRegistry { @Test def shouldFindThingsFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor = actorOf[TestActor] val actor = actorOf[TestActor]
actor.start 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.isDefined)
assert(found.get.actor.isInstanceOf[TestActor]) assert(found.get.actor.isInstanceOf[TestActor])
assert(found.get.id === "MyID") assert(found.get.id === "MyID")
@ -88,12 +88,12 @@ class ActorRegistrySpec extends JUnitSuite {
} }
@Test def shouldGetActorsByIdFromActorRegistry { @Test def shouldGetActorsByIdFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
actor1.start actor1.start
val actor2 = actorOf[TestActor] val actor2 = actorOf[TestActor]
actor2.start actor2.start
val actors = ActorRegistry.actorsFor("MyID") val actors = Actor.registry.actorsFor("MyID")
assert(actors.size === 2) assert(actors.size === 2)
assert(actors.head.actor.isInstanceOf[TestActor]) assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID") assert(actors.head.id === "MyID")
@ -104,12 +104,12 @@ class ActorRegistrySpec extends JUnitSuite {
} }
@Test def shouldGetActorsByClassFromActorRegistry { @Test def shouldGetActorsByClassFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
actor1.start actor1.start
val actor2 = actorOf[TestActor] val actor2 = actorOf[TestActor]
actor2.start actor2.start
val actors = ActorRegistry.actorsFor(classOf[TestActor]) val actors = Actor.registry.actorsFor(classOf[TestActor])
assert(actors.size === 2) assert(actors.size === 2)
assert(actors.head.actor.isInstanceOf[TestActor]) assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID") assert(actors.head.id === "MyID")
@ -120,12 +120,12 @@ class ActorRegistrySpec extends JUnitSuite {
} }
@Test def shouldGetActorsByManifestFromActorRegistry { @Test def shouldGetActorsByManifestFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
actor1.start actor1.start
val actor2 = actorOf[TestActor] val actor2 = actorOf[TestActor]
actor2.start actor2.start
val actors = ActorRegistry.actorsFor[TestActor] val actors = Actor.registry.actorsFor[TestActor]
assert(actors.size === 2) assert(actors.size === 2)
assert(actors.head.actor.isInstanceOf[TestActor]) assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID") assert(actors.head.id === "MyID")
@ -137,26 +137,26 @@ class ActorRegistrySpec extends JUnitSuite {
@Test def shouldGetActorsByMessageFromActorRegistry { @Test def shouldGetActorsByMessageFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
actor1.start actor1.start
val actor2 = actorOf[TestActor2] val actor2 = actorOf[TestActor2]
actor2.start actor2.start
val actorsForAcotrTestActor = ActorRegistry.actorsFor[TestActor] val actorsForAcotrTestActor = Actor.registry.actorsFor[TestActor]
assert(actorsForAcotrTestActor.size === 1) assert(actorsForAcotrTestActor.size === 1)
val actorsForAcotrTestActor2 = ActorRegistry.actorsFor[TestActor2] val actorsForAcotrTestActor2 = Actor.registry.actorsFor[TestActor2]
assert(actorsForAcotrTestActor2.size === 1) assert(actorsForAcotrTestActor2.size === 1)
val actorsForAcotr = ActorRegistry.actorsFor[Actor] val actorsForAcotr = Actor.registry.actorsFor[Actor]
assert(actorsForAcotr.size === 2) assert(actorsForAcotr.size === 2)
val actorsForMessagePing2 = ActorRegistry.actorsFor[Actor]("ping2") val actorsForMessagePing2 = Actor.registry.actorsFor[Actor]("ping2")
assert(actorsForMessagePing2.size === 1) assert(actorsForMessagePing2.size === 1)
val actorsForMessagePing = ActorRegistry.actorsFor[Actor]("ping") val actorsForMessagePing = Actor.registry.actorsFor[Actor]("ping")
assert(actorsForMessagePing.size === 2) assert(actorsForMessagePing.size === 2)
actor1.stop actor1.stop
@ -164,12 +164,12 @@ class ActorRegistrySpec extends JUnitSuite {
} }
@Test def shouldGetAllActorsFromActorRegistry { @Test def shouldGetAllActorsFromActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
actor1.start actor1.start
val actor2 = actorOf[TestActor] val actor2 = actorOf[TestActor]
actor2.start actor2.start
val actors = ActorRegistry.actors val actors = Actor.registry.actors
assert(actors.size === 2) assert(actors.size === 2)
assert(actors.head.actor.isInstanceOf[TestActor]) assert(actors.head.actor.isInstanceOf[TestActor])
assert(actors.head.id === "MyID") assert(actors.head.id === "MyID")
@ -180,43 +180,43 @@ class ActorRegistrySpec extends JUnitSuite {
} }
@Test def shouldGetResponseByAllActorsInActorRegistryWhenInvokingForeach { @Test def shouldGetResponseByAllActorsInActorRegistryWhenInvokingForeach {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
actor1.start actor1.start
val actor2 = actorOf[TestActor] val actor2 = actorOf[TestActor]
actor2.start actor2.start
record = "" record = ""
ActorRegistry.foreach(actor => actor !! "ping") Actor.registry.foreach(actor => actor !! "ping")
assert(record === "pongpong") assert(record === "pongpong")
actor1.stop actor1.stop
actor2.stop actor2.stop
} }
@Test def shouldShutdownAllActorsInActorRegistry { @Test def shouldShutdownAllActorsInActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
actor1.start actor1.start
val actor2 = actorOf[TestActor] val actor2 = actorOf[TestActor]
actor2.start actor2.start
ActorRegistry.shutdownAll Actor.registry.shutdownAll
assert(ActorRegistry.actors.size === 0) assert(Actor.registry.actors.size === 0)
} }
@Test def shouldRemoveUnregisterActorInActorRegistry { @Test def shouldRemoveUnregisterActorInActorRegistry {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
val actor1 = actorOf[TestActor] val actor1 = actorOf[TestActor]
actor1.start actor1.start
val actor2 = actorOf[TestActor] val actor2 = actorOf[TestActor]
actor2.start actor2.start
assert(ActorRegistry.actors.size === 2) assert(Actor.registry.actors.size === 2)
ActorRegistry.unregister(actor1) Actor.registry.unregister(actor1)
assert(ActorRegistry.actors.size === 1) assert(Actor.registry.actors.size === 1)
ActorRegistry.unregister(actor2) Actor.registry.unregister(actor2)
assert(ActorRegistry.actors.size === 0) assert(Actor.registry.actors.size === 0)
} }
@Test def shouldBeAbleToRegisterActorsConcurrently { @Test def shouldBeAbleToRegisterActorsConcurrently {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
def mkTestActors = for(i <- (1 to 10).toList;j <- 1 to 3000) yield actorOf( new Actor { def mkTestActors = for(i <- (1 to 10).toList;j <- 1 to 3000) yield actorOf( new Actor {
self.id = i.toString self.id = i.toString
@ -244,7 +244,7 @@ class ActorRegistrySpec extends JUnitSuite {
for(i <- 1 to 10) { for(i <- 1 to 10) {
val theId = i.toString 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) for(a <- actors if a.id == theId) assert(actors contains a)
assert(actors.size === 9000) assert(actors.size === 9000)
} }

View file

@ -12,7 +12,7 @@ class SchedulerSpec extends JUnitSuite {
def withCleanEndState(action: => Unit) { def withCleanEndState(action: => Unit) {
action action
Scheduler.restart Scheduler.restart
ActorRegistry.shutdownAll Actor.registry.shutdownAll
} }
@ -62,10 +62,10 @@ class SchedulerSpec extends JUnitSuite {
val actor = actorOf(new Actor { val actor = actorOf(new Actor {
def receive = { case Ping => ticks.countDown } def receive = { case Ping => ticks.countDown }
}).start }).start
val numActors = ActorRegistry.actors.length val numActors = Actor.registry.actors.length
(1 to 1000).foreach( _ => Scheduler.scheduleOnce(actor,Ping,1,TimeUnit.MILLISECONDS) ) (1 to 1000).foreach( _ => Scheduler.scheduleOnce(actor,Ping,1,TimeUnit.MILLISECONDS) )
assert(ticks.await(10,TimeUnit.SECONDS)) 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 * The root endpoint actor
*/ */
protected val _root = ActorRegistry.actorsFor(RootActorID).head protected val _root = Actor.registry.actorsFor(RootActorID).head
/** /**
* Server-specific method factory * 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 * 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 * 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] = def mkFilter(roles: Option[List[String]]): java.util.List[ResourceFilter] =
java.util.Collections.singletonList(new Filter(authenticator, roles)) java.util.Collections.singletonList(new Filter(authenticator, roles))

View file

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

View file

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

View file

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

View file

@ -1,6 +1,6 @@
package akka.actor.remote package akka.actor.remote
import akka.actor. {ActorRegistry, Actor} import akka.actor.{Actor}
object OptimizedLocalScopedSpec { object OptimizedLocalScopedSpec {
class TestActor extends Actor { class TestActor extends Actor {
@ -14,14 +14,14 @@ class OptimizedLocalScopedSpec extends AkkaRemoteTest {
"An enabled optimized local scoped remote" should { "An enabled optimized local scoped remote" should {
"Fetch local actor ref when scope is local" in { "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.register("foo", fooActor)
remote.actorFor("foo", host, port) must be (fooActor) remote.actorFor("foo", host, port) must be (fooActor)
} }
"Create local actor when client-managed is hosted locally" in { "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) localClientManaged.homeAddress must be (None)
} }

View file

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

View file

@ -96,10 +96,10 @@ class ServerInitiatedRemoteActorSpec extends AkkaRemoteTest {
implicit val sender = replyHandler(latch, "Pong") implicit val sender = replyHandler(latch, "Pong")
remote.register(actorOf[RemoteActorSpecActorUnidirectional]) remote.register(actorOf[RemoteActorSpecActorUnidirectional])
val actor = remote.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", host, port) val actor = remote.actorFor("akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", host, port)
val numberOfActorsInRegistry = ActorRegistry.actors.length val numberOfActorsInRegistry = Actor.registry.actors.length
actor ! "Ping" actor ! "Ping"
latch.await(1, TimeUnit.SECONDS) must be (true) latch.await(1, TimeUnit.SECONDS) must be (true)
numberOfActorsInRegistry must equal (ActorRegistry.actors.length) numberOfActorsInRegistry must equal (Actor.registry.actors.length)
} }
"UseServiceNameAsIdForRemoteActorRef" in { "UseServiceNameAsIdForRemoteActorRef" in {
@ -194,7 +194,7 @@ class ServerInitiatedRemoteActorSpec extends AkkaRemoteTest {
latch.countDown 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 must have size(2) //No new are allowed to have been created
decrementers.find( _ eq localFoo) must equal (Some(localFoo)) decrementers.find( _ eq localFoo) must equal (Some(localFoo))
decrementers.find( _ eq localBar) must equal (Some(localBar)) decrementers.find( _ eq localBar) must equal (Some(localBar))

View file

@ -39,10 +39,10 @@ class ServerInitiatedRemoteTypedActorSpec extends AkkaRemoteTest {
"should not recreate registered actors" in { "should not recreate registered actors" in {
val actor = createRemoteActorRef val actor = createRemoteActorRef
val numberOfActorsInRegistry = ActorRegistry.actors.length val numberOfActorsInRegistry = Actor.registry.actors.length
actor.oneWay actor.oneWay
oneWayLog.poll(5, TimeUnit.SECONDS) must equal ("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 { "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 { "An enabled optimized local scoped remote" should {
"Fetch remote actor ref when scope is local" in { "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.register("foo", fooActor)
remote.actorFor("foo", host, port) must not be (fooActor) remote.actorFor("foo", host, port) must not be (fooActor)
} }
"Create remote actor when client-managed is hosted locally" in { "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) localClientManaged.homeAddress must not be (None)
} }

View file

@ -43,7 +43,7 @@ class UntypedActorSerializationSpec extends
describe("Serializable untyped actor") { describe("Serializable untyped actor") {
it("should be able to serialize and de-serialize a stateful 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("hello") should equal("world 1")
actor1.sendRequestReply("debasish") should equal("hello debasish 2") actor1.sendRequestReply("debasish") should equal("hello debasish 2")

View file

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

View file

@ -19,9 +19,9 @@ class HelloWorldActor extends Actor {
object ServerManagedRemoteActorServer extends Logging { object ServerManagedRemoteActorServer extends Logging {
def run = { def run = {
ActorRegistry.remote.start("localhost", 2552) Actor.remote.start("localhost", 2552)
log.slf4j.info("Remote node started") 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") log.slf4j.info("Remote actor registered and started")
} }
@ -31,7 +31,7 @@ object ServerManagedRemoteActorServer extends Logging {
object ServerManagedRemoteActorClient extends Logging { object ServerManagedRemoteActorClient extends Logging {
def run = { 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("Remote client created")
log.slf4j.info("Sending 'Hello' to remote actor") log.slf4j.info("Sending 'Hello' to remote actor")
val result = actor !! "Hello" val result = actor !! "Hello"

View file

@ -2,7 +2,7 @@ package akka.transactor.example;
import akka.transactor.Coordinated; import akka.transactor.Coordinated;
import akka.transactor.Atomically; import akka.transactor.Atomically;
import akka.actor.ActorRef; import akka.actor.Actors;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.stm.Ref; import akka.stm.Ref;

View file

@ -2,7 +2,7 @@ package akka.transactor.example;
import akka.transactor.Coordinated; import akka.transactor.Coordinated;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.UntypedActor; import akka.actor.Actors;
import akka.dispatch.Future; import akka.dispatch.Future;
import akka.dispatch.Futures; import akka.dispatch.Futures;
@ -12,8 +12,8 @@ public class UntypedCoordinatedExample {
System.out.println("Untyped transactor example"); System.out.println("Untyped transactor example");
System.out.println(); System.out.println();
ActorRef counter1 = UntypedActor.actorOf(UntypedCoordinatedCounter.class).start(); ActorRef counter1 = Actors.actorOf(UntypedCoordinatedCounter.class).start();
ActorRef counter2 = UntypedActor.actorOf(UntypedCoordinatedCounter.class).start(); ActorRef counter2 = Actors.actorOf(UntypedCoordinatedCounter.class).start();
counter1.sendOneWay(new Coordinated(new Increment(counter2))); counter1.sendOneWay(new Coordinated(new Increment(counter2)));

View file

@ -1,7 +1,7 @@
package akka.transactor.example; package akka.transactor.example;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.UntypedActor; import akka.actor.Actors;
import akka.dispatch.Future; import akka.dispatch.Future;
import akka.dispatch.Futures; import akka.dispatch.Futures;
@ -11,8 +11,8 @@ public class UntypedTransactorExample {
System.out.println("Untyped transactor example"); System.out.println("Untyped transactor example");
System.out.println(); System.out.println();
ActorRef counter1 = UntypedActor.actorOf(UntypedCounter.class).start(); ActorRef counter1 = Actors.actorOf(UntypedCounter.class).start();
ActorRef counter2 = UntypedActor.actorOf(UntypedCounter.class).start(); ActorRef counter2 = Actors.actorOf(UntypedCounter.class).start();
counter1.sendOneWay(new Increment(counter2)); counter1.sendOneWay(new Increment(counter2));

View file

@ -3,6 +3,7 @@ package akka.transactor.test;
import akka.transactor.Coordinated; import akka.transactor.Coordinated;
import akka.transactor.Atomically; import akka.transactor.Atomically;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Actors;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.stm.*; import akka.stm.*;
import akka.util.Duration; import akka.util.Duration;

View file

@ -5,6 +5,7 @@ import org.junit.Test;
import org.junit.Before; import org.junit.Before;
import akka.transactor.Coordinated; import akka.transactor.Coordinated;
import akka.actor.Actors;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory; import akka.actor.UntypedActorFactory;
@ -28,7 +29,7 @@ public class UntypedCoordinatedIncrementTest {
counters = new ArrayList<ActorRef>(); counters = new ArrayList<ActorRef>();
for (int i = 1; i <= numCounters; i++) { for (int i = 1; i <= numCounters; i++) {
final String name = "counter" + i; final String name = "counter" + i;
ActorRef counter = UntypedActor.actorOf(new UntypedActorFactory() { ActorRef counter = Actors.actorOf(new UntypedActorFactory() {
public UntypedActor create() { public UntypedActor create() {
return new UntypedCoordinatedCounter(name); return new UntypedCoordinatedCounter(name);
} }
@ -36,7 +37,7 @@ public class UntypedCoordinatedIncrementTest {
counter.start(); counter.start();
counters.add(counter); counters.add(counter);
} }
failer = UntypedActor.actorOf(UntypedFailer.class); failer = Actors.actorOf(UntypedFailer.class);
failer.start(); failer.start();
} }

View file

@ -5,6 +5,7 @@ import org.junit.Test;
import org.junit.Before; import org.junit.Before;
import akka.actor.ActorRef; import akka.actor.ActorRef;
import akka.actor.Actors;
import akka.actor.UntypedActor; import akka.actor.UntypedActor;
import akka.actor.UntypedActorFactory; import akka.actor.UntypedActorFactory;
import akka.dispatch.Future; import akka.dispatch.Future;
@ -27,7 +28,7 @@ public class UntypedTransactorTest {
counters = new ArrayList<ActorRef>(); counters = new ArrayList<ActorRef>();
for (int i = 1; i <= numCounters; i++) { for (int i = 1; i <= numCounters; i++) {
final String name = "counter" + i; final String name = "counter" + i;
ActorRef counter = UntypedActor.actorOf(new UntypedActorFactory() { ActorRef counter = Actors.actorOf(new UntypedActorFactory() {
public UntypedActor create() { public UntypedActor create() {
return new UntypedCounter(name); return new UntypedCounter(name);
} }
@ -35,7 +36,7 @@ public class UntypedTransactorTest {
counter.start(); counter.start();
counters.add(counter); counters.add(counter);
} }
failer = UntypedActor.actorOf(UntypedFailer.class); failer = Actors.actorOf(UntypedFailer.class);
failer.start(); 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. * 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 { object TypedActorConfiguration {
@ -539,7 +539,7 @@ object TypedActor extends Logging {
config match { config match {
case null => actorOf(typedActor) case null => actorOf(typedActor)
case c: TypedActorConfiguration if (c._host.isDefined) => 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) case _ => actorOf(typedActor)
} }
} }
@ -676,9 +676,10 @@ object TypedActor extends Logging {
* Get the underlying typed actor for the given Typed Actor. * Get the underlying typed actor for the given Typed Actor.
*/ */
def actorFor(proxy: AnyRef): Option[ActorRef] = def actorFor(proxy: AnyRef): Option[ActorRef] =
ActorRegistry Actor.registry find {
.actorsFor(classOf[TypedActor]) case a if classOf[TypedActor].isAssignableFrom(a.actor.getClass) && a.actor.asInstanceOf[TypedActor].proxy == proxy =>
.find(a => a.actor.asInstanceOf[TypedActor].proxy == proxy) a
}
/** /**
* Get the typed actor proxy for the given Typed Actor. * 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 (message: Array[AnyRef], isEscaped) = escapeArguments(methodRtti.getParameterValues)
val future = ActorRegistry.remote.send[AnyRef]( val future = Actor.remote.send[AnyRef](
message, None, None, remoteAddress.get, message, None, None, remoteAddress.get,
timeout, isOneWay, actorRef, timeout, isOneWay, actorRef,
Some((interfaceClass.getName, methodRtti.getMethod.getName)), Some((interfaceClass.getName, methodRtti.getMethod.getName)),

View file

@ -112,7 +112,7 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa
component.remoteAddress match { component.remoteAddress match {
case Some(a) => case Some(a) =>
(Some(new InetSocketAddress(a.hostname, a.port)), (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 => case None =>
(None, Actor.actorOf(TypedActor.newTypedActor(implementationClass))) (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]) val obj = TypedActor.newInstance(classOf[SamplePojoAnnotated])
assert(AspectInitRegistry.initFor(obj) ne null) assert(AspectInitRegistry.initFor(obj) ne null)
assert("hello akka" === obj.greet("akka")) assert("hello akka" === obj.greet("akka"))
ActorRegistry.shutdownAll Actor.registry.shutdownAll
assert(AspectInitRegistry.initFor(obj) eq null) assert(AspectInitRegistry.initFor(obj) eq null)
assert(!obj.pre) assert(!obj.pre)
assert(!obj.post) assert(!obj.post)

View file

@ -17,59 +17,59 @@ class TypedActorRegistrySpec extends WordSpec with MustMatchers {
"Typed Actor" should { "Typed Actor" should {
"be able to be retreived from the registry by class" in { "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 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) actors.length must be (1)
ActorRegistry.shutdownAll Actor.registry.shutdownAll
} }
"be able to be retreived from the registry by manifest" in { "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 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 must not be (null)
option.isDefined must be (true) option.isDefined must be (true)
ActorRegistry.shutdownAll Actor.registry.shutdownAll
} }
"be able to be retreived from the registry by class two times" in { "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 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) actors1.length must be (1)
val actors2 = ActorRegistry.typedActorsFor(classOf[My]) val actors2 = Actor.registry.typedActorsFor(classOf[My])
actors2.length must be (1) actors2.length must be (1)
ActorRegistry.shutdownAll Actor.registry.shutdownAll
} }
"be able to be retreived from the registry by manifest two times" in { "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 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 must not be (null)
option1.isDefined must be (true) option1.isDefined must be (true)
val option2 = ActorRegistry.typedActorFor[My] val option2 = Actor.registry.typedActorFor[My]
option2 must not be (null) option2 must not be (null)
option2.isDefined must be (true) 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 { "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 val manager = new TypedActorConfigurator
manager.configure( manager.configure(
OneForOneStrategy(classOf[Exception] :: Nil, 3, 1000), OneForOneStrategy(classOf[Exception] :: Nil, 3, 1000),
Array(new SuperviseTypedActor(classOf[My], classOf[MyImpl], Permanent, 6000)) Array(new SuperviseTypedActor(classOf[My], classOf[MyImpl], Permanent, 6000))
).supervise ).supervise
val option1 = ActorRegistry.typedActorFor[My] val option1 = Actor.registry.typedActorFor[My]
option1 must not be (null) option1 must not be (null)
option1.isDefined must be (true) option1.isDefined must be (true)
val option2 = ActorRegistry.typedActorFor[My] val option2 = Actor.registry.typedActorFor[My]
option2 must not be (null) option2 must not be (null)
option2.isDefined must be (true) option2.isDefined must be (true)
ActorRegistry.shutdownAll Actor.registry.shutdownAll
} }
} }
} }

View file

@ -68,7 +68,7 @@ class TypedActorSpec extends
} }
override def afterEach() { override def afterEach() {
ActorRegistry.shutdownAll Actor.registry.shutdownAll
} }
describe("TypedActor") { describe("TypedActor") {
@ -115,67 +115,67 @@ class TypedActorSpec extends
it("should support finding a typed actor by uuid ") { it("should support finding a typed actor by uuid ") {
val typedActorRef = TypedActor.actorFor(simplePojo).get val typedActorRef = TypedActor.actorFor(simplePojo).get
val uuid = typedActorRef.uuid val uuid = typedActorRef.uuid
assert(ActorRegistry.typedActorFor(newUuid()) === None) assert(Actor.registry.typedActorFor(newUuid()) === None)
assert(ActorRegistry.typedActorFor(uuid).isDefined) assert(Actor.registry.typedActorFor(uuid).isDefined)
assert(ActorRegistry.typedActorFor(uuid).get === simplePojo) assert(Actor.registry.typedActorFor(uuid).get === simplePojo)
} }
it("should support finding typed actors by id ") { 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.length === 1)
assert(typedActors.contains(pojo)) assert(typedActors.contains(pojo))
// creating untyped actor with same custom id // creating untyped actor with same custom id
val actorRef = Actor.actorOf[MyActor].start 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.length === 1)
assert(typedActors2.contains(pojo)) assert(typedActors2.contains(pojo))
actorRef.stop actorRef.stop
} }
it("should support to filter typed actors") { 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.length === 1)
assert(actors.contains(pojo)) assert(actors.contains(pojo))
} }
it("should support to find typed actors by class") { 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.length === 1)
assert(actors.contains(pojo)) 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") { it("should support to get all typed actors") {
val actors = ActorRegistry.typedActors val actors = Actor.registry.typedActors
assert(actors.length === 2) assert(actors.length === 2)
assert(actors.contains(pojo)) assert(actors.contains(pojo))
assert(actors.contains(simplePojo)) assert(actors.contains(simplePojo))
} }
it("should support to find typed actors by manifest") { 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.length === 1)
assert(actors.contains(pojo)) assert(actors.contains(pojo))
assert(ActorRegistry.typedActorsFor[MyActor].isEmpty) assert(Actor.registry.typedActorsFor[MyActor].isEmpty)
} }
it("should support foreach for typed actors") { it("should support foreach for typed actors") {
val actorRef = Actor.actorOf[MyActor].start val actorRef = Actor.actorOf[MyActor].start
assert(ActorRegistry.actors.size === 3) assert(Actor.registry.actors.size === 3)
assert(ActorRegistry.typedActors.size === 2) assert(Actor.registry.typedActors.size === 2)
ActorRegistry.foreachTypedActor(TypedActor.stop(_)) Actor.registry.foreachTypedActor(TypedActor.stop(_))
assert(ActorRegistry.actors.size === 1) assert(Actor.registry.actors.size === 1)
assert(ActorRegistry.typedActors.size === 0) assert(Actor.registry.typedActors.size === 0)
} }
it("should shutdown all typed and untyped actors") { it("should shutdown all typed and untyped actors") {
val actorRef = Actor.actorOf[MyActor].start val actorRef = Actor.actorOf[MyActor].start
assert(ActorRegistry.actors.size === 3) assert(Actor.registry.actors.size === 3)
assert(ActorRegistry.typedActors.size === 2) assert(Actor.registry.typedActors.size === 2)
ActorRegistry.shutdownAll() Actor.registry.shutdownAll()
assert(ActorRegistry.actors.size === 0) assert(Actor.registry.actors.size === 0)
assert(ActorRegistry.typedActors.size === 0) assert(Actor.registry.typedActors.size === 0)
} }
} }
} }