context under a name defined by ActiveObjectComponent.InternalSchema.
+ * {@link se.scalablesolutions.akka.camel.component.TypedActorComponent} at
+ * context under a name defined by TypedActorComponent.InternalSchema.
*/
def init(context: CamelContext) {
- this.activeObjectComponent = new ActiveObjectComponent
- this.activeObjectRegistry = activeObjectComponent.activeObjectRegistry
+ this.typedActorComponent = new TypedActorComponent
+ this.typedActorRegistry = typedActorComponent.typedActorRegistry
this.context = context
this.context.setStreamCaching(true)
- this.context.addComponent(ActiveObjectComponent.InternalSchema, activeObjectComponent)
+ this.context.addComponent(TypedActorComponent.InternalSchema, typedActorComponent)
this.template = context.createProducerTemplate
_initialized = true
log.info("Camel context initialized")
diff --git a/akka-camel/src/main/scala/CamelService.scala b/akka-camel/src/main/scala/CamelService.scala
index 65a6a44fe5..d47daf8d9a 100644
--- a/akka-camel/src/main/scala/CamelService.scala
+++ b/akka-camel/src/main/scala/CamelService.scala
@@ -1,24 +1,24 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB se.scalablesolutions.akka.camel.CamelContextManager.context.
*
* @author Martin Krasser
*/
trait CamelService extends Bootable with Logging {
-
- import CamelContextManager._
-
private[camel] val consumerPublisher = actorOf[ConsumerPublisher]
private[camel] val publishRequestor = actorOf[PublishRequestor]
@@ -32,72 +32,142 @@ trait CamelService extends Bootable with Logging {
* Starts the CamelService. Any started actor that is a consumer actor will be (asynchronously)
* published as Camel endpoint. Consumer actors that are started after this method returned will
* be published as well. Actor publishing is done asynchronously. A started (loaded) CamelService
- * also publishes @consume annotated methods of active objects that have been created
- * with ActiveObject.newInstance(..) (and ActiveObject.newInstance(..)
+ * also publishes @consume annotated methods of typed actors that have been created
+ * with TypedActor.newInstance(..) (and TypedActor.newInstance(..)
* on a remote node).
*/
abstract override def onLoad = {
super.onLoad
// Only init and start if not already done by application
- if (!initialized) init
- if (!started) start
+ if (!CamelContextManager.initialized) CamelContextManager.init
+ if (!CamelContextManager.started) CamelContextManager.start
- // start actor that exposes consumer actors and active objects via Camel endpoints
+ // start actor that exposes consumer actors and typed actors via Camel endpoints
consumerPublisher.start
// init publishRequestor so that buffered and future events are delivered to consumerPublisher
publishRequestor ! PublishRequestorInit(consumerPublisher)
+
+ // Register this instance as current CamelService
+ CamelServiceManager.register(this)
}
/**
* Stops the CamelService.
*/
abstract override def onUnload = {
+ // Unregister this instance as current CamelService
+ CamelServiceManager.unregister(this)
+
+ // Remove related listeners from registry
ActorRegistry.removeListener(publishRequestor)
AspectInitRegistry.removeListener(publishRequestor)
+
+ // Stop related services
consumerPublisher.stop
- stop
+ CamelContextManager.stop
+
super.onUnload
}
+ @deprecated("use start() instead")
+ def load: CamelService = {
+ onLoad
+ this
+ }
+
+ @deprecated("use stop() instead")
+ def unload = onUnload
+
/**
* Starts the CamelService.
*
* @see onLoad
*/
- def load: CamelService = {
+ def start: CamelService = {
onLoad
this
}
-
+
/**
* Stops the CamelService.
*
* @see onUnload
*/
- def unload = onUnload
+ def stop = onUnload
+
+ /**
+ * Sets an expectation of the number of upcoming endpoint activations and returns
+ * a {@link CountDownLatch} that can be used to wait for the activations to occur.
+ * Endpoint activations that occurred in the past are not considered.
+ */
+ def expectEndpointActivationCount(count: Int): CountDownLatch =
+ (consumerPublisher !! SetExpectedRegistrationCount(count)).as[CountDownLatch].get
+
+ /**
+ * Sets an expectation of the number of upcoming endpoint de-activations and returns
+ * a {@link CountDownLatch} that can be used to wait for the de-activations to occur.
+ * Endpoint de-activations that occurred in the past are not considered.
+ */
+ def expectEndpointDeactivationCount(count: Int): CountDownLatch =
+ (consumerPublisher !! SetExpectedUnregistrationCount(count)).as[CountDownLatch].get
}
/**
- * CamelService companion object used by standalone applications to create their own
- * CamelService instance.
+ * ...
*
* @author Martin Krasser
*/
-object CamelService {
+object CamelServiceManager {
/**
- * Creates a new CamelService instance.
+ * The current (optional) CamelService. Is defined when a CamelService has been started.
*/
- def newInstance: CamelService = new DefaultCamelService
+ private var _current: Option[CamelService] = None
+
+ /**
+ * Starts a new CamelService and makes it the current CamelService.
+ */
+ def startCamelService = CamelServiceFactory.createCamelService.start
+
+ /**
+ * Stops the current CamelService.
+ */
+ def stopCamelService = service.stop
+
+ /**
+ * Returns the current CamelService.
+ *
+ * @throws IllegalStateException if there's no current CamelService.
+ */
+ def service =
+ if (_current.isDefined) _current.get
+ else throw new IllegalStateException("no current CamelService")
+
+ private[camel] def register(service: CamelService) =
+ if (_current.isDefined) throw new IllegalStateException("current CamelService already registered")
+ else _current = Some(service)
+
+ private[camel] def unregister(service: CamelService) =
+ if (_current == Some(service)) _current = None
+ else throw new IllegalStateException("only current CamelService can be unregistered")
}
/**
- * Default CamelService implementation to be created in Java applications with
- * - * CamelService service = new DefaultCamelService() - *+ * @author Martin Krasser */ -class DefaultCamelService extends CamelService { +object CamelServiceFactory { + /** + * Creates a new CamelService instance + */ + def createCamelService: CamelService = new CamelService { } + + /** + * Creates a new CamelService instance + */ + def createCamelService(camelContext: CamelContext): CamelService = { + CamelContextManager.init(camelContext) + createCamelService + } } diff --git a/akka-camel/src/main/scala/Consumer.scala b/akka-camel/src/main/scala/Consumer.scala index e5218a21f2..3b54c5df77 100644 --- a/akka-camel/src/main/scala/Consumer.scala +++ b/akka-camel/src/main/scala/Consumer.scala @@ -4,7 +4,9 @@ package se.scalablesolutions.akka.camel -import se.scalablesolutions.akka.actor.{ActorRef, Actor} +import se.scalablesolutions.akka.actor._ + +import java.net.InetSocketAddress /** * Mixed in by Actor implementations that consume message from Camel endpoints. @@ -25,6 +27,58 @@ trait Consumer { self: Actor => def blocking = false } +/** + * Java-friendly {@link Consumer} inherited by + * + *
PublishRequestorInit command message for obtaining a reference to
* a publisher actor. Before initialization it buffers all outbound messages
@@ -209,7 +208,7 @@ private[camel] class PublishRequestor extends Actor {
/**
* Command message to initialize a PublishRequestor to use consumerPublisher
- * for publishing actors or active object methods.
+ * for publishing actors or typed actor methods.
*/
private[camel] case class PublishRequestorInit(consumerPublisher: ActorRef)
@@ -244,32 +243,30 @@ private[camel] case class ConsumerRegistered(actorRef: ActorRef, uri: String, uu
private[camel] case class ConsumerUnregistered(actorRef: ActorRef, uri: String, uuid: String) extends ConsumerEvent
/**
- * Event indicating that an active object proxy has been created for a POJO. For each
- * @consume annotated POJO method a separate instance of this class is
- * created.
+ * Event indicating that an typed actor proxy has been created for a typed actor. For each @consume
+ * annotated typed actor method a separate instance of this class is created.
*
- * @param activeObject active object (proxy).
+ * @param typedActor typed actor (proxy).
* @param init
- * @param uri endpoint URI of the active object method
+ * @param uri endpoint URI of the typed actor method
* @param method method to be published.
*
* @author Martin Krasser
*/
-private[camel] case class ConsumerMethodRegistered(activeObject: AnyRef, init: AspectInit, uri: String, method: Method) extends ConsumerEvent
+private[camel] case class ConsumerMethodRegistered(typedActor: AnyRef, init: AspectInit, uri: String, method: Method) extends ConsumerEvent
/**
- * Event indicating that an active object has been stopped. For each
- * @consume annotated POJO method a separate instance of this class is
- * created.
+ * Event indicating that an typed actor has been stopped. For each @consume
+ * annotated typed object method a separate instance of this class is created.
*
- * @param activeObject active object (proxy).
+ * @param typedActor typed actor (proxy).
* @param init
- * @param uri endpoint URI of the active object method
+ * @param uri endpoint URI of the typed actor method
* @param method method to be un-published.
*
* @author Martin Krasser
*/
-private[camel] case class ConsumerMethodUnregistered(activeObject: AnyRef, init: AspectInit, uri: String, method: Method) extends ConsumerEvent
+private[camel] case class ConsumerMethodUnregistered(typedActor: AnyRef, init: AspectInit, uri: String, method: Method) extends ConsumerEvent
/**
* @author Martin Krasser
@@ -306,18 +303,22 @@ private[camel] object ConsumerUnregistered {
*/
private[camel] object ConsumerMethod {
/**
- * Applies a function f to each consumer method of activeObject and
+ * Applies a function f to each consumer method of TypedActor and
* returns the function results as a list. A consumer method is one that is annotated with
- * @consume. If activeObject is a proxy for a remote active object
+ * @consume. If typedActor is a proxy for a remote typed actor
* f is never called and Nil is returned.
*/
- def forConsumer[T](activeObject: AnyRef, init: AspectInit)(f: Method => T): List[T] = {
- // TODO: support consumer annotation inheritance
- // - visit overridden methods in superclasses
- // - visit implemented method declarations in interfaces
- if (init.remoteAddress.isDefined) Nil // let remote node publish active object methods on endpoints
- else for (m <- activeObject.getClass.getMethods.toList; if (m.isAnnotationPresent(classOf[consume])))
- yield f(m)
+ def forConsumer[T](typedActor: AnyRef, init: AspectInit)(f: Method => T): List[T] = {
+ if (init.remoteAddress.isDefined) Nil // let remote node publish typed actor methods on endpoints
+ else {
+ // TODO: support consumer annotation inheritance
+ // - visit overridden methods in superclasses
+ // - visit implemented method declarations in interfaces
+ val intfClass = typedActor.getClass
+ val implClass = init.targetInstance.getClass
+ (for (m <- intfClass.getMethods.toList; if (m.isAnnotationPresent(classOf[consume]))) yield f(m)) ++
+ (for (m <- implClass.getMethods.toList; if (m.isAnnotationPresent(classOf[consume]))) yield f(m))
+ }
}
}
@@ -326,13 +327,13 @@ private[camel] object ConsumerMethod {
*/
private[camel] object ConsumerMethodRegistered {
/**
- * Creates a list of ConsumerMethodRegistered event messages for an active object or an empty
- * list if the active object is a proxy for an remote active object or the active object doesn't
+ * Creates a list of ConsumerMethodRegistered event messages for an typed actor or an empty
+ * list if the typed actor is a proxy for an remote typed actor or the typed actor doesn't
* have any @consume annotated methods.
*/
- def forConsumer(activeObject: AnyRef, init: AspectInit): List[ConsumerMethodRegistered] = {
- ConsumerMethod.forConsumer(activeObject, init) {
- m => ConsumerMethodRegistered(activeObject, init, m.getAnnotation(classOf[consume]).value, m)
+ def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodRegistered] = {
+ ConsumerMethod.forConsumer(typedActor, init) {
+ m => ConsumerMethodRegistered(typedActor, init, m.getAnnotation(classOf[consume]).value, m)
}
}
}
@@ -342,13 +343,13 @@ private[camel] object ConsumerMethodRegistered {
*/
private[camel] object ConsumerMethodUnregistered {
/**
- * Creates a list of ConsumerMethodUnregistered event messages for an active object or an empty
- * list if the active object is a proxy for an remote active object or the active object doesn't
+ * Creates a list of ConsumerMethodUnregistered event messages for an typed actor or an empty
+ * list if the typed actor is a proxy for an remote typed actor or the typed actor doesn't
* have any @consume annotated methods.
*/
- def forConsumer(activeObject: AnyRef, init: AspectInit): List[ConsumerMethodUnregistered] = {
- ConsumerMethod.forConsumer(activeObject, init) {
- m => ConsumerMethodUnregistered(activeObject, init, m.getAnnotation(classOf[consume]).value, m)
+ def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodUnregistered] = {
+ ConsumerMethod.forConsumer(typedActor, init) {
+ m => ConsumerMethodUnregistered(typedActor, init, m.getAnnotation(classOf[consume]).value, m)
}
}
}
diff --git a/akka-camel/src/main/scala/Message.scala b/akka-camel/src/main/scala/Message.scala
index ec382df0c9..a834568a22 100644
--- a/akka-camel/src/main/scala/Message.scala
+++ b/akka-camel/src/main/scala/Message.scala
@@ -23,7 +23,6 @@ case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
*
* @see CamelContextManager.
*/
- @deprecated("use bodyAs[T](implicit m: Manifest[T]): T instead")
def bodyAs[T](clazz: Class[T]): T =
CamelContextManager.context.getTypeConverter.mandatoryConvertTo[T](clazz, body)
@@ -43,6 +42,26 @@ case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
*/
def headers(names: Set[String]): Map[String, Any] = headers.filter(names contains _._1)
+ /**
+ * Returns the header with given name. Throws NoSuchElementException
+ * if the header doesn't exist.
+ */
+ def header(name: String): Any = headers(name)
+
+ /**
+ * Returns the header with given name converted to type T. Throws
+ * NoSuchElementException if the header doesn't exist.
+ */
+ def headerAs[T](name: String)(implicit m: Manifest[T]): T =
+ CamelContextManager.context.getTypeConverter.mandatoryConvertTo[T](m.erasure.asInstanceOf[Class[T]], header(name))
+
+ /**
+ * Returns the header with given name converted to type given by the clazz
+ * argument. Throws NoSuchElementException if the header doesn't exist.
+ */
+ def headerAs[T](name: String, clazz: Class[T]): T =
+ CamelContextManager.context.getTypeConverter.mandatoryConvertTo[T](clazz, header(name))
+
/**
* Creates a Message with a new body using a transformer function.
*/
diff --git a/akka-camel/src/main/scala/Producer.scala b/akka-camel/src/main/scala/Producer.scala
index c49591ec7f..6f5c914a65 100644
--- a/akka-camel/src/main/scala/Producer.scala
+++ b/akka-camel/src/main/scala/Producer.scala
@@ -9,14 +9,14 @@ import CamelMessageConversion.toExchangeAdapter
import org.apache.camel._
import org.apache.camel.processor.SendProcessor
-import se.scalablesolutions.akka.actor.{Actor, ActorRef}
+import se.scalablesolutions.akka.actor.{Actor, ActorRef, UntypedActor}
/**
- * Mixed in by Actor implementations that produce messages to Camel endpoints.
+ * Support trait for producing messages to Camel endpoints.
*
* @author Martin Krasser
*/
-trait Producer { this: Actor =>
+trait ProducerSupport { this: Actor =>
/**
* Message headers to copy by default from request message to response-message.
@@ -141,11 +141,6 @@ trait Producer { this: Actor =>
case msg => if (!oneway) self.reply(msg)
}
- /**
- * Default implementation of Actor.receive
- */
- protected def receive = produce
-
/**
* Creates a new Exchange with given pattern from the endpoint specified by
* endpointUri.
@@ -162,6 +157,78 @@ trait Producer { this: Actor =>
}
}
+/**
+ * Mixed in by Actor implementations that produce messages to Camel endpoints.
+ */
+trait Producer extends ProducerSupport { this: Actor =>
+
+ /**
+ * Default implementation of Actor.receive
+ */
+ protected def receive = produce
+}
+
+/**
+ * Java-friendly {@link ProducerSupport} inherited by {@link UntypedProducerActor} implementations.
+ *
+ * @author Martin Krasser
+ */
+trait UntypedProducer extends ProducerSupport { this: UntypedActor =>
+
+ final override def endpointUri = getEndpointUri
+
+ final override def oneway = isOneway
+
+ final override def receiveBeforeProduce = {
+ case msg => onReceiveBeforeProduce(msg)
+ }
+
+ final override def receiveAfterProduce = {
+ case msg => onReceiveAfterProduce(msg)
+ }
+
+ /**
+ * Default implementation of UntypedActor.onReceive
+ */
+ def onReceive(message: Any) = produce(message)
+
+ /**
+ * Returns the Camel endpoint URI to produce messages to.
+ */
+ def getEndpointUri(): String
+
+ /**
+ * If set to false (default), this producer expects a response message from the Camel endpoint.
+ * If set to true, this producer communicates with the Camel endpoint with an in-only message
+ * exchange pattern (fire and forget).
+ */
+ def isOneway() = super.oneway
+
+ /**
+ * Called before the message is sent to the endpoint specified by getEndpointUri. The original
+ * message is passed as argument. By default, this method simply returns the argument but may be overridden
+ * by subclasses.
+ */
+ @throws(classOf[Exception])
+ def onReceiveBeforeProduce(message: Any): Any = super.receiveBeforeProduce(message)
+
+ /**
+ * Called after the a result was received from the endpoint specified by getEndpointUri. The
+ * result is passed as argument. By default, this method replies the result back to the original sender
+ * if isOneway returns false. If isOneway returns true then nothing is done. This
+ * method may be overridden by subclasses.
+ */
+ @throws(classOf[Exception])
+ def onReceiveAfterProduce(message: Any): Unit = super.receiveAfterProduce(message)
+}
+
+/**
+ * Subclass this abstract class to create an untyped producer actor. This class is meant to be used from Java.
+ *
+ * @author Martin Krasser
+ */
+abstract class UntypedProducerActor extends UntypedActor with UntypedProducer
+
/**
* @author Martin Krasser
*/
diff --git a/akka-camel/src/main/scala/component/ActorComponent.scala b/akka-camel/src/main/scala/component/ActorComponent.scala
index e267fcd077..a5d56dd9dc 100644
--- a/akka-camel/src/main/scala/component/ActorComponent.scala
+++ b/akka-camel/src/main/scala/component/ActorComponent.scala
@@ -14,18 +14,17 @@ import jsr166x.Deque
import org.apache.camel._
import org.apache.camel.impl.{DefaultProducer, DefaultEndpoint, DefaultComponent}
-import se.scalablesolutions.akka.actor.{ActorRegistry, Actor, ActorRef}
import se.scalablesolutions.akka.camel.{Failure, CamelMessageConversion, Message}
+import CamelMessageConversion.toExchangeAdapter
import se.scalablesolutions.akka.dispatch.{CompletableFuture, MessageInvocation, MessageDispatcher}
import se.scalablesolutions.akka.stm.TransactionConfig
+import se.scalablesolutions.akka.actor.{ScalaActorRef, ActorRegistry, Actor, ActorRef}
+import se.scalablesolutions.akka.AkkaException
import scala.reflect.BeanProperty
-import CamelMessageConversion.toExchangeAdapter
-import java.lang.Throwable
-
/**
- * Camel component for sending messages to and receiving replies from actors.
+ * Camel component for sending messages to and receiving replies from (untyped) actors.
*
* @see se.scalablesolutions.akka.camel.component.ActorEndpoint
* @see se.scalablesolutions.akka.camel.component.ActorProducer
@@ -50,7 +49,7 @@ class ActorComponent extends DefaultComponent {
}
/**
- * Camel endpoint for referencing an actor. The actor reference is given by the endpoint URI.
+ * Camel endpoint for referencing an (untyped) actor. The actor reference is given by the endpoint URI.
* An actor can be referenced by its ActorRef.id or its ActorRef.uuid.
* Supported endpoint URI formats are
* actor:<actorid>,
@@ -68,7 +67,7 @@ class ActorEndpoint(uri: String,
val uuid: Option[String]) extends DefaultEndpoint(uri, comp) {
/**
- * Blocking of client thread during two-way message exchanges with consumer actors. This is set
+ * Blocking of caller thread during two-way message exchanges with consumer actors. This is set
* via the blocking=true|false endpoint URI parameter. If omitted blocking is false.
*/
@BeanProperty var blocking: Boolean = false
@@ -91,7 +90,7 @@ class ActorEndpoint(uri: String,
}
/**
- * Sends the in-message of an exchange to an actor. If the exchange pattern is out-capable and
+ * Sends the in-message of an exchange to an (untyped) actor. If the exchange pattern is out-capable and
* blocking is enabled then the producer waits for a reply (using the !! operator),
* otherwise the ! operator is used for sending the message.
*
@@ -132,10 +131,8 @@ class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) with Asyn
result match {
case Some(msg: Failure) => exchange.fromFailureMessage(msg)
case Some(msg) => exchange.fromResponseMessage(Message.canonicalize(msg))
- case None => {
- throw new TimeoutException("timeout (%d ms) while waiting response from %s"
- format (actor.timeout, ep.getEndpointUri))
- }
+ case None => throw new TimeoutException("timeout (%d ms) while waiting response from %s"
+ format (actor.timeout, ep.getEndpointUri))
}
}
@@ -150,9 +147,8 @@ class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) with Asyn
else targetByUuid(ep.uuid.get)
private def targetById(id: String) = ActorRegistry.actorsFor(id) match {
- case Nil => None
- case actor :: Nil => Some(actor)
- case actors => Some(actors.head)
+ case actors if actors.length == 0 => None
+ case actors => Some(actors(0))
}
private def targetByUuid(uuid: String) = ActorRegistry.actorFor(uuid)
@@ -200,7 +196,7 @@ private[akka] object AsyncCallbackAdapter {
*
* @author Martin Krasser
*/
-private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCallback) extends ActorRef {
+private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCallback) extends ActorRef with ScalaActorRef {
def start = {
_isRunning = true
@@ -242,15 +238,15 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
def unlink(actorRef: ActorRef): Unit = unsupported
def startLink(actorRef: ActorRef): Unit = unsupported
def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = unsupported
- def spawn[T <: Actor : Manifest]: ActorRef = unsupported
- def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef = unsupported
- def spawnLink[T <: Actor: Manifest]: ActorRef = unsupported
- def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef = unsupported
+ def spawn(clazz: Class[_ <: Actor]): ActorRef = unsupported
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
+ def spawnLink(clazz: Class[_ <: Actor]): ActorRef = unsupported
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
def shutdownLinkedActors: Unit = unsupported
- def mailboxSize: Int = unsupported
def supervisor: Option[ActorRef] = unsupported
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T](message: Any, timeout: Long, senderOption: Option[ActorRef], senderFuture: Option[CompletableFuture[T]]) = unsupported
- protected[akka] def mailbox: Deque[MessageInvocation] = unsupported
+ protected[akka] def mailbox: AnyRef = unsupported
+ protected[akka] def mailbox_=(msg: AnyRef):AnyRef = unsupported
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
@@ -260,7 +256,7 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported
protected[akka] def registerSupervisorAsRemoteActor = unsupported
protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = unsupported
- protected[this] def actorInstance: AtomicReference[Actor] = unsupported
+ protected[akka] def actorInstance: AtomicReference[Actor] = unsupported
private def unsupported = throw new UnsupportedOperationException("Not supported for %s" format classOf[AsyncCallbackAdapter].getName)
-}
\ No newline at end of file
+}
diff --git a/akka-camel/src/main/scala/component/ActiveObjectComponent.scala b/akka-camel/src/main/scala/component/TypedActorComponent.scala
similarity index 64%
rename from akka-camel/src/main/scala/component/ActiveObjectComponent.scala
rename to akka-camel/src/main/scala/component/TypedActorComponent.scala
index 05fa026e04..2a48cf9fc4 100644
--- a/akka-camel/src/main/scala/component/ActiveObjectComponent.scala
+++ b/akka-camel/src/main/scala/component/TypedActorComponent.scala
@@ -12,31 +12,31 @@ import org.apache.camel.component.bean._
/**
* @author Martin Krasser
*/
-object ActiveObjectComponent {
+object TypedActorComponent {
/**
- * Default schema name for active object endpoint URIs.
+ * Default schema name for typed actor endpoint URIs.
*/
- val InternalSchema = "active-object-internal"
+ val InternalSchema = "typed-actor-internal"
}
/**
- * Camel component for exchanging messages with active objects. This component
- * tries to obtain the active object from the activeObjectRegistry
+ * Camel component for exchanging messages with typed actors. This component
+ * tries to obtain the typed actor from the typedActorRegistry
* first. If it's not there it tries to obtain it from the CamelContext's registry.
*
* @see org.apache.camel.component.bean.BeanComponent
*
* @author Martin Krasser
*/
-class ActiveObjectComponent extends BeanComponent {
- val activeObjectRegistry = new ConcurrentHashMap[String, AnyRef]
+class TypedActorComponent extends BeanComponent {
+ val typedActorRegistry = new ConcurrentHashMap[String, AnyRef]
/**
* Creates a {@link org.apache.camel.component.bean.BeanEndpoint} with a custom
- * bean holder that uses activeObjectRegistry for getting access to
- * active objects (beans).
+ * bean holder that uses typedActorRegistry for getting access to
+ * typed actors (beans).
*
- * @see se.scalablesolutions.akka.camel.component.ActiveObjectHolder
+ * @see se.scalablesolutions.akka.camel.component.TypedActorHolder
*/
override def createEndpoint(uri: String, remaining: String, parameters: Map[String, AnyRef]) = {
val endpoint = new BeanEndpoint(uri, this)
@@ -47,39 +47,39 @@ class ActiveObjectComponent extends BeanComponent {
}
private def createBeanHolder(beanName: String) =
- new ActiveObjectHolder(activeObjectRegistry, getCamelContext, beanName).createCacheHolder
+ new TypedActorHolder(typedActorRegistry, getCamelContext, beanName).createCacheHolder
}
/**
* {@link org.apache.camel.component.bean.BeanHolder} implementation that uses a custom
- * registry for getting access to active objects.
+ * registry for getting access to typed actors.
*
* @author Martin Krasser
*/
-class ActiveObjectHolder(activeObjectRegistry: Map[String, AnyRef], context: CamelContext, name: String)
+class TypedActorHolder(typedActorRegistry: Map[String, AnyRef], context: CamelContext, name: String)
extends RegistryBean(context, name) {
/**
- * Returns an {@link se.scalablesolutions.akka.camel.component.ActiveObjectInfo} instance.
+ * Returns an {@link se.scalablesolutions.akka.camel.component.TypedActorInfo} instance.
*/
override def getBeanInfo: BeanInfo =
- new ActiveObjectInfo(getContext, getBean.getClass, getParameterMappingStrategy)
+ new TypedActorInfo(getContext, getBean.getClass, getParameterMappingStrategy)
/**
- * Obtains an active object from activeObjectRegistry.
+ * Obtains an typed actor from typedActorRegistry.
*/
override def getBean: AnyRef = {
- val bean = activeObjectRegistry.get(getName)
+ val bean = typedActorRegistry.get(getName)
if (bean eq null) super.getBean else bean
}
}
/**
- * Provides active object meta information.
+ * Provides typed actor meta information.
*
* @author Martin Krasser
*/
-class ActiveObjectInfo(context: CamelContext, clazz: Class[_], strategy: ParameterMappingStrategy)
+class TypedActorInfo(context: CamelContext, clazz: Class[_], strategy: ParameterMappingStrategy)
extends BeanInfo(context, clazz, strategy) {
/**
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoBase.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoBase.java
deleted file mode 100644
index 05bf1625bb..0000000000
--- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoBase.java
+++ /dev/null
@@ -1,34 +0,0 @@
-package se.scalablesolutions.akka.camel;
-
-import org.apache.camel.Body;
-import org.apache.camel.Header;
-
-import se.scalablesolutions.akka.actor.annotation.consume;
-
-/**
- * @author Martin Krasser
- */
-public class PojoBase {
-
- public String m1(String b, String h) {
- return "m1base: " + b + " " + h;
- }
-
- @consume("direct:m2base")
- public String m2(@Body String b, @Header("test") String h) {
- return "m2base: " + b + " " + h;
- }
-
- @consume("direct:m3base")
- public String m3(@Body String b, @Header("test") String h) {
- return "m3base: " + b + " " + h;
- }
-
- @consume("direct:m4base")
- public String m4(@Body String b, @Header("test") String h) {
- return "m4base: " + b + " " + h;
- }
-
- public void m5(@Body String b, @Header("test") String h) {
- }
-}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoImpl.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoImpl.java
deleted file mode 100644
index b48202d4dc..0000000000
--- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoImpl.java
+++ /dev/null
@@ -1,23 +0,0 @@
-package se.scalablesolutions.akka.camel;
-
-import org.apache.camel.Body;
-import org.apache.camel.Header;
-
-import se.scalablesolutions.akka.actor.annotation.consume;
-
-/**
- * @author Martin Krasser
- */
-public class PojoImpl implements PojoIntf {
-
- public String m1(String b, String h) {
- return "m1impl: " + b + " " + h;
- }
-
- @consume("direct:m2impl")
- public String m2(@Body String b, @Header("test") String h) {
- return "m2impl: " + b + " " + h;
- }
-
-
-}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoIntf.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoIntf.java
deleted file mode 100644
index 14f63afd2e..0000000000
--- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoIntf.java
+++ /dev/null
@@ -1,18 +0,0 @@
-package se.scalablesolutions.akka.camel;
-
-import org.apache.camel.Body;
-import org.apache.camel.Header;
-
-import se.scalablesolutions.akka.actor.annotation.consume;
-
-/**
- * @author Martin Krasser
- */
-public interface PojoIntf {
-
- public String m1(String b, String h);
-
- @consume("direct:m2intf")
- public String m2(@Body String b, @Header("test") String h);
-
-}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoRemote.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoRemote.java
deleted file mode 100644
index 57b0999b8f..0000000000
--- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoRemote.java
+++ /dev/null
@@ -1,15 +0,0 @@
-package se.scalablesolutions.akka.camel;
-
-import se.scalablesolutions.akka.actor.annotation.consume;
-
-/**
- * @author Martin Krasser
- */
-public class PojoRemote {
-
- @consume("direct:remote-active-object")
- public String foo(String s) {
- return String.format("remote active object: %s", s);
- }
-
-}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoSingle.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoSingle.java
deleted file mode 100644
index 7d577535b2..0000000000
--- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoSingle.java
+++ /dev/null
@@ -1,14 +0,0 @@
-package se.scalablesolutions.akka.camel;
-
-import se.scalablesolutions.akka.actor.annotation.consume;
-
-/**
- * @author Martin Krasser
- */
-public class PojoSingle {
-
- @consume("direct:foo")
- public void foo(String b) {
- }
-
-}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoSub.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoSub.java
deleted file mode 100644
index be5b453698..0000000000
--- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/PojoSub.java
+++ /dev/null
@@ -1,27 +0,0 @@
-package se.scalablesolutions.akka.camel;
-
-import org.apache.camel.Body;
-import org.apache.camel.Header;
-
-import se.scalablesolutions.akka.actor.annotation.consume;
-
-public class PojoSub extends PojoBase {
-
- @Override
- @consume("direct:m1sub")
- public String m1(@Body String b, @Header("test") String h) {
- return "m1sub: " + b + " " + h;
- }
-
- @Override
- public String m2(String b, String h) {
- return "m2sub: " + b + " " + h;
- }
-
- @Override
- @consume("direct:m3sub")
- public String m3(@Body String b, @Header("test") String h) {
- return "m3sub: " + b + " " + h;
- }
-
-}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumer.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumer.java
new file mode 100644
index 0000000000..5fd39f07d9
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumer.java
@@ -0,0 +1,12 @@
+package se.scalablesolutions.akka.camel;
+
+import se.scalablesolutions.akka.camel.consume;
+
+/**
+ * @author Martin Krasser
+ */
+public interface SampleRemoteTypedConsumer {
+
+ @consume("direct:remote-typed-consumer")
+ public String foo(String s);
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumerImpl.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumerImpl.java
new file mode 100644
index 0000000000..f6b0076e73
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteTypedConsumerImpl.java
@@ -0,0 +1,14 @@
+package se.scalablesolutions.akka.camel;
+
+import se.scalablesolutions.akka.actor.TypedActor;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleRemoteTypedConsumerImpl extends TypedActor implements SampleRemoteTypedConsumer {
+
+ public String foo(String s) {
+ return String.format("remote typed actor: %s", s);
+ }
+
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteUntypedConsumer.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteUntypedConsumer.java
new file mode 100644
index 0000000000..c35bd92b71
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleRemoteUntypedConsumer.java
@@ -0,0 +1,29 @@
+package se.scalablesolutions.akka.camel;
+
+import se.scalablesolutions.akka.camel.RemoteUntypedConsumerActor;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleRemoteUntypedConsumer extends RemoteUntypedConsumerActor {
+
+ public SampleRemoteUntypedConsumer() {
+ this("localhost", 7774);
+ }
+
+ public SampleRemoteUntypedConsumer(String host, int port) {
+ super(host, port);
+ }
+
+ public String getEndpointUri() {
+ return "direct:remote-untyped-consumer";
+ }
+
+ public void onReceive(Object message) {
+ Message msg = (Message)message;
+ String body = msg.bodyAs(String.class);
+ String header = msg.headerAs("test", String.class);
+ getContext().replySafe(String.format("%s %s", body, header));
+ }
+
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActor.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActor.java
new file mode 100644
index 0000000000..d4d7f152bf
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActor.java
@@ -0,0 +1,9 @@
+package se.scalablesolutions.akka.camel;
+
+/**
+ * @author Martin Krasser
+ */
+public interface SampleTypedActor {
+
+ public String foo(String s);
+}
\ No newline at end of file
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/Pojo.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActorImpl.java
similarity index 53%
rename from akka-camel/src/test/java/se/scalablesolutions/akka/camel/Pojo.java
rename to akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActorImpl.java
index d1848c49ee..dc91fc3f7f 100644
--- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/Pojo.java
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedActorImpl.java
@@ -1,14 +1,14 @@
package se.scalablesolutions.akka.camel;
-import se.scalablesolutions.akka.actor.annotation.consume;
+import se.scalablesolutions.akka.actor.TypedActor;
/**
* @author Martin Krasser
*/
-public class Pojo {
+public class SampleTypedActorImpl extends TypedActor implements SampleTypedActor {
public String foo(String s) {
return String.format("foo: %s", s);
}
-
+
}
\ No newline at end of file
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumer.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumer.java
new file mode 100644
index 0000000000..a6a695f8d0
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumer.java
@@ -0,0 +1,20 @@
+package se.scalablesolutions.akka.camel;
+
+import org.apache.camel.Body;
+import org.apache.camel.Header;
+
+import se.scalablesolutions.akka.camel.consume;
+
+/**
+ * @author Martin Krasser
+ */
+public interface SampleTypedConsumer {
+
+ public String m1(String b, String h);
+ public String m2(@Body String b, @Header("test") String h);
+ public String m3(@Body String b, @Header("test") String h);
+
+ @consume("direct:m4")
+ public String m4(@Body String b, @Header("test") String h);
+ public void m5(@Body String b, @Header("test") String h);
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumerImpl.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumerImpl.java
new file mode 100644
index 0000000000..4fa00f2da0
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedConsumerImpl.java
@@ -0,0 +1,30 @@
+package se.scalablesolutions.akka.camel;
+
+import se.scalablesolutions.akka.actor.TypedActor;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleTypedConsumerImpl extends TypedActor implements SampleTypedConsumer {
+
+ public String m1(String b, String h) {
+ return "m1: " + b + " " + h;
+ }
+
+ @consume("direct:m2")
+ public String m2(String b, String h) {
+ return "m2: " + b + " " + h;
+ }
+
+ @consume("direct:m3")
+ public String m3(String b, String h) {
+ return "m3: " + b + " " + h;
+ }
+
+ public String m4(String b, String h) {
+ return "m4: " + b + " " + h;
+ }
+
+ public void m5(String b, String h) {
+ }
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumer.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumer.java
new file mode 100644
index 0000000000..5d31a35ae2
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumer.java
@@ -0,0 +1,13 @@
+package se.scalablesolutions.akka.camel;
+
+import se.scalablesolutions.akka.camel.consume;
+
+/**
+ * @author Martin Krasser
+ */
+public interface SampleTypedSingleConsumer {
+
+ @consume("direct:foo")
+ public void foo(String b);
+
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumerImpl.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumerImpl.java
new file mode 100644
index 0000000000..608a74d5e9
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleTypedSingleConsumerImpl.java
@@ -0,0 +1,13 @@
+package se.scalablesolutions.akka.camel;
+
+import se.scalablesolutions.akka.actor.TypedActor;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleTypedSingleConsumerImpl extends TypedActor implements SampleTypedSingleConsumer {
+
+ public void foo(String b) {
+ }
+
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedActor.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedActor.java
new file mode 100644
index 0000000000..d8cb1dd1b9
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedActor.java
@@ -0,0 +1,11 @@
+package se.scalablesolutions.akka.camel;
+
+import se.scalablesolutions.akka.actor.UntypedActor;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleUntypedActor extends UntypedActor {
+ public void onReceive(Object message) {
+ }
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumer.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumer.java
new file mode 100644
index 0000000000..303b4302f3
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumer.java
@@ -0,0 +1,21 @@
+package se.scalablesolutions.akka.camel;
+
+import se.scalablesolutions.akka.camel.UntypedConsumerActor;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleUntypedConsumer extends UntypedConsumerActor {
+
+ public String getEndpointUri() {
+ return "direct:test-untyped-consumer";
+ }
+
+ public void onReceive(Object message) {
+ Message msg = (Message)message;
+ String body = msg.bodyAs(String.class);
+ String header = msg.headerAs("test", String.class);
+ getContext().replySafe(String.format("%s %s", body, header));
+ }
+
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumerBlocking.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumerBlocking.java
new file mode 100644
index 0000000000..c653d421bc
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedConsumerBlocking.java
@@ -0,0 +1,23 @@
+package se.scalablesolutions.akka.camel;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleUntypedConsumerBlocking extends UntypedConsumerActor {
+
+ public String getEndpointUri() {
+ return "direct:test-untyped-consumer-blocking";
+ }
+
+ public boolean isBlocking() {
+ return true;
+ }
+
+ public void onReceive(Object message) {
+ Message msg = (Message)message;
+ String body = msg.bodyAs(String.class);
+ String header = msg.headerAs("test", String.class);
+ getContext().replySafe(String.format("%s %s", body, header));
+ }
+
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java
new file mode 100644
index 0000000000..e909947de8
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java
@@ -0,0 +1,18 @@
+package se.scalablesolutions.akka.camel;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleUntypedForwardingProducer extends UntypedProducerActor {
+
+ public String getEndpointUri() {
+ return "direct:producer-test-1";
+ }
+
+ @Override
+ public void onReceiveAfterProduce(Object message) {
+ Message msg = (Message)message;
+ String body = msg.bodyAs(String.class);
+ CamelContextManager.template().sendBody("direct:forward-test-1", body);
+ }
+}
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedReplyingProducer.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedReplyingProducer.java
new file mode 100644
index 0000000000..cc3fbf110d
--- /dev/null
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedReplyingProducer.java
@@ -0,0 +1,12 @@
+package se.scalablesolutions.akka.camel;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleUntypedReplyingProducer extends UntypedProducerActor {
+
+ public String getEndpointUri() {
+ return "direct:producer-test-1";
+ }
+
+}
diff --git a/akka-camel/src/test/scala/CamelServiceFeatureTest.scala b/akka-camel/src/test/scala/CamelServiceFeatureTest.scala
deleted file mode 100644
index df42237a61..0000000000
--- a/akka-camel/src/test/scala/CamelServiceFeatureTest.scala
+++ /dev/null
@@ -1,203 +0,0 @@
-package se.scalablesolutions.akka.camel
-
-import java.util.concurrent.{TimeoutException, CountDownLatch, TimeUnit}
-
-import org.apache.camel.CamelExecutionException
-import org.apache.camel.builder.RouteBuilder
-import org.scalatest.{GivenWhenThen, BeforeAndAfterAll, FeatureSpec}
-
-import se.scalablesolutions.akka.actor.Actor._
-import se.scalablesolutions.akka.actor.{ActiveObject, Actor, ActorRegistry}
-
-class CamelServiceFeatureTest extends FeatureSpec with BeforeAndAfterAll with GivenWhenThen {
- import CamelServiceFeatureTest._
-
- var service: CamelService = _
-
- override protected def beforeAll = {
- ActorRegistry.shutdownAll
- // create new CamelService instance
- service = CamelService.newInstance
- // register test consumer before starting the CamelService
- actorOf(new TestConsumer("direct:publish-test-1")).start
- // Configure a custom camel route
- CamelContextManager.init
- CamelContextManager.context.addRoutes(new TestRoute)
- // start consumer publisher, otherwise we cannot set message
- // count expectations in the next step (needed for testing only).
- service.consumerPublisher.start
- // set expectations on publish count
- val latch = (service.consumerPublisher !! SetExpectedMessageCount(1)).as[CountDownLatch].get
- // start the CamelService
- service.load
- // await publication of first test consumer
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
- }
-
- override protected def afterAll = {
- service.unload
- ActorRegistry.shutdownAll
- }
-
- feature("Publish registered consumer actors in the global CamelContext") {
-
- scenario("access non-blocking consumer actors via Camel direct-endpoints") {
-
- given("two consumer actors registered before and after CamelService startup")
- val latch = (service.consumerPublisher !! SetExpectedMessageCount(1)).as[CountDownLatch].get
- actorOf(new TestConsumer("direct:publish-test-2")).start
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
-
- when("requests are sent to these actors")
- val response1 = CamelContextManager.template.requestBody("direct:publish-test-1", "msg1")
- val response2 = CamelContextManager.template.requestBody("direct:publish-test-2", "msg2")
-
- then("both actors should have replied with expected responses")
- assert(response1 === "received msg1")
- assert(response2 === "received msg2")
- }
-
- scenario("access blocking, non-responding consumer actor via a Camel direct-endpoint") {
-
- given("a consumer actor registered after CamelService startup")
- val latch = (service.consumerPublisher !! SetExpectedMessageCount(1)).as[CountDownLatch].get
- actorOf(new TestBlocker("direct:publish-test-3")).start
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
-
- try {
- when("a request is sent to this actor")
- CamelContextManager.template.requestBody("direct:publish-test-3", "msg3")
- fail("expected TimoutException not thrown")
- } catch {
- case e => {
- then("a TimoutException should be thrown")
- assert(e.getCause.isInstanceOf[TimeoutException])
- }
- }
- }
- }
-
- feature("Unpublish registered consumer actor from the global CamelContext") {
-
- scenario("access to unregistered consumer actor via Camel direct-endpoint fails") {
- val endpointUri = "direct:unpublish-test-1"
-
- given("a consumer actor registered after CamelService startup")
- assert(CamelContextManager.context.hasEndpoint(endpointUri) eq null)
- var latch = (service.consumerPublisher !! SetExpectedMessageCount(1)).as[CountDownLatch].get
- val consumer = actorOf(new TestConsumer(endpointUri)).start
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
- assert(CamelContextManager.context.hasEndpoint(endpointUri) ne null)
-
- when("the actor is stopped")
- latch = (service.consumerPublisher !! SetExpectedMessageCount(1)).as[CountDownLatch].get
- consumer.stop
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
-
- then("the associated endpoint isn't accessible any more")
- intercept[CamelExecutionException] {
- CamelContextManager.template.requestBody(endpointUri, "msg1")
- }
- }
- }
-
- feature("Configure a custom Camel route for the global CamelContext") {
-
- scenario("access an actor from the custom Camel route") {
-
- given("a registered actor and a custom route to that actor")
- val actor = actorOf[TestActor].start
-
- when("sending a a message to that route")
- val response = CamelContextManager.template.requestBody("direct:custom-route-test-1", "msg3")
-
- then("an expected response generated by the actor should be returned")
- assert(response === "received msg3")
- }
- }
-
- feature("Publish active object methods in the global CamelContext") {
-
- scenario("access active object methods via Camel direct-endpoints") {
-
- given("an active object registered after CamelService startup")
- var latch = (service.consumerPublisher !! SetExpectedMessageCount(3)).as[CountDownLatch].get
- val obj = ActiveObject.newInstance(classOf[PojoBase])
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
-
- when("requests are sent to published methods")
- val response1 = CamelContextManager.template.requestBodyAndHeader("direct:m2base", "x", "test", "y")
- val response2 = CamelContextManager.template.requestBodyAndHeader("direct:m3base", "x", "test", "y")
- val response3 = CamelContextManager.template.requestBodyAndHeader("direct:m4base", "x", "test", "y")
-
- then("each should have returned a different response")
- assert(response1 === "m2base: x y")
- assert(response2 === "m3base: x y")
- assert(response3 === "m4base: x y")
-
- // cleanup to avoid conflicts with next test (i.e. avoid multiple consumers on direct-endpoints)
- latch = (service.consumerPublisher !! SetExpectedMessageCount(3)).as[CountDownLatch].get
- ActiveObject.stop(obj)
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
- }
- }
-
- feature("Unpublish active object method from the global CamelContext") {
-
- scenario("access to unregistered active object methof via Camel direct-endpoint fails") {
-
- given("an active object registered after CamelService startup")
- var latch = (service.consumerPublisher !! SetExpectedMessageCount(3)).as[CountDownLatch].get
- val obj = ActiveObject.newInstance(classOf[PojoBase])
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
-
- when("the active object is stopped")
- latch = (service.consumerPublisher !! SetExpectedMessageCount(3)).as[CountDownLatch].get
- ActiveObject.stop(obj)
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
-
- then("the associated endpoints aren't accessible any more")
- intercept[CamelExecutionException] {
- CamelContextManager.template.requestBodyAndHeader("direct:m2base", "x", "test", "y")
- }
- intercept[CamelExecutionException] {
- CamelContextManager.template.requestBodyAndHeader("direct:m3base", "x", "test", "y")
- }
- intercept[CamelExecutionException] {
- CamelContextManager.template.requestBodyAndHeader("direct:m4base", "x", "test", "y")
- }
- }
- }
-}
-
-object CamelServiceFeatureTest {
-
- class TestConsumer(uri: String) extends Actor with Consumer {
- def endpointUri = uri
- protected def receive = {
- case msg: Message => self.reply("received %s" format msg.body)
- }
- }
-
- class TestBlocker(uri: String) extends Actor with Consumer {
- self.timeout = 1
- def endpointUri = uri
- override def blocking = true
- protected def receive = {
- case msg: Message => { /* do not reply */ }
- }
- }
-
- class TestActor extends Actor {
- self.id = "custom-actor-id"
- protected def receive = {
- case msg: Message => self.reply("received %s" format msg.body)
- }
- }
-
- class TestRoute extends RouteBuilder {
- def configure {
- from("direct:custom-route-test-1").to("actor:custom-actor-id")
- }
- }
-}
diff --git a/akka-camel/src/test/scala/CamelServiceManagerSpec.scala b/akka-camel/src/test/scala/CamelServiceManagerSpec.scala
new file mode 100644
index 0000000000..9de84f1c27
--- /dev/null
+++ b/akka-camel/src/test/scala/CamelServiceManagerSpec.scala
@@ -0,0 +1,63 @@
+package se.scalablesolutions.akka.camel
+
+import org.scalatest.{BeforeAndAfterAll, WordSpec}
+import org.scalatest.matchers.MustMatchers
+
+import se.scalablesolutions.akka.actor.ActorRegistry
+
+/**
+ * @author Martin Krasser
+ */
+class CamelServiceManagerSpec extends WordSpec with BeforeAndAfterAll with MustMatchers {
+
+ override def afterAll = ActorRegistry.shutdownAll
+
+ "A CamelServiceManager" when {
+ "the startCamelService method been has been called" must {
+ "have registered the started CamelService instance" in {
+ val service = CamelServiceManager.startCamelService
+ CamelServiceManager.service must be theSameInstanceAs (service)
+ }
+ }
+ "the stopCamelService method been has been called" must {
+ "have unregistered the current CamelService instance" in {
+ val service = CamelServiceManager.stopCamelService
+ intercept[IllegalStateException] { CamelServiceManager.service }
+ }
+ }
+ }
+
+ "A CamelServiceManager" when {
+ val service = CamelServiceFactory.createCamelService
+ "a CamelService instance has been started externally" must {
+ "have registered the started CamelService instance" in {
+ service.start
+ CamelServiceManager.service must be theSameInstanceAs (service)
+ }
+ }
+ "the current CamelService instance has been stopped externally" must {
+ "have unregistered the current CamelService instance" in {
+ service.stop
+ intercept[IllegalStateException] { CamelServiceManager.service }
+ }
+ }
+ }
+
+ "A CamelServiceManager" when {
+ "a CamelService has been started" must {
+ "not allow further CamelService instances to be started" in {
+ CamelServiceManager.startCamelService
+ intercept[IllegalStateException] { CamelServiceManager.startCamelService }
+ }
+ }
+ "a CamelService has been stopped" must {
+ "only allow the current CamelService instance to be stopped" in {
+ intercept[IllegalStateException] { CamelServiceFactory.createCamelService.stop }
+ }
+ "ensure that the current CamelService instance has been actually started" in {
+ CamelServiceManager.stopCamelService
+ intercept[IllegalStateException] { CamelServiceManager.stopCamelService }
+ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/akka-camel/src/test/scala/ConsumerMethodRegisteredTest.scala b/akka-camel/src/test/scala/ConsumerMethodRegisteredTest.scala
deleted file mode 100644
index 7c28c7d8ee..0000000000
--- a/akka-camel/src/test/scala/ConsumerMethodRegisteredTest.scala
+++ /dev/null
@@ -1,57 +0,0 @@
-package se.scalablesolutions.akka.camel
-
-import java.net.InetSocketAddress
-
-import org.scalatest.junit.JUnitSuite
-
-import se.scalablesolutions.akka.actor.{AspectInit, ActiveObject}
-import se.scalablesolutions.akka.camel.ConsumerMethodRegistered._
-import org.junit.{AfterClass, Test}
-
-class ConsumerMethodRegisteredTest extends JUnitSuite {
- import ConsumerMethodRegisteredTest._
-
- val remoteAddress = new InetSocketAddress("localhost", 8888);
- val remoteAspectInit = AspectInit(classOf[String], null, Some(remoteAddress), 1000)
- val localAspectInit = AspectInit(classOf[String], null, None, 1000)
-
- val ascendingMethodName = (r1: ConsumerMethodRegistered, r2: ConsumerMethodRegistered) =>
- r1.method.getName < r2.method.getName
-
- @Test def shouldSelectPojoBaseMethods234 = {
- val registered = forConsumer(activePojoBase, localAspectInit).sortWith(ascendingMethodName)
- assert(registered.size === 3)
- assert(registered.map(_.method.getName) === List("m2", "m3", "m4"))
- }
-
- @Test def shouldSelectPojoSubMethods134 = {
- val registered = forConsumer(activePojoSub, localAspectInit).sortWith(ascendingMethodName)
- assert(registered.size === 3)
- assert(registered.map(_.method.getName) === List("m1", "m3", "m4"))
- }
-
- @Test def shouldSelectPojoIntfMethod2 = {
- val registered = forConsumer(activePojoIntf, localAspectInit)
- assert(registered.size === 1)
- assert(registered(0).method.getName === "m2")
- }
-
- @Test def shouldIgnoreRemoteProxies = {
- val registered = forConsumer(activePojoBase, remoteAspectInit)
- assert(registered.size === 0)
- }
-
-}
-
-object ConsumerMethodRegisteredTest {
- val activePojoBase = ActiveObject.newInstance(classOf[PojoBase])
- val activePojoSub = ActiveObject.newInstance(classOf[PojoSub])
- val activePojoIntf = ActiveObject.newInstance(classOf[PojoIntf], new PojoImpl)
-
- @AfterClass
- def afterClass = {
- ActiveObject.stop(activePojoBase)
- ActiveObject.stop(activePojoSub)
- ActiveObject.stop(activePojoIntf)
- }
-}
diff --git a/akka-camel/src/test/scala/ConsumerRegisteredTest.scala b/akka-camel/src/test/scala/ConsumerRegisteredTest.scala
index 3339caacf2..787142d50a 100644
--- a/akka-camel/src/test/scala/ConsumerRegisteredTest.scala
+++ b/akka-camel/src/test/scala/ConsumerRegisteredTest.scala
@@ -3,8 +3,46 @@ package se.scalablesolutions.akka.camel
import org.junit.Test
import org.scalatest.junit.JUnitSuite
-import se.scalablesolutions.akka.actor.Actor
-import se.scalablesolutions.akka.actor.Actor._
+import se.scalablesolutions.akka.actor.{Actor, UntypedActor}
+
+class ConsumerRegisteredTest extends JUnitSuite {
+ import ConsumerRegisteredTest._
+
+ @Test def shouldCreateSomeNonBlockingPublishRequestFromConsumer = {
+ val c = Actor.actorOf[ConsumerActor1]
+ val event = ConsumerRegistered.forConsumer(c)
+ assert(event === Some(ConsumerRegistered(c, "mock:test1", c.uuid, false)))
+ }
+
+ @Test def shouldCreateSomeBlockingPublishRequestFromConsumer = {
+ val c = Actor.actorOf[ConsumerActor2]
+ val event = ConsumerRegistered.forConsumer(c)
+ assert(event === Some(ConsumerRegistered(c, "mock:test2", c.uuid, true)))
+ }
+
+ @Test def shouldCreateNoneFromConsumer = {
+ val event = ConsumerRegistered.forConsumer(Actor.actorOf[PlainActor])
+ assert(event === None)
+ }
+
+ @Test def shouldCreateSomeNonBlockingPublishRequestFromUntypedConsumer = {
+ val uc = UntypedActor.actorOf(classOf[SampleUntypedConsumer])
+ val event = ConsumerRegistered.forConsumer(uc)
+ assert(event === Some(ConsumerRegistered(uc, "direct:test-untyped-consumer", uc.uuid, false)))
+ }
+
+ @Test def shouldCreateSomeBlockingPublishRequestFromUntypedConsumer = {
+ val uc = UntypedActor.actorOf(classOf[SampleUntypedConsumerBlocking])
+ val event = ConsumerRegistered.forConsumer(uc)
+ assert(event === Some(ConsumerRegistered(uc, "direct:test-untyped-consumer-blocking", uc.uuid, true)))
+ }
+
+ @Test def shouldCreateNoneFromUntypedConsumer = {
+ val a = UntypedActor.actorOf(classOf[SampleUntypedActor])
+ val event = ConsumerRegistered.forConsumer(a)
+ assert(event === None)
+ }
+}
object ConsumerRegisteredTest {
class ConsumerActor1 extends Actor with Consumer {
@@ -22,24 +60,3 @@ object ConsumerRegisteredTest {
protected def receive = null
}
}
-
-class ConsumerRegisteredTest extends JUnitSuite {
- import ConsumerRegisteredTest._
-
- @Test def shouldCreateSomeNonBlockingPublishRequest = {
- val ca = actorOf[ConsumerActor1]
- val event = ConsumerRegistered.forConsumer(ca)
- assert(event === Some(ConsumerRegistered(ca, "mock:test1", ca.uuid, false)))
- }
-
- @Test def shouldCreateSomeBlockingPublishRequest = {
- val ca = actorOf[ConsumerActor2]
- val event = ConsumerRegistered.forConsumer(ca)
- assert(event === Some(ConsumerRegistered(ca, "mock:test2", ca.uuid, true)))
- }
-
- @Test def shouldCreateNone = {
- val event = ConsumerRegistered.forConsumer(actorOf[PlainActor])
- assert(event === None)
- }
-}
diff --git a/akka-camel/src/test/scala/ConsumerSpec.scala b/akka-camel/src/test/scala/ConsumerSpec.scala
new file mode 100644
index 0000000000..14f5e1cf40
--- /dev/null
+++ b/akka-camel/src/test/scala/ConsumerSpec.scala
@@ -0,0 +1,205 @@
+package se.scalablesolutions.akka.camel
+
+import java.util.concurrent.{TimeoutException, CountDownLatch, TimeUnit}
+
+import org.apache.camel.CamelExecutionException
+import org.apache.camel.builder.RouteBuilder
+import org.scalatest.{BeforeAndAfterAll, WordSpec}
+import org.scalatest.matchers.MustMatchers
+
+import se.scalablesolutions.akka.actor.Actor._
+import se.scalablesolutions.akka.actor._
+
+/**
+ * @author Martin Krasser
+ */
+class ConsumerSpec extends WordSpec with BeforeAndAfterAll with MustMatchers {
+ import CamelContextManager.template
+ import ConsumerSpec._
+
+ var service: CamelService = _
+
+ override protected def beforeAll = {
+ ActorRegistry.shutdownAll
+ // create new CamelService instance
+ service = CamelServiceFactory.createCamelService
+ // register test consumer before starting the CamelService
+ actorOf(new TestConsumer("direct:publish-test-1")).start
+ // start consumer publisher, otherwise we cannot set message
+ // count expectations in the next step (needed for testing only).
+ service.consumerPublisher.start
+ // set expectations on publish count
+ val latch = service.expectEndpointActivationCount(1)
+ // start the CamelService
+ service.start
+ // await publication of first test consumer
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+ }
+
+ override protected def afterAll = {
+ service.stop
+ ActorRegistry.shutdownAll
+ }
+
+ "A responding consumer" when {
+ val consumer = actorOf(new TestConsumer("direct:publish-test-2"))
+ "started before starting the CamelService" must {
+ "support an in-out message exchange via its endpoint" in {
+ template.requestBody("direct:publish-test-1", "msg1") must equal ("received msg1")
+ }
+ }
+ "not started" must {
+ "not have an associated endpoint in the CamelContext" in {
+ CamelContextManager.context.hasEndpoint("direct:publish-test-2") must be (null)
+ }
+ }
+ "started" must {
+ "support an in-out message exchange via its endpoint" in {
+ val latch = service.expectEndpointActivationCount(1)
+ consumer.start
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+ template.requestBody("direct:publish-test-2", "msg2") must equal ("received msg2")
+ }
+ "have an associated endpoint in the CamelContext" in {
+ CamelContextManager.context.hasEndpoint("direct:publish-test-2") must not be (null)
+ }
+ }
+ "stopped" must {
+ "not support an in-out message exchange via its endpoint" in {
+ val latch = service.expectEndpointDeactivationCount(1)
+ consumer.stop
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+ intercept[CamelExecutionException] {
+ template.requestBody("direct:publish-test-2", "msg2")
+ }
+ }
+ }
+ }
+
+ "A responding, typed consumer" when {
+ var actor: SampleTypedConsumer = null
+ "started" must {
+ "support in-out message exchanges via its endpoints" in {
+ val latch = service.expectEndpointActivationCount(3)
+ actor = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+ template.requestBodyAndHeader("direct:m2", "x", "test", "y") must equal ("m2: x y")
+ template.requestBodyAndHeader("direct:m3", "x", "test", "y") must equal ("m3: x y")
+ template.requestBodyAndHeader("direct:m4", "x", "test", "y") must equal ("m4: x y")
+ }
+ }
+ "stopped" must {
+ "not support in-out message exchanges via its endpoints" in {
+ val latch = service.expectEndpointDeactivationCount(3)
+ TypedActor.stop(actor)
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+ intercept[CamelExecutionException] {
+ template.requestBodyAndHeader("direct:m2", "x", "test", "y")
+ }
+ intercept[CamelExecutionException] {
+ template.requestBodyAndHeader("direct:m3", "x", "test", "y")
+ }
+ intercept[CamelExecutionException] {
+ template.requestBodyAndHeader("direct:m4", "x", "test", "y")
+ }
+ }
+ }
+ }
+
+ "A responding, typed consumer (Scala)" when {
+ var actor: TestTypedConsumer = null
+ "started" must {
+ "support in-out message exchanges via its endpoints" in {
+ val latch = service.expectEndpointActivationCount(2)
+ actor = TypedActor.newInstance(classOf[TestTypedConsumer], classOf[TestTypedConsumerImpl])
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+ template.requestBody("direct:publish-test-3", "x") must equal ("foo: x")
+ template.requestBody("direct:publish-test-4", "x") must equal ("bar: x")
+ }
+ }
+ "stopped" must {
+ "not support in-out message exchanges via its endpoints" in {
+ val latch = service.expectEndpointDeactivationCount(2)
+ TypedActor.stop(actor)
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+ intercept[CamelExecutionException] {
+ template.requestBody("direct:publish-test-3", "x")
+ }
+ intercept[CamelExecutionException] {
+ template.requestBody("direct:publish-test-4", "x")
+ }
+ }
+ }
+ }
+
+ "A responding, untyped consumer" when {
+ val consumer = UntypedActor.actorOf(classOf[SampleUntypedConsumer])
+ "started" must {
+ "support an in-out message exchange via its endpoint" in {
+ val latch = service.expectEndpointActivationCount(1)
+ consumer.start
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+ template.requestBodyAndHeader("direct:test-untyped-consumer", "x", "test", "y") must equal ("x y")
+ }
+ }
+ "stopped" must {
+ "not support an in-out message exchange via its endpoint" in {
+ val latch = service.expectEndpointDeactivationCount(1)
+ consumer.stop
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+ intercept[CamelExecutionException] {
+ template.sendBodyAndHeader("direct:test-untyped-consumer", "blah", "test", "blub")
+ }
+ }
+ }
+ }
+
+ "A non-responding, blocking consumer" when {
+ "receiving an in-out message exchange" must {
+ "lead to a TimeoutException" in {
+ val latch = service.expectEndpointActivationCount(1)
+ actorOf(new TestBlocker("direct:publish-test-5")).start
+ latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
+
+ try {
+ template.requestBody("direct:publish-test-5", "msg3")
+ fail("expected TimoutException not thrown")
+ } catch {
+ case e => {
+ assert(e.getCause.isInstanceOf[TimeoutException])
+ }
+ }
+ }
+ }
+ }
+}
+
+object ConsumerSpec {
+ class TestConsumer(uri: String) extends Actor with Consumer {
+ def endpointUri = uri
+ protected def receive = {
+ case msg: Message => self.reply("received %s" format msg.body)
+ }
+ }
+
+ trait TestTypedConsumer {
+ @consume("direct:publish-test-3")
+ def foo(s: String): String
+ def bar(s: String): String
+ }
+
+ class TestTypedConsumerImpl extends TypedActor with TestTypedConsumer {
+ def foo(s: String) = "foo: %s" format s
+ @consume("direct:publish-test-4")
+ def bar(s: String) = "bar: %s" format s
+ }
+
+ class TestBlocker(uri: String) extends Actor with Consumer {
+ self.timeout = 1000
+ def endpointUri = uri
+ override def blocking = true
+ protected def receive = {
+ case msg: Message => { /* do not reply */ }
+ }
+ }
+}
\ No newline at end of file
diff --git a/akka-camel/src/test/scala/MessageTest.scala b/akka-camel/src/test/scala/MessageTest.scala
index b87a99e18c..1467402b9a 100644
--- a/akka-camel/src/test/scala/MessageTest.scala
+++ b/akka-camel/src/test/scala/MessageTest.scala
@@ -15,6 +15,7 @@ class MessageTest extends JUnitSuite with BeforeAndAfterAll {
@Test def shouldConvertDoubleBodyToString = {
assertEquals("1.4", Message(1.4, null).bodyAs[String])
+ assertEquals("1.4", Message(1.4, null).bodyAs(classOf[String]))
}
@Test def shouldThrowExceptionWhenConvertingDoubleBodyToInputStream {
@@ -23,6 +24,17 @@ class MessageTest extends JUnitSuite with BeforeAndAfterAll {
}
}
+ @Test def shouldReturnDoubleHeader = {
+ val message = Message("test" , Map("test" -> 1.4))
+ assertEquals(1.4, message.header("test"))
+ }
+
+ @Test def shouldConvertDoubleHeaderToString = {
+ val message = Message("test" , Map("test" -> 1.4))
+ assertEquals("1.4", message.headerAs[String]("test"))
+ assertEquals("1.4", message.headerAs("test", classOf[String]))
+ }
+
@Test def shouldReturnSubsetOfHeaders = {
val message = Message("test" , Map("A" -> "1", "B" -> "2"))
assertEquals(Map("B" -> "2"), message.headers(Set("B")))
diff --git a/akka-camel/src/test/scala/ProducerFeatureTest.scala b/akka-camel/src/test/scala/ProducerFeatureTest.scala
index 5cf24eaaa3..a27e05a54f 100644
--- a/akka-camel/src/test/scala/ProducerFeatureTest.scala
+++ b/akka-camel/src/test/scala/ProducerFeatureTest.scala
@@ -27,7 +27,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
mockEndpoint.reset
}
- feature("Produce a message to a Camel endpoint") {
+ feature("Produce a message to a sync Camel route") {
scenario("produce message and receive normal response") {
given("a registered two-way producer")
@@ -86,9 +86,9 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
}
}
- feature("Produce a message to an async Camel endpoint") {
+ feature("Produce a message to an async Camel route") {
- scenario("produce message and async receive normal response") {
+ scenario("produce message and receive normal response") {
given("a registered two-way producer")
val producer = actorOf(new TestProducer("direct:producer-test-3"))
producer.start
@@ -102,7 +102,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
assert(result === Some(expected))
}
- scenario("produce message and async receive failure response") {
+ scenario("produce message and receive failure response") {
given("a registered two-way producer")
val producer = actorOf(new TestProducer("direct:producer-test-3"))
producer.start
@@ -119,9 +119,9 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
}
}
- feature("Produce a message to a Camel endpoint and then forward the result") {
+ feature("Produce a message to a sync Camel route and then forward the response") {
- scenario("produce message, forward and receive normal response") {
+ scenario("produce message, forward normal response to a replying target actor and receive response") {
given("a registered two-way producer configured with a forward target")
val target = actorOf[ReplyingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
@@ -135,7 +135,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
assert(result === Some(expected))
}
- scenario("produce message, forward and receive failure response") {
+ scenario("produce message, forward failure response to a replying target actor and receive response") {
given("a registered two-way producer configured with a forward target")
val target = actorOf[ReplyingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
@@ -151,7 +151,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
assert(expectedHeaders === Map(Message.MessageExchangeId -> "123", "test" -> "failure"))
}
- scenario("produce message, forward and produce normal response") {
+ scenario("produce message, forward normal response to a producing target actor and produce response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val target = actorOf[ProducingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
@@ -164,7 +164,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
mockEndpoint.assertIsSatisfied
}
- scenario("produce message, forward and produce failure response") {
+ scenario("produce message, forward failure response to a producing target actor and produce response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val target = actorOf[ProducingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-2", target)).start
@@ -179,9 +179,9 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
}
}
- feature("Produce a message to an async Camel endpoint and then forward the result") {
+ feature("Produce a message to an async Camel route and then forward the response") {
- scenario("produce message, forward and async receive normal response") {
+ scenario("produce message, forward normal response to a replying target actor and receive response") {
given("a registered two-way producer configured with a forward target")
val target = actorOf[ReplyingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
@@ -195,7 +195,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
assert(result === Some(expected))
}
- scenario("produce message, forward and async receive failure response") {
+ scenario("produce message, forward failure response to a replying target actor and receive response") {
given("a registered two-way producer configured with a forward target")
val target = actorOf[ReplyingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
@@ -211,7 +211,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
assert(expectedHeaders === Map(Message.MessageExchangeId -> "123", "test" -> "failure"))
}
- scenario("produce message, forward and async produce normal response") {
+ scenario("produce message, forward normal response to a producing target actor and produce response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val target = actorOf[ProducingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
@@ -224,7 +224,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
mockEndpoint.assertIsSatisfied
}
- scenario("produce message, forward and async produce failure response") {
+ scenario("produce message, forward failure response to a producing target actor and produce response to direct:forward-test-1") {
given("a registered one-way producer configured with a forward target")
val target = actorOf[ProducingForwardTarget].start
val producer = actorOf(new TestForwarder("direct:producer-test-3", target)).start
@@ -298,4 +298,4 @@ object ProducerFeatureTest {
})
}
}
-}
\ No newline at end of file
+}
diff --git a/akka-camel/src/test/scala/PublishRequestorTest.scala b/akka-camel/src/test/scala/PublishRequestorTest.scala
index 131f4fe2b5..7cfced57e1 100644
--- a/akka-camel/src/test/scala/PublishRequestorTest.scala
+++ b/akka-camel/src/test/scala/PublishRequestorTest.scala
@@ -16,7 +16,10 @@ class PublishRequestorTest extends JUnitSuite {
var requestor: ActorRef = _
var consumer: ActorRef = _
- @Before def setUp = {
+ val ascendingMethodName = (r1: ConsumerMethodRegistered, r2: ConsumerMethodRegistered) =>
+ r1.method.getName < r2.method.getName
+
+ @Before def setUp: Unit = {
publisher = actorOf[PublisherMock].start
requestor = actorOf[PublishRequestor].start
requestor ! PublishRequestorInit(publisher)
@@ -24,40 +27,58 @@ class PublishRequestorTest extends JUnitSuite {
def endpointUri = "mock:test"
protected def receive = null
}).start
-
}
@After def tearDown = {
+ AspectInitRegistry.removeListener(requestor);
ActorRegistry.shutdownAll
}
- @Test def shouldReceiveConsumerMethodRegisteredEvent = {
- val obj = ActiveObject.newInstance(classOf[PojoSingle])
- val init = AspectInit(classOf[PojoSingle], null, None, 1000)
+ @Test def shouldReceiveOneConsumerMethodRegisteredEvent = {
+ AspectInitRegistry.addListener(requestor)
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
- requestor ! AspectInitRegistered(obj, init)
+ val obj = TypedActor.newInstance(classOf[SampleTypedSingleConsumer], classOf[SampleTypedSingleConsumerImpl])
assert(latch.await(5000, TimeUnit.MILLISECONDS))
- val event = (publisher !! GetRetainedMessage).get.asInstanceOf[ConsumerMethodRegistered]
- assert(event.init === init)
+ val event = (publisher !! GetRetainedMessage).as[ConsumerMethodRegistered].get
assert(event.uri === "direct:foo")
- assert(event.activeObject === obj)
+ assert(event.typedActor === obj)
assert(event.method.getName === "foo")
}
- @Test def shouldReceiveConsumerMethodUnregisteredEvent = {
- val obj = ActiveObject.newInstance(classOf[PojoSingle])
- val init = AspectInit(classOf[PojoSingle], null, None, 1000)
+ @Test def shouldReceiveOneConsumerMethodUnregisteredEvent = {
+ val obj = TypedActor.newInstance(classOf[SampleTypedSingleConsumer], classOf[SampleTypedSingleConsumerImpl])
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
- requestor ! AspectInitUnregistered(obj, init)
+ AspectInitRegistry.addListener(requestor)
+ TypedActor.stop(obj)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
- val event = (publisher !! GetRetainedMessage).get.asInstanceOf[ConsumerMethodUnregistered]
- assert(event.init === init)
+ val event = (publisher !! GetRetainedMessage).as[ConsumerMethodUnregistered].get
assert(event.uri === "direct:foo")
- assert(event.activeObject === obj)
+ assert(event.typedActor === obj)
assert(event.method.getName === "foo")
}
- @Test def shouldReceiveConsumerRegisteredEvent = {
+ @Test def shouldReceiveThreeConsumerMethodRegisteredEvents = {
+ AspectInitRegistry.addListener(requestor)
+ val latch = (publisher !! SetExpectedTestMessageCount(3)).as[CountDownLatch].get
+ val obj = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
+ assert(latch.await(5000, TimeUnit.MILLISECONDS))
+ val request = GetRetainedMessages(_.isInstanceOf[ConsumerMethodRegistered])
+ val events = (publisher !! request).as[List[ConsumerMethodRegistered]].get
+ assert(events.map(_.method.getName).sortWith(_ < _) === List("m2", "m3", "m4"))
+ }
+
+ @Test def shouldReceiveThreeConsumerMethodUnregisteredEvents = {
+ val obj = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
+ val latch = (publisher !! SetExpectedTestMessageCount(3)).as[CountDownLatch].get
+ AspectInitRegistry.addListener(requestor)
+ TypedActor.stop(obj)
+ assert(latch.await(5000, TimeUnit.MILLISECONDS))
+ val request = GetRetainedMessages(_.isInstanceOf[ConsumerMethodUnregistered])
+ val events = (publisher !! request).as[List[ConsumerMethodUnregistered]].get
+ assert(events.map(_.method.getName).sortWith(_ < _) === List("m2", "m3", "m4"))
+ }
+
+ @Test def shouldReceiveOneConsumerRegisteredEvent = {
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
requestor ! ActorRegistered(consumer)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
@@ -65,7 +86,7 @@ class PublishRequestorTest extends JUnitSuite {
Some(ConsumerRegistered(consumer, "mock:test", consumer.uuid, false)))
}
- @Test def shouldReceiveConsumerUnregisteredEvent = {
+ @Test def shouldReceiveOneConsumerUnregisteredEvent = {
val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
requestor ! ActorUnregistered(consumer)
assert(latch.await(5000, TimeUnit.MILLISECONDS))
diff --git a/akka-camel/src/test/scala/RemoteConsumerTest.scala b/akka-camel/src/test/scala/RemoteConsumerTest.scala
index 7e3b666590..afba2011d5 100644
--- a/akka-camel/src/test/scala/RemoteConsumerTest.scala
+++ b/akka-camel/src/test/scala/RemoteConsumerTest.scala
@@ -4,24 +4,23 @@ import java.util.concurrent.{CountDownLatch, TimeUnit}
import org.scalatest.{GivenWhenThen, BeforeAndAfterAll, FeatureSpec}
+import se.scalablesolutions.akka.actor._
import se.scalablesolutions.akka.actor.Actor._
-import se.scalablesolutions.akka.actor.{ActiveObject, ActorRegistry, RemoteActor}
import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer}
/**
* @author Martin Krasser
*/
class RemoteConsumerTest extends FeatureSpec with BeforeAndAfterAll with GivenWhenThen {
+ import CamelServiceManager._
import RemoteConsumerTest._
- var service: CamelService = _
var server: RemoteServer = _
override protected def beforeAll = {
ActorRegistry.shutdownAll
- service = CamelService.newInstance
- service.load
+ startCamelService
server = new RemoteServer()
server.start(host, port)
@@ -31,7 +30,8 @@ class RemoteConsumerTest extends FeatureSpec with BeforeAndAfterAll with GivenWh
override protected def afterAll = {
server.shutdown
- service.unload
+
+ stopCamelService
RemoteClient.shutdownAll
ActorRegistry.shutdownAll
@@ -39,35 +39,51 @@ class RemoteConsumerTest extends FeatureSpec with BeforeAndAfterAll with GivenWh
Thread.sleep(1000)
}
- feature("Client-initiated remote consumer actor") {
- scenario("access published remote consumer actor") {
- given("a client-initiated remote consumer actor")
+ feature("Publish consumer on remote node") {
+ scenario("access published remote consumer") {
+ given("a client-initiated remote consumer")
val consumer = actorOf[RemoteConsumer].start
when("remote consumer publication is triggered")
- val latch = (service.consumerPublisher !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ var latch = service.expectEndpointActivationCount(1)
consumer !! "init"
assert(latch.await(5000, TimeUnit.MILLISECONDS))
- then("the published actor is accessible via its endpoint URI")
- val response = CamelContextManager.template.requestBody("direct:remote-actor", "test")
+ then("the published consumer is accessible via its endpoint URI")
+ val response = CamelContextManager.template.requestBody("direct:remote-consumer", "test")
assert(response === "remote actor: test")
}
}
- feature("Client-initiated remote consumer active object") {
+ feature("Publish typed consumer on remote node") {
scenario("access published remote consumer method") {
- given("a client-initiated remote consumer active object")
- val consumer = ActiveObject.newRemoteInstance(classOf[PojoRemote], host, port)
+ given("a client-initiated remote typed consumer")
+ val consumer = TypedActor.newRemoteInstance(classOf[SampleRemoteTypedConsumer], classOf[SampleRemoteTypedConsumerImpl], host, port)
- when("remote consumer publication is triggered")
- val latch = (service.consumerPublisher !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ when("remote typed consumer publication is triggered")
+ var latch = service.expectEndpointActivationCount(1)
consumer.foo("init")
assert(latch.await(5000, TimeUnit.MILLISECONDS))
then("the published method is accessible via its endpoint URI")
- val response = CamelContextManager.template.requestBody("direct:remote-active-object", "test")
- assert(response === "remote active object: test")
+ val response = CamelContextManager.template.requestBody("direct:remote-typed-consumer", "test")
+ assert(response === "remote typed actor: test")
+ }
+ }
+
+ feature("Publish untyped consumer on remote node") {
+ scenario("access published remote untyped consumer") {
+ given("a client-initiated remote untyped consumer")
+ val consumer = UntypedActor.actorOf(classOf[SampleRemoteUntypedConsumer]).start
+
+ when("remote untyped consumer publication is triggered")
+ var latch = service.expectEndpointActivationCount(1)
+ consumer.sendRequestReply(Message("init", Map("test" -> "init")))
+ assert(latch.await(5000, TimeUnit.MILLISECONDS))
+
+ then("the published untyped consumer is accessible via its endpoint URI")
+ val response = CamelContextManager.template.requestBodyAndHeader("direct:remote-untyped-consumer", "a", "test", "b")
+ assert(response === "a b")
}
}
}
@@ -77,7 +93,7 @@ object RemoteConsumerTest {
val port = 7774
class RemoteConsumer extends RemoteActor(host, port) with Consumer {
- def endpointUri = "direct:remote-actor"
+ def endpointUri = "direct:remote-consumer"
protected def receive = {
case "init" => self.reply("done")
diff --git a/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala b/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala
new file mode 100644
index 0000000000..c8a0bd8542
--- /dev/null
+++ b/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala
@@ -0,0 +1,98 @@
+package se.scalablesolutions.akka.camel
+
+import org.apache.camel.{Exchange, Processor}
+import org.apache.camel.builder.RouteBuilder
+import org.apache.camel.component.mock.MockEndpoint
+import org.scalatest.{GivenWhenThen, BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
+
+import se.scalablesolutions.akka.actor.UntypedActor._
+import se.scalablesolutions.akka.actor.ActorRegistry
+
+class UntypedProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen {
+ import UntypedProducerFeatureTest._
+
+ override protected def beforeAll = {
+ ActorRegistry.shutdownAll
+ CamelContextManager.init
+ CamelContextManager.context.addRoutes(new TestRoute)
+ CamelContextManager.start
+ }
+
+ override protected def afterAll = {
+ CamelContextManager.stop
+ ActorRegistry.shutdownAll
+ }
+
+ override protected def afterEach = {
+ mockEndpoint.reset
+ }
+
+ feature("Produce a message to a sync Camel route") {
+
+ scenario("produce message and receive normal response") {
+ given("a registered two-way producer")
+ val producer = actorOf(classOf[SampleUntypedReplyingProducer])
+ producer.start
+
+ when("a test message is sent to the producer with !!")
+ val message = Message("test", Map(Message.MessageExchangeId -> "123"))
+ val result = producer.sendRequestReply(message)
+
+ then("a normal response should have been returned by the producer")
+ val expected = Message("received test", Map(Message.MessageExchangeId -> "123"))
+ assert(result === expected)
+ }
+
+ scenario("produce message and receive failure response") {
+ given("a registered two-way producer")
+ val producer = actorOf(classOf[SampleUntypedReplyingProducer])
+ producer.start
+
+ when("a test message causing an exception is sent to the producer with !!")
+ val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
+ val result = producer.sendRequestReply(message).asInstanceOf[Failure]
+
+ then("a failure response should have been returned by the producer")
+ val expectedFailureText = result.cause.getMessage
+ val expectedHeaders = result.headers
+ assert(expectedFailureText === "failure")
+ assert(expectedHeaders === Map(Message.MessageExchangeId -> "123"))
+ }
+
+ }
+
+ feature("Produce a message to a sync Camel route and then forward the response") {
+
+ scenario("produce message and send normal response to direct:forward-test-1") {
+ given("a registered one-way producer configured with a forward target")
+ val producer = actorOf(classOf[SampleUntypedForwardingProducer])
+ producer.start
+
+ when("a test message is sent to the producer with !")
+ mockEndpoint.expectedBodiesReceived("received test")
+ val result = producer.sendOneWay(Message("test"), producer)
+
+ then("a normal response should have been sent")
+ mockEndpoint.assertIsSatisfied
+ }
+
+ }
+
+ private def mockEndpoint = CamelContextManager.context.getEndpoint("mock:mock", classOf[MockEndpoint])
+}
+
+object UntypedProducerFeatureTest {
+ class TestRoute extends RouteBuilder {
+ def configure {
+ from("direct:forward-test-1").to("mock:mock")
+ from("direct:producer-test-1").process(new Processor() {
+ def process(exchange: Exchange) = {
+ exchange.getIn.getBody match {
+ case "fail" => throw new Exception("failure")
+ case body => exchange.getOut.setBody("received %s" format body)
+ }
+ }
+ })
+ }
+ }
+}
diff --git a/akka-camel/src/test/scala/component/ActiveObjectComponentFeatureTest.scala b/akka-camel/src/test/scala/component/ActiveObjectComponentFeatureTest.scala
deleted file mode 100644
index d80eedfd7a..0000000000
--- a/akka-camel/src/test/scala/component/ActiveObjectComponentFeatureTest.scala
+++ /dev/null
@@ -1,105 +0,0 @@
-package se.scalablesolutions.akka.camel.component
-
-import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
-
-import org.apache.camel.builder.RouteBuilder
-import se.scalablesolutions.akka.actor.Actor._
-import se.scalablesolutions.akka.actor.{ActorRegistry, ActiveObject}
-import se.scalablesolutions.akka.camel._
-import org.apache.camel.impl.{DefaultCamelContext, SimpleRegistry}
-import org.apache.camel.{ResolveEndpointFailedException, ExchangePattern, Exchange, Processor}
-
-/**
- * @author Martin Krasser
- */
-class ActiveObjectComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach {
- import ActiveObjectComponentFeatureTest._
- import CamelContextManager.template
-
- override protected def beforeAll = {
- val activePojo = ActiveObject.newInstance(classOf[Pojo]) // not a consumer
- val activePojoBase = ActiveObject.newInstance(classOf[PojoBase])
- val activePojoIntf = ActiveObject.newInstance(classOf[PojoIntf], new PojoImpl)
-
- val registry = new SimpleRegistry
- registry.put("pojo", activePojo)
-
- CamelContextManager.init(new DefaultCamelContext(registry))
- CamelContextManager.context.addRoutes(new CustomRouteBuilder)
- CamelContextManager.start
-
- CamelContextManager.activeObjectRegistry.put("base", activePojoBase)
- CamelContextManager.activeObjectRegistry.put("intf", activePojoIntf)
- }
-
- override protected def afterAll = {
- CamelContextManager.stop
- ActorRegistry.shutdownAll
- }
-
- feature("Communicate with an active object from a Camel application using active object endpoint URIs") {
- import ActiveObjectComponent.InternalSchema
- import ExchangePattern._
-
- scenario("in-out exchange with proxy created from interface and method returning String") {
- val result = template.requestBodyAndHeader("%s:intf?method=m2" format InternalSchema, "x", "test", "y")
- assert(result === "m2impl: x y")
- }
-
- scenario("in-out exchange with proxy created from class and method returning String") {
- val result = template.requestBodyAndHeader("%s:base?method=m2" format InternalSchema, "x", "test", "y")
- assert(result === "m2base: x y")
- }
-
- scenario("in-out exchange with proxy created from class and method returning void") {
- val result = template.requestBodyAndHeader("%s:base?method=m5" format InternalSchema, "x", "test", "y")
- assert(result === "x") // returns initial body
- }
-
- scenario("in-only exchange with proxy created from class and method returning String") {
- val result = template.send("%s:base?method=m2" format InternalSchema, InOnly, new Processor {
- def process(exchange: Exchange) = {
- exchange.getIn.setBody("x")
- exchange.getIn.setHeader("test", "y")
- }
- });
- assert(result.getPattern === InOnly)
- assert(result.getIn.getBody === "m2base: x y")
- assert(result.getOut.getBody === null)
- }
-
- scenario("in-only exchange with proxy created from class and method returning void") {
- val result = template.send("%s:base?method=m5" format InternalSchema, InOnly, new Processor {
- def process(exchange: Exchange) = {
- exchange.getIn.setBody("x")
- exchange.getIn.setHeader("test", "y")
- }
- });
- assert(result.getPattern === InOnly)
- assert(result.getIn.getBody === "x")
- assert(result.getOut.getBody === null)
- }
- }
-
- feature("Communicate with an active object from a Camel application from a custom Camel route") {
-
- scenario("in-out exchange with externally registered active object") {
- val result = template.requestBody("direct:test", "test")
- assert(result === "foo: test")
- }
-
- scenario("in-out exchange with internally registered active object not possible") {
- intercept[ResolveEndpointFailedException] {
- template.requestBodyAndHeader("active-object:intf?method=m2", "x", "test", "y")
- }
- }
- }
-}
-
-object ActiveObjectComponentFeatureTest {
- class CustomRouteBuilder extends RouteBuilder {
- def configure = {
- from("direct:test").to("active-object:pojo?method=foo")
- }
- }
-}
diff --git a/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala b/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala
index 7d1482c36c..331f2c23b6 100644
--- a/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala
+++ b/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala
@@ -3,37 +3,36 @@ package se.scalablesolutions.akka.camel.component
import java.util.concurrent.{TimeUnit, CountDownLatch}
import org.apache.camel.RuntimeCamelException
+import org.apache.camel.builder.RouteBuilder
+import org.apache.camel.component.mock.MockEndpoint
import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.actor.{ActorRegistry, Actor}
-import se.scalablesolutions.akka.camel.{Message, CamelContextManager}
+import se.scalablesolutions.akka.camel.{Failure, Message, CamelContextManager}
import se.scalablesolutions.akka.camel.support._
class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach {
+ import ActorComponentFeatureTest._
+
override protected def beforeAll = {
ActorRegistry.shutdownAll
CamelContextManager.init
+ CamelContextManager.context.addRoutes(new TestRoute)
CamelContextManager.start
}
override protected def afterAll = CamelContextManager.stop
- override protected def afterEach = ActorRegistry.shutdownAll
+ override protected def afterEach = {
+ ActorRegistry.shutdownAll
+ mockEndpoint.reset
+ }
- feature("Communicate with an actor from a Camel application using actor endpoint URIs") {
+ feature("Communicate with an actor via an actor:uuid endpoint") {
import CamelContextManager.template
- scenario("one-way communication using actor id") {
- val actor = actorOf[Tester1].start
- val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
- template.sendBody("actor:%s" format actor.id, "Martin")
- assert(latch.await(5000, TimeUnit.MILLISECONDS))
- val reply = (actor !! GetRetainedMessage).get.asInstanceOf[Message]
- assert(reply.body === "Martin")
- }
-
- scenario("one-way communication using actor uuid") {
+ scenario("one-way communication") {
val actor = actorOf[Tester1].start
val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
template.sendBody("actor:uuid:%s" format actor.uuid, "Martin")
@@ -42,12 +41,7 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
assert(reply.body === "Martin")
}
- scenario("two-way communication using actor id") {
- val actor = actorOf[Tester2].start
- assert(template.requestBody("actor:%s" format actor.id, "Martin") === "Hello Martin")
- }
-
- scenario("two-way communication using actor uuid") {
+ scenario("two-way communication") {
val actor = actorOf[Tester2].start
assert(template.requestBody("actor:uuid:%s" format actor.uuid, "Martin") === "Hello Martin")
}
@@ -58,5 +52,79 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
template.requestBody("actor:uuid:%s?blocking=true" format actor.uuid, "Martin")
}
}
+
+ scenario("two-way communication via a custom route with failure response") {
+ mockEndpoint.expectedBodiesReceived("whatever")
+ template.requestBody("direct:failure-test-1", "whatever")
+ mockEndpoint.assertIsSatisfied
+ }
+
+ scenario("two-way communication via a custom route with exception") {
+ mockEndpoint.expectedBodiesReceived("whatever")
+ template.requestBody("direct:failure-test-2", "whatever")
+ mockEndpoint.assertIsSatisfied
+ }
+ }
+
+ feature("Communicate with an actor via an actor:id endpoint") {
+ import CamelContextManager.template
+
+ scenario("one-way communication") {
+ val actor = actorOf[Tester1].start
+ val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ template.sendBody("actor:%s" format actor.id, "Martin")
+ assert(latch.await(5000, TimeUnit.MILLISECONDS))
+ val reply = (actor !! GetRetainedMessage).get.asInstanceOf[Message]
+ assert(reply.body === "Martin")
+ }
+
+ scenario("two-way communication") {
+ val actor = actorOf[Tester2].start
+ assert(template.requestBody("actor:%s" format actor.id, "Martin") === "Hello Martin")
+ }
+
+ scenario("two-way communication via a custom route") {
+ val actor = actorOf[CustomIdActor].start
+ assert(template.requestBody("direct:custom-id-test-1", "Martin") === "Received Martin")
+ assert(template.requestBody("direct:custom-id-test-2", "Martin") === "Received Martin")
+ }
+ }
+
+ private def mockEndpoint = CamelContextManager.context.getEndpoint("mock:mock", classOf[MockEndpoint])
+}
+
+object ActorComponentFeatureTest {
+ class CustomIdActor extends Actor {
+ self.id = "custom-id"
+ protected def receive = {
+ case msg: Message => self.reply("Received %s" format msg.body)
+ }
+ }
+
+ class FailWithMessage extends Actor {
+ protected def receive = {
+ case msg: Message => self.reply(Failure(new Exception("test")))
+ }
+ }
+
+ class FailWithException extends Actor {
+ protected def receive = {
+ case msg: Message => throw new Exception("test")
+ }
+ }
+
+ class TestRoute extends RouteBuilder {
+ val failWithMessage = actorOf[FailWithMessage].start
+ val failWithException = actorOf[FailWithException].start
+ def configure {
+ from("direct:custom-id-test-1").to("actor:custom-id")
+ from("direct:custom-id-test-2").to("actor:id:custom-id")
+ from("direct:failure-test-1")
+ .onException(classOf[Exception]).to("mock:mock").handled(true).end
+ .to("actor:uuid:%s" format failWithMessage.uuid)
+ from("direct:failure-test-2")
+ .onException(classOf[Exception]).to("mock:mock").handled(true).end
+ .to("actor:uuid:%s?blocking=true" format failWithException.uuid)
+ }
}
}
diff --git a/akka-camel/src/test/scala/component/ActorProducerTest.scala b/akka-camel/src/test/scala/component/ActorProducerTest.scala
index 300d0ca617..5e8a674e55 100644
--- a/akka-camel/src/test/scala/component/ActorProducerTest.scala
+++ b/akka-camel/src/test/scala/component/ActorProducerTest.scala
@@ -116,4 +116,4 @@ object ActorProducerTest {
}
}
-}
\ No newline at end of file
+}
diff --git a/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala b/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala
new file mode 100644
index 0000000000..06f7e29173
--- /dev/null
+++ b/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala
@@ -0,0 +1,109 @@
+package se.scalablesolutions.akka.camel.component
+
+import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
+
+import org.apache.camel.builder.RouteBuilder
+import se.scalablesolutions.akka.actor.Actor._
+import se.scalablesolutions.akka.actor.{ActorRegistry, TypedActor}
+import se.scalablesolutions.akka.camel._
+import org.apache.camel.impl.{DefaultCamelContext, SimpleRegistry}
+import org.apache.camel.{ResolveEndpointFailedException, ExchangePattern, Exchange, Processor}
+
+/**
+ * @author Martin Krasser
+ */
+class TypedActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach {
+ import TypedActorComponentFeatureTest._
+ import CamelContextManager.template
+
+ override protected def beforeAll = {
+ val typedActor = TypedActor.newInstance(classOf[SampleTypedActor], classOf[SampleTypedActorImpl]) // not a consumer
+ val typedConsumer = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
+
+ val registry = new SimpleRegistry
+ // external registration
+ registry.put("ta", typedActor)
+
+ CamelContextManager.init(new DefaultCamelContext(registry))
+ CamelContextManager.context.addRoutes(new CustomRouteBuilder)
+ CamelContextManager.start
+
+ // Internal registration
+ CamelContextManager.typedActorRegistry.put("tc", typedConsumer)
+ }
+
+ override protected def afterAll = {
+ CamelContextManager.stop
+ ActorRegistry.shutdownAll
+ }
+
+ feature("Communicate with an internally-registered typed actor using typed-actor-internal endpoint URIs") {
+ import TypedActorComponent.InternalSchema
+ import ExchangePattern._
+
+ scenario("two-way communication with method returning String") {
+ val result1 = template.requestBodyAndHeader("%s:tc?method=m2" format InternalSchema, "x", "test", "y")
+ val result2 = template.requestBodyAndHeader("%s:tc?method=m4" format InternalSchema, "x", "test", "y")
+ assert(result1 === "m2: x y")
+ assert(result2 === "m4: x y")
+ }
+
+ scenario("two-way communication with method returning void") {
+ val result = template.requestBodyAndHeader("%s:tc?method=m5" format InternalSchema, "x", "test", "y")
+ assert(result === "x") // returns initial body
+ }
+
+ scenario("one-way communication with method returning String") {
+ val result = template.send("%s:tc?method=m2" format InternalSchema, InOnly, new Processor {
+ def process(exchange: Exchange) = {
+ exchange.getIn.setBody("x")
+ exchange.getIn.setHeader("test", "y")
+ }
+ });
+ assert(result.getPattern === InOnly)
+ assert(result.getIn.getBody === "m2: x y")
+ assert(result.getOut.getBody === null)
+ }
+
+ scenario("one-way communication with method returning void") {
+ val result = template.send("%s:tc?method=m5" format InternalSchema, InOnly, new Processor {
+ def process(exchange: Exchange) = {
+ exchange.getIn.setBody("x")
+ exchange.getIn.setHeader("test", "y")
+ }
+ });
+ assert(result.getPattern === InOnly)
+ assert(result.getIn.getBody === "x")
+ assert(result.getOut.getBody === null)
+ }
+
+ }
+
+ feature("Communicate with an internally-registered typed actor using typed-actor endpoint URIs") {
+ scenario("communication not possible") {
+ intercept[ResolveEndpointFailedException] {
+ template.requestBodyAndHeader("typed-actor:tc?method=m2", "x", "test", "y")
+ }
+ }
+ }
+
+ feature("Communicate with an externally-registered typed actor using typed-actor endpoint URIs") {
+ scenario("two-way communication with method returning String") {
+ val result = template.requestBody("typed-actor:ta?method=foo", "test")
+ assert(result === "foo: test")
+ }
+
+ scenario("two-way communication with method returning String via custom route") {
+ val result = template.requestBody("direct:test", "test")
+ assert(result === "foo: test")
+ }
+ }
+}
+
+object TypedActorComponentFeatureTest {
+ class CustomRouteBuilder extends RouteBuilder {
+ def configure = {
+ from("direct:test").to("typed-actor:ta?method=foo")
+ }
+ }
+}
diff --git a/akka-camel/src/test/scala/support/TestSupport.scala b/akka-camel/src/test/scala/support/TestSupport.scala
index 61118ef3e3..8e1322e14f 100644
--- a/akka-camel/src/test/scala/support/TestSupport.scala
+++ b/akka-camel/src/test/scala/support/TestSupport.scala
@@ -2,6 +2,8 @@ package se.scalablesolutions.akka.camel.support
import java.util.concurrent.{TimeUnit, CountDownLatch}
+import collection.mutable.Buffer
+
import se.scalablesolutions.akka.camel.Message
import se.scalablesolutions.akka.actor.Actor
@@ -54,12 +56,13 @@ trait Respond { this: Actor =>
}
trait Retain { this: Actor =>
- var message: Any = _
+ val messages = Buffer[Any]()
def retain: Handler = {
- case GetRetainedMessage => self.reply(message)
+ case GetRetainedMessage => self.reply(messages.last)
+ case GetRetainedMessages(p) => self.reply(messages.toList.filter(p))
case msg => {
- message = msg
+ messages += msg
msg
}
}
@@ -73,3 +76,6 @@ trait Noop { this: Actor =>
case class SetExpectedMessageCount(num: Int)
case class GetRetainedMessage()
+case class GetRetainedMessages(p: Any => Boolean) {
+ def this() = this(_ => true)
+}
diff --git a/akka-core/.ensime b/akka-core/.ensime
index 15e1ae85be..0b21e8eb5c 100644
--- a/akka-core/.ensime
+++ b/akka-core/.ensime
@@ -1,11 +1,11 @@
(
;; Where you unpacked the ENSIME distribution.
- :server-root "/home/jboner/emacs-config/lib/ensime"
+ :server-root "/Users/jboner/config/emacs-config/lib/ensime"
;; The command with which to invoke the ENSIME server. Change this to
;; "bin/server.bat" if your're on Windows.
- :server-cmd "bin/server.bat"
+ :server-cmd "bin/server.sh"
;; The host to connect to. Connecting to remote ENSIME servers is not
diff --git a/akka-core/src/main/java/se/scalablesolutions/akka/annotation/configuration.java b/akka-core/src/main/java/se/scalablesolutions/akka/annotation/configuration.java
deleted file mode 100644
index 9c5375398b..0000000000
--- a/akka-core/src/main/java/se/scalablesolutions/akka/annotation/configuration.java
+++ /dev/null
@@ -1,11 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- * class Ping {
- * // This context will be injected, holds RTTI (runtime type information)
- * // for the current message send
- * private ActiveObjectContext context = null;
- *
- * public void hit(int count) {
- * Pong pong = (Pong) context.getSender();
- * pong.hit(count++)
- * }
- * }
- *
- *
- * @author Jonas Bonér
- */
-final class ActiveObjectContext {
- private[akka] var _sender: AnyRef = _
- private[akka] var _senderFuture: CompletableFuture[Any] = _
-
- /**
- * Returns the current sender Active Object reference.
- * Scala style getter.
- */
- def sender: AnyRef = {
- if (_sender eq null) throw new IllegalActorStateException("Sender reference should not be null.")
- else _sender
- }
-
- /**
- * Returns the current sender Active Object reference.
- * Java style getter.
- */
- def getSender: AnyRef = {
- if (_sender eq null) throw new IllegalActorStateException("Sender reference should not be null.")
- else _sender
- }
-
- /**
- * Returns the current sender future Active Object reference.
- * Scala style getter.
- */
- def senderFuture: Option[CompletableFuture[Any]] = if (_senderFuture eq null) None else Some(_senderFuture)
-
- /**
- * Returns the current sender future Active Object reference.
- * Java style getter.
- * This method returns 'null' if the sender future is not available.
- */
- def getSenderFuture = _senderFuture
-}
-
-/**
- * Internal helper class to help pass the contextual information between threads.
- *
- * @author Jonas Bonér
- */
-private[akka] object ActiveObjectContext {
- import scala.util.DynamicVariable
- private[actor] val sender = new DynamicVariable[AnyRef](null)
- private[actor] val senderFuture = new DynamicVariable[CompletableFuture[Any]](null)
-}
-
-/**
- * Factory class for creating Active Objects out of plain POJOs and/or POJOs with interfaces.
- *
- * @author Jonas Bonér
- */
-object ActiveObject extends Logging {
- import Actor.actorOf
-
- val AKKA_CAMEL_ROUTING_SCHEME = "akka"
- private[actor] val AW_PROXY_PREFIX = "$$ProxiedByAW".intern
-
- def newInstance[T](target: Class[T], timeout: Long): T =
- newInstance(target, actorOf(new Dispatcher(false)), None, timeout)
-
- def newInstance[T](target: Class[T]): T =
- newInstance(target, actorOf(new Dispatcher(false)), None, Actor.TIMEOUT)
-
- def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long): T =
- newInstance(intf, target, actorOf(new Dispatcher(false)), None, timeout)
-
- def newInstance[T](intf: Class[T], target: AnyRef): T =
- newInstance(intf, target, actorOf(new Dispatcher(false)), None, Actor.TIMEOUT)
-
- def newRemoteInstance[T](target: Class[T], timeout: Long, hostname: String, port: Int): T =
- newInstance(target, actorOf(new Dispatcher(false)), Some(new InetSocketAddress(hostname, port)), timeout)
-
- def newRemoteInstance[T](target: Class[T], hostname: String, port: Int): T =
- newInstance(target, actorOf(new Dispatcher(false)), Some(new InetSocketAddress(hostname, port)), Actor.TIMEOUT)
-
- def newInstance[T](target: Class[T], config: ActiveObjectConfiguration): T = {
- val actor = actorOf(new Dispatcher(config._transactionRequired, config._restartCallbacks, config._shutdownCallback))
- if (config._messageDispatcher.isDefined) {
- actor.dispatcher = config._messageDispatcher.get
- }
- newInstance(target, actor, config._host, config._timeout)
- }
-
- def newInstance[T](intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration): T = {
- val actor = actorOf(new Dispatcher(config._transactionRequired, config._restartCallbacks, config._shutdownCallback))
- if (config._messageDispatcher.isDefined) {
- actor.dispatcher = config._messageDispatcher.get
- }
- newInstance(intf, target, actor, config._host, config._timeout)
- }
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newInstance[T](target: Class[T], timeout: Long, restartCallbacks: Option[RestartCallbacks]): T =
- newInstance(target, actorOf(new Dispatcher(false, restartCallbacks)), None, timeout)
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, restartCallbacks: Option[RestartCallbacks]): T =
- newInstance(intf, target, actorOf(new Dispatcher(false, restartCallbacks)), None, timeout)
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newInstance[T](target: Class[T], timeout: Long, transactionRequired: Boolean): T =
- newInstance(target, actorOf(new Dispatcher(transactionRequired, None)), None, timeout)
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newInstance[T](target: Class[T], timeout: Long, transactionRequired: Boolean, restartCallbacks: Option[RestartCallbacks]): T =
- newInstance(target, actorOf(new Dispatcher(transactionRequired, restartCallbacks)), None, timeout)
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, transactionRequired: Boolean): T =
- newInstance(intf, target, actorOf(new Dispatcher(transactionRequired, None)), None, timeout)
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, transactionRequired: Boolean, restartCallbacks: Option[RestartCallbacks]): T =
- newInstance(intf, target, actorOf(new Dispatcher(transactionRequired, restartCallbacks)), None, timeout)
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, hostname: String, port: Int): T =
- newInstance(intf, target, actorOf(new Dispatcher(false, None)), Some(new InetSocketAddress(hostname, port)), timeout)
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T =
- newInstance(intf, target, actorOf(new Dispatcher(false, restartCallbacks)), Some(new InetSocketAddress(hostname, port)), timeout)
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](target: Class[T], timeout: Long, transactionRequired: Boolean, hostname: String, port: Int): T =
- newInstance(target, actorOf(new Dispatcher(transactionRequired, None)), Some(new InetSocketAddress(hostname, port)), timeout)
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](target: Class[T], timeout: Long, transactionRequired: Boolean, hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T =
- newInstance(target, actorOf(new Dispatcher(transactionRequired, restartCallbacks)), Some(new InetSocketAddress(hostname, port)), timeout)
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, transactionRequired: Boolean, hostname: String, port: Int): T =
- newInstance(intf, target, actorOf(new Dispatcher(transactionRequired, None)), Some(new InetSocketAddress(hostname, port)), timeout)
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, transactionRequired: Boolean, hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T =
- newInstance(intf, target, actorOf(new Dispatcher(transactionRequired, restartCallbacks)), Some(new InetSocketAddress(hostname, port)), timeout)
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher): T = {
- val actor = actorOf(new Dispatcher(false, None))
- actor.dispatcher = dispatcher
- newInstance(target, actor, None, timeout)
- }
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher, restartCallbacks: Option[RestartCallbacks]): T = {
- val actor = actorOf(new Dispatcher(false, restartCallbacks))
- actor.dispatcher = dispatcher
- newInstance(target, actor, None, timeout)
- }
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher): T = {
- val actor = actorOf(new Dispatcher(false, None))
- actor.dispatcher = dispatcher
- newInstance(intf, target, actor, None, timeout)
- }
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long,
- dispatcher: MessageDispatcher, restartCallbacks: Option[RestartCallbacks]): T = {
- val actor = actorOf(new Dispatcher(false, restartCallbacks))
- actor.dispatcher = dispatcher
- newInstance(intf, target, actor, None, timeout)
- }
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newInstance[T](target: Class[T], timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher): T = {
- val actor = actorOf(new Dispatcher(transactionRequired, None))
- actor.dispatcher = dispatcher
- newInstance(target, actor, None, timeout)
- }
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newInstance[T](target: Class[T], timeout: Long, transactionRequired: Boolean,
- dispatcher: MessageDispatcher, restartCallbacks: Option[RestartCallbacks]): T = {
- val actor = actorOf(new Dispatcher(transactionRequired, restartCallbacks))
- actor.dispatcher = dispatcher
- newInstance(target, actor, None, timeout)
- }
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher): T = {
- val actor = actorOf(new Dispatcher(transactionRequired, None))
- actor.dispatcher = dispatcher
- newInstance(intf, target, actor, None, timeout)
- }
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newInstance[T](intf: Class[T], target: AnyRef, timeout: Long, transactionRequired: Boolean,
- dispatcher: MessageDispatcher, restartCallbacks: Option[RestartCallbacks]): T = {
- val actor = actorOf(new Dispatcher(transactionRequired, restartCallbacks))
- actor.dispatcher = dispatcher
- newInstance(intf, target, actor, None, timeout)
- }
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int): T = {
- val actor = actorOf(new Dispatcher(false, None))
- actor.dispatcher = dispatcher
- newInstance(target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
- }
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](target: Class[T], timeout: Long, dispatcher: MessageDispatcher,
- hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T = {
- val actor = actorOf(new Dispatcher(false, restartCallbacks))
- actor.dispatcher = dispatcher
- newInstance(target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
- }
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher, hostname: String, port: Int): T = {
- val actor = actorOf(new Dispatcher(false, None))
- actor.dispatcher = dispatcher
- newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
- }
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, dispatcher: MessageDispatcher,
- hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T = {
- val actor = actorOf(new Dispatcher(false, restartCallbacks))
- actor.dispatcher = dispatcher
- newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
- }
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](target: Class[T], timeout: Long, transactionRequired: Boolean,
- dispatcher: MessageDispatcher, hostname: String, port: Int): T = {
- val actor = actorOf(new Dispatcher(transactionRequired, None))
- actor.dispatcher = dispatcher
- newInstance(target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
- }
-
- @deprecated("use newInstance(target: Class[T], config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](target: Class[T], timeout: Long, transactionRequired: Boolean, dispatcher: MessageDispatcher,
- hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T = {
- val actor = actorOf(new Dispatcher(transactionRequired, restartCallbacks))
- actor.dispatcher = dispatcher
- newInstance(target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
- }
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, transactionRequired: Boolean,
- dispatcher: MessageDispatcher, hostname: String, port: Int): T = {
- val actor = actorOf(new Dispatcher(transactionRequired, None))
- actor.dispatcher = dispatcher
- newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
- }
-
- @deprecated("use newInstance(intf: Class[T], target: AnyRef, config: ActiveObjectConfiguration) instead")
- def newRemoteInstance[T](intf: Class[T], target: AnyRef, timeout: Long, transactionRequired: Boolean,
- dispatcher: MessageDispatcher, hostname: String, port: Int, restartCallbacks: Option[RestartCallbacks]): T = {
- val actor = actorOf(new Dispatcher(transactionRequired, restartCallbacks))
- actor.dispatcher = dispatcher
- newInstance(intf, target, actor, Some(new InetSocketAddress(hostname, port)), timeout)
- }
-
- private[akka] def newInstance[T](target: Class[T], actorRef: ActorRef, remoteAddress: Option[InetSocketAddress], timeout: Long): T = {
- val proxy = Proxy.newInstance(target, true, false)
- val context = injectActiveObjectContext(proxy)
- actorRef.actor.asInstanceOf[Dispatcher].initialize(target, proxy, context)
- actorRef.timeout = timeout
- if (remoteAddress.isDefined) actorRef.makeRemote(remoteAddress.get)
- AspectInitRegistry.register(proxy, AspectInit(target, actorRef, remoteAddress, timeout))
- actorRef.start
- proxy.asInstanceOf[T]
- }
-
- private[akka] def newInstance[T](intf: Class[T], target: AnyRef, actorRef: ActorRef,
- remoteAddress: Option[InetSocketAddress], timeout: Long): T = {
- val context = injectActiveObjectContext(target)
- val proxy = Proxy.newInstance(Array(intf), Array(target), true, false)
- actorRef.actor.asInstanceOf[Dispatcher].initialize(target.getClass, target, context)
- actorRef.timeout = timeout
- if (remoteAddress.isDefined) actorRef.makeRemote(remoteAddress.get)
- AspectInitRegistry.register(proxy, AspectInit(intf, actorRef, remoteAddress, timeout))
- actorRef.start
- proxy.asInstanceOf[T]
- }
-
- def stop(obj: AnyRef): Unit = {
- val init = AspectInitRegistry.initFor(obj)
- init.actorRef.stop
- }
-
- /**
- * Get the underlying dispatcher actor for the given active object.
- */
- def actorFor(obj: AnyRef): Option[ActorRef] =
- ActorRegistry.actorsFor(classOf[Dispatcher]).find(a => a.actor.asInstanceOf[Dispatcher].target == Some(obj))
-
- /**
- * Links an other active object to this active object.
- * @param supervisor the supervisor active object
- * @param supervised the active object to link
- */
- def link(supervisor: AnyRef, supervised: AnyRef) = {
- val supervisorActor = actorFor(supervisor).getOrElse(
- throw new IllegalActorStateException("Can't link when the supervisor is not an active object"))
- val supervisedActor = actorFor(supervised).getOrElse(
- throw new IllegalActorStateException("Can't link when the supervised is not an active object"))
- supervisorActor.link(supervisedActor)
- }
-
- /**
- * Links an other active object to this active object and sets the fault handling for the supervisor.
- * @param supervisor the supervisor active object
- * @param supervised the active object to link
- * @param handler fault handling strategy
- * @param trapExceptions array of exceptions that should be handled by the supervisor
- */
- def link(supervisor: AnyRef, supervised: AnyRef, handler: FaultHandlingStrategy, trapExceptions: Array[Class[_ <: Throwable]]) = {
- val supervisorActor = actorFor(supervisor).getOrElse(
- throw new IllegalActorStateException("Can't link when the supervisor is not an active object"))
- val supervisedActor = actorFor(supervised).getOrElse(
- throw new IllegalActorStateException("Can't link when the supervised is not an active object"))
- supervisorActor.trapExit = trapExceptions.toList
- supervisorActor.faultHandler = Some(handler)
- supervisorActor.link(supervisedActor)
- }
-
- /**
- * Unlink the supervised active object from the supervisor.
- * @param supervisor the supervisor active object
- * @param supervised the active object to unlink
- */
- def unlink(supervisor: AnyRef, supervised: AnyRef) = {
- val supervisorActor = actorFor(supervisor).getOrElse(
- throw new IllegalActorStateException("Can't unlink when the supervisor is not an active object"))
- val supervisedActor = actorFor(supervised).getOrElse(
- throw new IllegalActorStateException("Can't unlink when the supervised is not an active object"))
- supervisorActor.unlink(supervisedActor)
- }
-
- /**
- * Sets the trap exit for the given supervisor active object.
- * @param supervisor the supervisor active object
- * @param trapExceptions array of exceptions that should be handled by the supervisor
- */
- def trapExit(supervisor: AnyRef, trapExceptions: Array[Class[_ <: Throwable]]) = {
- val supervisorActor = actorFor(supervisor).getOrElse(
- throw new IllegalActorStateException("Can't set trap exceptions when the supervisor is not an active object"))
- supervisorActor.trapExit = trapExceptions.toList
- this
- }
-
- /**
- * Sets the fault handling strategy for the given supervisor active object.
- * @param supervisor the supervisor active object
- * @param handler fault handling strategy
- */
- def faultHandler(supervisor: AnyRef, handler: FaultHandlingStrategy) = {
- val supervisorActor = actorFor(supervisor).getOrElse(
- throw new IllegalActorStateException("Can't set fault handler when the supervisor is not an active object"))
- supervisorActor.faultHandler = Some(handler)
- this
- }
-
- private def injectActiveObjectContext(activeObject: AnyRef): Option[ActiveObjectContext] = {
- def injectActiveObjectContext0(activeObject: AnyRef, clazz: Class[_]): Option[ActiveObjectContext] = {
- val contextField = clazz.getDeclaredFields.toList.find(_.getType == classOf[ActiveObjectContext])
- if (contextField.isDefined) {
- contextField.get.setAccessible(true)
- val context = new ActiveObjectContext
- contextField.get.set(activeObject, context)
- Some(context)
- } else {
- val parent = clazz.getSuperclass
- if (parent != null) injectActiveObjectContext0(activeObject, parent)
- else {
- log.ifTrace("Can't set 'ActiveObjectContext' for ActiveObject [" +
- activeObject.getClass.getName +
- "] since no field of this type could be found.")
- None
- }
- }
- }
- injectActiveObjectContext0(activeObject, activeObject.getClass)
- }
-
- private[akka] def supervise(restartStrategy: RestartStrategy, components: List[Supervise]): Supervisor =
- Supervisor(SupervisorConfig(restartStrategy, components))
-}
-
-private[akka] object AspectInitRegistry extends ListenerManagement {
- private val initializations = new java.util.concurrent.ConcurrentHashMap[AnyRef, AspectInit]
-
- def initFor(target: AnyRef) = {
- initializations.get(target)
- }
-
- def register(target: AnyRef, init: AspectInit) = {
- val res = initializations.put(target, init)
- foreachListener(_ ! AspectInitRegistered(target, init))
- res
- }
-
- def unregister(target: AnyRef) = {
- val res = initializations.remove(target)
- foreachListener(_ ! AspectInitUnregistered(target, res))
- res
- }
-}
-
-private[akka] sealed trait AspectInitRegistryEvent
-private[akka] case class AspectInitRegistered(proxy: AnyRef, init: AspectInit) extends AspectInitRegistryEvent
-private[akka] case class AspectInitUnregistered(proxy: AnyRef, init: AspectInit) extends AspectInitRegistryEvent
-
-private[akka] sealed case class AspectInit(
- val target: Class[_],
- val actorRef: ActorRef,
- val remoteAddress: Option[InetSocketAddress],
- val timeout: Long) {
- def this(target: Class[_], actorRef: ActorRef, timeout: Long) = this(target, actorRef, None, timeout)
-}
-
-/**
- * AspectWerkz Aspect that is turning POJOs into Active Object.
- * Is deployed on a 'per-instance' basis.
- *
- * @author Jonas Bonér
- */
-@Aspect("perInstance")
-private[akka] sealed class ActiveObjectAspect {
- @volatile private var isInitialized = false
- @volatile private var isStopped = false
- private var target: Class[_] = _
- private var actorRef: ActorRef = _
- private var remoteAddress: Option[InetSocketAddress] = _
- private var timeout: Long = _
- @volatile private var instance: AnyRef = _
-
- @Around("execution(* *.*(..))")
- def invoke(joinPoint: JoinPoint): AnyRef = {
- if (!isInitialized) {
- val init = AspectInitRegistry.initFor(joinPoint.getThis)
- target = init.target
- actorRef = init.actorRef
- remoteAddress = init.remoteAddress
- timeout = init.timeout
- isInitialized = true
-
- }
- dispatch(joinPoint)
- }
-
- private def dispatch(joinPoint: JoinPoint) = {
- if (remoteAddress.isDefined) remoteDispatch(joinPoint)
- else localDispatch(joinPoint)
- }
-
- private def localDispatch(joinPoint: JoinPoint): AnyRef = {
- val rtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
- val isOneWay = isVoid(rtti)
- val sender = ActiveObjectContext.sender.value
- val senderFuture = ActiveObjectContext.senderFuture.value
-
- if (!actorRef.isRunning && !isStopped) {
- isStopped = true
- joinPoint.proceed
- } else if (isOneWay) {
- actorRef ! Invocation(joinPoint, true, true, sender, senderFuture)
- null.asInstanceOf[AnyRef]
- } else {
- val result = (actorRef !! (Invocation(joinPoint, false, isOneWay, sender, senderFuture), timeout)).as[AnyRef]
- if (result.isDefined) result.get
- else throw new IllegalActorStateException("No result defined for invocation [" + joinPoint + "]")
- }
- }
-
- private def remoteDispatch(joinPoint: JoinPoint): AnyRef = {
- val rtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
- val isOneWay = isVoid(rtti)
- val (message: Array[AnyRef], isEscaped) = escapeArguments(rtti.getParameterValues)
- val requestBuilder = RemoteRequestProtocol.newBuilder
- .setId(RemoteRequestProtocolIdFactory.nextId)
- .setMessage(MessageSerializer.serialize(message))
- .setMethod(rtti.getMethod.getName)
- .setTarget(target.getName)
- .setUuid(actorRef.uuid)
- .setTimeout(timeout)
- .setIsActor(false)
- .setIsOneWay(isOneWay)
- .setIsEscaped(false)
- val id = actorRef.registerSupervisorAsRemoteActor
- if (id.isDefined) requestBuilder.setSupervisorUuid(id.get)
- val remoteMessage = requestBuilder.build
- val future = RemoteClient.clientFor(remoteAddress.get).send(remoteMessage, None)
- if (isOneWay) null // for void methods
- else {
- if (future.isDefined) {
- future.get.await
- val result = getResultOrThrowException(future.get)
- if (result.isDefined) result.get
- else throw new IllegalActorStateException("No result returned from call to [" + joinPoint + "]")
- } else throw new IllegalActorStateException("No future returned from call to [" + joinPoint + "]")
- }
- }
-
- private def getResultOrThrowException[T](future: Future[T]): Option[T] =
- if (future.exception.isDefined) {
- val (_, cause) = future.exception.get
- throw cause
- } else future.result
-
- private def isVoid(rtti: MethodRtti) = rtti.getMethod.getReturnType == java.lang.Void.TYPE
-
- private def escapeArguments(args: Array[AnyRef]): Tuple2[Array[AnyRef], Boolean] = {
- var isEscaped = false
- val escapedArgs = for (arg <- args) yield {
- val clazz = arg.getClass
- if (clazz.getName.contains(ActiveObject.AW_PROXY_PREFIX)) {
- isEscaped = true
- ActiveObject.AW_PROXY_PREFIX + clazz.getSuperclass.getName
- } else arg
- }
- (escapedArgs, isEscaped)
- }
-}
-
-/**
- * Represents a snapshot of the current invocation.
- *
- * @author Jonas Bonér
- */
-@serializable private[akka] case class Invocation(
- joinPoint: JoinPoint, isOneWay: Boolean, isVoid: Boolean, sender: AnyRef, senderFuture: CompletableFuture[Any]) {
-
- override def toString: String = synchronized {
- "Invocation [" +
- "\n\t\tmethod = " + joinPoint.getRtti.asInstanceOf[MethodRtti].getMethod.getName + " @ " + joinPoint.getTarget.getClass.getName +
- "\n\t\tisOneWay = " + isOneWay +
- "\n\t\tisVoid = " + isVoid +
- "\n\t\tsender = " + sender +
- "\n\t\tsenderFuture = " + senderFuture +
- "]"
- }
-
- override def hashCode: Int = synchronized {
- var result = HashCode.SEED
- result = HashCode.hash(result, joinPoint)
- result = HashCode.hash(result, isOneWay)
- result = HashCode.hash(result, isVoid)
- result = HashCode.hash(result, sender)
- result = HashCode.hash(result, senderFuture)
- result
- }
-
- override def equals(that: Any): Boolean = synchronized {
- that != null &&
- that.isInstanceOf[Invocation] &&
- that.asInstanceOf[Invocation].joinPoint == joinPoint &&
- that.asInstanceOf[Invocation].isOneWay == isOneWay &&
- that.asInstanceOf[Invocation].isVoid == isVoid &&
- that.asInstanceOf[Invocation].sender == sender &&
- that.asInstanceOf[Invocation].senderFuture == senderFuture
- }
-}
-
-object Dispatcher {
- val ZERO_ITEM_CLASS_ARRAY = Array[Class[_]]()
- val ZERO_ITEM_OBJECT_ARRAY = Array[Object]()
- var crashedActorTl:ThreadLocal[Dispatcher] = new ThreadLocal();
-}
-
-/**
- * Generic Actor managing Invocation dispatch, transaction and error management.
- *
- * @author Jonas Bonér
- */
-private[akka] class Dispatcher(transactionalRequired: Boolean,
- var restartCallbacks: Option[RestartCallbacks] = None,
- var shutdownCallback: Option[ShutdownCallback] = None) extends Actor {
- import Dispatcher._
-
- private[actor] var target: Option[AnyRef] = None
- private var zhutdown: Option[Method] = None
- private var preRestart: Option[Method] = None
- private var postRestart: Option[Method] = None
- private var initTxState: Option[Method] = None
- private var context: Option[ActiveObjectContext] = None
- private var targetClass:Class[_] = _
-
- def this(transactionalRequired: Boolean) = this(transactionalRequired,None)
-
- private[actor] def initialize(targetClass: Class[_], targetInstance: AnyRef, ctx: Option[ActiveObjectContext]) = {
-
- if (transactionalRequired || targetClass.isAnnotationPresent(Annotations.transactionrequired))
- self.makeTransactionRequired
- self.id = targetClass.getName
- this.targetClass = targetClass
- target = Some(targetInstance)
- context = ctx
- val methods = targetInstance.getClass.getDeclaredMethods.toList
-
- if (self.lifeCycle.isEmpty) self.lifeCycle = Some(LifeCycle(Permanent))
-
- // See if we have any config define restart callbacks
- restartCallbacks match {
- case None => {}
- case Some(RestartCallbacks(pre, post)) =>
- preRestart = Some(try {
- targetInstance.getClass.getDeclaredMethod(pre, ZERO_ITEM_CLASS_ARRAY: _*)
- } catch { case e => throw new IllegalActorStateException(
- "Could not find pre restart method [" + pre + "] \nin [" +
- targetClass.getName + "]. \nIt must have a zero argument definition.") })
- postRestart = Some(try {
- targetInstance.getClass.getDeclaredMethod(post, ZERO_ITEM_CLASS_ARRAY: _*)
- } catch { case e => throw new IllegalActorStateException(
- "Could not find post restart method [" + post + "] \nin [" +
- targetClass.getName + "]. \nIt must have a zero argument definition.") })
- }
- // See if we have any config define a shutdown callback
- shutdownCallback match {
- case None => {}
- case Some(ShutdownCallback(down)) =>
- zhutdown = Some(try {
- targetInstance.getClass.getDeclaredMethod(down, ZERO_ITEM_CLASS_ARRAY: _*)
- } catch { case e => throw new IllegalStateException(
- "Could not find shutdown method [" + down + "] \nin [" +
- targetClass.getName + "]. \nIt must have a zero argument definition.") })
- }
-
- // See if we have any annotation defined restart callbacks
- if (!preRestart.isDefined) preRestart = methods.find(m => m.isAnnotationPresent(Annotations.prerestart))
- if (!postRestart.isDefined) postRestart = methods.find(m => m.isAnnotationPresent(Annotations.postrestart))
- // See if we have an annotation defined shutdown callback
- if (!zhutdown.isDefined) zhutdown = methods.find(m => m.isAnnotationPresent(Annotations.shutdown))
-
- if (preRestart.isDefined && preRestart.get.getParameterTypes.length != 0)
- throw new IllegalActorStateException(
- "Method annotated with @prerestart or defined as a restart callback in \n[" +
- targetClass.getName + "] must have a zero argument definition")
- if (postRestart.isDefined && postRestart.get.getParameterTypes.length != 0)
- throw new IllegalActorStateException(
- "Method annotated with @postrestart or defined as a restart callback in \n[" +
- targetClass.getName + "] must have a zero argument definition")
- if (zhutdown.isDefined && zhutdown.get.getParameterTypes.length != 0)
- throw new IllegalStateException(
- "Method annotated with @shutdown or defined as a shutdown callback in \n[" +
- targetClass.getName + "] must have a zero argument definition")
-
- if (preRestart.isDefined) preRestart.get.setAccessible(true)
- if (postRestart.isDefined) postRestart.get.setAccessible(true)
- if (zhutdown.isDefined) zhutdown.get.setAccessible(true)
-
- // see if we have a method annotated with @inittransactionalstate, if so invoke it
- initTxState = methods.find(m => m.isAnnotationPresent(Annotations.inittransactionalstate))
- if (initTxState.isDefined && initTxState.get.getParameterTypes.length != 0)
- throw new IllegalActorStateException("Method annotated with @inittransactionalstate must have a zero argument definition")
- if (initTxState.isDefined) initTxState.get.setAccessible(true)
- }
-
- def receive = {
- case invocation @ Invocation(joinPoint, isOneWay, _, sender, senderFuture) =>
- ActiveObject.log.ifTrace("Invoking active object with message:\n" + invocation)
- context.foreach { ctx =>
- if (sender ne null) ctx._sender = sender
- if (senderFuture ne null) ctx._senderFuture = senderFuture
- }
- ActiveObjectContext.sender.value = joinPoint.getThis // set next sender
- self.senderFuture.foreach(ActiveObjectContext.senderFuture.value = _)
- if (Actor.SERIALIZE_MESSAGES) serializeArguments(joinPoint)
- if (isOneWay) joinPoint.proceed
- else self.reply(joinPoint.proceed)
-
- // Jan Kronquist: started work on issue 121
- case Link(target) => self.link(target)
- case Unlink(target) => self.unlink(target)
- case unexpected => throw new IllegalActorStateException(
- "Unexpected message [" + unexpected + "] sent to [" + this + "]")
- }
-
- override def preRestart(reason: Throwable) {
- try {
- // Since preRestart is called we know that this dispatcher
- // is about to be restarted. Put the instance in a thread
- // local so the new dispatcher can be initialized with the
- // contents of the old.
- //FIXME - This should be considered as a workaround.
- crashedActorTl.set(this)
- preRestart.foreach(_.invoke(target.get, ZERO_ITEM_OBJECT_ARRAY: _*))
- } catch { case e: InvocationTargetException => throw e.getCause }
- }
-
- override def postRestart(reason: Throwable) {
- try {
- postRestart.foreach(_.invoke(target.get, ZERO_ITEM_OBJECT_ARRAY: _*))
- } catch { case e: InvocationTargetException => throw e.getCause }
- }
-
- override def init = {
- // Get the crashed dispatcher from thread local and intitialize this actor with the
- // contents of the old dispatcher
- val oldActor = crashedActorTl.get();
- if (oldActor != null) {
- initialize(oldActor.targetClass, oldActor.target.get, oldActor.context)
- crashedActorTl.set(null)
- }
- }
-
- override def shutdown = {
- try {
- zhutdown.foreach(_.invoke(target.get, ZERO_ITEM_OBJECT_ARRAY: _*))
- } catch { case e: InvocationTargetException => throw e.getCause
- } finally {
- AspectInitRegistry.unregister(target.get);
- }
- }
-
- override def initTransactionalState = {
- try {
- if (initTxState.isDefined && target.isDefined) initTxState.get.invoke(target.get, ZERO_ITEM_OBJECT_ARRAY: _*)
- } catch { case e: InvocationTargetException => throw e.getCause }
- }
-
- private def serializeArguments(joinPoint: JoinPoint) = {
- val args = joinPoint.getRtti.asInstanceOf[MethodRtti].getParameterValues
- var unserializable = false
- var hasMutableArgument = false
- for (arg <- args.toList) {
- if (!arg.isInstanceOf[String] &&
- !arg.isInstanceOf[Byte] &&
- !arg.isInstanceOf[Int] &&
- !arg.isInstanceOf[Long] &&
- !arg.isInstanceOf[Float] &&
- !arg.isInstanceOf[Double] &&
- !arg.isInstanceOf[Boolean] &&
- !arg.isInstanceOf[Char] &&
- !arg.isInstanceOf[java.lang.Byte] &&
- !arg.isInstanceOf[java.lang.Integer] &&
- !arg.isInstanceOf[java.lang.Long] &&
- !arg.isInstanceOf[java.lang.Float] &&
- !arg.isInstanceOf[java.lang.Double] &&
- !arg.isInstanceOf[java.lang.Boolean] &&
- !arg.isInstanceOf[java.lang.Character]) {
- hasMutableArgument = true
- }
- if (arg.getClass.getName.contains(ActiveObject.AW_PROXY_PREFIX)) unserializable = true
- }
- if (!unserializable && hasMutableArgument) {
- val copyOfArgs = Serializer.Java.deepClone(args)
- joinPoint.getRtti.asInstanceOf[MethodRtti].setParameterValues(copyOfArgs.asInstanceOf[Array[AnyRef]])
- }
- }
-}
diff --git a/akka-core/src/main/scala/actor/Actor.scala b/akka-core/src/main/scala/actor/Actor.scala
index 76adf9c729..4e22e5db0b 100644
--- a/akka-core/src/main/scala/actor/Actor.scala
+++ b/akka-core/src/main/scala/actor/Actor.scala
@@ -9,10 +9,15 @@ import se.scalablesolutions.akka.config.Config._
import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.serialization.Serializer
import se.scalablesolutions.akka.util.Helpers.{narrow, narrowSilently}
-import se.scalablesolutions.akka.util.Logging
+import se.scalablesolutions.akka.util.{Logging, Duration}
+import se.scalablesolutions.akka.AkkaException
import com.google.protobuf.Message
+
import java.util.concurrent.TimeUnit
+import java.net.InetSocketAddress
+
+import scala.reflect.BeanProperty
/**
* Implements the Transactor abstraction. E.g. a transactional actor.
@@ -32,29 +37,42 @@ trait Transactor extends Actor {
*
* @author Jonas Bonér
*/
-abstract class RemoteActor(hostname: String, port: Int) extends Actor {
- self.makeRemote(hostname, port)
+abstract class RemoteActor(address: InetSocketAddress) extends Actor {
+ def this(hostname: String, port: Int) = this(new InetSocketAddress(hostname, port))
+ self.makeRemote(address)
}
/**
* Life-cycle messages for the Actors
*/
@serializable sealed trait LifeCycleMessage
+
case class HotSwap(code: Option[Actor.Receive]) extends LifeCycleMessage
+
case class Restart(reason: Throwable) extends LifeCycleMessage
+
case class Exit(dead: ActorRef, killer: Throwable) extends LifeCycleMessage
+
case class Link(child: ActorRef) extends LifeCycleMessage
+
case class Unlink(child: ActorRef) extends LifeCycleMessage
+
case class UnlinkAndStop(child: ActorRef) extends LifeCycleMessage
+
case object ReceiveTimeout extends LifeCycleMessage
+
case class MaximumNumberOfRestartsWithinTimeRangeReached(
- victim: ActorRef, maxNrOfRetries: Int, withinTimeRange: Int, lastExceptionCausingRestart: Throwable) extends LifeCycleMessage
+ @BeanProperty val victim: ActorRef,
+ @BeanProperty val maxNrOfRetries: Int,
+ @BeanProperty val withinTimeRange: Int,
+ @BeanProperty val lastExceptionCausingRestart: Throwable) extends LifeCycleMessage
// Exceptions for Actors
-class ActorStartException private[akka](message: String) extends RuntimeException(message)
-class IllegalActorStateException private[akka](message: String) extends RuntimeException(message)
-class ActorKilledException private[akka](message: String) extends RuntimeException(message)
-class ActorInitializationException private[akka](message: String) extends RuntimeException(message)
+class ActorStartException private[akka](message: String) extends AkkaException(message)
+class IllegalActorStateException private[akka](message: String) extends AkkaException(message)
+class ActorKilledException private[akka](message: String) extends AkkaException(message)
+class ActorInitializationException private[akka](message: String) extends AkkaException(message)
+class ActorTimeoutException private[akka](message: String) extends AkkaException(message)
/**
* Actor factory module with factory methods for creating various kinds of Actors.
@@ -62,7 +80,7 @@ class ActorInitializationException private[akka](message: String) extends Runtim
* @author Jonas Bonér
*/
object Actor extends Logging {
- val TIMEOUT = config.getInt("akka.actor.timeout", 5000)
+ val TIMEOUT = Duration(config.getInt("akka.actor.timeout", 5), TIME_UNIT).toMillis
val SERIALIZE_MESSAGES = config.getBool("akka.actor.serialize-messages", false)
/**
@@ -74,7 +92,7 @@ object Actor extends Logging {
private[actor] val actorRefInCreation = new scala.util.DynamicVariable[Option[ActorRef]](None)
/**
- * Creates a Actor.actorOf out of the Actor with type T.
+ * Creates an ActorRef out of the Actor with type T.
*
* import Actor._
* val actor = actorOf[MyActor]
@@ -87,10 +105,27 @@ object Actor extends Logging {
* val actor = actorOf[MyActor].start
*
*/
- def actorOf[T <: Actor : Manifest]: ActorRef = new LocalActorRef(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
+ def actorOf[T <: Actor : Manifest]: ActorRef = actorOf(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
/**
- * Creates a Actor.actorOf out of the Actor. Allows you to pass in a factory function
+ * Creates an ActorRef out of the Actor with type T.
+ * + * import Actor._ + * val actor = actorOf[MyActor] + * actor.start + * actor ! message + * actor.stop + *+ * You can create and start the actor in one statement like this: + *
+ * val actor = actorOf[MyActor].start + *+ */ + def actorOf(clazz: Class[_ <: Actor]): ActorRef = new LocalActorRef(clazz) + + + /** + * 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. * @@ -221,7 +256,7 @@ object Actor extends Logging { case object Spawn actorOf(new Actor() { def receive = { - case Spawn => body; self.stop + case Spawn => try { body } finally { self.stop } } }).start ! Spawn } @@ -292,15 +327,14 @@ trait Actor extends Logging { type Receive = Actor.Receive /* - * Option[ActorRef] representation of the 'self' ActorRef reference. - * - * Mainly for internal use, functions as the implicit sender references when invoking - * one of the message send functions ('!', '!!' and '!!!'). - */ - @transient implicit val optionSelf: Option[ActorRef] = { - val ref = Actor.actorRefInCreation.value - Actor.actorRefInCreation.value = None - if (ref.isEmpty) throw new ActorInitializationException( + * Some[ActorRef] representation of the 'self' ActorRef reference. + * + * Mainly for internal use, functions as the implicit sender references when invoking + * the 'forward' function. + */ + @transient implicit val someSelf: Some[ActorRef] = { + val optRef = Actor.actorRefInCreation.value + if (optRef.isEmpty) throw new ActorInitializationException( "ActorRef for instance of actor [" + getClass.getName + "] is not in scope." + "\n\tYou can not create an instance of an actor explicitly using 'new MyActor'." + "\n\tYou have to use one of the factory methods in the 'Actor' object to create a new actor." + @@ -308,16 +342,19 @@ trait Actor extends Logging { "\n\t\t'val actor = Actor.actorOf[MyActor]', or" + "\n\t\t'val actor = Actor.actorOf(new MyActor(..))', or" + "\n\t\t'val actor = Actor.actor { case msg => .. } }'") - else ref + + val ref = optRef.asInstanceOf[Some[ActorRef]].get + ref.id = getClass.getName //FIXME: Is this needed? + optRef.asInstanceOf[Some[ActorRef]] } - /* - * Some[ActorRef] representation of the 'self' ActorRef reference. + /* + * Option[ActorRef] representation of the 'self' ActorRef reference. * * Mainly for internal use, functions as the implicit sender references when invoking - * the 'forward' function. + * one of the message send functions ('!', '!!' and '!!!'). */ - @transient implicit val someSelf: Some[ActorRef] = optionSelf.asInstanceOf[Some[ActorRef]] + implicit def optionSelf: Option[ActorRef] = someSelf /** * The 'self' field holds the ActorRef for this actor. @@ -346,11 +383,7 @@ trait Actor extends Logging { * self.stop(..) * */ - @transient val self: ActorRef = { - val zelf = optionSelf.get - zelf.id = getClass.getName - zelf - } + @transient val self: ScalaActorRef = someSelf.get /** * User overridable callback/setting. @@ -413,26 +446,45 @@ trait Actor extends Logging { /** * Is the actor able to handle the message passed in as arguments? */ - def isDefinedAt(message: Any): Boolean = base.isDefinedAt(message) + def isDefinedAt(message: Any): Boolean = processingBehavior.isDefinedAt(message) + + /** One of the fundamental methods of the ActorsModel + * Actor assumes a new behavior, + * None reverts the current behavior to the original behavior + */ + def become(behavior: Option[Receive]) { + self.hotswap = behavior + self.checkReceiveTimeout // FIXME : how to reschedule receivetimeout on hotswap? + } + + /** Akka Java API + * One of the fundamental methods of the ActorsModel + * Actor assumes a new behavior, + * null reverts the current behavior to the original behavior + */ + def become(behavior: Receive): Unit = become(Option(behavior)) // ========================================= // ==== INTERNAL IMPLEMENTATION DETAILS ==== // ========================================= + + private[akka] def apply(msg: Any) = processingBehavior(msg) - private[akka] def base: Receive = try { - lifeCycles orElse (self.hotswap getOrElse receive) - } catch { - case e: NullPointerException => throw new IllegalActorStateException( - "The 'self' ActorRef reference for [" + getClass.getName + "] is NULL, error in the ActorRef initialization process.") - } - - private val lifeCycles: Receive = { - case HotSwap(code) => self.hotswap = code; self.checkReceiveTimeout // FIXME : how to reschedule receivetimeout on hotswap? - case Exit(dead, reason) => self.handleTrapExit(dead, reason) - case Link(child) => self.link(child) - case Unlink(child) => self.unlink(child) - case UnlinkAndStop(child) => self.unlink(child); child.stop - case Restart(reason) => throw reason + private lazy val processingBehavior: Receive = { + lazy val defaultBehavior = receive + val actorBehavior: Receive = { + case HotSwap(code) => become(code) + case Exit(dead, reason) => self.handleTrapExit(dead, reason) + case Link(child) => self.link(child) + case Unlink(child) => self.unlink(child) + case UnlinkAndStop(child) => self.unlink(child); child.stop + case Restart(reason) => throw reason + case msg if self.hotswap.isDefined && + self.hotswap.get.isDefinedAt(msg) => self.hotswap.get.apply(msg) + case msg if self.hotswap.isEmpty && + defaultBehavior.isDefinedAt(msg) => defaultBehavior.apply(msg) + } + actorBehavior } } diff --git a/akka-core/src/main/scala/actor/ActorRef.scala b/akka-core/src/main/scala/actor/ActorRef.scala index 82f035f311..6f818b9c50 100644 --- a/akka-core/src/main/scala/actor/ActorRef.scala +++ b/akka-core/src/main/scala/actor/ActorRef.scala @@ -13,22 +13,25 @@ import se.scalablesolutions.akka.stm.TransactionManagement._ import se.scalablesolutions.akka.stm.{TransactionManagement, TransactionSetAbortedException} import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._ import se.scalablesolutions.akka.remote.{RemoteNode, RemoteServer, RemoteClient, MessageSerializer, RemoteRequestProtocolIdFactory} -import se.scalablesolutions.akka.serialization.Serializer +import se.scalablesolutions.akka.serialization.{Serializer, BinaryString} import se.scalablesolutions.akka.util.{HashCode, Logging, UUID, ReentrantGuard} +import se.scalablesolutions.akka.AkkaException import RemoteActorSerialization._ import org.multiverse.api.ThreadLocalTransaction._ import org.multiverse.commitbarriers.CountDownCommitBarrier import org.multiverse.api.exceptions.DeadTransactionException +import org.codehaus.aspectwerkz.joinpoint.JoinPoint + import java.net.InetSocketAddress import java.util.concurrent.locks.ReentrantLock import java.util.concurrent.atomic.AtomicReference -import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import java.util.concurrent.{ScheduledFuture, ConcurrentHashMap, TimeUnit} import java.util.{Map => JMap} import java.lang.reflect.Field -import jsr166x.{Deque, ConcurrentLinkedDeque} +import scala.reflect.BeanProperty import com.google.protobuf.ByteString @@ -64,7 +67,8 @@ import com.google.protobuf.ByteString * * @author Jonas Bonér */ -trait ActorRef extends TransactionManagement { +trait ActorRef extends ActorRefShared with TransactionManagement with java.lang.Comparable[ActorRef] { + scalaRef: ScalaActorRef => // Only mutable for RemoteServer in order to maintain identity across nodes @volatile protected[akka] var _uuid = UUID.newUuid.toString @@ -72,10 +76,10 @@ trait ActorRef extends TransactionManagement { @volatile protected[this] var _isShutDown = false @volatile protected[akka] var _isBeingRestarted = false @volatile protected[akka] var _homeAddress = new InetSocketAddress(RemoteServer.HOSTNAME, RemoteServer.PORT) - @volatile protected[akka] var _timeoutActor: Option[ActorRef] = None + @volatile protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None @volatile protected[akka] var startOnCreation = false @volatile protected[akka] var registeredInRemoteNodeDuringSerialization = false - protected[this] val guard = new ReentrantGuard + protected[akka] val guard = new ReentrantGuard /** * User overridable callback/setting. @@ -87,7 +91,7 @@ trait ActorRef extends TransactionManagement { * that you can use a custom name to be able to retrieve the "correct" persisted state * upon restart, remote restart etc. */ - @volatile var id: String = _uuid + @BeanProperty @volatile var id: String = _uuid /** * User overridable callback/setting. @@ -95,7 +99,7 @@ trait ActorRef extends TransactionManagement { * Defines the default timeout for '!!' and '!!!' invocations, * e.g. the timeout for the future returned by the call to '!!' and '!!!'. */ - @volatile var timeout: Long = Actor.TIMEOUT + @BeanProperty @volatile var timeout: Long = Actor.TIMEOUT /** * User overridable callback/setting. @@ -106,55 +110,60 @@ trait ActorRef extends TransactionManagement { @volatile var receiveTimeout: Option[Long] = None /** - * User overridable callback/setting. - * - * - * Set trapExit to the list of exception classes that the actor should be able to trap + * Akka Java API + * Defines the default timeout for an initial receive invocation. + * When specified, the receive function should be able to handle a 'ReceiveTimeout' message. + */ + def setReceiveTimeout(timeout: Long) = this.receiveTimeout = Some(timeout) + def getReceiveTimeout(): Option[Long] = receiveTimeout + + /** + * Akka Java API + * Set 'trapExit' to the list of exception classes that the actor should be able to trap * from the actor it is supervising. When the supervising actor throws these exceptions * then they will trigger a restart. * * - * Trap no exceptions: - *
- * trapExit = Nil - *- * * Trap all exceptions: *
- * trapExit = List(classOf[Throwable])
+ * getContext().setTrapExit(new Class[]{Throwable.class});
*
*
* Trap specific exceptions only:
*
- * trapExit = List(classOf[MyApplicationException], classOf[MyApplicationError])
+ * getContext().setTrapExit(new Class[]{MyApplicationException.class, MyApplicationError.class});
*
*/
- @volatile var trapExit: List[Class[_ <: Throwable]] = Nil
+ def setTrapExit(exceptions: Array[Class[_ <: Throwable]]) = trapExit = exceptions.toList
+ def getTrapExit(): Array[Class[_ <: Throwable]] = trapExit.toArray
/**
- * User overridable callback/setting.
- *
- * If 'trapExit' is set for the actor to act as supervisor, then a faultHandler must be defined.
+ * Akka Java API
+ * If 'trapExit' is set for the actor to act as supervisor, then a 'faultHandler' must be defined.
*
* Can be one of:
*
- * faultHandler = Some(AllForOneStrategy(maxNrOfRetries, withinTimeRange))
+ * getContext().setFaultHandler(new AllForOneStrategy(maxNrOfRetries, withinTimeRange));
*
* Or:
*
- * faultHandler = Some(OneForOneStrategy(maxNrOfRetries, withinTimeRange))
+ * getContext().setFaultHandler(new OneForOneStrategy(maxNrOfRetries, withinTimeRange));
*
*/
- @volatile var faultHandler: Option[FaultHandlingStrategy] = None
+ def setFaultHandler(handler: FaultHandlingStrategy) = this.faultHandler = Some(handler)
+ def getFaultHandler(): Option[FaultHandlingStrategy] = faultHandler
/**
- * User overridable callback/setting.
- *
* Defines the life-cycle for a supervised actor.
*/
- @volatile var lifeCycle: Option[LifeCycle] = None
+ def setLifeCycle(lifeCycle: LifeCycle) = this.lifeCycle = Some(lifeCycle)
+ def getLifeCycle(): Option[LifeCycle] = lifeCycle
+
+
+ @volatile private[akka] var _dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher
/**
+ * Akka Java API
* The default dispatcher is the Dispatchers.globalExecutorBasedEventDrivenDispatcher.
* This means that all actors will share the same event-driven executor based dispatcher.
*
@@ -165,7 +174,9 @@ trait ActorRef extends TransactionManagement {
* The default is also that all actors that are created and spawned from within this actor
* is sharing the same dispatcher as its creator.
*/
- @volatile private[akka] var _dispatcher: MessageDispatcher = Dispatchers.globalExecutorBasedEventDrivenDispatcher
+ def setDispatcher(dispatcher: MessageDispatcher) = this.dispatcher = dispatcher
+ def getDispatcher(): MessageDispatcher = dispatcher
+
/**
* Holds the hot swapped partial function.
@@ -204,32 +215,30 @@ trait ActorRef extends TransactionManagement {
protected[akka] def currentMessage_=(msg: Option[MessageInvocation]) = guard.withGuard { _currentMessage = msg }
protected[akka] def currentMessage = guard.withGuard { _currentMessage }
+ /**
+ * Comparison only takes uuid into account.
+ */
+ def compareTo(other: ActorRef) = this.uuid.compareTo(other.uuid)
+
/**
* Returns the uuid for the actor.
*/
+ def getUuid() = _uuid
def uuid = _uuid
/**
+ * Akka Java API
* The reference sender Actor of the last received message.
* Is defined if the message was sent from another Actor, else None.
*/
- def sender: Option[ActorRef] = {
- // Five lines of map-performance-avoidance, could be just: currentMessage map { _.sender }
- val msg = currentMessage
- if(msg.isEmpty) None
- else msg.get.sender
- }
+ def getSender(): Option[ActorRef] = sender
/**
+ * Akka Java API
* The reference sender future of the last received message.
* Is defined if the message was sent with sent with '!!' or '!!!', else None.
*/
- def senderFuture: Option[CompletableFuture[Any]] = {
- // Five lines of map-performance-avoidance, could be just: currentMessage map { _.senderFuture }
- val msg = currentMessage
- if(msg.isEmpty) None
- else msg.get.senderFuture
- }
+ def getSenderFuture(): Option[CompletableFuture[Any]] = senderFuture
/**
* Is the actor being restarted?
@@ -257,125 +266,144 @@ trait ActorRef extends TransactionManagement {
protected[akka] def uuid_=(uid: String) = _uuid = uid
/**
+ * Akka Java API
* Sends a one-way asynchronous message. E.g. fire-and-forget semantics.
*
- *
- * If invoked from within an actor then the actor reference is implicitly passed on as the implicit 'sender' argument.
- *
- *
- * This actor 'sender' reference is then available in the receiving actor in the 'sender' member variable,
- * if invoked from within an Actor. If not then no sender is available.
*
- * actor ! message
+ * actor.sendOneWay(message);
*
*
*/
- def !(message: Any)(implicit sender: Option[ActorRef] = None) = {
- if (isRunning) postMessageToMailbox(message, sender)
- else throw new ActorInitializationException(
- "Actor has not been started, you need to invoke 'actor.start' before using it")
- }
+ def sendOneWay(message: AnyRef): Unit = sendOneWay(message,null)
/**
- * Sends a message asynchronously and waits on a future for a reply message.
+ * Akka Java API
+ * Sends a one-way asynchronous message. E.g. fire-and-forget semantics.
*
- * It waits on the reply either until it receives it (in the form of Some(replyMessage))
- * or until the timeout expires (which will return None). E.g. send-and-receive-eventually semantics.
+ * Allows you to pass along the sender of the messag.
+ *
+ * + * actor.sendOneWay(message, context); + *+ * + */ + def sendOneWay(message: AnyRef, sender: ActorRef): Unit = this.!(message)(Option(sender)) + + /** + * Akka Java API + * @see sendRequestReply(message: AnyRef, timeout: Long, sender: ActorRef) + * Uses the defualt timeout of the Actor (setTimeout()) and omits the sender reference + */ + def sendRequestReply(message: AnyRef): AnyRef = sendRequestReply(message,timeout,null) + + /** + * Akka Java API + * @see sendRequestReply(message: AnyRef, timeout: Long, sender: ActorRef) + * Uses the defualt timeout of the Actor (setTimeout()) + */ + def sendRequestReply(message: AnyRef, sender: ActorRef): AnyRef = sendRequestReply(message,timeout,sender) + + /** + * Akka Java API + * Sends a message asynchronously and waits on a future for a reply message under the hood. + * + * It waits on the reply either until it receives it or until the timeout expires + * (which will throw an ActorTimeoutException). E.g. send-and-receive-eventually semantics. * * NOTE: - * Use this method with care. In most cases it is better to use '!' together with the 'sender' member field to + * Use this method with care. In most cases it is better to use 'sendOneWay' together with 'getContext().getSender()' to * implement request/response message exchanges. - * If you are sending messages using
!! then you have to use self.reply(..)
+ *
+ * If you are sending messages using sendRequestReply then you have to use getContext().reply(..)
* to send a reply message to the original sender. If not then the sender will block until the timeout expires.
*/
- def !!(message: Any, timeout: Long = this.timeout)(implicit sender: Option[ActorRef] = None): Option[Any] = {
- if (isRunning) {
- val future = postMessageToMailboxAndCreateFutureResultWithTimeout[Any](message, timeout, sender, None)
- val isActiveObject = message.isInstanceOf[Invocation]
- if (isActiveObject && message.asInstanceOf[Invocation].isVoid) {
- future.asInstanceOf[CompletableFuture[Option[_]]].completeWithResult(None)
- }
- try {
- future.await
- } catch {
- case e: FutureTimeoutException =>
- if (isActiveObject) throw e
- else None
- }
- if (future.exception.isDefined) throw future.exception.get._2
- else future.result
- } else throw new ActorInitializationException(
- "Actor has not been started, you need to invoke 'actor.start' before using it")
+ def sendRequestReply(message: AnyRef, timeout: Long, sender: ActorRef): AnyRef = {
+ !!(message,timeout)(Option(sender)).getOrElse(throw new ActorTimeoutException(
+ "Message [" + message +
+ "]\n\tsent to [" + actorClassName +
+ "]\n\tfrom [" + (if(sender ne null) sender.actorClassName else "nowhere") +
+ "]\n\twith timeout [" + timeout +
+ "]\n\ttimed out."))
+ .asInstanceOf[AnyRef]
}
/**
+ * Akka Java API
+ * @see sendRequestReplyFuture(message: AnyRef, sender: ActorRef): Future[_]
+ * Uses the Actors default timeout (setTimeout()) and omits the sender
+ */
+ def sendRequestReplyFuture(message: AnyRef): Future[_] = sendRequestReplyFuture(message,timeout,null)
+
+ /**
+ * Akka Java API
+ * @see sendRequestReplyFuture(message: AnyRef, sender: ActorRef): Future[_]
+ * Uses the Actors default timeout (setTimeout())
+ */
+ def sendRequestReplyFuture(message: AnyRef, sender: ActorRef): Future[_] = sendRequestReplyFuture(message,timeout,sender)
+
+ /**
+ * Akka Java API
* Sends a message asynchronously returns a future holding the eventual reply message.
*
* NOTE:
- * Use this method with care. In most cases it is better to use '!' together with the 'sender' member field to
+ * Use this method with care. In most cases it is better to use 'sendOneWay' together with the 'getContext().getSender()' to
* implement request/response message exchanges.
- * If you are sending messages using !!! then you have to use self.reply(..)
+ *
+ * If you are sending messages using sendRequestReplyFuture then you have to use getContext().reply(..)
* to send a reply message to the original sender. If not then the sender will block until the timeout expires.
*/
- def !!(implicit sender: Option[ActorRef] = None): Future[T] = {
- if (isRunning) postMessageToMailboxAndCreateFutureResultWithTimeout[T](message, timeout, sender, None)
- else throw new ActorInitializationException(
- "Actor has not been started, you need to invoke 'actor.start' before using it")
- }
+ def sendRequestReplyFuture(message: AnyRef, timeout: Long, sender: ActorRef): Future[_] = !!!(message,timeout)(Option(sender))
/**
- * Forwards the message and passes the original sender actor as the sender.
- *
- * Works with '!', '!!' and '!!!'.
+ * Akka Java API
+ * Forwards the message specified to this actor and preserves the original sender of the message
*/
- def forward(message: Any)(implicit sender: Some[ActorRef]) = {
- if (isRunning) {
- if (sender.get.senderFuture.isDefined) postMessageToMailboxAndCreateFutureResultWithTimeout(
- message, timeout, sender.get.sender, sender.get.senderFuture)
- else if (sender.get.sender.isDefined) postMessageToMailbox(message, Some(sender.get.sender.get))
- else throw new IllegalActorStateException("Can't forward message when initial sender is not an actor")
- } else throw new ActorInitializationException("Actor has not been started, you need to invoke 'actor.start' before using it")
- }
+ def forward(message: AnyRef, sender: ActorRef): Unit =
+ if (sender eq null) throw new IllegalArgumentException("The 'sender' argument to 'forward' can't be null")
+ else forward(message)(Some(sender))
+
/**
- * Use self.reply(..) to reply with a message to the original sender of the message currently
+ * Akka Java API
+ * Use getContext().replyUnsafe(..) to reply with a message to the original sender of the message currently
* being processed.
*
* Throws an IllegalStateException if unable to determine what to reply to.
*/
- def reply(message: Any) = if(!reply_?(message)) throw new IllegalActorStateException(
- "\n\tNo sender in scope, can't reply. " +
- "\n\tYou have probably: " +
- "\n\t\t1. Sent a message to an Actor from an instance that is NOT an Actor." +
- "\n\t\t2. Invoked a method on an Active Object from an instance NOT an Active Object." +
- "\n\tElse you might want to use 'reply_?' which returns Boolean(true) if succes and Boolean(false) if no sender in scope")
+ def replyUnsafe(message: AnyRef) = reply(message)
/**
- * Use reply_?(..) to reply with a message to the original sender of the message currently
+ * Akka Java API
+ * Use getContext().replySafe(..) to reply with a message to the original sender of the message currently
* being processed.
*
* Returns true if reply was sent, and false if unable to determine what to reply to.
*/
- def reply_?(message: Any): Boolean = {
- if (senderFuture.isDefined) {
- senderFuture.get completeWithResult message
- true
- } else if (sender.isDefined) {
- sender.get ! message
- true
- } else false
- }
+ def replySafe(message: AnyRef): Boolean = reply_?(message)
/**
* Returns the class for the Actor instance that is managed by the ActorRef.
*/
def actorClass: Class[_ <: Actor]
+ /**
+ * Akka Java API
+ * Returns the class for the Actor instance that is managed by the ActorRef.
+ */
+ def getActorClass(): Class[_ <: Actor] = actorClass
+
+
/**
* Returns the class name for the Actor instance that is managed by the ActorRef.
*/
def actorClassName: String
+ /**
+ * Akka Java API
+ * Returns the class name for the Actor instance that is managed by the ActorRef.
+ */
+ def getActorClassName(): String = actorClassName
+
/**
* Sets the dispatcher for this actor. Needs to be invoked before the actor is started.
*/
@@ -407,33 +435,74 @@ trait ActorRef extends TransactionManagement {
*/
def transactionConfig_=(config: TransactionConfig): Unit
+ /**
+ * Akka Java API
+ * Sets the transaction configuration for this actor. Needs to be invoked before the actor is started.
+ */
+ def setTransactionConfig(config: TransactionConfig): Unit = transactionConfig = config
+
+
/**
* Get the transaction configuration for this actor.
*/
def transactionConfig: TransactionConfig
+ /**
+ * Akka Java API
+ * Get the transaction configuration for this actor.
+ */
+ def getTransactionConfig(): TransactionConfig = transactionConfig
+
+
/**
* Returns the home address and port for this actor.
*/
def homeAddress: InetSocketAddress = _homeAddress
-
+
+ /**
+ * Akka Java API
+ * Returns the home address and port for this actor.
+ */
+ def getHomeAddress(): InetSocketAddress = homeAddress
+
/**
* Set the home address and port for this actor.
*/
def homeAddress_=(hostnameAndPort: Tuple2[String, Int]): Unit =
homeAddress_=(new InetSocketAddress(hostnameAndPort._1, hostnameAndPort._2))
-
+
+ /**
+ * Akka Java API
+ * Set the home address and port for this actor.
+ */
+ def setHomeAddress(hostname: String, port: Int): Unit = homeAddress = (hostname,port)
+
+
/**
* Set the home address and port for this actor.
*/
def homeAddress_=(address: InetSocketAddress): Unit
+ /**
+ * Akka Java API
+ * Set the home address and port for this actor.
+ */
+ def setHomeAddress(address: InetSocketAddress): Unit = homeAddress = address
+
+
/**
* Returns the remote address for the actor, if any, else None.
*/
def remoteAddress: Option[InetSocketAddress]
protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit
+ /**
+ * Akka Java API
+ * Gets the remote address for the actor, if any, else None.
+ */
+ def getRemoteAddress(): Option[InetSocketAddress] = remoteAddress
+
+
/**
* Starts up the actor and its message queue.
*/
@@ -457,29 +526,21 @@ trait ActorRef extends TransactionManagement {
* If the 'trapExit' member field has been set to at contain at least one exception class then it will
* 'trap' these exceptions and automatically restart the linked actors according to the restart strategy
* defined by the 'faultHandler'.
- *
- * To be invoked from within the actor itself.
*/
def link(actorRef: ActorRef): Unit
/**
* Unlink the actor.
- *
- * To be invoked from within the actor itself.
*/
def unlink(actorRef: ActorRef): Unit
/**
* Atomically start and link an actor.
- *
- * To be invoked from within the actor itself.
*/
def startLink(actorRef: ActorRef): Unit
/**
* Atomically start, link and make an actor remote.
- *
- * To be invoked from within the actor itself.
*/
def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit
@@ -488,33 +549,39 @@ trait ActorRef extends TransactionManagement {
*
* To be invoked from within the actor itself.
*/
- def spawn[T <: Actor : Manifest]: ActorRef
+ def spawn(clazz: Class[_ <: Actor]): ActorRef
/**
- * Atomically create (from actor class), start and make an actor remote.
+ * Atomically create (from actor class), make it remote and start an actor.
*
* To be invoked from within the actor itself.
*/
- def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef
/**
- * Atomically create (from actor class), start and link an actor.
+ * Atomically create (from actor class), link and start an actor.
*
* To be invoked from within the actor itself.
*/
- def spawnLink[T <: Actor: Manifest]: ActorRef
+ def spawnLink(clazz: Class[_ <: Actor]): ActorRef
- /**
- * Atomically create (from actor class), start, link and make an actor remote.
+ /**
+ * Atomically create (from actor class), make it remote, link and start an actor.
*
* To be invoked from within the actor itself.
*/
- def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef
/**
* Returns the mailbox size.
*/
- def mailboxSize: Int
+ def mailboxSize = dispatcher.mailboxSize(this)
+
+ /**
+ * Akka Java API
+ * Returns the mailbox size.
+ */
+ def getMailboxSize(): Int = mailboxSize
/**
* Returns the supervisor, if there is one.
@@ -522,9 +589,10 @@ trait ActorRef extends TransactionManagement {
def supervisor: Option[ActorRef]
/**
- * Shuts down and removes all linked actors.
+ * Akka Java API
+ * Returns the supervisor, if there is one.
*/
- def shutdownLinkedActors(): Unit
+ def getSupervisor(): ActorRef = supervisor getOrElse null
protected[akka] def invoke(messageHandle: MessageInvocation): Unit
@@ -536,14 +604,15 @@ trait ActorRef extends TransactionManagement {
senderOption: Option[ActorRef],
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T]
- protected[this] def actorInstance: AtomicReference[Actor]
+ protected[akka] def actorInstance: AtomicReference[Actor]
protected[akka] def actor: Actor = actorInstance.get
protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit
- protected[akka] def mailbox: Deque[MessageInvocation]
-
+ protected[akka] def mailbox: AnyRef
+ protected[akka] def mailbox_=(value: AnyRef): AnyRef
+
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit
@@ -570,14 +639,16 @@ trait ActorRef extends TransactionManagement {
cancelReceiveTimeout
receiveTimeout.foreach { time =>
log.debug("Scheduling timeout for %s", this)
- _timeoutActor = Some(Scheduler.scheduleOnce(this, ReceiveTimeout, time, TimeUnit.MILLISECONDS))
+ _futureTimeout = Some(Scheduler.scheduleOnce(this, ReceiveTimeout, time, TimeUnit.MILLISECONDS))
}
}
- protected[akka] def cancelReceiveTimeout = _timeoutActor.foreach { timeoutActor =>
- if (timeoutActor.isRunning) Scheduler.unschedule(timeoutActor)
- _timeoutActor = None
- log.debug("Timeout canceled for %s", this)
+ protected[akka] def cancelReceiveTimeout = {
+ if(_futureTimeout.isDefined) {
+ _futureTimeout.get.cancel(true)
+ _futureTimeout = None
+ log.debug("Timeout canceled for %s", this)
+ }
}
}
@@ -586,9 +657,9 @@ trait ActorRef extends TransactionManagement {
*
* @author Jonas Bonér
*/
-sealed class LocalActorRef private[akka](
+class LocalActorRef private[akka](
private[this] var actorFactory: Either[Option[Class[_ <: Actor]], Option[() => Actor]] = Left(None))
- extends ActorRef {
+ extends ActorRef with ScalaActorRef {
@volatile private[akka] var _remoteAddress: Option[InetSocketAddress] = None // only mutable to maintain identity across nodes
@volatile private[akka] var _linkedActors: Option[ConcurrentHashMap[String, ActorRef]] = None
@@ -599,9 +670,9 @@ sealed class LocalActorRef private[akka](
@volatile private var loader: Option[ClassLoader] = None
@volatile private var maxNrOfRetriesCount: Int = 0
@volatile private var restartsWithinTimeRangeTimestamp: Long = 0L
-
- protected[akka] val _mailbox: Deque[MessageInvocation] = new ConcurrentLinkedDeque[MessageInvocation]
- protected[this] val actorInstance = guard.withGuard { new AtomicReference[Actor](newActor) }
+ @volatile private var _mailbox: AnyRef = _
+
+ protected[akka] val actorInstance = guard.withGuard { new AtomicReference[Actor](newActor) }
// Needed to be able to null out the 'val self: ActorRef' member variables to make the Actor
// instance elegible for garbage collection
@@ -649,7 +720,6 @@ sealed class LocalActorRef private[akka](
hotswap = __hotswap
actorSelfFields._1.set(actor, this)
actorSelfFields._2.set(actor, Some(this))
- actorSelfFields._3.set(actor, Some(this))
start
__messages.foreach(message => this ! MessageSerializer.deserialize(message.getMessage))
checkReceiveTimeout
@@ -839,8 +909,8 @@ sealed class LocalActorRef private[akka](
*
* To be invoked from within the actor itself.
*/
- def spawn[T <: Actor : Manifest]: ActorRef = guard.withGuard {
- val actorRef = spawnButDoNotStart[T]
+ def spawn(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard {
+ val actorRef = spawnButDoNotStart(clazz)
actorRef.start
actorRef
}
@@ -850,8 +920,8 @@ sealed class LocalActorRef private[akka](
*
* To be invoked from within the actor itself.
*/
- def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef = guard.withGuard {
- val actor = spawnButDoNotStart[T]
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = guard.withGuard {
+ val actor = spawnButDoNotStart(clazz)
actor.makeRemote(hostname, port)
actor.start
actor
@@ -862,8 +932,8 @@ sealed class LocalActorRef private[akka](
*
* To be invoked from within the actor itself.
*/
- def spawnLink[T <: Actor: Manifest]: ActorRef = guard.withGuard {
- val actor = spawnButDoNotStart[T]
+ def spawnLink(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard {
+ val actor = spawnButDoNotStart(clazz)
try {
actor.start
} finally {
@@ -877,8 +947,8 @@ sealed class LocalActorRef private[akka](
*
* To be invoked from within the actor itself.
*/
- def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef = guard.withGuard {
- val actor = spawnButDoNotStart[T]
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = guard.withGuard {
+ val actor = spawnButDoNotStart(clazz)
try {
actor.makeRemote(hostname, port)
actor.start
@@ -890,17 +960,9 @@ sealed class LocalActorRef private[akka](
/**
* Returns the mailbox.
*/
- def mailbox: Deque[MessageInvocation] = _mailbox
+ def mailbox: AnyRef = _mailbox
- /**
- * Returns the mailbox size.
- */
- def mailboxSize: Int = _mailbox.size
-
- /**
- * Returns a copy of all the messages, put into a List[MessageInvocation].
- */
- def messagesInMailbox: List[MessageInvocation] = _mailbox.toArray.toList.asInstanceOf[List[MessageInvocation]]
+ protected[akka] def mailbox_=(value: AnyRef):AnyRef = { _mailbox = value; value }
/**
* Shuts down and removes all linked actors.
@@ -927,10 +989,7 @@ sealed class LocalActorRef private[akka](
createRemoteRequestProtocolBuilder(this, message, true, senderOption).build, None)
} else {
val invocation = new MessageInvocation(this, message, senderOption, None, transactionSet.get)
- if (dispatcher.usesActorMailbox) {
- _mailbox.add(invocation)
- invocation.send
- } else invocation.send
+ invocation.send
}
}
@@ -951,7 +1010,6 @@ sealed class LocalActorRef private[akka](
else new DefaultCompletableFuture[T](timeout)
val invocation = new MessageInvocation(
this, message, senderOption, Some(future.asInstanceOf[CompletableFuture[Any]]), transactionSet.get)
- if (dispatcher.usesActorMailbox) _mailbox.add(invocation)
invocation.send
future
}
@@ -961,7 +1019,8 @@ sealed class LocalActorRef private[akka](
* Callback for the dispatcher. This is the ingle entry point to the user Actor implementation.
*/
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = guard.withGuard {
- if (isShutdown) Actor.log.warning("Actor [%s] is shut down,\n\tignoring message [%s]", toString, messageHandle)
+ if (isShutdown)
+ Actor.log.warning("Actor [%s] is shut down,\n\tignoring message [%s]", toString, messageHandle)
else {
currentMessage = Option(messageHandle)
try {
@@ -972,14 +1031,13 @@ sealed class LocalActorRef private[akka](
throw e
} finally {
currentMessage = None //TODO: Don't reset this, we might want to resend the message
- }
+ }
}
}
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = {
if (trapExit.exists(_.isAssignableFrom(reason.getClass))) {
faultHandler match {
- // FIXME: implement support for maxNrOfRetries and withinTimeRange in RestartStrategy
case Some(AllForOneStrategy(maxNrOfRetries, withinTimeRange)) =>
restartLinkedActors(reason, maxNrOfRetries, withinTimeRange)
@@ -991,7 +1049,6 @@ sealed class LocalActorRef private[akka](
"\n\tto non-empty list of exception classes - can't proceed " + toString)
}
} else {
- if (lifeCycle.isEmpty) lifeCycle = Some(LifeCycle(Permanent)) // when passing on make sure we have a lifecycle
notifySupervisorWithMessage(Exit(this, reason)) // if 'trapExit' is not defined then pass the Exit on
}
}
@@ -999,49 +1056,41 @@ sealed class LocalActorRef private[akka](
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = {
if (maxNrOfRetriesCount == 0) restartsWithinTimeRangeTimestamp = System.currentTimeMillis // first time around
maxNrOfRetriesCount += 1
-
+
val tooManyRestarts = maxNrOfRetriesCount > maxNrOfRetries
val restartingHasExpired = (System.currentTimeMillis - restartsWithinTimeRangeTimestamp) > withinTimeRange
-
if (tooManyRestarts || restartingHasExpired) {
val notification = MaximumNumberOfRestartsWithinTimeRangeReached(this, maxNrOfRetries, withinTimeRange, reason)
Actor.log.warning(
- "Maximum number of restarts [%s] within time range [%s] reached." +
- "\n\tWill *not* restart actor [%s] anymore." +
- "\n\tLast exception causing restart was" +
- "\n\t[%s].",
+ "Maximum number of restarts [%s] within time range [%s] reached." +
+ "\n\tWill *not* restart actor [%s] anymore." +
+ "\n\tLast exception causing restart was" +
+ "\n\t[%s].",
maxNrOfRetries, withinTimeRange, this, reason)
- _supervisor.foreach { sup =>
+ _supervisor.foreach { sup =>
// can supervisor handle the notification?
- if (sup.isDefinedAt(notification)) notifySupervisorWithMessage(notification)
+ if (sup.isDefinedAt(notification)) notifySupervisorWithMessage(notification)
else Actor.log.warning(
"No message handler defined for system message [MaximumNumberOfRestartsWithinTimeRangeReached]" +
"\n\tCan't send the message to the supervisor [%s].", sup)
}
- } else {
+
+ stop
+ } else {
_isBeingRestarted = true
val failedActor = actorInstance.get
guard.withGuard {
- lifeCycle.get match {
- case LifeCycle(scope, _, _) => {
- scope match {
- case Permanent =>
- Actor.log.info("Restarting actor [%s] configured as PERMANENT.", id)
- restartLinkedActors(reason, maxNrOfRetries, withinTimeRange)
- Actor.log.debug("Restarting linked actors for actor [%s].", id)
- Actor.log.debug("Invoking 'preRestart' for failed actor instance [%s].", id)
- failedActor.preRestart(reason)
- nullOutActorRefReferencesFor(failedActor)
- val freshActor = newActor
- freshActor.init
- freshActor.initTransactionalState
- actorInstance.set(freshActor)
- Actor.log.debug("Invoking 'postRestart' for new actor instance [%s].", id)
- freshActor.postRestart(reason)
- _isBeingRestarted = false
- case Temporary => shutDownTemporaryActor(this)
- }
- }
+ lifeCycle match {
+ case Some(LifeCycle(Temporary)) => shutDownTemporaryActor(this)
+ case _ =>
+ // either permanent or none where default is permanent
+ Actor.log.info("Restarting actor [%s] configured as PERMANENT.", id)
+ Actor.log.debug("Restarting linked actors for actor [%s].", id)
+ restartLinkedActors(reason, maxNrOfRetries, withinTimeRange)
+ Actor.log.debug("Invoking 'preRestart' for failed actor instance [%s].", id)
+ if (isTypedActorDispatcher(failedActor)) restartTypedActorDispatcher(failedActor, reason)
+ else restartActor(failedActor, reason)
+ _isBeingRestarted = false
}
}
}
@@ -1049,14 +1098,10 @@ sealed class LocalActorRef private[akka](
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int) = {
linkedActorsAsList.foreach { actorRef =>
- if (actorRef.lifeCycle.isEmpty) actorRef.lifeCycle = Some(LifeCycle(Permanent))
- actorRef.lifeCycle.get match {
- case LifeCycle(scope, _, _) => {
- scope match {
- case Permanent => actorRef.restart(reason, maxNrOfRetries, withinTimeRange)
- case Temporary => shutDownTemporaryActor(actorRef)
- }
- }
+ actorRef.lifeCycle match {
+ // either permanent or none where default is permanent
+ case Some(LifeCycle(Temporary)) => shutDownTemporaryActor(actorRef)
+ case _ => actorRef.restart(reason, maxNrOfRetries, withinTimeRange)
}
}
}
@@ -1081,15 +1126,34 @@ sealed class LocalActorRef private[akka](
// ========= PRIVATE FUNCTIONS =========
- private def spawnButDoNotStart[T <: Actor: Manifest]: ActorRef = guard.withGuard {
- val actorRef = Actor.actorOf(manifest[T].erasure.asInstanceOf[Class[T]].newInstance)
+ private def isTypedActorDispatcher(a: Actor): Boolean = a.isInstanceOf[TypedActor]
+
+ private def restartTypedActorDispatcher(failedActor: Actor, reason: Throwable) = {
+ failedActor.preRestart(reason)
+ failedActor.postRestart(reason)
+ }
+
+ private def restartActor(failedActor: Actor, reason: Throwable) = {
+ failedActor.preRestart(reason)
+ nullOutActorRefReferencesFor(failedActor)
+ val freshActor = newActor
+ freshActor.init
+ freshActor.initTransactionalState
+ actorInstance.set(freshActor)
+ if (failedActor.isInstanceOf[TypedActor]) failedActor.asInstanceOf[TypedActor].swapInstanceInProxy(freshActor)
+ Actor.log.debug("Invoking 'postRestart' for new actor instance [%s].", id)
+ freshActor.postRestart(reason)
+ }
+
+ private def spawnButDoNotStart(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard {
+ val actorRef = Actor.actorOf(clazz.newInstance)
if (!dispatcher.isInstanceOf[ThreadBasedDispatcher]) actorRef.dispatcher = dispatcher
actorRef
}
private[this] def newActor: Actor = {
+ Actor.actorRefInCreation.withValue(Some(this)){
isInInitialization = true
- Actor.actorRefInCreation.value = Some(this)
val actor = actorFactory match {
case Left(Some(clazz)) =>
try {
@@ -1111,6 +1175,7 @@ sealed class LocalActorRef private[akka](
"Actor instance passed to ActorRef can not be 'null'")
isInInitialization = false
actor
+ }
}
private def joinTransaction(message: Any) = if (isTransactionSetInScope) {
@@ -1120,15 +1185,16 @@ sealed class LocalActorRef private[akka](
clearTransactionSet
createNewTransactionSet
} else oldTxSet
- Actor.log.ifTrace("Joining transaction set [" + currentTxSet +
- "];\n\tactor " + toString + "\n\twith message [" + message + "]")
+ Actor.log.trace("Joining transaction set [" + currentTxSet +
+ "];\n\tactor " + toString +
+ "\n\twith message [" + message + "]")
val mtx = ThreadLocalTransaction.getThreadLocalTransaction
if ((mtx eq null) || mtx.getStatus.isDead) currentTxSet.incParties
else currentTxSet.incParties(mtx, 1)
}
private def dispatch[T](messageHandle: MessageInvocation) = {
- Actor.log.ifTrace("Invoking actor with message:\n" + messageHandle)
+ Actor.log.trace("Invoking actor with message:\n" + messageHandle)
val message = messageHandle.message //serializeMessage(messageHandle.message)
var topLevelTransaction = false
val txSet: Option[CountDownCommitBarrier] =
@@ -1136,7 +1202,7 @@ sealed class LocalActorRef private[akka](
else {
topLevelTransaction = true // FIXME create a new internal atomic block that can wait for X seconds if top level tx
if (isTransactor) {
- Actor.log.ifTrace("Creating a new transaction set (top-level transaction)\n\tfor actor " + toString +
+ Actor.log.trace("Creating a new transaction set (top-level transaction)\n\tfor actor " + toString +
"\n\twith message " + messageHandle)
Some(createNewTransactionSet)
} else None
@@ -1148,20 +1214,20 @@ sealed class LocalActorRef private[akka](
if (isTransactor) {
val txFactory = _transactionFactory.getOrElse(DefaultGlobalTransactionFactory)
atomic(txFactory) {
- actor.base(message)
+ actor(message)
setTransactionSet(txSet) // restore transaction set to allow atomic block to do commit
}
} else {
- actor.base(message)
+ actor(message)
setTransactionSet(txSet) // restore transaction set to allow atomic block to do commit
}
} catch {
case e: DeadTransactionException =>
handleExceptionInDispatch(
- new TransactionSetAbortedException("Transaction set has been aborted by another participant"),
+ new TransactionSetAbortedException("Transaction set has been aborted by another participant"),
message, topLevelTransaction)
- case e =>
- handleExceptionInDispatch(e, message, topLevelTransaction)
+ case e: InterruptedException => {} // received message while actor is shutting down, ignore
+ case e => handleExceptionInDispatch(e, message, topLevelTransaction)
} finally {
clearTransaction
if (topLevelTransaction) clearTransactionSet
@@ -1195,39 +1261,42 @@ sealed class LocalActorRef private[akka](
}
}
- senderFuture.foreach(_.completeWithException(this, reason))
+ senderFuture.foreach(_.completeWithException(reason))
clearTransaction
if (topLevelTransaction) clearTransactionSet
- notifySupervisorWithMessage(Exit(this, reason))
+ if (supervisor.isDefined) notifySupervisorWithMessage(Exit(this, reason))
+ else {
+ lifeCycle match {
+ case Some(LifeCycle(Temporary)) => shutDownTemporaryActor(this)
+ case _ =>
+ }
+ }
}
private def notifySupervisorWithMessage(notification: LifeCycleMessage) = {
// FIXME to fix supervisor restart of remote actor for oneway calls, inject a supervisor proxy that can send notification back to client
- _supervisor.foreach { sup =>
+ _supervisor.foreach { sup =>
if (sup.isShutdown) { // if supervisor is shut down, game over for all linked actors
-// shutdownLinkedActors
-// stop
+ shutdownLinkedActors
+ stop
} else sup ! notification // else notify supervisor
}
}
-
+
private def nullOutActorRefReferencesFor(actor: Actor) = {
actorSelfFields._1.set(actor, null)
actorSelfFields._2.set(actor, null)
- actorSelfFields._3.set(actor, null)
}
- private def findActorSelfField(clazz: Class[_]): Tuple3[Field, Field, Field] = {
+ private def findActorSelfField(clazz: Class[_]): Tuple2[Field, Field] = {
try {
val selfField = clazz.getDeclaredField("self")
- val optionSelfField = clazz.getDeclaredField("optionSelf")
val someSelfField = clazz.getDeclaredField("someSelf")
selfField.setAccessible(true)
- optionSelfField.setAccessible(true)
someSelfField.setAccessible(true)
- (selfField, optionSelfField, someSelfField)
+ (selfField, someSelfField)
} catch {
case e: NoSuchFieldException =>
val parent = clazz.getSuperclass
@@ -1240,7 +1309,7 @@ sealed class LocalActorRef private[akka](
private def initializeActorInstance = {
actor.init // run actor init and initTransactionalState callbacks
actor.initTransactionalState
- Actor.log.debug("[%s] has started", toString)
+ Actor.log.trace("[%s] has started", toString)
ActorRegistry.register(this)
if (id == "N/A") id = actorClass.getName // if no name set, then use default name (class name)
clearTransactionSet // clear transaction set that might have been created if atomic block has been used within the Actor constructor body
@@ -1272,6 +1341,15 @@ sealed class LocalActorRef private[akka](
} else message
}
+/**
+ * System messages for RemoteActorRef.
+ *
+ * @author Jonas Bonér
+ */
+object RemoteActorSystemMessage {
+ val Stop = BinaryString("RemoteActorRef:stop")
+}
+
/**
* Remote ActorRef that is used when referencing the Actor on a different node than its "home" node.
* This reference is network-aware (remembers its origin) and immutable.
@@ -1279,9 +1357,10 @@ sealed class LocalActorRef private[akka](
* @author Jonas Bonér
*/
private[akka] case class RemoteActorRef private[akka] (
-// uuid: String, className: String, hostname: String, port: Int, timeOut: Long, isOnRemoteHost: Boolean) extends ActorRef {
uuuid: String, val className: String, val hostname: String, val port: Int, _timeout: Long, loader: Option[ClassLoader])
- extends ActorRef {
+ // uuid: String, className: String, hostname: String, port: Int, timeOut: Long, isOnRemoteHost: Boolean) extends ActorRef {
+ extends ActorRef with ScalaActorRef {
+
_uuid = uuuid
timeout = _timeout
@@ -1310,6 +1389,7 @@ private[akka] case class RemoteActorRef private[akka] (
def stop: Unit = {
_isRunning = false
_isShutDown = true
+ postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
}
/**
@@ -1334,14 +1414,14 @@ private[akka] case class RemoteActorRef private[akka] (
def unlink(actorRef: ActorRef): Unit = unsupported
def startLink(actorRef: ActorRef): Unit = unsupported
def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = unsupported
- def spawn[T <: Actor : Manifest]: ActorRef = unsupported
- def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef = unsupported
- def spawnLink[T <: Actor: Manifest]: ActorRef = unsupported
- def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef = unsupported
- def mailboxSize: Int = unsupported
+ def spawn(clazz: Class[_ <: Actor]): ActorRef = unsupported
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
+ def spawnLink(clazz: Class[_ <: Actor]): ActorRef = unsupported
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
def supervisor: Option[ActorRef] = unsupported
def shutdownLinkedActors: Unit = unsupported
- protected[akka] def mailbox: Deque[MessageInvocation] = unsupported
+ protected[akka] def mailbox: AnyRef = unsupported
+ protected[akka] def mailbox_=(value: AnyRef):AnyRef = unsupported
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
@@ -1350,7 +1430,259 @@ private[akka] case class RemoteActorRef private[akka] (
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported
protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = unsupported
- protected[this] def actorInstance: AtomicReference[Actor] = unsupported
-
+ protected[akka] def actorInstance: AtomicReference[Actor] = unsupported
private def unsupported = throw new UnsupportedOperationException("Not supported for RemoteActorRef")
}
+
+/**
+ * This trait represents the common (external) methods for all ActorRefs
+ * Needed because implicit conversions aren't applied when instance imports are used
+ *
+ * i.e.
+ * var self: ScalaActorRef = ...
+ * import self._
+ * //can't call ActorRef methods here unless they are declared in a common
+ * //superclass, which ActorRefShared is.
+ */
+trait ActorRefShared {
+ /**
+ * Returns the uuid for the actor.
+ */
+ def uuid: String
+
+ /**
+ * Shuts down and removes all linked actors.
+ */
+ def shutdownLinkedActors(): Unit
+}
+
+/**
+ * This trait represents the Scala Actor API
+ * There are implicit conversions in ../actor/Implicits.scala
+ * from ActorRef -> ScalaActorRef and back
+ */
+trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
+ /**
+ * Identifier for actor, does not have to be a unique one. Default is the 'uuid'.
+ *
+ * This field is used for logging, AspectRegistry.actorsFor(id), identifier for remote
+ * actor in RemoteServer etc.But also as the identifier for persistence, which means
+ * that you can use a custom name to be able to retrieve the "correct" persisted state
+ * upon restart, remote restart etc.
+ */
+ def id: String
+ def id_=(id: String):Unit
+
+ /**
+ * User overridable callback/setting.
+ *
+ * Defines the life-cycle for a supervised actor.
+ */
+ @volatile var lifeCycle: Option[LifeCycle] = None
+
+ /**
+ * User overridable callback/setting.
+ *
+ *
+ * Set trapExit to the list of exception classes that the actor should be able to trap
+ * from the actor it is supervising. When the supervising actor throws these exceptions
+ * then they will trigger a restart.
+ *
+ *
+ * Trap no exceptions:
+ * + * trapExit = Nil + *+ * + * Trap all exceptions: + *
+ * trapExit = List(classOf[Throwable]) + *+ * + * Trap specific exceptions only: + *
+ * trapExit = List(classOf[MyApplicationException], classOf[MyApplicationError]) + *+ */ + @volatile var trapExit: List[Class[_ <: Throwable]] = Nil + + + /** + * User overridable callback/setting. + * + * If 'trapExit' is set for the actor to act as supervisor, then a faultHandler must be defined. + * + * Can be one of: + *
+ * faultHandler = Some(AllForOneStrategy(maxNrOfRetries, withinTimeRange)) + *+ * Or: + *
+ * faultHandler = Some(OneForOneStrategy(maxNrOfRetries, withinTimeRange)) + *+ */ + @volatile var faultHandler: Option[FaultHandlingStrategy] = None + + + /** + * The reference sender Actor of the last received message. + * Is defined if the message was sent from another Actor, else None. + */ + def sender: Option[ActorRef] = { + // Five lines of map-performance-avoidance, could be just: currentMessage map { _.sender } + val msg = currentMessage + if(msg.isEmpty) None + else msg.get.sender + } + + /** + * The reference sender future of the last received message. + * Is defined if the message was sent with sent with '!!' or '!!!', else None. + */ + def senderFuture(): Option[CompletableFuture[Any]] = { + // Five lines of map-performance-avoidance, could be just: currentMessage map { _.senderFuture } + val msg = currentMessage + if(msg.isEmpty) None + else msg.get.senderFuture + } + + + /** + * Sends a one-way asynchronous message. E.g. fire-and-forget semantics. + * + * + * If invoked from within an actor then the actor reference is implicitly passed on as the implicit 'sender' argument. + * + * + * This actor 'sender' reference is then available in the receiving actor in the 'sender' member variable, + * if invoked from within an Actor. If not then no sender is available. + *
+ * actor ! message + *+ * + */ + def !(message: Any)(implicit sender: Option[ActorRef] = None): Unit = { + if (isRunning) postMessageToMailbox(message, sender) + else throw new ActorInitializationException( + "Actor has not been started, you need to invoke 'actor.start' before using it") + } + + /** + * Sends a message asynchronously and waits on a future for a reply message. + * + * It waits on the reply either until it receives it (in the form of
Some(replyMessage))
+ * or until the timeout expires (which will return None). E.g. send-and-receive-eventually semantics.
+ *
+ * NOTE:
+ * Use this method with care. In most cases it is better to use '!' together with the 'sender' member field to
+ * implement request/response message exchanges.
+ * If you are sending messages using !! then you have to use self.reply(..)
+ * to send a reply message to the original sender. If not then the sender will block until the timeout expires.
+ */
+ def !!(message: Any, timeout: Long = this.timeout)(implicit sender: Option[ActorRef] = None): Option[Any] = {
+ if (isRunning) {
+ val future = postMessageToMailboxAndCreateFutureResultWithTimeout[Any](message, timeout, sender, None)
+ val isTypedActor = message.isInstanceOf[JoinPoint]
+ if (isTypedActor && TypedActor.isOneWay(message.asInstanceOf[JoinPoint])) {
+ future.asInstanceOf[CompletableFuture[Option[_]]].completeWithResult(None)
+ }
+ try {
+ future.await
+ } catch {
+ case e: FutureTimeoutException =>
+ if (isTypedActor) throw e
+ else None
+ }
+ if (future.exception.isDefined) throw future.exception.get
+ else future.result
+ } else throw new ActorInitializationException(
+ "Actor has not been started, you need to invoke 'actor.start' before using it")
+ }
+
+
+ /**
+ * Sends a message asynchronously returns a future holding the eventual reply message.
+ *
+ * NOTE:
+ * Use this method with care. In most cases it is better to use '!' together with the 'sender' member field to
+ * implement request/response message exchanges.
+ * If you are sending messages using !!! then you have to use self.reply(..)
+ * to send a reply message to the original sender. If not then the sender will block until the timeout expires.
+ */
+ def !!(implicit sender: Option[ActorRef] = None): Future[T] = {
+ if (isRunning) postMessageToMailboxAndCreateFutureResultWithTimeout[T](message, timeout, sender, None)
+ else throw new ActorInitializationException(
+ "Actor has not been started, you need to invoke 'actor.start' before using it")
+ }
+
+ /**
+ * Forwards the message and passes the original sender actor as the sender.
+ *
+ * Works with '!', '!!' and '!!!'.
+ */
+ def forward(message: Any)(implicit sender: Some[ActorRef]) = {
+ if (isRunning) {
+ if (sender.get.senderFuture.isDefined) postMessageToMailboxAndCreateFutureResultWithTimeout(
+ message, timeout, sender.get.sender, sender.get.senderFuture)
+ else if (sender.get.sender.isDefined) postMessageToMailbox(message, Some(sender.get.sender.get))
+ else throw new IllegalActorStateException("Can't forward message when initial sender is not an actor")
+ } else throw new ActorInitializationException("Actor has not been started, you need to invoke 'actor.start' before using it")
+ }
+
+ /**
+ * Use self.reply(..) to reply with a message to the original sender of the message currently
+ * being processed.
+ *
+ * Throws an IllegalStateException if unable to determine what to reply to.
+ */
+ def reply(message: Any) = if(!reply_?(message)) throw new IllegalActorStateException(
+ "\n\tNo sender in scope, can't reply. " +
+ "\n\tYou have probably: " +
+ "\n\t\t1. Sent a message to an Actor from an instance that is NOT an Actor." +
+ "\n\t\t2. Invoked a method on an TypedActor from an instance NOT an TypedActor." +
+ "\n\tElse you might want to use 'reply_?' which returns Boolean(true) if succes and Boolean(false) if no sender in scope")
+
+ /**
+ * Use reply_?(..) to reply with a message to the original sender of the message currently
+ * being processed.
+ *
+ * Returns true if reply was sent, and false if unable to determine what to reply to.
+ */
+ def reply_?(message: Any): Boolean = {
+ if (senderFuture.isDefined) {
+ senderFuture.get completeWithResult message
+ true
+ } else if (sender.isDefined) {
+ sender.get ! message
+ true
+ } else false
+ }
+
+
+
+ /**
+ * Atomically create (from actor class) and start an actor.
+ */
+ def spawn[T <: Actor : Manifest]: ActorRef =
+ spawn(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
+
+ /**
+ * Atomically create (from actor class), start and make an actor remote.
+ */
+ def spawnRemote[T <: Actor: Manifest](hostname: String, port: Int): ActorRef =
+ spawnRemote(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]],hostname,port)
+
+
+ /**
+ * Atomically create (from actor class), start and link an actor.
+ */
+ def spawnLink[T <: Actor: Manifest]: ActorRef =
+ spawnLink(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
+
+
+ /**
+ * Atomically create (from actor class), start, link and make an actor remote.
+ */
+ def spawnLinkRemote[T <: Actor : Manifest](hostname: String, port: Int): ActorRef =
+ spawnLinkRemote(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]],hostname,port)
+}
diff --git a/akka-core/src/main/scala/actor/ActorRegistry.scala b/akka-core/src/main/scala/actor/ActorRegistry.scala
index 88113a30a0..51de155723 100644
--- a/akka-core/src/main/scala/actor/ActorRegistry.scala
+++ b/akka-core/src/main/scala/actor/ActorRegistry.scala
@@ -29,19 +29,15 @@ case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent
* @author Jonas Bonér
*/
object ActorRegistry extends ListenerManagement {
-
- private val refComparator = new java.util.Comparator[ActorRef]{
- def compare(a: ActorRef,b: ActorRef) = a.uuid.compareTo(b.uuid)
- }
-
private val actorsByUUID = new ConcurrentHashMap[String, ActorRef]
private val actorsById = new ConcurrentHashMap[String, JSet[ActorRef]]
- private val actorsByClassName = new ConcurrentHashMap[String, JSet[ActorRef]]
+
+ private val Naught = Array[ActorRef]() //Nil for Arrays
/**
* Returns all actors in the system.
*/
- def actors: List[ActorRef] = filter(_ => true)
+ def actors: Array[ActorRef] = filter(_ => true)
/**
* Invokes a function for all actors.
@@ -51,16 +47,30 @@ object ActorRegistry extends ListenerManagement {
while (elements.hasMoreElements) f(elements.nextElement)
}
+ /**
+ * Invokes the function on all known actors until it returns Some
+ * Returns None if the function never returns Some
+ */
+ def find[T](f: PartialFunction[ActorRef,T]) : Option[T] = {
+ val elements = actorsByUUID.elements
+ while (elements.hasMoreElements) {
+ val element = elements.nextElement
+ if(f isDefinedAt element)
+ return Some(f(element))
+ }
+ None
+ }
+
/**
* Finds all actors that are subtypes of the class passed in as the Manifest argument and supproting passed message.
*/
- def actorsFor[T <: Actor](message: Any)(implicit manifest: Manifest[T] ): List[ActorRef] =
+ def actorsFor[T <: Actor](message: Any)(implicit manifest: Manifest[T] ): Array[ActorRef] =
filter(a => manifest.erasure.isAssignableFrom(a.actor.getClass) && a.isDefinedAt(message))
/**
* Finds all actors that satisfy a predicate.
*/
- def filter(p: ActorRef => Boolean): List[ActorRef] = {
+ def filter(p: ActorRef => Boolean): Array[ActorRef] = {
val all = new ListBuffer[ActorRef]
val elements = actorsByUUID.elements
while (elements.hasMoreElements) {
@@ -69,37 +79,34 @@ object ActorRegistry extends ListenerManagement {
all += actorId
}
}
- all.toList
+ all.toArray
}
/**
* Finds all actors that are subtypes of the class passed in as the Manifest argument.
*/
- def actorsFor[T <: Actor](implicit manifest: Manifest[T]): List[ActorRef] =
- filter(a => manifest.erasure.isAssignableFrom(a.actor.getClass))
+ def actorsFor[T <: Actor](implicit manifest: Manifest[T]): Array[ActorRef] =
+ actorsFor[T](manifest.erasure.asInstanceOf[Class[T]])
/**
* Finds any actor that matches T.
*/
def actorFor[T <: Actor](implicit manifest: Manifest[T]): Option[ActorRef] =
- actorsFor[T](manifest).headOption
+ find({ case a:ActorRef if manifest.erasure.isAssignableFrom(a.actor.getClass) => a })
/**
- * Finds all actors of the exact type specified by the class passed in as the Class argument.
+ * Finds all actors of type or sub-type specified by the class passed in as the Class argument.
*/
- def actorsFor[T <: Actor](clazz: Class[T]): List[ActorRef] = {
- if (actorsByClassName.containsKey(clazz.getName)) {
- actorsByClassName.get(clazz.getName).toArray.toList.asInstanceOf[List[ActorRef]]
- } else Nil
- }
+ def actorsFor[T <: Actor](clazz: Class[T]): Array[ActorRef] =
+ filter(a => clazz.isAssignableFrom(a.actor.getClass))
/**
* Finds all actors that has a specific id.
*/
- def actorsFor(id: String): List[ActorRef] = {
+ def actorsFor(id: String): Array[ActorRef] = {
if (actorsById.containsKey(id)) {
- actorsById.get(id).toArray.toList.asInstanceOf[List[ActorRef]]
- } else Nil
+ actorsById.get(id).toArray(Naught)
+ } else Naught
}
/**
@@ -114,27 +121,22 @@ object ActorRegistry extends ListenerManagement {
* Registers an actor in the ActorRegistry.
*/
def register(actor: ActorRef) = {
- // UUID
- actorsByUUID.put(actor.uuid, actor)
-
// ID
val id = actor.id
if (id eq null) throw new IllegalActorStateException("Actor.id is null " + actor)
- if (actorsById.containsKey(id)) actorsById.get(id).add(actor)
+
+ val set = actorsById get id
+ if (set ne null) set add actor
else {
- val set = new ConcurrentSkipListSet[ActorRef](refComparator)
- set.add(actor)
- actorsById.put(id, set)
+ val newSet = new ConcurrentSkipListSet[ActorRef]
+ newSet add actor
+ val oldSet = actorsById.putIfAbsent(id,newSet)
+ // Parry for two simultaneous putIfAbsent(id,newSet)
+ if (oldSet ne null) oldSet add actor
}
- // Class name
- val className = actor.actorClassName
- if (actorsByClassName.containsKey(className)) actorsByClassName.get(className).add(actor)
- else {
- val set = new ConcurrentSkipListSet[ActorRef](refComparator)
- set.add(actor)
- actorsByClassName.put(className, set)
- }
+ // UUID
+ actorsByUUID.put(actor.uuid, actor)
// notify listeners
foreachListener(_ ! ActorRegistered(actor))
@@ -146,11 +148,10 @@ object ActorRegistry extends ListenerManagement {
def unregister(actor: ActorRef) = {
actorsByUUID remove actor.uuid
- val id = actor.id
- if (actorsById.containsKey(id)) actorsById.get(id).remove(actor)
+ val set = actorsById get actor.id
+ if (set ne null) set remove actor
- val className = actor.actorClassName
- if (actorsByClassName.containsKey(className)) actorsByClassName.get(className).remove(actor)
+ //FIXME: safely remove set if empty, leaks memory
// notify listeners
foreachListener(_ ! ActorUnregistered(actor))
@@ -159,12 +160,11 @@ object ActorRegistry extends ListenerManagement {
/**
* Shuts down and unregisters all actors in the system.
*/
- def shutdownAll = {
+ def shutdownAll() {
log.info("Shutting down all actors in the system...")
foreach(_.stop)
actorsByUUID.clear
actorsById.clear
- actorsByClassName.clear
log.info("All actors have been shut down and unregistered from ActorRegistry")
}
}
diff --git a/akka-core/src/main/scala/actor/Agent.scala b/akka-core/src/main/scala/actor/Agent.scala
index b800c94f23..df358cdfc4 100644
--- a/akka-core/src/main/scala/actor/Agent.scala
+++ b/akka-core/src/main/scala/actor/Agent.scala
@@ -5,11 +5,12 @@
package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.stm.Ref
+import se.scalablesolutions.akka.AkkaException
import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.CountDownLatch
-class AgentException private[akka](message: String) extends RuntimeException(message)
+class AgentException private[akka](message: String) extends AkkaException(message)
/**
* The Agent class was strongly inspired by the agent principle in Clojure.
diff --git a/akka-core/src/main/scala/actor/BootableActorLoaderService.scala b/akka-core/src/main/scala/actor/BootableActorLoaderService.scala
index 22878b416a..dfb8541396 100644
--- a/akka-core/src/main/scala/actor/BootableActorLoaderService.scala
+++ b/akka-core/src/main/scala/actor/BootableActorLoaderService.scala
@@ -26,7 +26,7 @@ class AkkaDeployClassLoader(urls : List[URL], parent : ClassLoader) extends URLC
}
def listClassesInPackage(jar : URL, pkg : String) = {
- val f = new File(jar.getFile)
+ val f = new File(jar.getFile)
val jf = new JarFile(f)
try {
val es = jf.entries
@@ -84,11 +84,14 @@ trait BootableActorLoaderService extends Bootable with Logging {
}
abstract override def onLoad = {
+ applicationLoader.foreach(_ => log.info("Creating /deploy class-loader"))
+
+ super.onLoad
+
for (loader <- applicationLoader; clazz <- BOOT_CLASSES) {
log.info("Loading boot class [%s]", clazz)
loader.loadClass(clazz).newInstance
}
- super.onLoad
}
abstract override def onUnload = {
diff --git a/akka-core/src/main/scala/actor/Implicits.scala b/akka-core/src/main/scala/actor/Implicits.scala
new file mode 100644
index 0000000000..16bce4b016
--- /dev/null
+++ b/akka-core/src/main/scala/actor/Implicits.scala
@@ -0,0 +1,15 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ * class TestActorImpl extends TypedActor implements TestActor {
+ *
+ * public void hit(int count) {
+ * Pong pong = (Pong) getContext().getSender();
+ * pong.hit(count++);
+ * }
+ *
+ * public Future square(int x) {
+ * return future(x * x);
+ * }
+ *
+ * @Override
+ * public void init() {
+ * ... // optional initialization on start
+ * }
+ *
+ * @Override
+ * public void shutdown() {
+ * ... // optional cleanup on stop
+ * }
+ *
+ * ... // more life-cycle callbacks if needed
+ * }
+ *
+ * // create the ping actor
+ * TestActor actor = TypedActor.newInstance(TestActor.class, TestActorImpl.class);
+ *
+ * actor.hit(1); // use the actor
+ * actor.hit(1);
+ *
+ * // This method will return immediately when called, caller should wait on the Future for the result
+ * Future future = actor.square(10);
+ * future.await();
+ * Integer result = future.get();
+ *
+ * // stop the actor
+ * TypedActor.stop(actor);
+ *
+ *
+ * Here is an example of usage (in Scala):
+ *
+ * class TestActorImpl extends TypedActor with TestActor {
+ *
+ * def hit(count: Int) = {
+ * val pong = context.sender.asInstanceOf[Pong]
+ * pong.hit(count += 1)
+ * }
+ *
+ * def square(x: Int): Future[Integer] = future(x * x)
+ *
+ * override def init = {
+ * ... // optional initialization on start
+ * }
+ *
+ * override def shutdown = {
+ * ... // optional cleanup on stop
+ * }
+ *
+ * ... // more life-cycle callbacks if needed
+ * }
+ *
+ * // create the ping actor
+ * val ping = TypedActor.newInstance(classOf[Ping], classOf[PingImpl])
+ *
+ * ping.hit(1) // use the actor
+ * ping.hit(1)
+ *
+ * // This method will return immediately when called, caller should wait on the Future for the result
+ * val future = actor.square(10)
+ * future.await
+ * val result: Int = future.get
+ *
+ * // stop the actor
+ * TypedActor.stop(ping)
+ *
+ *
+ * @author Jonas Bonér
+ */
+abstract class TypedActor extends Actor {
+ val DELEGATE_FIELD_NAME = "DELEGATE_0".intern
+
+ @volatile private[actor] var proxy: AnyRef = _
+ @volatile private var proxyDelegate: Field = _
+
+ /**
+ * Holds RTTI (runtime type information) for the TypedActor, f.e. current 'sender'
+ * reference, the 'senderFuture' reference etc.
+ *
+ * This class does not contain static information but is updated by the runtime system
+ * at runtime.
+ *
+ * You can get a hold of the context using either the 'getContext()' or 'context'
+ * methods from the 'TypedActor' base class.
+ *
+ *
+ * Here is an example of usage (in Java):
+ *
+ * class PingImpl extends TypedActor implements Ping {
+ * public void hit(int count) {
+ * Pong pong = (Pong) getContext().getSender();
+ * pong.hit(count++);
+ * }
+ * }
+ *
+ *
+ * Here is an example of usage (in Scala):
+ *
+ * class PingImpl extends TypedActor with Ping {
+ * def hit(count: Int) = {
+ * val pong = context.sender.asInstanceOf[Pong]
+ * pong.hit(count += 1)
+ * }
+ * }
+ *
+ */
+ @BeanProperty val context: TypedActorContext = new TypedActorContext(self)
+
+ /**
+ * This method is used to resolve the Future for TypedActor methods that are defined to return a
+ * {@link se.scalablesolutions.akka.actor.dispatch.Future }.
+ *
+ * Here is an example:
+ *
+ * class MyTypedActorImpl extends TypedActor implements MyTypedActor {
+ * public Future square(int x) {
+ * return future(x * x);
+ * }
+ * }
+ *
+ * MyTypedActor actor = TypedActor.actorOf(MyTypedActor.class, MyTypedActorImpl.class);
+ *
+ * // This method will return immediately when called, caller should wait on the Future for the result
+ * Future future = actor.square(10);
+ * future.await();
+ * Integer result = future.get();
+ *
+ */
+ def future[T](value: T): Future[T] =
+ self.senderFuture
+ .map{f => f.completeWithResult(value); f }
+ .getOrElse(throw new IllegalActorStateException("No sender future in scope"))
+ .asInstanceOf[Future[T]]
+
+ def receive = {
+ case joinPoint: JoinPoint =>
+ SenderContextInfo.senderActorRef.value = self
+ SenderContextInfo.senderProxy.value = proxy
+
+ if (Actor.SERIALIZE_MESSAGES) serializeArguments(joinPoint)
+ if (TypedActor.isOneWay(joinPoint)) joinPoint.proceed
+ else self.reply(joinPoint.proceed)
+
+ case Link(proxy) => self.link(proxy)
+ case Unlink(proxy) => self.unlink(proxy)
+ case unexpected => throw new IllegalActorStateException(
+ "Unexpected message [" + unexpected + "] sent to [" + this + "]")
+ }
+
+ /**
+ * Rewrite target instance in AspectWerkz Proxy.
+ */
+ private[actor] def swapInstanceInProxy(newInstance: Actor) = proxyDelegate.set(proxy, newInstance)
+
+ private[akka] def initialize(typedActorProxy: AnyRef) = {
+ proxy = typedActorProxy
+ proxyDelegate = {
+ val field = proxy.getClass.getDeclaredField(DELEGATE_FIELD_NAME)
+ field.setAccessible(true)
+ field
+ }
+ }
+
+ private def serializeArguments(joinPoint: JoinPoint) = {
+ val args = joinPoint.getRtti.asInstanceOf[MethodRtti].getParameterValues
+ var unserializable = false
+ var hasMutableArgument = false
+ for (arg <- args.toList) {
+ if (!arg.isInstanceOf[String] &&
+ !arg.isInstanceOf[Byte] &&
+ !arg.isInstanceOf[Int] &&
+ !arg.isInstanceOf[Long] &&
+ !arg.isInstanceOf[Float] &&
+ !arg.isInstanceOf[Double] &&
+ !arg.isInstanceOf[Boolean] &&
+ !arg.isInstanceOf[Char] &&
+ !arg.isInstanceOf[java.lang.Byte] &&
+ !arg.isInstanceOf[java.lang.Integer] &&
+ !arg.isInstanceOf[java.lang.Long] &&
+ !arg.isInstanceOf[java.lang.Float] &&
+ !arg.isInstanceOf[java.lang.Double] &&
+ !arg.isInstanceOf[java.lang.Boolean] &&
+ !arg.isInstanceOf[java.lang.Character]) hasMutableArgument = true
+ if (arg.getClass.getName.contains(TypedActor.AW_PROXY_PREFIX)) unserializable = true
+ }
+ if (!unserializable && hasMutableArgument) {
+ val copyOfArgs = Serializer.Java.deepClone(args)
+ joinPoint.getRtti.asInstanceOf[MethodRtti].setParameterValues(copyOfArgs.asInstanceOf[Array[AnyRef]])
+ }
+ }
+}
+
+/**
+ * Transactional TypedActor. All messages send to this actor as sent in a transaction. If an enclosing transaction
+ * exists it will be joined, if not then a new transaction will be created.
+ *
+ * @author Jonas Bonér
+ */
+abstract class TypedTransactor extends TypedActor {
+ self.makeTransactionRequired
+}
+
+/**
+ * Holds RTTI (runtime type information) for the TypedActor, f.e. current 'sender'
+ * reference, the 'senderFuture' reference etc.
+ *
+ * This class does not contain static information but is updated by the runtime system
+ * at runtime.
+ *
+ * You can get a hold of the context using either the 'getContext()' or 'context'
+ * methods from the 'TypedActor' base class.
+ *
+ * Here is an example of usage (from Java):
+ *
+ * class PingImpl extends TypedActor implements Ping {
+ * public void hit(int count) {
+ * Pong pong = (Pong) getContext().getSender();
+ * pong.hit(count++);
+ * }
+ * }
+ *
+ *
+ * Here is an example of usage (in Scala):
+ *
+ * class PingImpl extends TypedActor with Ping {
+ * def hit(count: Int) = {
+ * val pong = context.sender.asInstanceOf[Pong]
+ * pong.hit(count += 1)
+ * }
+ * }
+ *
+ *
+ * @author Jonas Bonér
+ */
+final class TypedActorContext(private val actorRef: ActorRef) {
+ private[akka] var _sender: AnyRef = _
+
+ /**
+ * Returns the current sender reference.
+ * Scala style getter.
+ */
+ def sender: AnyRef = {
+ if (_sender eq null) throw new IllegalActorStateException("Sender reference should not be null.")
+ else _sender
+ }
+
+ /**
+ * Returns the current sender reference.
+ * Java style getter.
+ */
+ def getSender: AnyRef = {
+ if (_sender eq null) throw new IllegalActorStateException("Sender reference should not be null.")
+ else _sender
+ }
+
+ /**
+ * Returns the current sender future TypedActor reference.
+ * Scala style getter.
+ */
+ def senderFuture: Option[CompletableFuture[Any]] = actorRef.senderFuture
+
+ /**
+ * Returns the current sender future TypedActor reference.
+ * Java style getter.
+ * This method returns 'null' if the sender future is not available.
+ */
+ def getSenderFuture = senderFuture
+}
+
+/**
+ * Configuration factory for TypedActors.
+ *
+ * @author Jonas Bonér
+ */
+final class TypedActorConfiguration {
+ private[akka] var _timeout: Long = Actor.TIMEOUT
+ private[akka] var _transactionRequired = false
+ private[akka] var _host: Option[InetSocketAddress] = None
+ private[akka] var _messageDispatcher: Option[MessageDispatcher] = None
+ private[akka] var _threadBasedDispatcher: Option[Boolean] = None
+
+ def timeout = _timeout
+ def timeout(timeout: Duration) : TypedActorConfiguration = {
+ _timeout = timeout.toMillis
+ this
+ }
+
+ def makeTransactionRequired() : TypedActorConfiguration = {
+ _transactionRequired = true;
+ this
+ }
+
+ def makeRemote(hostname: String, port: Int) : TypedActorConfiguration = {
+ _host = Some(new InetSocketAddress(hostname, port))
+ this
+ }
+
+ def dispatcher(messageDispatcher: MessageDispatcher) : TypedActorConfiguration = {
+ if (_threadBasedDispatcher.isDefined) throw new IllegalArgumentException(
+ "Cannot specify both 'threadBasedDispatcher()' and 'dispatcher()'")
+ _messageDispatcher = Some(messageDispatcher)
+ this
+ }
+
+ def threadBasedDispatcher() : TypedActorConfiguration = {
+ if (_messageDispatcher.isDefined) throw new IllegalArgumentException(
+ "Cannot specify both 'threadBasedDispatcher()' and 'dispatcher()'")
+ _threadBasedDispatcher = Some(true)
+ this
+ }
+}
+
+/**
+ * Factory class for creating TypedActors out of plain POJOs and/or POJOs with interfaces.
+ *
+ * @author Jonas Bonér
+ */
+object TypedActor extends Logging {
+ import Actor.actorOf
+
+ val ZERO_ITEM_CLASS_ARRAY = Array[Class[_]]()
+ val ZERO_ITEM_OBJECT_ARRAY = Array[Object]()
+
+ val AKKA_CAMEL_ROUTING_SCHEME = "akka".intern
+ private[actor] val AW_PROXY_PREFIX = "$$ProxiedByAW".intern
+
+ def newInstance[T](intfClass: Class[T], targetClass: Class[_]): T = {
+ newInstance(intfClass, targetClass, None, Actor.TIMEOUT)
+ }
+
+ def newRemoteInstance[T](intfClass: Class[T], targetClass: Class[_], hostname: String, port: Int): T = {
+ newInstance(intfClass, targetClass, Some(new InetSocketAddress(hostname, port)), Actor.TIMEOUT)
+ }
+
+ def newInstance[T](intfClass: Class[T], targetClass: Class[_], timeout: Long = Actor.TIMEOUT): T = {
+ newInstance(intfClass, targetClass, None, timeout)
+ }
+
+ def newRemoteInstance[T](intfClass: Class[T], targetClass: Class[_], timeout: Long = Actor.TIMEOUT, hostname: String, port: Int): T = {
+ newInstance(intfClass, targetClass, Some(new InetSocketAddress(hostname, port)), timeout)
+ }
+
+ def newInstance[T](intfClass: Class[T], targetClass: Class[_], config: TypedActorConfiguration): T = {
+ val actorRef = actorOf(newTypedActor(targetClass))
+ val typedActor = actorRef.actorInstance.get.asInstanceOf[TypedActor]
+ val proxy = Proxy.newInstance(Array(intfClass), Array(typedActor), true, false)
+ typedActor.initialize(proxy)
+ if (config._messageDispatcher.isDefined) actorRef.dispatcher = config._messageDispatcher.get
+ if (config._threadBasedDispatcher.isDefined) actorRef.dispatcher = Dispatchers.newThreadBasedDispatcher(actorRef)
+ AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, None, config.timeout))
+ actorRef.start
+ proxy.asInstanceOf[T]
+ }
+
+ private[akka] def newInstance[T](intfClass: Class[T], targetClass: Class[_],
+ remoteAddress: Option[InetSocketAddress], timeout: Long): T = {
+ val actorRef = actorOf(newTypedActor(targetClass))
+ val typedActor = actorRef.actorInstance.get.asInstanceOf[TypedActor]
+ val proxy = Proxy.newInstance(Array(intfClass), Array(typedActor), true, false)
+ typedActor.initialize(proxy)
+ actorRef.timeout = timeout
+ if (remoteAddress.isDefined) actorRef.makeRemote(remoteAddress.get)
+ AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, remoteAddress, timeout))
+ actorRef.start
+ proxy.asInstanceOf[T]
+ }
+
+/*
+ // NOTE: currently not used - but keep it around
+ private[akka] def newInstance[T <: TypedActor](targetClass: Class[T],
+ remoteAddress: Option[InetSocketAddress], timeout: Long): T = {
+ val proxy = {
+ val instance = Proxy.newInstance(targetClass, true, false)
+ if (instance.isInstanceOf[TypedActor]) instance.asInstanceOf[TypedActor]
+ else throw new IllegalActorStateException("Actor [" + targetClass.getName + "] is not a sub class of 'TypedActor'")
+ }
+ val context = injectTypedActorContext(proxy)
+ actorRef.actor.asInstanceOf[Dispatcher].initialize(targetClass, proxy, proxy, context)
+ actorRef.timeout = timeout
+ if (remoteAddress.isDefined) actorRef.makeRemote(remoteAddress.get)
+ AspectInitRegistry.register(proxy, AspectInit(targetClass, proxy, actorRef, remoteAddress, timeout))
+ actorRef.start
+ proxy.asInstanceOf[T]
+ }
+*/
+
+ /**
+ * Stops the current Typed Actor.
+ */
+ def stop(proxy: AnyRef): Unit = AspectInitRegistry.unregister(proxy)
+
+ /**
+ * Get the underlying dispatcher actor for the given Typed Actor.
+ */
+ def actorFor(proxy: AnyRef): Option[ActorRef] =
+ ActorRegistry
+ .actorsFor(classOf[TypedActor])
+ .find(a => a.actor.asInstanceOf[TypedActor].proxy == proxy)
+
+ /**
+ * Links an other Typed Actor to this Typed Actor.
+ * @param supervisor the supervisor Typed Actor
+ * @param supervised the Typed Actor to link
+ */
+ def link(supervisor: AnyRef, supervised: AnyRef) = {
+ val supervisorActor = actorFor(supervisor).getOrElse(
+ throw new IllegalActorStateException("Can't link when the supervisor is not an Typed Actor"))
+ val supervisedActor = actorFor(supervised).getOrElse(
+ throw new IllegalActorStateException("Can't link when the supervised is not an Typed Actor"))
+ supervisorActor.link(supervisedActor)
+ }
+
+ /**
+ * Links an other Typed Actor to this Typed Actor and sets the fault handling for the supervisor.
+ * @param supervisor the supervisor Typed Actor
+ * @param supervised the Typed Actor to link
+ * @param handler fault handling strategy
+ * @param trapExceptions array of exceptions that should be handled by the supervisor
+ */
+ def link(supervisor: AnyRef, supervised: AnyRef,
+ handler: FaultHandlingStrategy, trapExceptions: Array[Class[_ <: Throwable]]) = {
+ val supervisorActor = actorFor(supervisor).getOrElse(
+ throw new IllegalActorStateException("Can't link when the supervisor is not an Typed Actor"))
+ val supervisedActor = actorFor(supervised).getOrElse(
+ throw new IllegalActorStateException("Can't link when the supervised is not an Typed Actor"))
+ supervisorActor.trapExit = trapExceptions.toList
+ supervisorActor.faultHandler = Some(handler)
+ supervisorActor.link(supervisedActor)
+ }
+
+ /**
+ * Unlink the supervised Typed Actor from the supervisor.
+ * @param supervisor the supervisor Typed Actor
+ * @param supervised the Typed Actor to unlink
+ */
+ def unlink(supervisor: AnyRef, supervised: AnyRef) = {
+ val supervisorActor = actorFor(supervisor).getOrElse(
+ throw new IllegalActorStateException("Can't unlink when the supervisor is not an Typed Actor"))
+ val supervisedActor = actorFor(supervised).getOrElse(
+ throw new IllegalActorStateException("Can't unlink when the supervised is not an Typed Actor"))
+ supervisorActor.unlink(supervisedActor)
+ }
+
+ /**
+ * Sets the trap exit for the given supervisor Typed Actor.
+ * @param supervisor the supervisor Typed Actor
+ * @param trapExceptions array of exceptions that should be handled by the supervisor
+ */
+ def trapExit(supervisor: AnyRef, trapExceptions: Array[Class[_ <: Throwable]]) = {
+ val supervisorActor = actorFor(supervisor).getOrElse(
+ throw new IllegalActorStateException("Can't set trap exceptions when the supervisor is not an Typed Actor"))
+ supervisorActor.trapExit = trapExceptions.toList
+ this
+ }
+
+ /**
+ * Sets the fault handling strategy for the given supervisor Typed Actor.
+ * @param supervisor the supervisor Typed Actor
+ * @param handler fault handling strategy
+ */
+ def faultHandler(supervisor: AnyRef, handler: FaultHandlingStrategy) = {
+ val supervisorActor = actorFor(supervisor).getOrElse(
+ throw new IllegalActorStateException("Can't set fault handler when the supervisor is not an Typed Actor"))
+ supervisorActor.faultHandler = Some(handler)
+ this
+ }
+
+ def isTransactional(clazz: Class[_]): Boolean = {
+ if (clazz == null) false
+ else if (clazz.isAssignableFrom(classOf[TypedTransactor])) true
+ else isTransactional(clazz.getSuperclass)
+ }
+
+ private[akka] def newTypedActor(targetClass: Class[_]): TypedActor = {
+ val instance = targetClass.newInstance
+ val typedActor =
+ if (instance.isInstanceOf[TypedActor]) instance.asInstanceOf[TypedActor]
+ else throw new IllegalArgumentException("Actor [" + targetClass.getName + "] is not a sub class of 'TypedActor'")
+ typedActor.init
+ import se.scalablesolutions.akka.stm.local.atomic
+ atomic {
+ typedActor.initTransactionalState
+ }
+ typedActor
+ }
+
+ private[akka] def isOneWay(joinPoint: JoinPoint): Boolean =
+ isOneWay(joinPoint.getRtti.asInstanceOf[MethodRtti])
+
+ private[akka] def isOneWay(methodRtti: MethodRtti): Boolean =
+ methodRtti.getMethod.getReturnType == java.lang.Void.TYPE
+
+ private[akka] def returnsFuture_?(methodRtti: MethodRtti): Boolean =
+ classOf[Future[_]].isAssignableFrom(methodRtti.getMethod.getReturnType)
+
+ private[akka] def supervise(restartStrategy: RestartStrategy, components: List[Supervise]): Supervisor =
+ Supervisor(SupervisorConfig(restartStrategy, components))
+}
+
+/**
+ * AspectWerkz Aspect that is turning POJO into TypedActor.
+ *
+ * Is deployed on a 'perInstance' basis with the pointcut 'execution(* *.*(..))',
+ * e.g. all methods on the instance.
+ *
+ * @author Jonas Bonér
+ */
+@Aspect("perInstance")
+private[akka] sealed class TypedActorAspect {
+ @volatile private var isInitialized = false
+ @volatile private var isStopped = false
+ private var interfaceClass: Class[_] = _
+ private var typedActor: TypedActor = _
+ private var actorRef: ActorRef = _
+ private var remoteAddress: Option[InetSocketAddress] = _
+ private var timeout: Long = _
+ private var uuid: String = _
+ @volatile private var instance: TypedActor = _
+
+ @Around("execution(* *.*(..))")
+ def invoke(joinPoint: JoinPoint): AnyRef = {
+ if (!isInitialized) initialize(joinPoint)
+ dispatch(joinPoint)
+ }
+
+ private def dispatch(joinPoint: JoinPoint) = {
+ if (remoteAddress.isDefined) remoteDispatch(joinPoint)
+ else localDispatch(joinPoint)
+ }
+
+ private def localDispatch(joinPoint: JoinPoint): AnyRef = {
+ val methodRtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
+ val isOneWay = TypedActor.isOneWay(methodRtti)
+ val senderActorRef = Some(SenderContextInfo.senderActorRef.value)
+ val senderProxy = Some(SenderContextInfo.senderProxy.value)
+
+ typedActor.context._sender = senderProxy
+ if (!actorRef.isRunning && !isStopped) {
+ isStopped = true
+ joinPoint.proceed
+
+ } else if (isOneWay) {
+ actorRef.!(joinPoint)(senderActorRef)
+ null.asInstanceOf[AnyRef]
+
+ } else if (TypedActor.returnsFuture_?(methodRtti)) {
+ actorRef.!!!(joinPoint, timeout)(senderActorRef)
+
+ } else {
+ val result = (actorRef.!!(joinPoint, timeout)(senderActorRef)).as[AnyRef]
+ if (result.isDefined) result.get
+ else throw new ActorTimeoutException("Invocation to [" + joinPoint + "] timed out.")
+ }
+ }
+
+ private def remoteDispatch(joinPoint: JoinPoint): AnyRef = {
+ val methodRtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
+ val isOneWay = TypedActor.isOneWay(methodRtti)
+ val (message: Array[AnyRef], isEscaped) = escapeArguments(methodRtti.getParameterValues)
+
+ val typedActorInfo = TypedActorInfoProtocol.newBuilder
+ .setInterface(interfaceClass.getName)
+ .setMethod(methodRtti.getMethod.getName)
+ .build
+
+ val actorInfo = ActorInfoProtocol.newBuilder
+ .setUuid(uuid)
+ .setTarget(typedActor.getClass.getName)
+ .setTimeout(timeout)
+ .setActorType(ActorType.TYPED_ACTOR)
+ .setTypedActorInfo(typedActorInfo)
+ .build
+
+ val requestBuilder = RemoteRequestProtocol.newBuilder
+ .setId(RemoteRequestProtocolIdFactory.nextId)
+ .setMessage(MessageSerializer.serialize(message))
+ .setActorInfo(actorInfo)
+ .setIsOneWay(isOneWay)
+
+ val id = actorRef.registerSupervisorAsRemoteActor
+ if (id.isDefined) requestBuilder.setSupervisorUuid(id.get)
+
+ val remoteMessage = requestBuilder.build
+
+ val future = RemoteClient.clientFor(remoteAddress.get).send(remoteMessage, None)
+
+ if (isOneWay) null // for void methods
+ else {
+ if (future.isDefined) {
+ future.get.await
+ val result = getResultOrThrowException(future.get)
+ if (result.isDefined) result.get
+ else throw new IllegalActorStateException("No result returned from call to [" + joinPoint + "]")
+ } else throw new IllegalActorStateException("No future returned from call to [" + joinPoint + "]")
+ }
+ }
+
+ private def getResultOrThrowException[T](future: Future[T]): Option[T] =
+ if (future.exception.isDefined) throw future.exception.get
+ else future.result
+
+ private def escapeArguments(args: Array[AnyRef]): Tuple2[Array[AnyRef], Boolean] = {
+ var isEscaped = false
+ val escapedArgs = for (arg <- args) yield {
+ val clazz = arg.getClass
+ if (clazz.getName.contains(TypedActor.AW_PROXY_PREFIX)) {
+ isEscaped = true
+ TypedActor.AW_PROXY_PREFIX + clazz.getSuperclass.getName
+ } else arg
+ }
+ (escapedArgs, isEscaped)
+ }
+
+ private def initialize(joinPoint: JoinPoint): Unit = {
+ val init = AspectInitRegistry.initFor(joinPoint.getThis)
+ interfaceClass = init.interfaceClass
+ typedActor = init.targetInstance
+ actorRef = init.actorRef
+ uuid = actorRef.uuid
+ remoteAddress = init.remoteAddress
+ timeout = init.timeout
+ isInitialized = true
+ }
+}
+
+/**
+ * Internal helper class to help pass the contextual information between threads.
+ *
+ * @author Jonas Bonér
+ */
+private[akka] object SenderContextInfo {
+ import scala.util.DynamicVariable
+ private[actor] val senderActorRef = new DynamicVariable[ActorRef](null)
+ private[actor] val senderProxy = new DynamicVariable[AnyRef](null)
+}
+
+/**
+ * @author Jonas Bonér
+ */
+private[akka] object AspectInitRegistry extends ListenerManagement {
+ private val initializations = new java.util.concurrent.ConcurrentHashMap[AnyRef, AspectInit]
+
+ def initFor(proxy: AnyRef) = initializations.get(proxy)
+
+ def register(proxy: AnyRef, init: AspectInit) = {
+ val res = initializations.put(proxy, init)
+ foreachListener(_ ! AspectInitRegistered(proxy, init))
+ res
+ }
+
+ /**
+ * Unregisters initialization and stops its ActorRef.
+ */
+ def unregister(proxy: AnyRef): AspectInit = {
+ val init = initializations.remove(proxy)
+ foreachListener(_ ! AspectInitUnregistered(proxy, init))
+ init.actorRef.stop
+ init
+ }
+}
+
+private[akka] sealed trait AspectInitRegistryEvent
+private[akka] case class AspectInitRegistered(proxy: AnyRef, init: AspectInit) extends AspectInitRegistryEvent
+private[akka] case class AspectInitUnregistered(proxy: AnyRef, init: AspectInit) extends AspectInitRegistryEvent
+
+/**
+ * @author Jonas Bonér
+ */
+private[akka] sealed case class AspectInit(
+ val interfaceClass: Class[_],
+ val targetInstance: TypedActor,
+ val actorRef: ActorRef,
+ val remoteAddress: Option[InetSocketAddress],
+ val timeout: Long) {
+ def this(interfaceClass: Class[_], targetInstance: TypedActor, actorRef: ActorRef, timeout: Long) =
+ this(interfaceClass, targetInstance, actorRef, None, timeout)
+}
+
diff --git a/akka-core/src/main/scala/actor/UntypedActor.scala b/akka-core/src/main/scala/actor/UntypedActor.scala
new file mode 100644
index 0000000000..e94ea94b3e
--- /dev/null
+++ b/akka-core/src/main/scala/actor/UntypedActor.scala
@@ -0,0 +1,158 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ * public class SampleUntypedActor extends UntypedActor {
+ * public void onReceive(Object message) throws Exception {
+ * if (message instanceof String) {
+ * String msg = (String)message;
+ *
+ * if (msg.equals("UseReply")) {
+ * // Reply to original sender of message using the 'replyUnsafe' method
+ * getContext().replyUnsafe(msg + ":" + getContext().getUuid());
+ *
+ * } else if (msg.equals("UseSender") && getContext().getSender().isDefined()) {
+ * // Reply to original sender of message using the sender reference
+ * // also passing along my own refererence (the context)
+ * getContext().getSender().get().sendOneWay(msg, context);
+ *
+ * } else if (msg.equals("UseSenderFuture") && getContext().getSenderFuture().isDefined()) {
+ * // Reply to original sender of message using the sender future reference
+ * getContext().getSenderFuture().get().completeWithResult(msg);
+ *
+ * } else if (msg.equals("SendToSelf")) {
+ * // Send message to the actor itself recursively
+ * getContext().sendOneWay(msg)
+ *
+ * } 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();
+ *
+ * } 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);
+ * actor.start();
+ * actor.sendOneWay("SendToSelf");
+ * actor.stop();
+ * }
+ * }
+ *
+ *
+ * @author Jonas Bonér
+ */
+abstract class UntypedActor extends Actor {
+ def getContext(): ActorRef = self
+
+ final protected def receive = {
+ case msg => onReceive(msg)
+ }
+
+ @throws(classOf[Exception])
+ def onReceive(message: Any): Unit
+}
+
+/**
+ * Implements the Transactor abstraction. E.g. a transactional UntypedActor.
+ *
+ * @author Jonas Bonér
+ */
+abstract class UntypedTransactor extends UntypedActor {
+ self.makeTransactionRequired
+}
+
+/**
+ * Factory closure for an UntypedActor, to be used with 'UntypedActor.actorOf(factory)'.
+ *
+ * @author Jonas Bonér
+ */
+trait UntypedActorFactory {
+ def create: UntypedActor
+}
+
+/**
+ * Extend this abstract class to create a remote UntypedActor.
+ *
+ * @author Jonas Bonér
+ */
+abstract class RemoteUntypedActor(address: InetSocketAddress) extends UntypedActor {
+ def this(hostname: String, port: Int) = this(new InetSocketAddress(hostname, port))
+ self.makeRemote(address)
+}
+
+/**
+ * Factory object for creating and managing 'UntypedActor's. Meant to be used from Java.
+ *
+ * Example on how to create an actor:
+ * + * ActorRef actor = UntypedActor.actorOf(MyUntypedActor.class); + * actor.start(); + * actor.sendOneWay(message, context) + * actor.stop(); + *+ * You can create and start the actor in one statement like this: + *
+ * ActorRef actor = UntypedActor.actorOf(MyUntypedActor.class).start(); + *+ * + * @author Jonas Bonér + */ +object UntypedActor { + /** + * Creates an ActorRef out of the Actor type represented by the class provided. + * Example in Java: + *
+ * ActorRef actor = UntypedActor.actorOf(MyUntypedActor.class); + * actor.start(); + * actor.sendOneWay(message, context); + * actor.stop(); + *+ * You can create and start the actor in one statement like this: + *
+ * val actor = actorOf(classOf[MyActor]).start + *+ */ + def actorOf[T <: Actor](clazz: Class[T]): ActorRef = Actor.actorOf(clazz) + + /** + * NOTE: Use this convenience method with care, do NOT make it possible to get a reference to the + * UntypedActor instance directly, but only through its 'ActorRef' wrapper reference. + * + * Creates an ActorRef out of the Actor. Allows you to pass in the instance for the UntypedActor. + * Only use this method when you need to pass in constructor arguments into the 'UntypedActor'. + * + * You use it by implementing the UntypedActorFactory interface. + * Example in Java: + *
+ * ActorRef actor = UntypedActor.actorOf(new UntypedActorFactory() {
+ * public UntypedActor create() {
+ * return new MyUntypedActor("service:name", 5);
+ * }
+ * });
+ * actor.start();
+ * actor.sendOneWay(message, context);
+ * actor.stop();
+ *
+ */
+ def actorOf(factory: UntypedActorFactory): ActorRef = Actor.actorOf(factory.create)
+}
\ No newline at end of file
diff --git a/akka-core/src/main/scala/config/Config.scala b/akka-core/src/main/scala/config/Config.scala
index 68842ad1e3..aca46e6249 100644
--- a/akka-core/src/main/scala/config/Config.scala
+++ b/akka-core/src/main/scala/config/Config.scala
@@ -5,10 +5,11 @@
package se.scalablesolutions.akka.config
import se.scalablesolutions.akka.util.Logging
+import se.scalablesolutions.akka.AkkaException
import net.lag.configgy.{Config => CConfig, Configgy, ParseException}
-class ConfigurationException(message: String) extends RuntimeException(message)
+class ConfigurationException(message: String) extends AkkaException(message)
/**
* Loads up the configuration (from the akka.conf file).
@@ -82,6 +83,8 @@ object Config extends Logging {
if (VERSION != CONFIG_VERSION) throw new ConfigurationException(
"Akka JAR version [" + VERSION + "] is different than the provided config ('akka.conf') version [" + CONFIG_VERSION + "]")
+ val TIME_UNIT = config.getString("akka.time-unit", "seconds")
+
val startTime = System.currentTimeMillis
def uptime = (System.currentTimeMillis - startTime) / 1000
}
diff --git a/akka-core/src/main/scala/config/Configuration.scala b/akka-core/src/main/scala/config/Configuration.scala
index faa5c912f1..e257c739a9 100644
--- a/akka-core/src/main/scala/config/Configuration.scala
+++ b/akka-core/src/main/scala/config/Configuration.scala
@@ -5,7 +5,7 @@
package se.scalablesolutions.akka.config
/*
-import se.scalablesolutions.akka.kernel.{ActiveObject, ActiveObjectProxy}
+import se.scalablesolutions.akka.kernel.{TypedActor, TypedActorProxy}
import com.google.inject.{AbstractModule}
import java.util.{List => JList, ArrayList}
import scala.reflect.BeanProperty
@@ -55,6 +55,6 @@ class Component(@BeanProperty val intf: Class[_],
@BeanProperty val target: Class[_],
@BeanProperty val lifeCycle: LifeCycle,
@BeanProperty val timeout: Int) extends Server {
- def newWorker(proxy: ActiveObjectProxy) = se.scalablesolutions.akka.kernel.Supervise(proxy.server, lifeCycle.transform)
+ def newWorker(proxy: TypedActorProxy) = se.scalablesolutions.akka.kernel.Supervise(proxy.server, lifeCycle.transform)
}
*/
diff --git a/akka-core/src/main/scala/config/Configurator.scala b/akka-core/src/main/scala/config/Configurator.scala
index db92c5f35b..ba7e1f35f2 100644
--- a/akka-core/src/main/scala/config/Configurator.scala
+++ b/akka-core/src/main/scala/config/Configurator.scala
@@ -6,14 +6,14 @@ package se.scalablesolutions.akka.config
import ScalaConfig.{RestartStrategy, Component}
-private[akka] trait ActiveObjectConfiguratorBase {
+private[akka] trait TypedActorConfiguratorBase {
def getExternalDependency[T](clazz: Class[T]): T
- def configure(restartStrategy: RestartStrategy, components: List[Component]): ActiveObjectConfiguratorBase
+ def configure(restartStrategy: RestartStrategy, components: List[Component]): TypedActorConfiguratorBase
- def inject: ActiveObjectConfiguratorBase
+ def inject: TypedActorConfiguratorBase
- def supervise: ActiveObjectConfiguratorBase
+ def supervise: TypedActorConfiguratorBase
def reset
diff --git a/akka-core/src/main/scala/config/SupervisionConfig.scala b/akka-core/src/main/scala/config/SupervisionConfig.scala
index 1f5fd15a9b..2f25f4ed33 100644
--- a/akka-core/src/main/scala/config/SupervisionConfig.scala
+++ b/akka-core/src/main/scala/config/SupervisionConfig.scala
@@ -4,7 +4,7 @@
package se.scalablesolutions.akka.config
-import se.scalablesolutions.akka.actor.{Actor, ActorRef}
+import se.scalablesolutions.akka.actor.{ActorRef}
import se.scalablesolutions.akka.dispatch.MessageDispatcher
sealed abstract class FaultHandlingStrategy
@@ -42,16 +42,7 @@ object ScalaConfig {
case object AllForOne extends FailOverScheme
case object OneForOne extends FailOverScheme
- case class LifeCycle(scope: Scope,
- restartCallbacks: Option[RestartCallbacks] = None,
- shutdownCallback: Option[ShutdownCallback] = None) extends ConfigElement
- case class RestartCallbacks(preRestart: String, postRestart: String) {
- if ((preRestart eq null) || (postRestart eq null)) throw new IllegalArgumentException("Restart callback methods can't be null")
- }
- case class ShutdownCallback(shutdown: String) {
- if (shutdown eq null) throw new IllegalArgumentException("Shutdown callback method can't be null")
- }
-
+ case class LifeCycle(scope: Scope) extends ConfigElement
case object Permanent extends Scope
case object Temporary extends Scope
@@ -137,26 +128,12 @@ object JavaConfig {
scheme.transform, maxNrOfRetries, withinTimeRange, trapExceptions.toList)
}
- class LifeCycle(@BeanProperty val scope: Scope,
- @BeanProperty val restartCallbacks: RestartCallbacks,
- @BeanProperty val shutdownCallback: ShutdownCallback) extends ConfigElement {
- def this(scope: Scope) = this(scope, null, null)
- def this(scope: Scope, restartCallbacks: RestartCallbacks) = this(scope, restartCallbacks, null)
- def this(scope: Scope, shutdownCallback: ShutdownCallback) = this(scope, null, shutdownCallback)
+ class LifeCycle(@BeanProperty val scope: Scope) extends ConfigElement {
def transform = {
- val restartCallbacksOption = if (restartCallbacks eq null) None else Some(restartCallbacks.transform)
- val shutdownCallbackOption = if (shutdownCallback eq null) None else Some(shutdownCallback.transform)
- se.scalablesolutions.akka.config.ScalaConfig.LifeCycle(scope.transform, restartCallbacksOption, shutdownCallbackOption)
+ se.scalablesolutions.akka.config.ScalaConfig.LifeCycle(scope.transform)
}
}
- class RestartCallbacks(@BeanProperty val preRestart: String, @BeanProperty val postRestart: String) {
- def transform = se.scalablesolutions.akka.config.ScalaConfig.RestartCallbacks(preRestart, postRestart)
- }
- class ShutdownCallback(@BeanProperty val shutdown: String) {
- def transform = se.scalablesolutions.akka.config.ScalaConfig.ShutdownCallback(shutdown)
- }
-
abstract class Scope extends ConfigElement {
def transform: se.scalablesolutions.akka.config.ScalaConfig.Scope
}
diff --git a/akka-core/src/main/scala/config/ActiveObjectConfigurator.scala b/akka-core/src/main/scala/config/TypedActorConfigurator.scala
similarity index 56%
rename from akka-core/src/main/scala/config/ActiveObjectConfigurator.scala
rename to akka-core/src/main/scala/config/TypedActorConfigurator.scala
index 88e495bbd0..d639d21f5f 100644
--- a/akka-core/src/main/scala/config/ActiveObjectConfigurator.scala
+++ b/akka-core/src/main/scala/config/TypedActorConfigurator.scala
@@ -12,54 +12,55 @@ import java.util.{ArrayList}
import com.google.inject._
/**
- * Configurator for the Active Objects. Used to do declarative configuration of supervision.
- * It also does dependency injection with and into Active Objects using dependency injection
+ * Configurator for the TypedActors. Used to do declarative configuration of supervision.
+ * It also does dependency injection with and into TypedActors using dependency injection
* frameworks such as Google Guice or Spring.
*
- * If you don't want declarative configuration then you should use the ActiveObject
+ * If you don't want declarative configuration then you should use the TypedActor
* factory methods.
*
* @author Jonas Bonér
*/
-class ActiveObjectConfigurator {
+class TypedActorConfigurator {
import scala.collection.JavaConversions._
// TODO: make pluggable once we have f.e a SpringConfigurator
- private val INSTANCE = new ActiveObjectGuiceConfigurator
+ private val INSTANCE = new TypedActorGuiceConfigurator
/**
- * Returns the a list with all active objects that has been put under supervision for the class specified.
+ * Returns the a list with all typed actors that has been put under supervision for the class specified.
*
- * @param clazz the class for the active object
- * @return a list with all the active objects for the class
+ * @param clazz the class for the typed actor
+ * @return a list with all the typed actors for the class
*/
- def getInstances[T](clazz: Class[T]): JList[T] = INSTANCE.getInstance(clazz).foldLeft(new ArrayList[T]){ (l, i) => l add i ; l }
+ def getInstances[T](clazz: Class[T]): JList[T] =
+ INSTANCE.getInstance(clazz).foldLeft(new ArrayList[T]){ (l, i) => l add i ; l }
/**
- * Returns the first item in a list of all active objects that has been put under supervision for the class specified.
+ * Returns the first item in a list of all typed actors that has been put under supervision for the class specified.
*
- * @param clazz the class for the active object
- * @return the active object for the class
+ * @param clazz the class for the typed actor
+ * @return the typed actor for the class
*/
def getInstance[T](clazz: Class[T]): T = INSTANCE.getInstance(clazz).head
- def configure(restartStrategy: RestartStrategy, components: Array[Component]): ActiveObjectConfigurator = {
+ def configure(restartStrategy: RestartStrategy, components: Array[Component]): TypedActorConfigurator = {
INSTANCE.configure(
restartStrategy.transform,
components.toList.asInstanceOf[scala.List[Component]].map(_.transform))
this
}
- def inject: ActiveObjectConfigurator = {
+ def inject: TypedActorConfigurator = {
INSTANCE.inject
this
}
- def supervise: ActiveObjectConfigurator = {
+ def supervise: TypedActorConfigurator = {
INSTANCE.supervise
this
}
- def addExternalGuiceModule(module: Module): ActiveObjectConfigurator = {
+ def addExternalGuiceModule(module: Module): TypedActorConfigurator = {
INSTANCE.addExternalGuiceModule(module)
this
}
diff --git a/akka-core/src/main/scala/config/ActiveObjectGuiceConfigurator.scala b/akka-core/src/main/scala/config/TypedActorGuiceConfigurator.scala
similarity index 59%
rename from akka-core/src/main/scala/config/ActiveObjectGuiceConfigurator.scala
rename to akka-core/src/main/scala/config/TypedActorGuiceConfigurator.scala
index 54174b6030..cced864721 100644
--- a/akka-core/src/main/scala/config/ActiveObjectGuiceConfigurator.scala
+++ b/akka-core/src/main/scala/config/TypedActorGuiceConfigurator.scala
@@ -4,25 +4,27 @@
package se.scalablesolutions.akka.config
-import com.google.inject._
-
+import se.scalablesolutions.akka.actor._
import se.scalablesolutions.akka.config.ScalaConfig._
-import se.scalablesolutions.akka.actor.{Supervisor, ActiveObject, Dispatcher, ActorRef, Actor, IllegalActorStateException}
import se.scalablesolutions.akka.remote.RemoteServer
import se.scalablesolutions.akka.util.Logging
+import org.codehaus.aspectwerkz.proxy.Proxy
+
import scala.collection.mutable.HashMap
import java.net.InetSocketAddress
import java.lang.reflect.Method
+import com.google.inject._
+
/**
- * This is an class for internal usage. Instead use the se.scalablesolutions.akka.config.ActiveObjectConfigurator
- * class for creating ActiveObjects.
+ * This is an class for internal usage. Instead use the se.scalablesolutions.akka.config.TypedActorConfigurator
+ * class for creating TypedActors.
*
* @author Jonas Bonér
*/
-private[akka] class ActiveObjectGuiceConfigurator extends ActiveObjectConfiguratorBase with Logging {
+private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBase with Logging {
private var injector: Injector = _
private var supervisor: Option[Supervisor] = None
private var restartStrategy: RestartStrategy = _
@@ -30,22 +32,22 @@ private[akka] class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurat
private var supervised: List[Supervise] = Nil
private var bindings: List[DependencyBinding] = Nil
private var configRegistry = new HashMap[Class[_], Component] // TODO is configRegistry needed?
- private var activeObjectRegistry = new HashMap[Class[_], Tuple3[AnyRef, AnyRef, Component]]
+ private var typedActorRegistry = new HashMap[Class[_], Tuple3[AnyRef, AnyRef, Component]]
private var modules = new java.util.ArrayList[Module]
private var methodToUriRegistry = new HashMap[Method, String]
/**
* Returns the active abject that has been put under supervision for the class specified.
*
- * @param clazz the class for the active object
- * @return the active objects for the class
+ * @param clazz the class for the typed actor
+ * @return the typed actors for the class
*/
def getInstance[T](clazz: Class[T]): List[T] = synchronized {
- log.debug("Retrieving active object [%s]", clazz.getName)
+ log.debug("Retrieving typed actor [%s]", clazz.getName)
if (injector eq null) throw new IllegalActorStateException(
"inject() and/or supervise() must be called before invoking getInstance(clazz)")
val (proxy, targetInstance, component) =
- activeObjectRegistry.getOrElse(clazz, throw new IllegalActorStateException(
+ typedActorRegistry.getOrElse(clazz, throw new IllegalActorStateException(
"Class [" + clazz.getName + "] has not been put under supervision" +
"\n(by passing in the config to the 'configure' and then invoking 'supervise') method"))
injector.injectMembers(targetInstance)
@@ -53,7 +55,7 @@ private[akka] class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurat
}
def isDefined(clazz: Class[_]): Boolean = synchronized {
- activeObjectRegistry.get(clazz).isDefined
+ typedActorRegistry.get(clazz).isDefined
}
override def getExternalDependency[T](clazz: Class[T]): T = synchronized {
@@ -67,72 +69,90 @@ private[akka] class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurat
}
override def configure(restartStrategy: RestartStrategy, components: List[Component]):
- ActiveObjectConfiguratorBase = synchronized {
+ TypedActorConfiguratorBase = synchronized {
this.restartStrategy = restartStrategy
this.components = components.toArray.toList.asInstanceOf[List[Component]]
bindings = for (component <- this.components) yield {
- if (component.intf.isDefined) newDelegatingProxy(component)
- else newSubclassingProxy(component)
+ newDelegatingProxy(component)
+// if (component.intf.isDefined) newDelegatingProxy(component)
+// else newSubclassingProxy(component)
}
val deps = new java.util.ArrayList[DependencyBinding](bindings.size)
for (b <- bindings) deps.add(b)
- modules.add(new ActiveObjectGuiceModule(deps))
+ modules.add(new TypedActorGuiceModule(deps))
this
}
+/*
private def newSubclassingProxy(component: Component): DependencyBinding = {
- val targetClass = component.target
- val actorRef = Actor.actorOf(new Dispatcher(component.transactionRequired,
- component.lifeCycle.restartCallbacks,
- component.lifeCycle.shutdownCallback))
+ val targetClass =
+ if (component.target.isInstanceOf[Class[_ <: TypedActor]]) component.target.asInstanceOf[Class[_ <: TypedActor]]
+ else throw new IllegalArgumentException("TypedActor [" + component.target.getName + "] must be a subclass of TypedActor")
+ val actorRef = Actor.actorOf(new Dispatcher(component.transactionRequired))
if (component.dispatcher.isDefined) actorRef.dispatcher = component.dispatcher.get
val remoteAddress =
if (component.remoteAddress.isDefined)
Some(new InetSocketAddress(component.remoteAddress.get.hostname, component.remoteAddress.get.port))
else None
- val proxy = ActiveObject.newInstance(targetClass, actorRef, remoteAddress, component.timeout).asInstanceOf[AnyRef]
- remoteAddress.foreach(address => RemoteServer.registerActiveObject(address, targetClass.getName, proxy))
+ val proxy = TypedActor.newInstance(targetClass, actorRef, remoteAddress, component.timeout).asInstanceOf[AnyRef]
+ remoteAddress.foreach(address => RemoteServer.registerTypedActor(address, targetClass.getName, proxy))
supervised ::= Supervise(actorRef, component.lifeCycle)
- activeObjectRegistry.put(targetClass, (proxy, proxy, component))
+ typedActorRegistry.put(targetClass, (proxy, proxy, component))
new DependencyBinding(targetClass, proxy)
}
-
+*/
private def newDelegatingProxy(component: Component): DependencyBinding = {
- val targetClass = component.intf.get
- val targetInstance = component.target.newInstance.asInstanceOf[AnyRef] // TODO: perhaps need to put in registry
component.target.getConstructor(Array[Class[_]](): _*).setAccessible(true)
- val actorRef = Actor.actorOf(new Dispatcher(component.transactionRequired,
- component.lifeCycle.restartCallbacks,
- component.lifeCycle.shutdownCallback))
+ val interfaceClass = if (component.intf.isDefined) component.intf.get
+ else throw new IllegalActorStateException("No interface for TypedActor specified")
+ val implementationClass = component.target
+ val timeout = component.timeout
+
+ val actorRef = Actor.actorOf(TypedActor.newTypedActor(implementationClass))
+ actorRef.timeout = timeout
if (component.dispatcher.isDefined) actorRef.dispatcher = component.dispatcher.get
+ val typedActor = actorRef.actorInstance.get.asInstanceOf[TypedActor]
+
+ val proxy = Proxy.newInstance(Array(interfaceClass), Array(typedActor), true, false)
+
val remoteAddress =
if (component.remoteAddress.isDefined)
Some(new InetSocketAddress(component.remoteAddress.get.hostname, component.remoteAddress.get.port))
else None
- val proxy = ActiveObject.newInstance(
- targetClass, targetInstance, actorRef, remoteAddress, component.timeout).asInstanceOf[AnyRef]
- remoteAddress.foreach(address => RemoteServer.registerActiveObject(address, targetClass.getName, proxy))
+
+ remoteAddress.foreach { address =>
+ actorRef.makeRemote(remoteAddress.get)
+ RemoteServer.registerTypedActor(address, implementationClass.getName, proxy)
+ }
+
+ AspectInitRegistry.register(
+ proxy,
+ AspectInit(interfaceClass, typedActor, actorRef, remoteAddress, timeout))
+ typedActor.initialize(proxy)
+ actorRef.start
+
supervised ::= Supervise(actorRef, component.lifeCycle)
- activeObjectRegistry.put(targetClass, (proxy, targetInstance, component))
- new DependencyBinding(targetClass, proxy)
+
+ typedActorRegistry.put(interfaceClass, (proxy, typedActor, component))
+ new DependencyBinding(interfaceClass, proxy)
}
- override def inject: ActiveObjectConfiguratorBase = synchronized {
+ override def inject: TypedActorConfiguratorBase = synchronized {
if (injector ne null) throw new IllegalActorStateException("inject() has already been called on this configurator")
injector = Guice.createInjector(modules)
this
}
- override def supervise: ActiveObjectConfiguratorBase = synchronized {
+ override def supervise: TypedActorConfiguratorBase = synchronized {
if (injector eq null) inject
- supervisor = Some(ActiveObject.supervise(restartStrategy, supervised))
+ supervisor = Some(TypedActor.supervise(restartStrategy, supervised))
this
}
/**
* Add additional services to be wired in.
*
- * activeObjectConfigurator.addExternalGuiceModule(new AbstractModule {
+ * typedActorConfigurator.addExternalGuiceModule(new AbstractModule {
* protected void configure() {
* bind(Foo.class).to(FooImpl.class).in(Scopes.SINGLETON);
* bind(BarImpl.class);
@@ -141,7 +161,7 @@ private[akka] class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurat
* }})
*
*/
- def addExternalGuiceModule(module: Module): ActiveObjectConfiguratorBase = synchronized {
+ def addExternalGuiceModule(module: Module): TypedActorConfiguratorBase = synchronized {
modules.add(module)
this
}
@@ -151,7 +171,7 @@ private[akka] class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurat
def reset = synchronized {
modules = new java.util.ArrayList[Module]
configRegistry = new HashMap[Class[_], Component]
- activeObjectRegistry = new HashMap[Class[_], Tuple3[AnyRef, AnyRef, Component]]
+ typedActorRegistry = new HashMap[Class[_], Tuple3[AnyRef, AnyRef, Component]]
methodToUriRegistry = new HashMap[Method, String]
injector = null
restartStrategy = null
diff --git a/akka-core/src/main/scala/dataflow/DataFlowVariable.scala b/akka-core/src/main/scala/dataflow/DataFlowVariable.scala
index baf3e33f6e..cd7ba704ad 100644
--- a/akka-core/src/main/scala/dataflow/DataFlowVariable.scala
+++ b/akka-core/src/main/scala/dataflow/DataFlowVariable.scala
@@ -10,6 +10,7 @@ import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue}
import se.scalablesolutions.akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.dispatch.CompletableFuture
+import se.scalablesolutions.akka.AkkaException
/**
* Implements Oz-style dataflow (single assignment) variables.
@@ -155,7 +156,7 @@ import se.scalablesolutions.akka.dispatch.CompletableFuture
/**
* @author Jonas Bonér
*/
- class DataFlowVariableException(msg: String) extends RuntimeException(msg)
+ class DataFlowVariableException(msg: String) extends AkkaException(msg)
}
diff --git a/akka-core/src/main/scala/dispatch/Dispatchers.scala b/akka-core/src/main/scala/dispatch/Dispatchers.scala
index e938e36e4e..82c51c57de 100644
--- a/akka-core/src/main/scala/dispatch/Dispatchers.scala
+++ b/akka-core/src/main/scala/dispatch/Dispatchers.scala
@@ -1,11 +1,15 @@
/**
- * Copyright (C) 2009-2010 Scalable Solutions AB HawtDispatcher.pin(self) to "pin" an actor to a specific thread.
+ *
+ * See the ScalaDoc for the {@link se.scalablesolutions.akka.dispatch.HawtDispatcher} for details.
+ */
+ def newHawtDispatcher(aggregate: Boolean) = new HawtDispatcher(aggregate)
+
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
*
@@ -91,4 +111,74 @@ object Dispatchers {
* E.g. each actor consumes its own thread.
*/
def newThreadBasedDispatcher(actor: ActorRef) = new ThreadBasedDispatcher(actor)
+
+ /**
+ * Utility function that tries to load the specified dispatcher config from the akka.conf
+ * or else use the supplied default dispatcher
+ */
+ def fromConfig(key: String, default: => MessageDispatcher = defaultGlobalDispatcher): MessageDispatcher =
+ config.getConfigMap(key).flatMap(from).getOrElse(default)
+
+ /*
+ * Creates of obtains a dispatcher from a ConfigMap according to the format below
+ *
+ * default-dispatcher {
+ * type = "GlobalExecutorBasedEventDriven" # Must be one of the following, all "Global*" are non-configurable
+ * # ReactorBasedSingleThreadEventDriven, (ExecutorBasedEventDrivenWorkStealing), ExecutorBasedEventDriven,
+ * # ReactorBasedThreadPoolEventDriven, Hawt, GlobalReactorBasedSingleThreadEventDriven,
+ * # GlobalReactorBasedThreadPoolEventDriven, GlobalExecutorBasedEventDriven, GlobalHawt
+ * keep-alive-ms = 60000 # Keep alive time for threads
+ * core-pool-size-factor = 1.0 # No of core threads ... ceil(available processors * factor)
+ * max-pool-size-factor = 4.0 # Max no of threads ... ceil(available processors * factor)
+ * executor-bounds = -1 # Makes the Executor bounded, -1 is unbounded
+ * allow-core-timeout = on # Allow core threads to time out
+ * rejection-policy = "caller-runs" # abort, caller-runs, discard-oldest, discard
+ * throughput = 5 # Throughput for ExecutorBasedEventDrivenDispatcher
+ * aggregate = off # Aggregate on/off for HawtDispatchers
+ * }
+ * ex: from(config.getConfigMap(identifier).get)
+ *
+ * Gotcha: Only configures the dispatcher if possible
+ * Returns: None if "type" isn't specified in the config
+ * Throws: IllegalArgumentException if the value of "type" is not valid
+ */
+ def from(cfg: ConfigMap): Option[MessageDispatcher] = {
+ lazy val name = cfg.getString("name",UUID.newUuid.toString)
+
+ val dispatcher: Option[MessageDispatcher] = cfg.getString("type") map {
+ case "ReactorBasedSingleThreadEventDriven" => newReactorBasedSingleThreadEventDrivenDispatcher(name)
+ case "ExecutorBasedEventDrivenWorkStealing" => newExecutorBasedEventDrivenWorkStealingDispatcher(name)
+ case "ExecutorBasedEventDriven" => newExecutorBasedEventDrivenDispatcher(name,cfg.getInt("throughput",THROUGHPUT))
+ case "ReactorBasedThreadPoolEventDriven" => newReactorBasedThreadPoolEventDrivenDispatcher(name)
+ case "Hawt" => newHawtDispatcher(cfg.getBool("aggregate").getOrElse(true))
+ case "GlobalReactorBasedSingleThreadEventDriven" => globalReactorBasedSingleThreadEventDrivenDispatcher
+ case "GlobalReactorBasedThreadPoolEventDriven" => globalReactorBasedThreadPoolEventDrivenDispatcher
+ case "GlobalExecutorBasedEventDriven" => globalExecutorBasedEventDrivenDispatcher
+ case "GlobalHawt" => globalHawtDispatcher
+
+ case unknown => throw new IllegalArgumentException("Unknown dispatcher type %s" format unknown)
+ }
+
+ dispatcher foreach {
+ case d: ThreadPoolBuilder => d.configureIfPossible( builder => {
+
+ cfg.getInt("keep-alive-ms").foreach(builder.setKeepAliveTimeInMillis(_))
+ cfg.getDouble("core-pool-size-factor").foreach(builder.setCorePoolSizeFromFactor(_))
+ cfg.getDouble("max-pool-size-factor").foreach(builder.setMaxPoolSizeFromFactor(_))
+ cfg.getInt("executor-bounds").foreach(builder.setExecutorBounds(_))
+ cfg.getBool("allow-core-timeout").foreach(builder.setAllowCoreThreadTimeout(_))
+
+ cfg.getString("rejection-policy").map({
+ case "abort" => new AbortPolicy()
+ case "caller-runs" => new CallerRunsPolicy()
+ case "discard-oldest" => new DiscardOldestPolicy()
+ case "discard" => new DiscardPolicy()
+ case x => throw new IllegalArgumentException("[%s] is not a valid rejectionPolicy!" format x)
+ }).foreach(builder.setRejectionPolicy(_))
+ })
+ case _ =>
+ }
+
+ dispatcher
+ }
}
diff --git a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
index 1d34083e0a..1f03c1eba2 100644
--- a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
+++ b/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
@@ -5,6 +5,7 @@
package se.scalablesolutions.akka.dispatch
import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException}
+import jsr166x.ConcurrentLinkedDeque
/**
* Default settings are:
@@ -67,15 +68,34 @@ class ExecutorBasedEventDrivenDispatcher(_name: String, throughput: Int = Dispat
val name = "akka:event-driven:dispatcher:" + _name
init
- def dispatch(invocation: MessageInvocation) = dispatch(invocation.receiver)
+ def dispatch(invocation: MessageInvocation) = {
+ getMailbox(invocation.receiver).add(invocation)
+ dispatch(invocation.receiver)
+ }
+
+ /**
+ * @return the mailbox associated with the actor
+ */
+ private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[ConcurrentLinkedDeque[MessageInvocation]]
+
+ override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
+
+ override def register(actorRef: ActorRef) = {
+ // The actor will need a ConcurrentLinkedDeque based mailbox
+ if( actorRef.mailbox eq null ) {
+ actorRef.mailbox = new ConcurrentLinkedDeque[MessageInvocation]()
+ }
+ super.register(actorRef)
+ }
def dispatch(receiver: ActorRef): Unit = if (active) {
+
executor.execute(new Runnable() {
def run = {
var lockAcquiredOnce = false
var finishedBeforeMailboxEmpty = false
val lock = receiver.dispatcherLock
- val mailbox = receiver.mailbox
+ val mailbox = getMailbox(receiver)
// this do-while loop is required to prevent missing new messages between the end of the inner while
// loop and releasing the lock
do {
@@ -92,8 +112,10 @@ class ExecutorBasedEventDrivenDispatcher(_name: String, throughput: Int = Dispat
} while ((lockAcquiredOnce && !finishedBeforeMailboxEmpty && !mailbox.isEmpty))
}
})
- } else log.warning(
- "%s is shut down,\n\tignoring the rest of the messages in the mailbox of\n\t%s", toString, receiver)
+ } else {
+ log.warning("%s is shut down,\n\tignoring the rest of the messages in the mailbox of\n\t%s", toString, receiver)
+ }
+
/**
* Process the messages in the mailbox of the given actor.
@@ -102,15 +124,16 @@ class ExecutorBasedEventDrivenDispatcher(_name: String, throughput: Int = Dispat
*/
def processMailbox(receiver: ActorRef): Boolean = {
var processedMessages = 0
- var messageInvocation = receiver.mailbox.poll
+ val mailbox = getMailbox(receiver)
+ var messageInvocation = mailbox.poll
while (messageInvocation != null) {
messageInvocation.invoke
processedMessages += 1
// check if we simply continue with other messages, or reached the throughput limit
- if (throughput <= 0 || processedMessages < throughput) messageInvocation = receiver.mailbox.poll
+ if (throughput <= 0 || processedMessages < throughput) messageInvocation = mailbox.poll
else {
messageInvocation = null
- return !receiver.mailbox.isEmpty
+ return !mailbox.isEmpty
}
}
false
@@ -128,11 +151,9 @@ class ExecutorBasedEventDrivenDispatcher(_name: String, throughput: Int = Dispat
references.clear
}
- def usesActorMailbox = true
-
def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
"Can't build a new thread pool for a dispatcher that is already up and running")
-
+
override def toString = "ExecutorBasedEventDrivenDispatcher[" + name + "]"
// FIXME: should we have an unbounded queue and not bounded as default ????
diff --git a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
index 76138dce35..b9ff5d92f4 100644
--- a/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
+++ b/akka-core/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
@@ -7,6 +7,7 @@ package se.scalablesolutions.akka.dispatch
import java.util.concurrent.CopyOnWriteArrayList
import se.scalablesolutions.akka.actor.{Actor, ActorRef, IllegalActorStateException}
+import jsr166x.ConcurrentLinkedDeque
/**
* An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
@@ -44,7 +45,16 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
val name = "akka:event-driven-work-stealing:dispatcher:" + _name
init
+
+ /**
+ * @return the mailbox associated with the actor
+ */
+ private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[ConcurrentLinkedDeque[MessageInvocation]]
+
+ override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
+
def dispatch(invocation: MessageInvocation) = if (active) {
+ getMailbox(invocation.receiver).add(invocation)
executor.execute(new Runnable() {
def run = {
if (!tryProcessMailbox(invocation.receiver)) {
@@ -76,7 +86,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
lock.unlock
}
}
- } while ((lockAcquiredOnce && !receiver.mailbox.isEmpty))
+ } while ((lockAcquiredOnce && !getMailbox(receiver).isEmpty))
return lockAcquiredOnce
}
@@ -85,10 +95,11 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
* Process the messages in the mailbox of the given actor.
*/
private def processMailbox(receiver: ActorRef) = {
- var messageInvocation = receiver.mailbox.poll
+ val mailbox = getMailbox(receiver)
+ var messageInvocation = mailbox.poll
while (messageInvocation != null) {
messageInvocation.invoke
- messageInvocation = receiver.mailbox.poll
+ messageInvocation = mailbox.poll
}
}
@@ -116,7 +127,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
for (i <- 0 to actors.length) {
val index = (i + startIndex) % actors.length
val actor = actors(index)
- if (actor != receiver && actor.mailbox.isEmpty) return (Some(actor), index)
+ if (actor != receiver && getMailbox(actor).isEmpty) return (Some(actor), index)
}
(None, startIndex) // nothing found, reuse same start index next time
}
@@ -139,7 +150,7 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
* Steal a message from the receiver and give it to the thief.
*/
private def donateMessage(receiver: ActorRef, thief: ActorRef): Boolean = {
- val donated = receiver.mailbox.pollLast
+ val donated = getMailbox(receiver).pollLast
if (donated ne null) {
if (donated.senderFuture.isDefined) thief.self.postMessageToMailboxAndCreateFutureResultWithTimeout[Any](
donated.message, receiver.timeout, donated.sender, donated.senderFuture)
@@ -164,11 +175,15 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
"Can't build a new thread pool for a dispatcher that is already up and running")
override def toString = "ExecutorBasedEventDrivenWorkStealingDispatcher[" + name + "]"
-
+
private[akka] def init = withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool
override def register(actorRef: ActorRef) = {
verifyActorsAreOfSameType(actorRef)
+ // The actor will need a ConcurrentLinkedDeque based mailbox
+ if( actorRef.mailbox == null ) {
+ actorRef.mailbox = new ConcurrentLinkedDeque[MessageInvocation]()
+ }
pooledActors.add(actorRef)
super.register(actorRef)
}
@@ -178,8 +193,6 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(_name: String) extends Mess
super.unregister(actorRef)
}
- def usesActorMailbox = true
-
private def verifyActorsAreOfSameType(actorOfId: ActorRef) = {
actorType match {
case None => actorType = Some(actorOfId.actor.getClass)
diff --git a/akka-core/src/main/scala/dispatch/Future.scala b/akka-core/src/main/scala/dispatch/Future.scala
index d1b4f9572b..17c63bcd57 100644
--- a/akka-core/src/main/scala/dispatch/Future.scala
+++ b/akka-core/src/main/scala/dispatch/Future.scala
@@ -4,11 +4,12 @@
package se.scalablesolutions.akka.dispatch
-import java.util.concurrent.locks.ReentrantLock
+import se.scalablesolutions.akka.AkkaException
+import java.util.concurrent.locks.ReentrantLock
import java.util.concurrent.TimeUnit
-class FutureTimeoutException(message: String) extends RuntimeException(message)
+class FutureTimeoutException(message: String) extends AkkaException(message)
object Futures {
@@ -25,7 +26,7 @@ object Futures {
try {
promise completeWithResult body
} catch {
- case e => promise completeWithException (None, e)
+ case e => promise completeWithException e
}
promise
}
@@ -77,12 +78,12 @@ sealed trait Future[T] {
def isExpired: Boolean
def timeoutInNanos: Long
def result: Option[T]
- def exception: Option[Tuple2[AnyRef, Throwable]]
+ def exception: Option[Throwable]
}
trait CompletableFuture[T] extends Future[T] {
def completeWithResult(result: T)
- def completeWithException(toBlame: AnyRef, exception: Throwable)
+ def completeWithException(exception: Throwable)
}
// Based on code from the actorom actor framework by Sergio Bossa [http://code.google.com/p/actorom/].
@@ -96,7 +97,7 @@ class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] {
private val _signal = _lock.newCondition
private var _completed: Boolean = _
private var _result: Option[T] = None
- private var _exception: Option[Tuple2[AnyRef, Throwable]] = None
+ private var _exception: Option[Throwable] = None
def await = try {
_lock.lock
@@ -147,7 +148,7 @@ class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] {
_lock.unlock
}
- def exception: Option[Tuple2[AnyRef, Throwable]] = try {
+ def exception: Option[Throwable] = try {
_lock.lock
_exception
} finally {
@@ -165,11 +166,11 @@ class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] {
_lock.unlock
}
- def completeWithException(toBlame: AnyRef, exception: Throwable) = try {
+ def completeWithException(exception: Throwable) = try {
_lock.lock
if (!_completed) {
_completed = true
- _exception = Some((toBlame, exception))
+ _exception = Some(exception)
}
} finally {
_signal.signalAll
diff --git a/akka-core/src/main/scala/dispatch/HawtDispatcher.scala b/akka-core/src/main/scala/dispatch/HawtDispatcher.scala
new file mode 100644
index 0000000000..45e4468b3d
--- /dev/null
+++ b/akka-core/src/main/scala/dispatch/HawtDispatcher.scala
@@ -0,0 +1,249 @@
+/**
+ * Copyright (C) 2010, Progress Software Corporation and/or its
+ * subsidiaries or affiliates. All rights reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package se.scalablesolutions.akka.dispatch
+
+import se.scalablesolutions.akka.actor.ActorRef
+import org.fusesource.hawtdispatch.DispatchQueue
+import org.fusesource.hawtdispatch.ScalaDispatch._
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.concurrent.atomic.AtomicBoolean
+import java.util.concurrent.CountDownLatch
+import org.fusesource.hawtdispatch.DispatchQueue.QueueType
+import org.fusesource.hawtdispatch.ListEventAggregator
+
+/**
+ * Holds helper methods for working with actors that are using
+ * a HawtDispatcher as it's dispatcher.
+ */
+object HawtDispatcher {
+
+ private val retained = new AtomicInteger()
+ @volatile private var shutdownLatch: CountDownLatch = _
+
+ private def retainNonDaemon = {
+ if( retained.getAndIncrement == 0 ) {
+ shutdownLatch = new CountDownLatch(1)
+ new Thread("HawtDispatch Non-Daemon") {
+ override def run = {
+ try {
+ shutdownLatch.await
+ } catch {
+ case _ =>
+ }
+ println("done");
+ }
+ }.start()
+ }
+ }
+
+ private def releaseNonDaemon = {
+ if( retained.decrementAndGet == 0 ) {
+ shutdownLatch.countDown
+ shutdownLatch = null
+ }
+ }
+
+ /**
+ * @return the mailbox associated with the actor
+ */
+ private def mailbox(actorRef: ActorRef) = {
+ actorRef.mailbox.asInstanceOf[HawtDispatcherMailbox]
+ }
+
+ /**
+ * @return the dispatch queue associated with the actor
+ */
+ def queue(actorRef: ActorRef) = {
+ mailbox(actorRef).queue
+ }
+
+
+ /**
+ * + * Pins an actor to a random thread queue. Once pinned the actor will always execute + * on the same thread. + *
+ * + *+ * This method can only succeed if the actor it's dispatcher is set to a HawtDispatcher and it has been started + *
+ * + * @return true if the actor was pinned + */ + def pin(actorRef: ActorRef) = { + actorRef.mailbox match { + case x:HawtDispatcherMailbox=> + x.queue.setTargetQueue( getRandomThreadQueue ) + true + case _ => false + } + } + + /** + *+ * Unpins the actor so that all threads in the hawt dispatch thread pool + * compete to execute him. + *
+ * + *+ * This method can only succeed if the actor it's dispatcher is set to a HawtDispatcher and it has been started + *
+ * @return true if the actor was unpinned + */ + def unpin(actorRef: ActorRef) = { + target(actorRef, globalQueue) + } + + + /** + * @return true if the actor was pinned to a thread. + */ + def pinned(actorRef: ActorRef):Boolean = { + actorRef.mailbox match { + case x:HawtDispatcherMailbox=> + x.queue.getTargetQueue.getQueueType == QueueType.THREAD_QUEUE + case _ => false + } + } + + /** + *+ * Updates the actor's target dispatch queue to the value specified. This allows + * you to do odd things like targeting another serial queue. + *
+ * + *+ * This method can only succeed if the actor it's dispatcher is set to a HawtDispatcher and it has been started + *
+ * @return true if the actor was unpinned + */ + def target(actorRef: ActorRef, parent:DispatchQueue) = { + actorRef.mailbox match { + case x:HawtDispatcherMailbox=> + x.queue.setTargetQueue( parent ) + true + case _ => false + } + } + +} + +/** + *+ * A HawtDispatch based MessageDispatcher. Actors with this dispatcher are executed + * on the HawtDispatch fixed sized thread pool. The number of of threads will match + * the number of cores available on your system. + * + *
+ *+ * Actors using this dispatcher are restricted to only executing non blocking + * operations. The actor cannot synchronously call another actor or call 3rd party + * libraries that can block for a long time. You should use non blocking IO APIs + * instead of blocking IO apis to avoid blocking that actor for an extended amount + * of time. + *
+ * + *+ * This dispatcher delivers messages to the actors in the order that they + * were producer at the sender. + *
+ * + *
+ * HawtDispatch supports processing Non blocking Socket IO in both the reactor
+ * and proactor styles. For more details, see the HawtDispacherEchoServer.scala
+ * example.
+ *
- * import se.scalablesolutions.akka.stm.local._
- *
- * atomic {
- * // do something within a transaction
- * }
- *
- */
-class LocalStm extends TransactionManagement with Logging {
-
- val DefaultLocalTransactionConfig = TransactionConfig()
- val DefaultLocalTransactionFactory = TransactionFactory(DefaultLocalTransactionConfig, "DefaultLocalTransaction")
-
- def atomic[T](body: => T)(implicit factory: TransactionFactory = DefaultLocalTransactionFactory): T = atomic(factory)(body)
-
- def atomic[T](factory: TransactionFactory)(body: => T): T = {
- factory.boilerplate.execute(new TransactionalCallable[T]() {
- def call(mtx: MultiverseTransaction): T = {
- factory.addHooks
- val result = body
- log.ifTrace("Committing local transaction [" + mtx + "]")
- result
- }
- })
- }
-}
-
-/**
- * Global transaction management, global in the context of multiple threads.
- * Use this if you need to have one transaction span multiple threads (or Actors).
- *
- * Example of atomic transaction management using the atomic block:
- *
- *
- * import se.scalablesolutions.akka.stm.global._
- *
- * atomic {
- * // do something within a transaction
- * }
- *
- */
-class GlobalStm extends TransactionManagement with Logging {
-
- val DefaultGlobalTransactionConfig = TransactionConfig()
- val DefaultGlobalTransactionFactory = TransactionFactory(DefaultGlobalTransactionConfig, "DefaultGlobalTransaction")
-
- def atomic[T](body: => T)(implicit factory: TransactionFactory = DefaultGlobalTransactionFactory): T = atomic(factory)(body)
-
- def atomic[T](factory: TransactionFactory)(body: => T): T = {
- factory.boilerplate.execute(new TransactionalCallable[T]() {
- def call(mtx: MultiverseTransaction): T = {
- if (!isTransactionSetInScope) createNewTransactionSet
- factory.addHooks
- val result = body
- val txSet = getTransactionSetInScope
- log.ifTrace("Committing global transaction [" + mtx + "]\n\tand joining transaction set [" + txSet + "]")
- try {
- txSet.tryJoinCommit(
- mtx,
- TransactionConfig.DefaultTimeout.length,
- TransactionConfig.DefaultTimeout.unit)
- // Need to catch IllegalStateException until we have fix in Multiverse, since it throws it by mistake
- } catch { case e: IllegalStateException => {} }
- result
- }
- })
- }
-}
-
trait StmUtil {
-
/**
* Schedule a deferred task on the thread local transaction (use within an atomic).
* This is executed when the transaction commits.
*/
- def deferred[T](body: => T): Unit =
+ def deferred[T](body: => T): Unit =
MultiverseStmUtils.scheduleDeferredTask(new Runnable { def run = body })
/**
* Schedule a compensating task on the thread local transaction (use within an atomic).
* This is executed when the transaction aborts.
*/
- def compensating[T](body: => T): Unit =
+ def compensating[T](body: => T): Unit =
MultiverseStmUtils.scheduleCompensatingTask(new Runnable { def run = body })
/**
@@ -193,7 +108,7 @@ trait StmUtil {
/**
* Use either-orElse to combine two blocking transactions.
- * Usage:
+ * Usage:
*
* either {
* ...
@@ -209,3 +124,14 @@ trait StmUtil {
}.execute()
}
}
+
+trait StmCommon {
+ type TransactionConfig = se.scalablesolutions.akka.stm.TransactionConfig
+ val TransactionConfig = se.scalablesolutions.akka.stm.TransactionConfig
+
+ type TransactionFactory = se.scalablesolutions.akka.stm.TransactionFactory
+ val TransactionFactory = se.scalablesolutions.akka.stm.TransactionFactory
+
+ type Ref[T] = se.scalablesolutions.akka.stm.Ref[T]
+ val Ref = se.scalablesolutions.akka.stm.Ref
+}
diff --git a/akka-core/src/main/scala/stm/TransactionalMap.scala b/akka-core/src/main/scala/stm/TransactionalMap.scala
index be7b9c5189..d45396ad25 100644
--- a/akka-core/src/main/scala/stm/TransactionalMap.scala
+++ b/akka-core/src/main/scala/stm/TransactionalMap.scala
@@ -11,9 +11,9 @@ import se.scalablesolutions.akka.util.UUID
import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction
object TransactionalMap {
- def apply[K, V]() = new TransactionalMap[K, V]
+ def apply[K, V]() = new TransactionalMap[K, V]()
- def apply[K, V](pairs: (K, V)*) = new TransactionalMap(Some(HashMap(pairs: _*)))
+ def apply[K, V](pairs: (K, V)*) = new TransactionalMap(HashMap(pairs: _*))
}
/**
@@ -21,12 +21,12 @@ object TransactionalMap {
*
* @author Jonas Bonér
*/
-class TransactionalMap[K, V](initialOpt: Option[HashMap[K, V]] = None) extends Transactional with scala.collection.mutable.Map[K, V] {
- def this() = this(None) // Java compatibility
+class TransactionalMap[K, V](initialValue: HashMap[K, V]) extends Transactional with scala.collection.mutable.Map[K, V] {
+ def this() = this(HashMap[K, V]())
val uuid = UUID.newUuid.toString
- protected[this] val ref = new Ref(initialOpt.orElse(Some(HashMap[K, V]())))
+ private[this] val ref = Ref(initialValue)
def -=(key: K) = {
remove(key)
@@ -41,36 +41,36 @@ class TransactionalMap[K, V](initialOpt: Option[HashMap[K, V]] = None) extends T
}
override def remove(key: K) = {
- val map = ref.get.get
+ val map = ref.get
val oldValue = map.get(key)
- ref.swap(ref.get.get - key)
+ ref.swap(ref.get - key)
oldValue
}
- def get(key: K): Option[V] = ref.get.get.get(key)
+ def get(key: K): Option[V] = ref.get.get(key)
override def put(key: K, value: V): Option[V] = {
- val map = ref.get.get
+ val map = ref.get
val oldValue = map.get(key)
ref.swap(map.updated(key, value))
oldValue
}
override def update(key: K, value: V) = {
- val map = ref.get.get
+ val map = ref.get
val oldValue = map.get(key)
ref.swap(map.updated(key, value))
}
- def iterator = ref.get.get.iterator
+ def iterator = ref.get.iterator
- override def elements: Iterator[(K, V)] = ref.get.get.iterator
+ override def elements: Iterator[(K, V)] = ref.get.iterator
- override def contains(key: K): Boolean = ref.get.get.contains(key)
+ override def contains(key: K): Boolean = ref.get.contains(key)
override def clear = ref.swap(HashMap[K, V]())
- override def size: Int = ref.get.get.size
+ override def size: Int = ref.get.size
override def hashCode: Int = System.identityHashCode(this);
diff --git a/akka-core/src/main/scala/stm/TransactionalVector.scala b/akka-core/src/main/scala/stm/TransactionalVector.scala
index e2ad6a2aeb..2beeeecef0 100644
--- a/akka-core/src/main/scala/stm/TransactionalVector.scala
+++ b/akka-core/src/main/scala/stm/TransactionalVector.scala
@@ -11,9 +11,9 @@ import se.scalablesolutions.akka.util.UUID
import org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction
object TransactionalVector {
- def apply[T]() = new TransactionalVector[T]
+ def apply[T]() = new TransactionalVector[T]()
- def apply[T](elems: T*) = new TransactionalVector(Some(Vector(elems: _*)))
+ def apply[T](elems: T*) = new TransactionalVector(Vector(elems: _*))
}
/**
@@ -21,31 +21,31 @@ object TransactionalVector {
*
* @author Jonas Bonér
*/
-class TransactionalVector[T](initialOpt: Option[Vector[T]] = None) extends Transactional with IndexedSeq[T] {
- def this() = this(None) // Java compatibility
+class TransactionalVector[T](initialValue: Vector[T]) extends Transactional with IndexedSeq[T] {
+ def this() = this(Vector[T]())
val uuid = UUID.newUuid.toString
- private[this] val ref = new Ref(initialOpt.orElse(Some(Vector[T]())))
+ private[this] val ref = Ref(initialValue)
def clear = ref.swap(Vector[T]())
def +(elem: T) = add(elem)
- def add(elem: T) = ref.swap(ref.get.get :+ elem)
+ def add(elem: T) = ref.swap(ref.get :+ elem)
- def get(index: Int): T = ref.get.get.apply(index)
+ def get(index: Int): T = ref.get.apply(index)
/**
* Removes the tail element of this vector.
*/
- def pop = ref.swap(ref.get.get.dropRight(1))
+ def pop = ref.swap(ref.get.dropRight(1))
- def update(index: Int, elem: T) = ref.swap(ref.get.get.updated(index, elem))
+ def update(index: Int, elem: T) = ref.swap(ref.get.updated(index, elem))
- def length: Int = ref.get.get.length
+ def length: Int = ref.get.length
- def apply(index: Int): T = ref.get.get.apply(index)
+ def apply(index: Int): T = ref.get.apply(index)
override def hashCode: Int = System.identityHashCode(this);
diff --git a/akka-core/src/main/scala/stm/global/Atomic.scala b/akka-core/src/main/scala/stm/global/Atomic.scala
new file mode 100644
index 0000000000..d5a92fe047
--- /dev/null
+++ b/akka-core/src/main/scala/stm/global/Atomic.scala
@@ -0,0 +1,41 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.stm.global
+
+/**
+ * Java-friendly atomic blocks.
+ *
+ * Example usage (in Java):
+ *
+ *
+ * import se.scalablesolutions.akka.stm.*;
+ * import se.scalablesolutions.akka.stm.global.Atomic;
+ *
+ * final Ref ref = new Ref(0);
+ *
+ * new Atomic() {
+ * public Object atomically() {
+ * return ref.set(1);
+ * }
+ * }.execute();
+ *
+ * // To configure transactions pass a TransactionFactory
+ *
+ * TransactionFactory txFactory = new TransactionFactoryBuilder()
+ * .setReadonly(true)
+ * .build();
+ *
+ * Integer value = new Atomic(txFactory) {
+ * public Integer atomically() {
+ * return ref.get();
+ * }
+ * }.execute();
+ *
+ */
+abstract class Atomic[T](factory: TransactionFactory) {
+ def this() = this(DefaultGlobalTransactionFactory)
+ def atomically: T
+ def execute: T = atomic(factory)(atomically)
+}
diff --git a/akka-core/src/main/scala/stm/global/GlobalStm.scala b/akka-core/src/main/scala/stm/global/GlobalStm.scala
new file mode 100644
index 0000000000..1fd53ffe51
--- /dev/null
+++ b/akka-core/src/main/scala/stm/global/GlobalStm.scala
@@ -0,0 +1,53 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.stm
+
+import se.scalablesolutions.akka.util.Logging
+
+import org.multiverse.api.{Transaction => MultiverseTransaction}
+import org.multiverse.templates.TransactionalCallable
+
+/**
+ * Global transaction management, global in the context of multiple threads.
+ * Use this if you need to have one transaction span multiple threads (or Actors).
+ *
+ * Example of atomic transaction management using the atomic block:
+ *
+ *
+ * import se.scalablesolutions.akka.stm.global._
+ *
+ * atomic {
+ * // do something within a transaction
+ * }
+ *
+ */
+class GlobalStm extends TransactionManagement with Logging {
+
+ val DefaultGlobalTransactionConfig = TransactionConfig()
+ val DefaultGlobalTransactionFactory = TransactionFactory(DefaultGlobalTransactionConfig, "DefaultGlobalTransaction")
+
+ def atomic[T](body: => T)(implicit factory: TransactionFactory = DefaultGlobalTransactionFactory): T = atomic(factory)(body)
+
+ def atomic[T](factory: TransactionFactory)(body: => T): T = {
+ factory.boilerplate.execute(new TransactionalCallable[T]() {
+ def call(mtx: MultiverseTransaction): T = {
+ if (!isTransactionSetInScope) createNewTransactionSet
+ factory.addHooks
+ val result = body
+ val txSet = getTransactionSetInScope
+ log.trace("Committing global transaction [" + mtx + "]\n\tand joining transaction set [" + txSet + "]")
+ try {
+ txSet.tryJoinCommit(
+ mtx,
+ TransactionConfig.DefaultTimeout.length,
+ TransactionConfig.DefaultTimeout.unit)
+ // Need to catch IllegalStateException until we have fix in Multiverse, since it throws it by mistake
+ } catch { case e: IllegalStateException => {} }
+ result
+ }
+ })
+ }
+}
+
diff --git a/akka-core/src/main/scala/stm/global/package.scala b/akka-core/src/main/scala/stm/global/package.scala
new file mode 100644
index 0000000000..9b8a1b289e
--- /dev/null
+++ b/akka-core/src/main/scala/stm/global/package.scala
@@ -0,0 +1,10 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.stm
+
+/**
+ * For easily importing global STM.
+ */
+package object global extends GlobalStm with StmUtil with StmCommon
diff --git a/akka-core/src/main/scala/stm/local/Atomic.scala b/akka-core/src/main/scala/stm/local/Atomic.scala
new file mode 100644
index 0000000000..c06f99ba99
--- /dev/null
+++ b/akka-core/src/main/scala/stm/local/Atomic.scala
@@ -0,0 +1,41 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.stm.local
+
+/**
+ * Java-friendly atomic blocks.
+ *
+ * Example usage (in Java):
+ *
+ *
+ * import se.scalablesolutions.akka.stm.*;
+ * import se.scalablesolutions.akka.stm.local.Atomic;
+ *
+ * final Ref ref = new Ref(0);
+ *
+ * new Atomic() {
+ * public Object atomically() {
+ * return ref.set(1);
+ * }
+ * }.execute();
+ *
+ * // To configure transactions pass a TransactionFactory
+ *
+ * TransactionFactory txFactory = new TransactionFactoryBuilder()
+ * .setReadonly(true)
+ * .build();
+ *
+ * Integer value = new Atomic(txFactory) {
+ * public Integer atomically() {
+ * return ref.get();
+ * }
+ * }.execute();
+ *
+ */
+abstract class Atomic[T](factory: TransactionFactory) {
+ def this() = this(DefaultLocalTransactionFactory)
+ def atomically: T
+ def execute: T = atomic(factory)(atomically)
+}
diff --git a/akka-core/src/main/scala/stm/local/LocalStm.scala b/akka-core/src/main/scala/stm/local/LocalStm.scala
new file mode 100644
index 0000000000..477027aab2
--- /dev/null
+++ b/akka-core/src/main/scala/stm/local/LocalStm.scala
@@ -0,0 +1,44 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.stm
+
+import se.scalablesolutions.akka.util.Logging
+
+import org.multiverse.api.{Transaction => MultiverseTransaction}
+import org.multiverse.templates.TransactionalCallable
+
+/**
+ * Local transaction management, local in the context of threads.
+ * Use this if you do not need to have one transaction span
+ * multiple threads (or Actors).
+ *
+ * Example of atomic transaction management using the atomic block.
+ *
+ *
+ * import se.scalablesolutions.akka.stm.local._
+ *
+ * atomic {
+ * // do something within a transaction
+ * }
+ *
+ */
+class LocalStm extends TransactionManagement with Logging {
+
+ val DefaultLocalTransactionConfig = TransactionConfig()
+ val DefaultLocalTransactionFactory = TransactionFactory(DefaultLocalTransactionConfig, "DefaultLocalTransaction")
+
+ def atomic[T](body: => T)(implicit factory: TransactionFactory = DefaultLocalTransactionFactory): T = atomic(factory)(body)
+
+ def atomic[T](factory: TransactionFactory)(body: => T): T = {
+ factory.boilerplate.execute(new TransactionalCallable[T]() {
+ def call(mtx: MultiverseTransaction): T = {
+ factory.addHooks
+ val result = body
+ log.trace("Committing local transaction [" + mtx + "]")
+ result
+ }
+ })
+ }
+}
diff --git a/akka-core/src/main/scala/stm/local/package.scala b/akka-core/src/main/scala/stm/local/package.scala
new file mode 100644
index 0000000000..406d4880f6
--- /dev/null
+++ b/akka-core/src/main/scala/stm/local/package.scala
@@ -0,0 +1,10 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.stm
+
+/**
+ * For easily importing local STM.
+ */
+package object local extends LocalStm with StmUtil with StmCommon
diff --git a/akka-core/src/main/scala/stm/packages.scala b/akka-core/src/main/scala/stm/transactional.scala
similarity index 68%
rename from akka-core/src/main/scala/stm/packages.scala
rename to akka-core/src/main/scala/stm/transactional.scala
index cbb3ad4804..e00c7ef8e9 100644
--- a/akka-core/src/main/scala/stm/packages.scala
+++ b/akka-core/src/main/scala/stm/transactional.scala
@@ -5,28 +5,7 @@
package se.scalablesolutions.akka.stm
/**
- * For importing 'local' STM.
- */
-package object local extends LocalStm with StmUtil with StmCommon
-
-/**
- * For importing 'global' STM.
- */
-package object global extends GlobalStm with StmUtil with StmCommon
-
-trait StmCommon {
- type TransactionConfig = se.scalablesolutions.akka.stm.TransactionConfig
- val TransactionConfig = se.scalablesolutions.akka.stm.TransactionConfig
-
- type TransactionFactory = se.scalablesolutions.akka.stm.TransactionFactory
- val TransactionFactory = se.scalablesolutions.akka.stm.TransactionFactory
-
- type Ref[T] = se.scalablesolutions.akka.stm.Ref[T]
- val Ref = se.scalablesolutions.akka.stm.Ref
-}
-
-/**
- * For importing the transactional data structures, including the primitive refs
+ * For importing the transactional datastructures, including the primitive refs
* and transactional data structures from Multiverse.
*/
package object transactional {
diff --git a/akka-core/src/main/scala/util/AkkaException.scala b/akka-core/src/main/scala/util/AkkaException.scala
new file mode 100644
index 0000000000..ca8284ced5
--- /dev/null
+++ b/akka-core/src/main/scala/util/AkkaException.scala
@@ -0,0 +1,50 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka
+
+import se.scalablesolutions.akka.util.{UUID, Logging}
+
+import java.io.{StringWriter, PrintWriter}
+import java.net.{InetAddress, UnknownHostException}
+
+/**
+ * Akka base Exception. Each Exception gets:
+ *
+ * - a UUID for tracking purposes
+ * - a message including exception name, uuid, original message and the stacktrace
+ * - a method 'log' that will log the exception once and only once
+ *
+ *
+ * @author Jonas Bonér
+ */
+@serializable abstract class AkkaException(message: String) extends RuntimeException(message) {
+ @volatile private var isLogged = false
+ val exceptionName = getClass.getName
+
+ val uuid = String.format("%s_%s", AkkaException.hostname, UUID.newUuid.toString)
+
+ override val toString =
+ String.format("%s\n\t[%s]\n\t%s\n\t%s", exceptionName, uuid, message, stackTrace)
+
+ val stackTrace = {
+ val sw = new StringWriter
+ val pw = new PrintWriter(sw)
+ printStackTrace(pw)
+ sw.toString
+ }
+
+ def log = if (!isLogged) {
+ isLogged = true
+ AkkaException.log.error(toString)
+ }
+}
+
+object AkkaException extends Logging {
+ val hostname = try {
+ InetAddress.getLocalHost.getHostName
+ } catch {
+ case e: UnknownHostException => "unknown"
+ }
+}
diff --git a/akka-core/src/main/scala/util/Duration.scala b/akka-core/src/main/scala/util/Duration.scala
index f49e1ae04b..0dee2fc139 100644
--- a/akka-core/src/main/scala/util/Duration.scala
+++ b/akka-core/src/main/scala/util/Duration.scala
@@ -20,8 +20,21 @@ object Duration {
/**
* Utility for working with java.util.concurrent.TimeUnit durations.
+ *
*
- * Example:
+ * Examples of usage from Java:
+ *
+ * import se.scalablesolutions.akka.util.Duration;
+ * import java.util.concurrent.TimeUnit;
+ *
+ * Duration duration = new Duration(100, TimeUnit.MILLISECONDS);
+ * Duration duration = new Duration(5, "seconds");
+ *
+ * duration.toNanos();
+ *
+ *
+ *
+ * Examples of usage from Scala:
*
* import se.scalablesolutions.akka.util.Duration
* import java.util.concurrent.TimeUnit
@@ -31,6 +44,7 @@ object Duration {
*
* duration.toNanos
*
+ *
*
* Implicits are also provided for Int and Long. Example usage:
*
@@ -40,6 +54,7 @@ object Duration {
*
*/
class Duration(val length: Long, val unit: TimeUnit) {
+ def this(length: Long, unit: String) = this(length, Duration.timeUnit(unit))
def toNanos = unit.toNanos(length)
def toMicros = unit.toMicros(length)
def toMillis = unit.toMillis(length)
diff --git a/akka-core/src/main/scala/util/Helpers.scala b/akka-core/src/main/scala/util/Helpers.scala
index ccbd896610..eab9e1981d 100644
--- a/akka-core/src/main/scala/util/Helpers.scala
+++ b/akka-core/src/main/scala/util/Helpers.scala
@@ -6,8 +6,6 @@ package se.scalablesolutions.akka.util
import java.security.MessageDigest
-class SystemFailure(cause: Throwable) extends RuntimeException(cause)
-
/**
* @author Jonas Bonér
*/
diff --git a/akka-core/src/main/scala/util/ListenerManagement.scala b/akka-core/src/main/scala/util/ListenerManagement.scala
index 7316beba64..cfcb5ac2b6 100644
--- a/akka-core/src/main/scala/util/ListenerManagement.scala
+++ b/akka-core/src/main/scala/util/ListenerManagement.scala
@@ -4,7 +4,7 @@
package se.scalablesolutions.akka.util
-import java.util.concurrent.CopyOnWriteArrayList
+import java.util.concurrent.ConcurrentSkipListSet
import se.scalablesolutions.akka.actor.ActorRef
@@ -15,7 +15,7 @@ import se.scalablesolutions.akka.actor.ActorRef
*/
trait ListenerManagement extends Logging {
- private val listeners = new CopyOnWriteArrayList[ActorRef]
+ private val listeners = new ConcurrentSkipListSet[ActorRef]
/**
* Adds the listener this this registry's listener list.
@@ -23,7 +23,7 @@ trait ListenerManagement extends Logging {
*/
def addListener(listener: ActorRef) = {
listener.start
- listeners.add(listener)
+ listeners add listener
}
/**
@@ -31,8 +31,8 @@ trait ListenerManagement extends Logging {
* The listener is stopped by this method.
*/
def removeListener(listener: ActorRef) = {
+ listeners remove listener
listener.stop
- listeners.remove(listener)
}
/**
diff --git a/akka-core/src/main/scala/util/Logging.scala b/akka-core/src/main/scala/util/Logging.scala
index 8c6ea0fb99..1fed0d017a 100644
--- a/akka-core/src/main/scala/util/Logging.scala
+++ b/akka-core/src/main/scala/util/Logging.scala
@@ -4,7 +4,7 @@
package se.scalablesolutions.akka.util
-import net.lag.logging.Logger
+import org.slf4j.{Logger => SLFLogger,LoggerFactory => SLFLoggerFactory}
import java.io.StringWriter
import java.io.PrintWriter
@@ -17,55 +17,146 @@ import java.net.UnknownHostException
* @author Jonas Bonér
*/
trait Logging {
- @sjson.json.JSONProperty(ignore = true) @transient lazy val log = Logger.get(this.getClass.getName)
+ @sjson.json.JSONProperty(ignore = true) @transient lazy val log = Logger(this.getClass.getName)
}
/**
- * LoggableException is a subclass of Exception and can be used as the base exception
- * for application specific exceptions.
- *
- * It keeps track of the exception is logged or not and also stores the unique id,
- * so that it can be carried all along to the client tier and displayed to the end user.
- * The end user can call up the customer support using this number.
+ * Scala SLF4J wrapper
*
- * @author Jonas Bonér
+ * Example:
+ *
+ * class Foo extends Logging {
+ * log.info("My foo is %s","alive")
+ * log.error(new Exception(),"My foo is %s","broken")
+ * }
+ *
+ *
+ * The logger uses String.format:
+ * http://download-llnw.oracle.com/javase/6/docs/api/java/lang/String.html#format(java.lang.String,%20java.lang.Object...)
*/
- // FIXME make use of LoggableException
-class LoggableException extends Exception with Logging {
- private val uniqueId = getExceptionID
- private var originalException: Option[Exception] = None
- private var isLogged = false
+class Logger(val logger: SLFLogger) {
+ def name = logger.getName
- def this(baseException: Exception) = {
- this()
- originalException = Some(baseException)
+ def trace_? = logger.isTraceEnabled
+ def debug_? = logger.isDebugEnabled
+ def info_? = logger.isInfoEnabled
+ def warning_? = logger.isWarnEnabled
+ def error_? = logger.isErrorEnabled
+
+ //Trace
+ def trace(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
+ trace(t,message(fmt,arg,argN:_*))
}
- def logException = synchronized {
- if (!isLogged) {
- originalException match {
- case Some(e) => log.error("Logged Exception [%s] %s", uniqueId, getStackTraceAsString(e))
- case None => log.error("Logged Exception [%s] %s", uniqueId, getStackTraceAsString(this))
- }
- isLogged = true
- }
+ def trace(t: Throwable, msg: => String) {
+ if (trace_?) logger.trace(msg,t)
}
- private def getExceptionID: String = {
- val hostname: String = try {
- InetAddress.getLocalHost.getHostName
- } catch {
- case e: UnknownHostException =>
- log.error("Could not get hostname to generate loggable exception")
- "N/A"
- }
- hostname + "_" + System.currentTimeMillis
+ def trace(fmt: => String, arg: Any, argN: Any*) {
+ trace(message(fmt,arg,argN:_*))
}
- private def getStackTraceAsString(exception: Throwable): String = {
- val sw = new StringWriter
- val pw = new PrintWriter(sw)
- exception.printStackTrace(pw)
- sw.toString
+ def trace(msg: => String) {
+ if (trace_?) logger trace msg
+ }
+
+ //Debug
+ def debug(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
+ debug(t,message(fmt,arg,argN:_*))
+ }
+
+ def debug(t: Throwable, msg: => String) {
+ if (debug_?) logger.debug(msg,t)
+ }
+
+ def debug(fmt: => String, arg: Any, argN: Any*) {
+ debug(message(fmt,arg,argN:_*))
+ }
+
+ def debug(msg: => String) {
+ if (debug_?) logger debug msg
+ }
+
+ //Info
+ def info(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
+ info(t,message(fmt,arg,argN:_*))
+ }
+
+ def info(t: Throwable, msg: => String) {
+ if (info_?) logger.info(msg,t)
+ }
+
+ def info(fmt: => String, arg: Any, argN: Any*) {
+ info(message(fmt,arg,argN:_*))
+ }
+
+ def info(msg: => String) {
+ if (info_?) logger info msg
+ }
+
+ //Warning
+ def warning(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
+ warning(t,message(fmt,arg,argN:_*))
+ }
+
+ def warning(t: Throwable, msg: => String) {
+ if (warning_?) logger.warn(msg,t)
+ }
+
+ def warning(fmt: => String, arg: Any, argN: Any*) {
+ warning(message(fmt,arg,argN:_*))
+ }
+
+ def warning(msg: => String) {
+ if (warning_?) logger warn msg
+ }
+
+ //Error
+ def error(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
+ error(t,message(fmt,arg,argN:_*))
+ }
+
+ def error(t: Throwable, msg: => String) {
+ if (error_?) logger.error(msg,t)
+ }
+
+ def error(fmt: => String, arg: Any, argN: Any*) {
+ error(message(fmt,arg,argN:_*))
+ }
+
+ def error(msg: => String) {
+ if (error_?) logger error msg
+ }
+
+ protected def message(fmt: String, arg: Any, argN: Any*) : String = {
+ if ((argN eq null) || argN.isEmpty) fmt.format(arg)
+ else fmt.format((arg +: argN):_*)
}
}
+
+/**
+ * Logger factory
+ *
+ * ex.
+ *
+ * val logger = Logger("my.cool.logger")
+ * val logger = Logger(classOf[Banana])
+ * val rootLogger = Logger.root
+ *
+ */
+object Logger {
+
+ /* Uncomment to be able to debug what logging configuration will be used
+ {
+ import org.slf4j.LoggerFactory
+ import ch.qos.logback.classic.LoggerContext
+ import ch.qos.logback.core.util.StatusPrinter
+
+ // print logback's internal status
+ StatusPrinter.print(LoggerFactory.getILoggerFactory.asInstanceOf[LoggerContext])
+ }*/
+
+ def apply(logger: String) : Logger = new Logger(SLFLoggerFactory getLogger logger)
+ def apply(clazz: Class[_]) : Logger = apply(clazz.getName)
+ def root : Logger = apply(SLFLogger.ROOT_LOGGER_NAME)
+}
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/ActiveObjectFailer.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/ActiveObjectFailer.java
deleted file mode 100644
index 6e30a1a971..0000000000
--- a/akka-core/src/test/java/se/scalablesolutions/akka/actor/ActiveObjectFailer.java
+++ /dev/null
@@ -1,7 +0,0 @@
-package se.scalablesolutions.akka.actor;
-
-public class ActiveObjectFailer implements java.io.Serializable {
- public int fail() {
- throw new RuntimeException("expected");
- }
-}
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java
index 09b50a7347..9cb41a85cf 100644
--- a/akka-core/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/BarImpl.java
@@ -1,13 +1,16 @@
package se.scalablesolutions.akka.actor;
import com.google.inject.Inject;
+import se.scalablesolutions.akka.actor.*;
-public class BarImpl implements Bar {
+public class BarImpl extends TypedActor implements Bar {
@Inject
private Ext ext;
+
public Ext getExt() {
return ext;
}
+
public void bar(String msg) {
}
}
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/Foo.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/Foo.java
index 87eb809aba..4cc5b977dc 100644
--- a/akka-core/src/test/java/se/scalablesolutions/akka/actor/Foo.java
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/Foo.java
@@ -1,34 +1,14 @@
package se.scalablesolutions.akka.actor;
-import com.google.inject.Inject;
-
-public class Foo extends se.scalablesolutions.akka.serialization.Serializable.JavaJSON {
- @Inject
- private Bar bar;
- public Foo body() { return this; }
- public Bar getBar() {
- return bar;
- }
- public String foo(String msg) {
- return msg + "return_foo ";
- }
- public void bar(String msg) {
- bar.bar(msg);
- }
- public String longRunning() {
- try {
- Thread.sleep(1200);
- } catch (InterruptedException e) {
- }
- return "test";
- }
- public String throwsException() {
- if (true) throw new RuntimeException("Expected exception; to test fault-tolerance");
- return "test";
- }
+public interface Foo {
+ public Foo body();
+ public Bar getBar();
- public int $tag() throws java.rmi.RemoteException
- {
- return 0;
- }
+ public String foo(String msg);
+ public void bar(String msg);
+
+ public String longRunning();
+ public String throwsException();
+
+ public int $tag() throws java.rmi.RemoteException;
}
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java
new file mode 100644
index 0000000000..dc6aba481c
--- /dev/null
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/FooImpl.java
@@ -0,0 +1,40 @@
+package se.scalablesolutions.akka.actor;
+
+import com.google.inject.Inject;
+import se.scalablesolutions.akka.actor.*;
+
+public class FooImpl extends TypedActor implements Foo {
+ @Inject
+ private Bar bar;
+
+ public Foo body() { return this; }
+
+ public Bar getBar() {
+ return bar;
+ }
+
+ public String foo(String msg) {
+ return msg + "return_foo ";
+ }
+
+ public void bar(String msg) {
+ bar.bar(msg);
+ }
+
+ public String longRunning() {
+ try {
+ Thread.sleep(1200);
+ } catch (InterruptedException e) {
+ }
+ return "test";
+ }
+
+ public String throwsException() {
+ if (true) throw new RuntimeException("Expected exception; to test fault-tolerance");
+ return "test";
+ }
+
+ public int $tag() throws java.rmi.RemoteException {
+ return 0;
+ }
+}
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java
new file mode 100644
index 0000000000..ee7998f69a
--- /dev/null
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActor.java
@@ -0,0 +1,12 @@
+package se.scalablesolutions.akka.actor;
+
+public interface NestedTransactionalTypedActor {
+ public String getMapState(String key);
+ public String getVectorState();
+ public String getRefState();
+ public void setMapState(String key, String msg);
+ public void setVectorState(String msg);
+ public void setRefState(String msg);
+ public void success(String key, String msg);
+ public String failure(String key, String msg, TypedActorFailer failer);
+}
\ No newline at end of file
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalActiveObject.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java
similarity index 59%
rename from akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalActiveObject.java
rename to akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java
index af6bb8245c..5b7eab615e 100644
--- a/akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalActiveObject.java
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java
@@ -1,17 +1,15 @@
package se.scalablesolutions.akka.actor;
-import se.scalablesolutions.akka.actor.annotation.transactionrequired;
-import se.scalablesolutions.akka.actor.annotation.inittransactionalstate;
+import se.scalablesolutions.akka.actor.*;
import se.scalablesolutions.akka.stm.*;
-@transactionrequired
-public class NestedTransactionalActiveObject {
+public class NestedTransactionalTypedActorImpl extends TypedTransactor implements NestedTransactionalTypedActor {
private TransactionalMap mapState;
private TransactionalVector vectorState;
private Ref refState;
private boolean isInitialized = false;
- @inittransactionalstate
+ @Override
public void init() {
if (!isInitialized) {
mapState = new TransactionalMap();
@@ -25,62 +23,37 @@ public class NestedTransactionalActiveObject {
return (String) mapState.get(key).get();
}
-
public String getVectorState() {
return (String) vectorState.last();
}
-
public String getRefState() {
- return (String) refState.get().get();
+ return (String) refState.get();
}
-
public void setMapState(String key, String msg) {
mapState.put(key, msg);
}
-
public void setVectorState(String msg) {
vectorState.add(msg);
}
-
public void setRefState(String msg) {
refState.swap(msg);
}
-
public void success(String key, String msg) {
mapState.put(key, msg);
vectorState.add(msg);
refState.swap(msg);
}
-
- public String failure(String key, String msg, ActiveObjectFailer failer) {
+ public String failure(String key, String msg, TypedActorFailer failer) {
mapState.put(key, msg);
vectorState.add(msg);
refState.swap(msg);
failer.fail();
return msg;
}
-
-
- public void thisMethodHangs(String key, String msg, ActiveObjectFailer failer) {
- setMapState(key, msg);
- }
-
- /*
- public void clashOk(String key, String msg, InMemClasher clasher) {
- mapState.put(key, msg);
- clasher.clash();
- }
-
- public void clashNotOk(String key, String msg, InMemClasher clasher) {
- mapState.put(key, msg);
- clasher.clash();
- this.success("clash", "clash");
- }
- */
}
\ No newline at end of file
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java
new file mode 100644
index 0000000000..03df632582
--- /dev/null
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOne.java
@@ -0,0 +1,6 @@
+package se.scalablesolutions.akka.actor;
+
+public interface RemoteTypedActorOne {
+ public String requestReply(String s) throws Exception;
+ public void oneWay() throws Exception;
+}
\ No newline at end of file
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java
new file mode 100644
index 0000000000..0744652181
--- /dev/null
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorOneImpl.java
@@ -0,0 +1,29 @@
+package se.scalablesolutions.akka.actor.remote;
+
+import se.scalablesolutions.akka.actor.*;
+
+import java.util.concurrent.CountDownLatch;
+
+public class RemoteTypedActorOneImpl extends TypedActor implements RemoteTypedActorOne {
+
+ public static CountDownLatch latch = new CountDownLatch(1);
+
+ public String requestReply(String s) throws Exception {
+ if (s.equals("ping")) {
+ RemoteTypedActorLog.messageLog().put("ping");
+ return "pong";
+ } else if (s.equals("die")) {
+ throw new RuntimeException("Expected exception; to test fault-tolerance");
+ } else return null;
+ }
+
+ public void oneWay() throws Exception {
+ RemoteTypedActorLog.oneWayLog().put("oneway");
+ }
+
+ @Override
+ public void preRestart(Throwable e) {
+ try { RemoteTypedActorLog.messageLog().put(e.getMessage()); } catch(Exception ex) {}
+ latch.countDown();
+ }
+}
\ No newline at end of file
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java
new file mode 100644
index 0000000000..58f294c6cd
--- /dev/null
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwo.java
@@ -0,0 +1,6 @@
+package se.scalablesolutions.akka.actor;
+
+public interface RemoteTypedActorTwo {
+ public String requestReply(String s) throws Exception;
+ public void oneWay() throws Exception;
+}
\ No newline at end of file
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java
new file mode 100644
index 0000000000..36bb055ef8
--- /dev/null
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/RemoteTypedActorTwoImpl.java
@@ -0,0 +1,29 @@
+package se.scalablesolutions.akka.actor.remote;
+
+import se.scalablesolutions.akka.actor.*;
+
+import java.util.concurrent.CountDownLatch;
+
+public class RemoteTypedActorTwoImpl extends TypedActor implements RemoteTypedActorTwo {
+
+ public static CountDownLatch latch = new CountDownLatch(1);
+
+ public String requestReply(String s) throws Exception {
+ if (s.equals("ping")) {
+ RemoteTypedActorLog.messageLog().put("ping");
+ return "pong";
+ } else if (s.equals("die")) {
+ throw new RuntimeException("Expected exception; to test fault-tolerance");
+ } else return null;
+ }
+
+ public void oneWay() throws Exception {
+ RemoteTypedActorLog.oneWayLog().put("oneway");
+ }
+
+ @Override
+ public void preRestart(Throwable e) {
+ try { RemoteTypedActorLog.messageLog().put(e.getMessage()); } catch(Exception ex) {}
+ latch.countDown();
+ }
+}
\ No newline at end of file
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java
index 50f3e43221..ae47276ba6 100644
--- a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojo.java
@@ -2,36 +2,7 @@ package se.scalablesolutions.akka.actor;
import java.util.concurrent.CountDownLatch;
-public class SamplePojo {
-
- private CountDownLatch latch;
-
- public boolean _pre = false;
- public boolean _post = false;
- public boolean _down = false;
-
- public CountDownLatch newCountdownLatch(int count) {
- latch = new CountDownLatch(count);
- return latch;
- }
-
- public String fail() {
- throw new RuntimeException("expected");
- }
-
- public void pre() {
- _pre = true;
- latch.countDown();
- }
-
- public void post() {
- _post = true;
- latch.countDown();
- }
-
- public void down() {
- _down = true;
- latch.countDown();
- }
-
-}
+public interface SamplePojo {
+ public String greet(String s);
+ public String fail();
+}
\ No newline at end of file
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojoAnnotated.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojoAnnotated.java
deleted file mode 100644
index 8bf4ba36d3..0000000000
--- a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojoAnnotated.java
+++ /dev/null
@@ -1,52 +0,0 @@
-package se.scalablesolutions.akka.actor;
-
-import se.scalablesolutions.akka.actor.annotation.postrestart;
-import se.scalablesolutions.akka.actor.annotation.prerestart;
-import se.scalablesolutions.akka.actor.annotation.shutdown;
-
-import java.util.concurrent.CountDownLatch;
-
-public class SamplePojoAnnotated {
-
- private CountDownLatch latch;
-
- public boolean _pre = false;
- public boolean _post = false;
- public boolean _down = false;
-
- public SamplePojoAnnotated() {
- latch = new CountDownLatch(1);
- }
-
- public CountDownLatch newCountdownLatch(int count) {
- latch = new CountDownLatch(count);
- return latch;
- }
-
- public String greet(String s) {
- return "hello " + s;
- }
-
- public String fail() {
- throw new RuntimeException("expected");
- }
-
- @prerestart
- public void pre() {
- _pre = true;
- latch.countDown();
- }
-
- @postrestart
- public void post() {
- _post = true;
- latch.countDown();
- }
-
- @shutdown
- public void down() {
- _down = true;
- latch.countDown();
- }
-
-}
\ No newline at end of file
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java
new file mode 100644
index 0000000000..d57232b629
--- /dev/null
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java
@@ -0,0 +1,45 @@
+package se.scalablesolutions.akka.actor;
+
+import se.scalablesolutions.akka.actor.*;
+
+import java.util.concurrent.CountDownLatch;
+
+public class SamplePojoImpl extends TypedActor implements SamplePojo {
+
+ public static CountDownLatch latch = new CountDownLatch(1);
+
+ public static boolean _pre = false;
+ public static boolean _post = false;
+ public static boolean _down = false;
+ public static void reset() {
+ _pre = false;
+ _post = false;
+ _down = false;
+ }
+
+ public String greet(String s) {
+ return "hello " + s;
+ }
+
+ public String fail() {
+ throw new RuntimeException("expected");
+ }
+
+ @Override
+ public void preRestart(Throwable e) {
+ _pre = true;
+ latch.countDown();
+ }
+
+ @Override
+ public void postRestart(Throwable e) {
+ _post = true;
+ latch.countDown();
+ }
+
+ @Override
+ public void shutdown() {
+ _down = true;
+ latch.countDown();
+ }
+}
\ No newline at end of file
diff --git a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java b/akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java
index 31f22c217f..d3a18abbd9 100644
--- a/akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java
+++ b/akka-core/src/test/java/se/scalablesolutions/akka/actor/SimpleJavaPojo.java
@@ -1,48 +1,14 @@
package se.scalablesolutions.akka.actor;
-import se.scalablesolutions.akka.actor.annotation.prerestart;
-import se.scalablesolutions.akka.actor.annotation.postrestart;
-import se.scalablesolutions.akka.actor.ActiveObjectContext;
+import se.scalablesolutions.akka.dispatch.Future;
import se.scalablesolutions.akka.dispatch.CompletableFuture;
+import se.scalablesolutions.akka.dispatch.Future;
-public class SimpleJavaPojo {
-
- ActiveObjectContext context;
-
- public boolean pre = false;
- public boolean post = false;
-
- private String name;
-
- public Object getSender() {
- return context.getSender();
- }
-
- public CompletableFuture