+ * 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[T <: Actor : Manifest]: ActorRef = actorOf(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]])
+
+ /**
+ * 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(() => {
+ import ReflectiveAccess.{ createInstance, noParams, noArgs }
+ createInstance[Actor](clazz.asInstanceOf[Class[_]], noParams, noArgs).getOrElse(
+ throw new ActorInitializationException(
+ "Could not instantiate Actor" +
+ "\nMake sure Actor is NOT defined inside a class/trait," +
+ "\nif so put it outside the class/trait, f.e. in a companion object," +
+ "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'."))
+ })
+
+ /**
+ * 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.
+ *
+ * This function should NOT be used for remote actors.
+ *
+ * import Actor._
+ * val actor = actorOf(new MyActor)
+ * actor.start
+ * actor ! message
+ * actor.stop
+ *
+ * You can create and start the actor in one statement like this:
+ *
+ * val actor = actorOf(new MyActor).start
+ *
+ */
+ def actorOf(factory: => Actor): ActorRef = new LocalActorRef(() => factory)
+
+ /**
+ * Use to spawn out a block of code in an event-driven actor. Will shut actor down when
+ * the block has been executed.
+ *
+ * NOTE: If used from within an Actor then has to be qualified with 'Actor.spawn' since
+ * there is a method 'spawn[ActorType]' in the Actor trait already.
+ * Example:
+ *
+ */
+ def spawn(body: => Unit)(implicit dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher): Unit = {
+ case object Spawn
+ actorOf(new Actor() {
+ self.dispatcher = dispatcher
+ def receive = {
+ case Spawn => try { body } finally { self.stop }
+ }
+ }).start ! Spawn
+ }
+
+ /**
+ * Implicitly converts the given Option[Any] to a AnyOptionAsTypedOption which offers the method as[T]
+ * to convert an Option[Any] to an Option[T].
+ */
+ implicit def toAnyOptionAsTypedOption(anyOption: Option[Any]) = new AnyOptionAsTypedOption(anyOption)
+}
+
+/**
+ * Actor base trait that should be extended by or mixed to create an Actor with the semantics of the 'Actor Model':
+ * http://en.wikipedia.org/wiki/Actor_model
+ *
+ * An actor has a well-defined (non-cyclic) life-cycle.
+ *
+ * => NEW (newly created actor) - can't receive messages (yet)
+ * => STARTED (when 'start' is invoked) - can receive messages
+ * => SHUT DOWN (when 'exit' is invoked) - can't do anything
+ *
+ *
+ *
+ * The Actor's API is available in the 'self' member variable.
+ *
+ *
+ * Here you find functions like:
+ * - !, !!, !!! and forward
+ * - link, unlink, startLink, spawnLink etc
+ * - makeRemote etc.
+ * - start, stop
+ * - etc.
+ *
+ *
+ * Here you also find fields like
+ * - dispatcher = ...
+ * - id = ...
+ * - lifeCycle = ...
+ * - faultHandler = ...
+ * - trapExit = ...
+ * - etc.
+ *
+ *
+ * This means that to use them you have to prefix them with 'self', like this: self ! Message
+ *
+ * However, for convenience you can import these functions and fields like below, which will allow you do
+ * drop the 'self' prefix:
+ *
+ *
+ *
+ * The Actor trait also has a 'log' member field that can be used for logging within the Actor.
+ *
+ * @author Jonas Bonér
+ */
+trait Actor extends Logging {
+
+ /**
+ * Type alias because traits cannot have companion objects.
+ */
+ type Receive = Actor.Receive
+
+ /*
+ * 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." +
+ "\n\tEither use:" +
+ "\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 => .. } }'")
+ val ref = optRef.asInstanceOf[Some[ActorRef]].get
+ ref.id = getClass.getName //FIXME: Is this needed?
+ optRef.asInstanceOf[Some[ActorRef]]
+ }
+
+ /*
+ * 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 '!!!').
+ */
+ implicit def optionSelf: Option[ActorRef] = someSelf
+
+ /**
+ * The 'self' field holds the ActorRef for this actor.
+ *
+ * Can be used to send messages to itself:
+ *
+ * self ! message
+ *
+ * Here you also find most of the Actor API.
+ *
+ * For example fields like:
+ *
+ */
+ @transient val self: ScalaActorRef = someSelf.get
+
+ /**
+ * User overridable callback/setting.
+ *
+ * Partial function implementing the actor logic.
+ * To be implemented by concrete actor class.
+ *
+ * Example code:
+ *
+ */
+ protected def receive: Receive
+
+ /**
+ * User overridable callback.
+ *
+ * Is called when an Actor is started by invoking 'actor.start'.
+ */
+ def preStart {}
+
+ /**
+ * User overridable callback.
+ *
+ * Is called when 'actor.stop' is invoked.
+ */
+ def postStop {}
+
+ /**
+ * User overridable callback.
+ *
+ * Is called on a crashed Actor right BEFORE it is restarted to allow clean up of resources before Actor is terminated.
+ */
+ def preRestart(reason: Throwable) {}
+
+ /**
+ * User overridable callback.
+ *
+ * Is called right AFTER restart on the newly created Actor to allow reinitialization after an Actor crash.
+ */
+ def postRestart(reason: Throwable) {}
+
+ /**
+ * User overridable callback.
+ *
+ * Is called when a message isn't handled by the current behavior of the actor
+ * by default it throws an UnhandledMessageException
+ */
+ def unhandled(msg: Any){
+ throw new UnhandledMessageException(msg,self)
+ }
+
+ /**
+ * Is the actor able to handle the message passed in as arguments?
+ */
+ def isDefinedAt(message: Any): Boolean = processingBehavior.isDefinedAt(message)
+
+ /**
+ * Changes tha Actor's behavior to become the new 'Receive' (PartialFunction[Any, Unit]) handler.
+ * Puts the behavior on top of the hotswap stack.
+ * If "discardOld" is true, an unbecome will be issued prior to pushing the new behavior to the stack
+ */
+ def become(behavior: Receive, discardOld: Boolean = false) {
+ if (discardOld)
+ unbecome
+
+ self.hotswap = self.hotswap.push(behavior)
+ }
+
+ /**
+ * Reverts the Actor behavior to the previous one in the hotswap stack.
+ */
+ def unbecome: Unit = if (!self.hotswap.isEmpty) self.hotswap = self.hotswap.pop
+
+ // =========================================
+ // ==== INTERNAL IMPLEMENTATION DETAILS ====
+ // =========================================
+
+ private[akka] def apply(msg: Any) = fullBehavior(msg)
+
+ /*Processingbehavior and fullBehavior are duplicates so make sure changes are done to both */
+ private lazy val processingBehavior: Receive = {
+ lazy val defaultBehavior = receive
+ val actorBehavior: Receive = {
+ case HotSwap(code) => become(code(self))
+ case RevertHotSwap => unbecome
+ 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.isEmpty &&
+ self.hotswap.head.isDefinedAt(msg) => self.hotswap.head.apply(msg)
+ case msg if self.hotswap.isEmpty &&
+ defaultBehavior.isDefinedAt(msg) => defaultBehavior.apply(msg)
+ }
+ actorBehavior
+ }
+
+ private lazy val fullBehavior: Receive = {
+ lazy val defaultBehavior = receive
+ val actorBehavior: Receive = {
+ case HotSwap(code) => become(code(self))
+ case RevertHotSwap => unbecome
+ 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.isEmpty &&
+ self.hotswap.head.isDefinedAt(msg) => self.hotswap.head.apply(msg)
+ case msg if self.hotswap.isEmpty &&
+ defaultBehavior.isDefinedAt(msg) => defaultBehavior.apply(msg)
+ case unknown => unhandled(unknown) //This is the only line that differs from processingbehavior
+ }
+ actorBehavior
+ }
+}
+
+private[actor] class AnyOptionAsTypedOption(anyOption: Option[Any]) {
+
+ /**
+ * Convenience helper to cast the given Option of Any to an Option of the given type. Will throw a ClassCastException
+ * if the actual type is not assignable from the given one.
+ */
+ def as[T]: Option[T] = narrow[T](anyOption)
+
+ /**
+ * Convenience helper to cast the given Option of Any to an Option of the given type. Will swallow a possible
+ * ClassCastException and return None in that case.
+ */
+ def asSilently[T: Manifest]: Option[T] = narrowSilently[T](anyOption)
+}
+
+/**
+ * Marker interface for proxyable actors (such as typed actor).
+ *
+ * @author Jonas Bonér
+ */
+trait Proxyable {
+ private[actor] def swapProxiedActor(newInstance: Actor)
+}
+
+/**
+ * Represents the different Actor types.
+ *
+ * @author Jonas Bonér
+ */
+sealed trait ActorType
+object ActorType {
+ case object ScalaActor extends ActorType
+ case object TypedActor extends ActorType
+}
diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala
new file mode 100644
index 0000000000..79cf2cc920
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala
@@ -0,0 +1,1536 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import akka.dispatch._
+import akka.config.Config._
+import akka.config.Supervision._
+import akka.AkkaException
+import akka.util._
+import ReflectiveAccess._
+
+import java.net.InetSocketAddress
+import java.util.concurrent.atomic.{AtomicInteger, AtomicReference}
+import java.util.concurrent.locks.ReentrantLock
+import java.util.concurrent.{ ScheduledFuture, ConcurrentHashMap, TimeUnit }
+import java.util.{ Map => JMap }
+import java.lang.reflect.Field
+
+import scala.reflect.BeanProperty
+import scala.collection.immutable.Stack
+import scala.annotation.tailrec
+
+private[akka] object ActorRefInternals extends Logging {
+
+ /**
+ * LifeCycles for ActorRefs.
+ */
+ private[akka] sealed trait StatusType
+ object UNSTARTED extends StatusType
+ object RUNNING extends StatusType
+ object BEING_RESTARTED extends StatusType
+ object SHUTDOWN extends StatusType
+}
+
+
+/**
+ * ActorRef is an immutable and serializable handle to an Actor.
+ *
+ * Create an ActorRef for an Actor by using the factory method on the Actor object.
+ *
+ * Here is an example on how to create an actor with a default constructor.
+ *
+ * import Actor._
+ *
+ * val actor = actorOf[MyActor]
+ * actor.start
+ * actor ! message
+ * actor.stop
+ *
+ *
+ * You can also create and start actors like this:
+ *
+ * val actor = actorOf[MyActor].start
+ *
+ *
+ * Here is an example on how to create an actor with a non-default constructor.
+ *
+ * import Actor._
+ *
+ * val actor = actorOf(new MyActor(...))
+ * actor.start
+ * actor ! message
+ * actor.stop
+ *
+ *
+ * @author Jonas Bonér
+ */
+trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scalaRef: ScalaActorRef =>
+ //Reuse same logger
+ import Actor.log
+
+ // Only mutable for RemoteServer in order to maintain identity across nodes
+ @volatile
+ protected[akka] var _uuid = newUuid
+ @volatile
+ protected[this] var _status: ActorRefInternals.StatusType = ActorRefInternals.UNSTARTED
+ @volatile
+ protected[akka] var _homeAddress = new InetSocketAddress(RemoteServerModule.HOSTNAME, RemoteServerModule.PORT)
+ @volatile
+ protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None
+ protected[akka] val guard = new ReentrantGuard
+
+ /**
+ * User overridable callback/setting.
+ *
+ * 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.
+ */
+ @BeanProperty
+ @volatile
+ var id: String = _uuid.toString
+
+ /**
+ * User overridable callback/setting.
+ *
+ * Defines the default timeout for '!!' and '!!!' invocations,
+ * e.g. the timeout for the future returned by the call to '!!' and '!!!'.
+ */
+ @BeanProperty
+ @volatile
+ var timeout: Long = Actor.TIMEOUT
+
+ /**
+ * User overridable callback/setting.
+ *
+ * Defines the default timeout for an initial receive invocation.
+ * When specified, the receive function should be able to handle a 'ReceiveTimeout' message.
+ */
+ @volatile
+ var receiveTimeout: Option[Long] = None
+
+ /**
+ * 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
+ * A faultHandler defines what should be done when a linked actor signals an error.
+ *
+ * Can be one of:
+ *
+ */
+ def setFaultHandler(handler: FaultHandlingStrategy)
+ def getFaultHandler(): FaultHandlingStrategy
+
+ @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.
+ *
+ * You can override it so it fits the specific use-case that the actor is used for.
+ * See the akka.dispatch.Dispatchers class for the different
+ * dispatchers available.
+ *
+ * The default is also that all actors that are created and spawned from within this actor
+ * is sharing the same dispatcher as its creator.
+ */
+ def setDispatcher(dispatcher: MessageDispatcher) = this.dispatcher = dispatcher
+ def getDispatcher(): MessageDispatcher = dispatcher
+
+ /**
+ * Holds the hot swapped partial function.
+ */
+ @volatile
+ protected[akka] var hotswap = Stack[PartialFunction[Any, Unit]]()
+
+ /**
+ * This is a reference to the message currently being processed by the actor
+ */
+ @volatile
+ protected[akka] var currentMessage: MessageInvocation = null
+
+ /**
+ * 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 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 getSenderFuture(): Option[CompletableFuture[Any]] = senderFuture
+
+ /**
+ * Is the actor being restarted?
+ */
+ def isBeingRestarted: Boolean = _status == ActorRefInternals.BEING_RESTARTED
+
+ /**
+ * Is the actor running?
+ */
+ def isRunning: Boolean = _status match {
+ case ActorRefInternals.BEING_RESTARTED | ActorRefInternals.RUNNING => true
+ case _ => false
+ }
+
+ /**
+ * Is the actor shut down?
+ */
+ def isShutdown: Boolean = _status == ActorRefInternals.SHUTDOWN
+
+ /**
+ * Is the actor ever started?
+ */
+ def isUnstarted: Boolean = _status == ActorRefInternals.UNSTARTED
+
+ /**
+ * Is the actor able to handle the message passed in as arguments?
+ */
+ def isDefinedAt(message: Any): Boolean = actor.isDefinedAt(message)
+
+ /**
+ * Only for internal use. UUID is effectively final.
+ */
+ protected[akka] def uuid_=(uid: Uuid) = _uuid = uid
+
+ /**
+ * Akka Java API
+ * Sends a one-way asynchronous message. E.g. fire-and-forget semantics.
+ *
+ *
+ * actor.sendOneWay(message);
+ *
+ *
+ */
+ def sendOneWay(message: AnyRef): Unit = sendOneWay(message, null)
+
+ /**
+ * Akka Java API
+ * Sends a one-way asynchronous message. E.g. fire-and-forget 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 'sendOneWay' together with 'getContext().getSender()' to
+ * implement request/response message exchanges.
+ *
+ * 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 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 'sendOneWay' together with the 'getContext().getSender()' to
+ * implement request/response message exchanges.
+ *
+ * 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 sendRequestReplyFuture(message: AnyRef, timeout: Long, sender: ActorRef): Future[_] = !!!(message, timeout)(Option(sender))
+
+ /**
+ * Akka Java API
+ * Forwards the message specified to this actor and preserves the original sender of the message
+ */
+ 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))
+
+ /**
+ * 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 replyUnsafe(message: AnyRef) = reply(message)
+
+ /**
+ * 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 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.
+ */
+ def dispatcher_=(md: MessageDispatcher): Unit
+
+ /**
+ * Get the dispatcher for this actor.
+ */
+ def dispatcher: MessageDispatcher
+
+ /**
+ * Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
+ */
+ def makeRemote(hostname: String, port: Int): Unit
+
+ /**
+ * Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
+ */
+ def makeRemote(address: InetSocketAddress): Unit
+
+ /**
+ * Returns the home address and port for this actor.
+ */
+ def homeAddress: InetSocketAddress = _homeAddress
+
+ /**
+ * Akka Java API
+ * Returns the home address and port for this actor.
+ */
+ def getHomeAddress(): InetSocketAddress = homeAddress
+
+ /**
+ * Set the home address and port for this actor.
+ */
+ def homeAddress_=(hostnameAndPort: Tuple2[String, Int]): Unit =
+ homeAddress_=(new InetSocketAddress(hostnameAndPort._1, hostnameAndPort._2))
+
+ /**
+ * Akka Java API
+ * Set the home address and port for this actor.
+ */
+ def setHomeAddress(hostname: String, port: Int): Unit = homeAddress = (hostname, port)
+
+ /**
+ * Set the home address and port for this actor.
+ */
+ def homeAddress_=(address: InetSocketAddress): Unit
+
+ /**
+ * Akka Java API
+ * Set the home address and port for this actor.
+ */
+ def setHomeAddress(address: InetSocketAddress): Unit = homeAddress = address
+
+ /**
+ * Returns the remote address for the actor, if any, else None.
+ */
+ def remoteAddress: Option[InetSocketAddress]
+ protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit
+
+ /**
+ * Akka Java API
+ * Gets the remote address for the actor, if any, else None.
+ */
+ def getRemoteAddress(): Option[InetSocketAddress] = remoteAddress
+
+ /**
+ * Starts up the actor and its message queue.
+ */
+ def start(): ActorRef
+
+ /**
+ * Shuts down the actor its dispatcher and message queue.
+ * Alias for 'stop'.
+ */
+ def exit() = stop()
+
+ /**
+ * Shuts down the actor its dispatcher and message queue.
+ */
+ def stop(): Unit
+
+ /**
+ * Links an other actor to this actor. Links are unidirectional and means that a the linking actor will
+ * receive a notification if the linked actor has crashed.
+ *
+ * If the 'trapExit' member field of the 'faultHandler' 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'.
+ */
+ def link(actorRef: ActorRef): Unit
+
+ /**
+ * Unlink the actor.
+ */
+ def unlink(actorRef: ActorRef): Unit
+
+ /**
+ * Atomically start and link an actor.
+ */
+ def startLink(actorRef: ActorRef): Unit
+
+ /**
+ * Atomically start, link and make an actor remote.
+ */
+ def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit
+
+ /**
+ * Atomically create (from actor class) and start an actor.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def spawn(clazz: Class[_ <: Actor]): ActorRef
+
+ /**
+ * Atomically create (from actor class), make it remote and start an actor.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef
+
+ /**
+ * Atomically create (from actor class), link and start an actor.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def spawnLink(clazz: Class[_ <: Actor]): ActorRef
+
+ /**
+ * Atomically create (from actor class), make it remote, link and start an actor.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef
+
+ /**
+ * Returns the mailbox size.
+ */
+ def mailboxSize = dispatcher.mailboxSize(this)
+
+ /**
+ * Akka Java API
+ * Returns the mailbox size.
+ */
+ def getMailboxSize(): Int = mailboxSize
+
+ /**
+ * Returns the supervisor, if there is one.
+ */
+ def supervisor: Option[ActorRef]
+
+ /**
+ * Akka Java API
+ * Returns the supervisor, if there is one.
+ */
+ def getSupervisor(): ActorRef = supervisor getOrElse null
+
+ protected[akka] def invoke(messageHandle: MessageInvocation): Unit
+
+ protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit
+
+ protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T](
+ message: Any,
+ timeout: Long,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T]
+
+ 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: AnyRef
+ protected[akka] def mailbox_=(value: AnyRef): AnyRef
+
+ protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit
+
+ protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit
+
+ protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit
+
+ protected[akka] def registerSupervisorAsRemoteActor: Option[Uuid]
+
+ protected[akka] def linkedActors: JMap[Uuid, ActorRef]
+
+ override def hashCode: Int = HashCode.hash(HashCode.SEED, uuid)
+
+ override def equals(that: Any): Boolean = {
+ that.isInstanceOf[ActorRef] &&
+ that.asInstanceOf[ActorRef].uuid == uuid
+ }
+
+ override def toString = "Actor[" + id + ":" + uuid + "]"
+
+ protected[akka] def checkReceiveTimeout = {
+ cancelReceiveTimeout
+ if (receiveTimeout.isDefined && dispatcher.mailboxSize(this) <= 0) { //Only reschedule if desired and there are currently no more messages to be processed
+ log.debug("Scheduling timeout for %s", this)
+ _futureTimeout = Some(Scheduler.scheduleOnce(this, ReceiveTimeout, receiveTimeout.get, TimeUnit.MILLISECONDS))
+ }
+ }
+
+ protected[akka] def cancelReceiveTimeout = {
+ if (_futureTimeout.isDefined) {
+ _futureTimeout.get.cancel(true)
+ _futureTimeout = None
+ log.debug("Timeout canceled for %s", this)
+ }
+ }
+}
+
+/**
+ * Local (serializable) ActorRef that is used when referencing the Actor on its "home" node.
+ *
+ * @author Jonas Bonér
+ */
+class LocalActorRef private[akka] (
+ private[this] val actorFactory: () => Actor)
+ extends ActorRef with ScalaActorRef {
+
+ @volatile
+ private[akka] var _remoteAddress: Option[InetSocketAddress] = None // only mutable to maintain identity across nodes
+ @volatile
+ private[akka] lazy val _linkedActors = new ConcurrentHashMap[Uuid, ActorRef]
+ @volatile
+ private[akka] var _supervisor: Option[ActorRef] = None
+ @volatile
+ private var maxNrOfRetriesCount: Int = 0
+ @volatile
+ private var restartsWithinTimeRangeTimestamp: Long = 0L
+ @volatile
+ private var _mailbox: AnyRef = _
+
+ protected[akka] val actorInstance = guard.withGuard { new AtomicReference[Actor](newActor) }
+
+ //If it was started inside "newActor", initialize it
+ if (isRunning) initializeActorInstance
+
+ // used only for deserialization
+ private[akka] def this(__uuid: Uuid,
+ __id: String,
+ __hostname: String,
+ __port: Int,
+ __timeout: Long,
+ __receiveTimeout: Option[Long],
+ __lifeCycle: LifeCycle,
+ __supervisor: Option[ActorRef],
+ __hotswap: Stack[PartialFunction[Any, Unit]],
+ __factory: () => Actor) = {
+ this(__factory)
+ _uuid = __uuid
+ id = __id
+ homeAddress = (__hostname, __port)
+ timeout = __timeout
+ receiveTimeout = __receiveTimeout
+ lifeCycle = __lifeCycle
+ _supervisor = __supervisor
+ hotswap = __hotswap
+ setActorSelfFields(actor,this)
+ start
+ ActorRegistry.register(this)
+ }
+
+ // ========= PUBLIC FUNCTIONS =========
+
+ /**
+ * Returns the class for the Actor instance that is managed by the ActorRef.
+ */
+ def actorClass: Class[_ <: Actor] = actor.getClass.asInstanceOf[Class[_ <: Actor]]
+
+ /**
+ * Returns the class name for the Actor instance that is managed by the ActorRef.
+ */
+ def actorClassName: String = actorClass.getName
+
+ /**
+ * Sets the dispatcher for this actor. Needs to be invoked before the actor is started.
+ */
+ def dispatcher_=(md: MessageDispatcher): Unit = guard.withGuard {
+ if (!isBeingRestarted) {
+ if (!isRunning) _dispatcher = md
+ else throw new ActorInitializationException(
+ "Can not swap dispatcher for " + toString + " after it has been started")
+ }
+ }
+
+ /**
+ * Get the dispatcher for this actor.
+ */
+ def dispatcher: MessageDispatcher = _dispatcher
+
+ /**
+ * Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
+ */
+ def makeRemote(hostname: String, port: Int): Unit = {
+ ensureRemotingEnabled
+ if (!isRunning || isBeingRestarted) makeRemote(new InetSocketAddress(hostname, port))
+ else throw new ActorInitializationException(
+ "Can't make a running actor remote. Make sure you call 'makeRemote' before 'start'.")
+ }
+
+ /**
+ * Invoking 'makeRemote' means that an actor will be moved to and invoked on a remote host.
+ */
+ def makeRemote(address: InetSocketAddress): Unit = guard.withGuard {
+ ensureRemotingEnabled
+ if (!isRunning || isBeingRestarted) {
+ _remoteAddress = Some(address)
+ RemoteClientModule.register(address, uuid)
+ homeAddress = (RemoteServerModule.HOSTNAME, RemoteServerModule.PORT)
+ } else throw new ActorInitializationException(
+ "Can't make a running actor remote. Make sure you call 'makeRemote' before 'start'.")
+ }
+
+ /**
+ * Set the contact address for this actor. This is used for replying to messages
+ * sent asynchronously when no reply channel exists.
+ */
+ def homeAddress_=(address: InetSocketAddress): Unit = _homeAddress = address
+
+ /**
+ * Returns the remote address for the actor, if any, else None.
+ */
+ def remoteAddress: Option[InetSocketAddress] = _remoteAddress
+ protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = _remoteAddress = addr
+
+ /**
+ * Starts up the actor and its message queue.
+ */
+ def start: ActorRef = guard.withGuard {
+ if (isShutdown) throw new ActorStartException(
+ "Can't restart an actor that has been shut down with 'stop' or 'exit'")
+ if (!isRunning) {
+ dispatcher.attach(this)
+
+ _status = ActorRefInternals.RUNNING
+
+ // If we are not currently creating this ActorRef instance
+ if ((actorInstance ne null) && (actorInstance.get ne null))
+ initializeActorInstance
+
+ checkReceiveTimeout //Schedule the initial Receive timeout
+ }
+ this
+ }
+
+ /**
+ * Shuts down the actor its dispatcher and message queue.
+ */
+ def stop() = guard.withGuard {
+ if (isRunning) {
+ receiveTimeout = None
+ cancelReceiveTimeout
+ dispatcher.detach(this)
+ _status = ActorRefInternals.SHUTDOWN
+ actor.postStop
+ ActorRegistry.unregister(this)
+ if (isRemotingEnabled) {
+ if (remoteAddress.isDefined)
+ RemoteClientModule.unregister(remoteAddress.get, uuid)
+ RemoteServerModule.unregister(this)
+ }
+ setActorSelfFields(actorInstance.get,null)
+ } //else if (isBeingRestarted) throw new ActorKilledException("Actor [" + toString + "] is being restarted.")
+ }
+
+ /**
+ * Links an other actor to this actor. Links are unidirectional and means that a the linking actor will
+ * receive a notification if the linked actor has crashed.
+ *
+ * If the 'trapExit' member field of the 'faultHandler' 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) = guard.withGuard {
+ if (actorRef.supervisor.isDefined) throw new IllegalActorStateException(
+ "Actor can only have one supervisor [" + actorRef + "], e.g. link(actor) fails")
+ linkedActors.put(actorRef.uuid, actorRef)
+ actorRef.supervisor = Some(this)
+ Actor.log.debug("Linking actor [%s] to actor [%s]", actorRef, this)
+ }
+
+ /**
+ * Unlink the actor.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def unlink(actorRef: ActorRef) = guard.withGuard {
+ if (!linkedActors.containsKey(actorRef.uuid)) throw new IllegalActorStateException(
+ "Actor [" + actorRef + "] is not a linked actor, can't unlink")
+ linkedActors.remove(actorRef.uuid)
+ actorRef.supervisor = None
+ Actor.log.debug("Unlinking actor [%s] from actor [%s]", actorRef, this)
+ }
+
+ /**
+ * Atomically start and link an actor.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def startLink(actorRef: ActorRef): Unit = guard.withGuard {
+ link(actorRef)
+ actorRef.start
+ }
+
+ /**
+ * 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 = guard.withGuard {
+ ensureRemotingEnabled
+ actorRef.makeRemote(hostname, port)
+ link(actorRef)
+ actorRef.start
+ }
+
+ /**
+ * Atomically create (from actor class) and start an actor.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def spawn(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard {
+ Actor.actorOf(clazz).start
+ }
+
+ /**
+ * Atomically create (from actor class), start and make an actor remote.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = guard.withGuard {
+ ensureRemotingEnabled
+ val actor = Actor.actorOf(clazz)
+ actor.makeRemote(hostname, port)
+ actor.start
+ actor
+ }
+
+ /**
+ * Atomically create (from actor class), start and link an actor.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def spawnLink(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard {
+ val actor = Actor.actorOf(clazz)
+ link(actor)
+ actor.start
+ actor
+ }
+
+ /**
+ * Atomically create (from actor class), start, link and make an actor remote.
+ *
+ * To be invoked from within the actor itself.
+ */
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = guard.withGuard {
+ ensureRemotingEnabled
+ val actor = Actor.actorOf(clazz)
+ actor.makeRemote(hostname, port)
+ link(actor)
+ actor.start
+ actor
+ }
+
+ /**
+ * Returns the mailbox.
+ */
+ def mailbox: AnyRef = _mailbox
+
+ protected[akka] def mailbox_=(value: AnyRef): AnyRef = { _mailbox = value; value }
+
+ /**
+ * Shuts down and removes all linked actors.
+ */
+ def shutdownLinkedActors() {
+ val i = linkedActors.values.iterator
+ while(i.hasNext) {
+ i.next.stop
+ i.remove
+ }
+ }
+
+ /**
+ * Returns the supervisor, if there is one.
+ */
+ def supervisor: Option[ActorRef] = _supervisor
+
+ // ========= AKKA PROTECTED FUNCTIONS =========
+
+ protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = _supervisor = sup
+
+ protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = {
+ if (remoteAddress.isDefined && isRemotingEnabled) {
+ RemoteClientModule.send[Any](
+ message, senderOption, None, remoteAddress.get, timeout, true, this, None, ActorType.ScalaActor)
+ } else {
+ val invocation = new MessageInvocation(this, message, senderOption, None)
+ dispatcher dispatchMessage invocation
+ }
+ }
+
+ protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T](
+ message: Any,
+ timeout: Long,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
+
+ if (remoteAddress.isDefined && isRemotingEnabled) {
+ val future = RemoteClientModule.send[T](
+ message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, ActorType.ScalaActor)
+ if (future.isDefined) future.get
+ else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
+ } else {
+ val future = if (senderFuture.isDefined) senderFuture else Some(new DefaultCompletableFuture[T](timeout))
+ val invocation = new MessageInvocation(
+ this, message, senderOption, future.asInstanceOf[Some[CompletableFuture[Any]]])
+ dispatcher dispatchMessage invocation
+ future.get
+ }
+ }
+
+ /**
+ * Callback for the dispatcher. This is the single 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)
+ else {
+ currentMessage = messageHandle
+ try {
+ dispatch(messageHandle)
+ } catch {
+ case e =>
+ Actor.log.error(e, "Could not invoke actor [%s]", this)
+ throw e
+ } finally {
+ currentMessage = null //TODO: Don't reset this, we might want to resend the message
+ }
+ }
+ }
+
+ protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) {
+ faultHandler match {
+ case AllForOneStrategy(trapExit,maxRetries, within) if trapExit.exists(_.isAssignableFrom(reason.getClass)) =>
+ restartLinkedActors(reason, maxRetries, within)
+
+ case OneForOneStrategy(trapExit,maxRetries, within) if trapExit.exists(_.isAssignableFrom(reason.getClass)) =>
+ dead.restart(reason, maxRetries, within)
+
+ case _ =>
+ notifySupervisorWithMessage(Exit(this, reason))
+ }
+ }
+
+ private def requestRestartPermission(maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Boolean = {
+ val denied = if (maxNrOfRetries.isEmpty && withinTimeRange.isEmpty) { //Immortal
+ false
+ } else if (withinTimeRange.isEmpty) { // restrict number of restarts
+ maxNrOfRetriesCount += 1 //Increment number of retries
+ maxNrOfRetriesCount > maxNrOfRetries.get
+ } else { // cannot restart more than N within M timerange
+ maxNrOfRetriesCount += 1 //Increment number of retries
+ val windowStart = restartsWithinTimeRangeTimestamp
+ val now = System.currentTimeMillis
+ val retries = maxNrOfRetriesCount
+ //We are within the time window if it isn't the first restart, or if the window hasn't closed
+ val insideWindow = if (windowStart == 0) false
+ else (now - windowStart) <= withinTimeRange.get
+
+ //The actor is dead if it dies X times within the window of restart
+ val unrestartable = insideWindow && retries > maxNrOfRetries.getOrElse(1)
+
+ if (windowStart == 0 || !insideWindow) //(Re-)set the start of the window
+ restartsWithinTimeRangeTimestamp = now
+
+ if (windowStart != 0 && !insideWindow) //Reset number of restarts if window has expired
+ maxNrOfRetriesCount = 1
+
+ unrestartable
+ }
+
+ denied == false //If we weren't denied, we have a go
+ }
+
+ protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) {
+
+ def performRestart {
+ Actor.log.info("Restarting actor [%s] configured as PERMANENT.", id)
+ val failedActor = actorInstance.get
+ Actor.log.debug("Invoking 'preRestart' for failed actor instance [%s].", id)
+ failedActor.preRestart(reason)
+ val freshActor = newActor
+ setActorSelfFields(failedActor,null) //Only null out the references if we could instantiate the new actor
+ actorInstance.set(freshActor) //Assign it here so if preStart fails, we can null out the sef-refs next call
+ freshActor.preStart
+ failedActor match {
+ case p: Proxyable => p.swapProxiedActor(freshActor)
+ case _ =>
+ }
+ Actor.log.debug("Invoking 'postRestart' for new actor instance [%s].", id)
+ freshActor.postRestart(reason)
+ }
+
+ def tooManyRestarts {
+ 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].",
+ maxNrOfRetries, withinTimeRange, this, reason)
+ _supervisor.foreach { sup =>
+ // can supervisor handle the notification?
+ val notification = MaximumNumberOfRestartsWithinTimeRangeReached(this, maxNrOfRetries, withinTimeRange, reason)
+ 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)
+ }
+
+ stop
+ }
+
+ @tailrec def attemptRestart() {
+ val success = if (requestRestartPermission(maxNrOfRetries,withinTimeRange)) {
+ guard.withGuard[Boolean] {
+ _status = ActorRefInternals.BEING_RESTARTED
+
+ lifeCycle match {
+ case Temporary =>
+ shutDownTemporaryActor(this)
+ true
+ case _ => // either permanent or none where default is permanent
+ val success = try {
+ performRestart
+ true
+ } catch {
+ case e => false //An error or exception here should trigger a retry
+ }
+
+ Actor.log.debug("Restart: %s for [%s].", success, id)
+
+ if (success) {
+ _status = ActorRefInternals.RUNNING
+ dispatcher.resume(this)
+ restartLinkedActors(reason,maxNrOfRetries,withinTimeRange)
+ }
+
+ success
+ }
+ }
+ } else {
+ tooManyRestarts
+ true //Done
+ }
+
+ if (success)
+ () //Alles gut
+ else
+ attemptRestart
+ }
+
+ attemptRestart() //Tailrecursion
+ }
+
+ protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) = {
+ import scala.collection.JavaConversions._
+ linkedActors.values foreach { actorRef =>
+ actorRef.lifeCycle match {
+ // either permanent or none where default is permanent
+ case Temporary => shutDownTemporaryActor(actorRef)
+ case _ => actorRef.restart(reason, maxNrOfRetries, withinTimeRange)
+ }
+ }
+ }
+
+ protected[akka] def registerSupervisorAsRemoteActor: Option[Uuid] = guard.withGuard {
+ ensureRemotingEnabled
+ if (_supervisor.isDefined) {
+ remoteAddress.foreach(address => RemoteClientModule.registerSupervisorForActor(address, this))
+ Some(_supervisor.get.uuid)
+ } else None
+ }
+
+ protected[akka] def linkedActors: JMap[Uuid, ActorRef] = _linkedActors
+
+ // ========= PRIVATE FUNCTIONS =========
+
+ private[this] def newActor: Actor = {
+ val a = Actor.actorRefInCreation.withValue(Some(this)) { actorFactory() }
+ if (a eq null) throw new ActorInitializationException("Actor instance passed to ActorRef can not be 'null'")
+ a
+ }
+
+ private def dispatch[T](messageHandle: MessageInvocation) = {
+ Actor.log.trace("Invoking actor with message: %s\n", messageHandle)
+ val message = messageHandle.message //serializeMessage(messageHandle.message)
+
+ try {
+ cancelReceiveTimeout // FIXME: leave this here?
+ actor(message)
+ } catch {
+ case e: InterruptedException => {} // received message while actor is shutting down, ignore
+ case e => handleExceptionInDispatch(e, message)
+ }
+ finally {
+ checkReceiveTimeout // Reschedule receive timeout
+ }
+ }
+
+ private def shutDownTemporaryActor(temporaryActor: ActorRef) {
+ Actor.log.info("Actor [%s] configured as TEMPORARY and will not be restarted.", temporaryActor.id)
+ temporaryActor.stop
+ linkedActors.remove(temporaryActor.uuid) // remove the temporary actor
+ // if last temporary actor is gone, then unlink me from supervisor
+ if (linkedActors.isEmpty) {
+ Actor.log.info(
+ "All linked actors have died permanently (they were all configured as TEMPORARY)" +
+ "\n\tshutting down and unlinking supervisor actor as well [%s].",
+ temporaryActor.id)
+ notifySupervisorWithMessage(UnlinkAndStop(this))
+ }
+
+ true
+ }
+
+ private def handleExceptionInDispatch(reason: Throwable, message: Any) = {
+ Actor.log.error(reason, "Exception when invoking \n\tactor [%s] \n\twith message [%s]", this, message)
+
+ //Prevent any further messages to be processed until the actor has been restarted
+ dispatcher.suspend(this)
+
+ senderFuture.foreach(_.completeWithException(reason))
+
+ if (supervisor.isDefined) notifySupervisorWithMessage(Exit(this, reason))
+ else {
+ lifeCycle match {
+ case Temporary => shutDownTemporaryActor(this)
+ case _ => dispatcher.resume(this) //Resume processing for this actor
+ }
+ }
+ }
+
+ 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 =>
+ if (sup.isShutdown) { // if supervisor is shut down, game over for all linked actors
+ shutdownLinkedActors
+ stop
+ } else sup ! notification // else notify supervisor
+ }
+ }
+
+ private def setActorSelfFields(actor: Actor, value: ActorRef) {
+
+ @tailrec def lookupAndSetSelfFields(clazz: Class[_],actor: Actor, value: ActorRef): Boolean = {
+ val success = try {
+ val selfField = clazz.getDeclaredField("self")
+ val someSelfField = clazz.getDeclaredField("someSelf")
+ selfField.setAccessible(true)
+ someSelfField.setAccessible(true)
+ selfField.set(actor,value)
+ someSelfField.set(actor, if (value ne null) Some(value) else null)
+ true
+ } catch {
+ case e: NoSuchFieldException => false
+ }
+
+ if (success) {
+ true
+ }
+ else {
+ val parent = clazz.getSuperclass
+ if (parent eq null)
+ throw new IllegalActorStateException(toString + " is not an Actor since it have not mixed in the 'Actor' trait")
+ lookupAndSetSelfFields(parent,actor,value)
+ }
+ }
+
+ lookupAndSetSelfFields(actor.getClass,actor,value)
+ }
+
+ private def initializeActorInstance = {
+ actor.preStart // run actor preStart
+ Actor.log.trace("[%s] has started", toString)
+ ActorRegistry.register(this)
+ }
+
+ /*
+ private def serializeMessage(message: AnyRef): AnyRef = if (Actor.SERIALIZE_MESSAGES) {
+ if (!message.isInstanceOf[String] &&
+ !message.isInstanceOf[Byte] &&
+ !message.isInstanceOf[Int] &&
+ !message.isInstanceOf[Long] &&
+ !message.isInstanceOf[Float] &&
+ !message.isInstanceOf[Double] &&
+ !message.isInstanceOf[Boolean] &&
+ !message.isInstanceOf[Char] &&
+ !message.isInstanceOf[Tuple2[_, _]] &&
+ !message.isInstanceOf[Tuple3[_, _, _]] &&
+ !message.isInstanceOf[Tuple4[_, _, _, _]] &&
+ !message.isInstanceOf[Tuple5[_, _, _, _, _]] &&
+ !message.isInstanceOf[Tuple6[_, _, _, _, _, _]] &&
+ !message.isInstanceOf[Tuple7[_, _, _, _, _, _, _]] &&
+ !message.isInstanceOf[Tuple8[_, _, _, _, _, _, _, _]] &&
+ !message.getClass.isArray &&
+ !message.isInstanceOf[List[_]] &&
+ !message.isInstanceOf[scala.collection.immutable.Map[_, _]] &&
+ !message.isInstanceOf[scala.collection.immutable.Set[_]]) {
+ Serializer.Java.deepClone(message)
+ } else message
+ } else message
+ */
+}
+
+/**
+ * System messages for RemoteActorRef.
+ *
+ * @author Jonas Bonér
+ */
+object RemoteActorSystemMessage {
+ val Stop = "RemoteActorRef:stop".intern
+}
+
+/**
+ * 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.
+ *
+ * @author Jonas Bonér
+ */
+private[akka] case class RemoteActorRef private[akka] (
+ classOrServiceName: String,
+ val actorClassName: String,
+ val hostname: String,
+ val port: Int,
+ _timeout: Long,
+ loader: Option[ClassLoader],
+ val actorType: ActorType = ActorType.ScalaActor)
+ extends ActorRef with ScalaActorRef {
+
+ ensureRemotingEnabled
+
+ id = classOrServiceName
+ timeout = _timeout
+
+ start
+
+ def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit =
+ RemoteClientModule.send[Any](
+ message, senderOption, None, remoteAddress.get, timeout, true, this, None, actorType)
+
+ def postMessageToMailboxAndCreateFutureResultWithTimeout[T](
+ message: Any,
+ timeout: Long,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
+ val future = RemoteClientModule.send[T](
+ message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, actorType)
+ if (future.isDefined) future.get
+ else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
+ }
+
+ def start: ActorRef = synchronized {
+ _status = ActorRefInternals.RUNNING
+ this
+ }
+
+ def stop: Unit = synchronized {
+ if (_status == ActorRefInternals.RUNNING) {
+ _status = ActorRefInternals.SHUTDOWN
+ postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
+ }
+ }
+
+ protected[akka] def registerSupervisorAsRemoteActor: Option[Uuid] = None
+
+ val remoteAddress: Option[InetSocketAddress] = Some(new InetSocketAddress(hostname, port))
+
+ // ==== NOT SUPPORTED ====
+ def actorClass: Class[_ <: Actor] = unsupported
+ def dispatcher_=(md: MessageDispatcher): Unit = unsupported
+ def dispatcher: MessageDispatcher = unsupported
+ def makeRemote(hostname: String, port: Int): Unit = unsupported
+ def makeRemote(address: InetSocketAddress): Unit = unsupported
+ def homeAddress_=(address: InetSocketAddress): Unit = unsupported
+ def link(actorRef: ActorRef): Unit = unsupported
+ def unlink(actorRef: ActorRef): Unit = unsupported
+ def startLink(actorRef: ActorRef): Unit = unsupported
+ def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = unsupported
+ def spawn(clazz: Class[_ <: Actor]): ActorRef = unsupported
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
+ def 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: 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: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
+ protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
+ protected[akka] def linkedActors: JMap[Uuid, ActorRef] = unsupported
+ protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
+ protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported
+ protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = unsupported
+ protected[akka] def actorInstance: AtomicReference[Actor] = unsupported
+ private def unsupported = throw new UnsupportedOperationException("Not supported for RemoteActorRef")
+}
+
+/**
+ * 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: Uuid
+
+ /**
+ * 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
+ @BeanProperty
+ var lifeCycle: LifeCycle = UndefinedLifeCycle
+
+ /**
+ * User overridable callback/setting.
+ *
+ * Don't forget to supply a List of exception types to intercept (trapExit)
+ *
+ * Can be one of:
+ *
+ */
+ @volatile
+ @BeanProperty
+ var faultHandler: FaultHandlingStrategy = NoFaultHandlingStrategy
+
+ /**
+ * 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] = {
+ val msg = currentMessage
+ if (msg eq null) None
+ else msg.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]] = {
+ val msg = currentMessage
+ if (msg eq null) None
+ else msg.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 isMessageJoinPoint = if (isTypedActorEnabled) TypedActorModule.resolveFutureIfMessageIsJoinPoint(message, future)
+ else false
+ try {
+ future.await
+ } catch {
+ case e: FutureTimeoutException =>
+ if (isMessageJoinPoint) 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, sender.get.sender)
+ 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) {
+ //TODO: optimize away this allocation, perhaps by having implicit self: Option[ActorRef] in signature
+ sender.get.!(message)(Some(this))
+ true
+ } else false
+ }
+
+ /**
+ * Abstraction for unification of sender and senderFuture for later reply
+ */
+ def channel: Channel[Any] = {
+ if (senderFuture.isDefined) {
+ new Channel[Any] {
+ val future = senderFuture.get
+ def !(msg: Any) = future completeWithResult msg
+ }
+ } else if (sender.isDefined) {
+ val someSelf = Some(this)
+ new Channel[Any] {
+ val client = sender.get
+ def !(msg: Any) = client.!(msg)(someSelf)
+ }
+ } else throw new IllegalActorStateException("No channel available")
+ }
+
+ /**
+ * 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 = {
+ ensureRemotingEnabled
+ 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 = {
+ ensureRemotingEnabled
+ spawnLinkRemote(manifest[T].erasure.asInstanceOf[Class[_ <: Actor]], hostname, port)
+ }
+}
+
+/**
+ * Abstraction for unification of sender and senderFuture for later reply
+ */
+abstract class Channel[T] {
+ /**
+ * Sends the specified message to the channel
+ * Scala API
+ */
+ def !(msg: T): Unit
+
+ /**
+ * Sends the specified message to the channel
+ * Java API
+ */
+ def sendOneWay(msg: T): Unit = this.!(msg)
+}
diff --git a/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala b/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala
new file mode 100644
index 0000000000..bf0a479f7f
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/actor/ActorRegistry.scala
@@ -0,0 +1,438 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import scala.collection.mutable.{ListBuffer, Map}
+import scala.reflect.Manifest
+
+import java.util.concurrent.{ConcurrentSkipListSet, ConcurrentHashMap}
+import java.util.{Set => JSet}
+
+import annotation.tailrec
+import akka.util.ReflectiveAccess._
+import akka.util.{ReadWriteGuard, Address, ListenerManagement}
+import java.net.InetSocketAddress
+
+/**
+ * Base trait for ActorRegistry events, allows listen to when an actor is added and removed from the ActorRegistry.
+ *
+ * @author Jonas Bonér
+ */
+sealed trait ActorRegistryEvent
+case class ActorRegistered(actor: ActorRef) extends ActorRegistryEvent
+case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent
+
+/**
+ * Registry holding all Actor instances in the whole system.
+ * Mapped by:
+ *
+ *
the Actor's UUID
+ *
the Actor's id field (which can be set by user-code)
+ *
the Actor's class
+ *
all Actors that are subtypes of a specific type
+ *
+ *
+ * @author Jonas Bonér
+ */
+object ActorRegistry extends ListenerManagement {
+ private val actorsByUUID = new ConcurrentHashMap[Uuid, ActorRef]
+ private val actorsById = new Index[String,ActorRef]
+ private val remoteActorSets = Map[Address, RemoteActorSet]()
+ private val guard = new ReadWriteGuard
+
+ /**
+ * Returns all actors in the system.
+ */
+ def actors: Array[ActorRef] = filter(_ => true)
+
+ /**
+ * Returns the number of actors in the system.
+ */
+ def size : Int = actorsByUUID.size
+
+ /**
+ * Invokes a function for all actors.
+ */
+ def foreach(f: (ActorRef) => Unit) = {
+ val elements = actorsByUUID.elements
+ 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] ): 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): Array[ActorRef] = {
+ val all = new ListBuffer[ActorRef]
+ val elements = actorsByUUID.elements
+ while (elements.hasMoreElements) {
+ val actorId = elements.nextElement
+ if (p(actorId)) {
+ all += actorId
+ }
+ }
+ 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]): 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] =
+ find({ case a:ActorRef if manifest.erasure.isAssignableFrom(a.actor.getClass) => a })
+
+ /**
+ * 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]): Array[ActorRef] =
+ filter(a => clazz.isAssignableFrom(a.actor.getClass))
+
+ /**
+ * Finds all actors that has a specific id.
+ */
+ def actorsFor(id: String): Array[ActorRef] = actorsById values id
+
+ /**
+ * Finds the actor that has a specific UUID.
+ */
+ def actorFor(uuid: Uuid): Option[ActorRef] = Option(actorsByUUID get uuid)
+
+ /**
+ * Returns all typed actors in the system.
+ */
+ def typedActors: Array[AnyRef] = filterTypedActors(_ => true)
+
+ /**
+ * Invokes a function for all typed actors.
+ */
+ def foreachTypedActor(f: (AnyRef) => Unit) = {
+ TypedActorModule.ensureTypedActorEnabled
+ val elements = actorsByUUID.elements
+ while (elements.hasMoreElements) {
+ val proxy = typedActorFor(elements.nextElement)
+ if (proxy.isDefined) {
+ f(proxy.get)
+ }
+ }
+ }
+
+ /**
+ * Invokes the function on all known typed actors until it returns Some
+ * Returns None if the function never returns Some
+ */
+ def findTypedActor[T](f: PartialFunction[AnyRef,T]) : Option[T] = {
+ TypedActorModule.ensureTypedActorEnabled
+ val elements = actorsByUUID.elements
+ while (elements.hasMoreElements) {
+ val proxy = typedActorFor(elements.nextElement)
+ if(proxy.isDefined && (f isDefinedAt proxy))
+ return Some(f(proxy))
+ }
+ None
+ }
+
+ /**
+ * Finds all typed actors that satisfy a predicate.
+ */
+ def filterTypedActors(p: AnyRef => Boolean): Array[AnyRef] = {
+ TypedActorModule.ensureTypedActorEnabled
+ val all = new ListBuffer[AnyRef]
+ val elements = actorsByUUID.elements
+ while (elements.hasMoreElements) {
+ val proxy = typedActorFor(elements.nextElement)
+ if (proxy.isDefined && p(proxy.get)) {
+ all += proxy.get
+ }
+ }
+ all.toArray
+ }
+
+ /**
+ * Finds all typed actors that are subtypes of the class passed in as the Manifest argument.
+ */
+ def typedActorsFor[T <: AnyRef](implicit manifest: Manifest[T]): Array[AnyRef] = {
+ TypedActorModule.ensureTypedActorEnabled
+ typedActorsFor[T](manifest.erasure.asInstanceOf[Class[T]])
+ }
+
+ /**
+ * Finds any typed actor that matches T.
+ */
+ def typedActorFor[T <: AnyRef](implicit manifest: Manifest[T]): Option[AnyRef] = {
+ TypedActorModule.ensureTypedActorEnabled
+ def predicate(proxy: AnyRef) : Boolean = {
+ val actorRef = TypedActorModule.typedActorObjectInstance.get.actorFor(proxy)
+ actorRef.isDefined && manifest.erasure.isAssignableFrom(actorRef.get.actor.getClass)
+ }
+ findTypedActor({ case a:AnyRef if predicate(a) => a })
+ }
+
+ /**
+ * Finds all typed actors of type or sub-type specified by the class passed in as the Class argument.
+ */
+ def typedActorsFor[T <: AnyRef](clazz: Class[T]): Array[AnyRef] = {
+ TypedActorModule.ensureTypedActorEnabled
+ def predicate(proxy: AnyRef) : Boolean = {
+ val actorRef = TypedActorModule.typedActorObjectInstance.get.actorFor(proxy)
+ actorRef.isDefined && clazz.isAssignableFrom(actorRef.get.actor.getClass)
+ }
+ filterTypedActors(predicate)
+ }
+
+ /**
+ * Finds all typed actors that have a specific id.
+ */
+ def typedActorsFor(id: String): Array[AnyRef] = {
+ TypedActorModule.ensureTypedActorEnabled
+ val actorRefs = actorsById values id
+ actorRefs.flatMap(typedActorFor(_))
+ }
+
+ /**
+ * Finds the typed actor that has a specific UUID.
+ */
+ def typedActorFor(uuid: Uuid): Option[AnyRef] = {
+ TypedActorModule.ensureTypedActorEnabled
+ val actorRef = actorsByUUID get uuid
+ if (actorRef eq null)
+ None
+ else
+ typedActorFor(actorRef)
+ }
+
+ /**
+ * Get the typed actor proxy for a given typed actor ref.
+ */
+ private def typedActorFor(actorRef: ActorRef): Option[AnyRef] = {
+ TypedActorModule.typedActorObjectInstance.get.proxyFor(actorRef)
+ }
+
+
+ /**
+ * Registers an actor in the ActorRegistry.
+ */
+ private[akka] def register(actor: ActorRef) = {
+ // ID
+ actorsById.put(actor.id, actor)
+
+ // UUID
+ actorsByUUID.put(actor.uuid, actor)
+
+ // notify listeners
+ notifyListeners(ActorRegistered(actor))
+ }
+
+ /**
+ * Unregisters an actor in the ActorRegistry.
+ */
+ private[akka] def unregister(actor: ActorRef) = {
+ actorsByUUID remove actor.uuid
+
+ actorsById.remove(actor.id,actor)
+
+ // notify listeners
+ notifyListeners(ActorUnregistered(actor))
+ }
+
+ /**
+ * Shuts down and unregisters all actors in the system.
+ */
+ def shutdownAll() {
+ log.info("Shutting down all actors in the system...")
+ if (TypedActorModule.isTypedActorEnabled) {
+ val elements = actorsByUUID.elements
+ while (elements.hasMoreElements) {
+ val actorRef = elements.nextElement
+ val proxy = typedActorFor(actorRef)
+ if (proxy.isDefined) {
+ TypedActorModule.typedActorObjectInstance.get.stop(proxy.get)
+ } else {
+ actorRef.stop
+ }
+ }
+ } else {
+ foreach(_.stop)
+ }
+ actorsByUUID.clear
+ actorsById.clear
+ log.info("All actors have been shut down and unregistered from ActorRegistry")
+ }
+
+ /**
+ * Get the remote actors for the given server address. For internal use only.
+ */
+ private[akka] def actorsFor(remoteServerAddress: Address): RemoteActorSet = guard.withWriteGuard {
+ remoteActorSets.getOrElseUpdate(remoteServerAddress, new RemoteActorSet)
+ }
+
+ private[akka] def registerActorByUuid(address: InetSocketAddress, uuid: String, actor: ActorRef) {
+ actorsByUuid(Address(address.getHostName, address.getPort)).putIfAbsent(uuid, actor)
+ }
+
+ private[akka] def registerTypedActorByUuid(address: InetSocketAddress, uuid: String, typedActor: AnyRef) {
+ typedActorsByUuid(Address(address.getHostName, address.getPort)).putIfAbsent(uuid, typedActor)
+ }
+
+ private[akka] def actors(address: Address) = actorsFor(address).actors
+ private[akka] def actorsByUuid(address: Address) = actorsFor(address).actorsByUuid
+ private[akka] def typedActors(address: Address) = actorsFor(address).typedActors
+ private[akka] def typedActorsByUuid(address: Address) = actorsFor(address).typedActorsByUuid
+
+ private[akka] class RemoteActorSet {
+ private[ActorRegistry] val actors = new ConcurrentHashMap[String, ActorRef]
+ private[ActorRegistry] val actorsByUuid = new ConcurrentHashMap[String, ActorRef]
+ private[ActorRegistry] val typedActors = new ConcurrentHashMap[String, AnyRef]
+ private[ActorRegistry] val typedActorsByUuid = new ConcurrentHashMap[String, AnyRef]
+ }
+}
+
+/**
+ * An implementation of a ConcurrentMultiMap
+ * Adds/remove is serialized over the specified key
+ * Reads are fully concurrent <-- el-cheapo
+ *
+ * @author Viktor Klang
+ */
+class Index[K <: AnyRef,V <: AnyRef : Manifest] {
+ private val Naught = Array[V]() //Nil for Arrays
+ private val container = new ConcurrentHashMap[K, JSet[V]]
+ private val emptySet = new ConcurrentSkipListSet[V]
+
+ /**
+ * Associates the value of type V with the key of type K
+ * @returns true if the value didn't exist for the key previously, and false otherwise
+ */
+ def put(key: K, value: V): Boolean = {
+ //Tailrecursive spin-locking put
+ @tailrec def spinPut(k: K, v: V): Boolean = {
+ var retry = false
+ var added = false
+ val set = container get k
+
+ if (set ne null) {
+ set.synchronized {
+ if (set.isEmpty) {
+ retry = true //IF the set is empty then it has been removed, so signal retry
+ }
+ else { //Else add the value to the set and signal that retry is not needed
+ added = set add v
+ retry = false
+ }
+ }
+ }
+ else {
+ val newSet = new ConcurrentSkipListSet[V]
+ newSet add v
+
+ // Parry for two simultaneous putIfAbsent(id,newSet)
+ val oldSet = container.putIfAbsent(k,newSet)
+ if (oldSet ne null) {
+ oldSet.synchronized {
+ if (oldSet.isEmpty) {
+ retry = true //IF the set is empty then it has been removed, so signal retry
+ }
+ else { //Else try to add the value to the set and signal that retry is not needed
+ added = oldSet add v
+ retry = false
+ }
+ }
+ } else {
+ added = true
+ }
+ }
+
+ if (retry) spinPut(k,v)
+ else added
+ }
+
+ spinPut(key,value)
+ }
+
+ /**
+ * @returns a _new_ array of all existing values for the given key at the time of the call
+ */
+ def values(key: K): Array[V] = {
+ val set: JSet[V] = container get key
+ val result = if (set ne null) set toArray Naught else Naught
+ result.asInstanceOf[Array[V]]
+ }
+
+ /**
+ * @returns Some(value) for the first matching value where the supplied function returns true for the given key,
+ * if no matches it returns None
+ */
+ def findValue(key: K)(f: (V) => Boolean): Option[V] = {
+ import scala.collection.JavaConversions._
+ val set = container get key
+ if (set ne null)
+ set.iterator.find(f)
+ else
+ None
+ }
+
+ /**
+ * Applies the supplied function to all keys and their values
+ */
+ def foreach(fun: (K,V) => Unit) {
+ import scala.collection.JavaConversions._
+ container.entrySet foreach {
+ (e) => e.getValue.foreach(fun(e.getKey,_))
+ }
+ }
+
+ /**
+ * Disassociates the value of type V from the key of type K
+ * @returns true if the value was disassociated from the key and false if it wasn't previously associated with the key
+ */
+ def remove(key: K, value: V): Boolean = {
+ val set = container get key
+
+ if (set ne null) {
+ set.synchronized {
+ if (set.remove(value)) { //If we can remove the value
+ if (set.isEmpty) //and the set becomes empty
+ container.remove(key,emptySet) //We try to remove the key if it's mapped to an empty set
+
+ true //Remove succeeded
+ }
+ else false //Remove failed
+ }
+ } else false //Remove failed
+ }
+
+ /**
+ * @returns true if the underlying containers is empty, may report false negatives when the last remove is underway
+ */
+ def isEmpty: Boolean = container.isEmpty
+
+ /**
+ * Removes all keys and all values
+ */
+ def clear = foreach { case (k,v) => remove(k,v) }
+}
diff --git a/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala
new file mode 100644
index 0000000000..6c58203de2
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/actor/BootableActorLoaderService.scala
@@ -0,0 +1,101 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import java.io.File
+import java.net.{URL, URLClassLoader}
+import java.util.jar.JarFile
+import java.util.Enumeration
+
+import akka.util.{Bootable, Logging}
+import akka.config.Config._
+
+class AkkaDeployClassLoader(urls : List[URL], parent : ClassLoader) extends URLClassLoader(urls.toArray.asInstanceOf[Array[URL]],parent)
+{
+ override def findResources(resource : String) = {
+ val normalResult = super.findResources(resource)
+ if(normalResult.hasMoreElements) normalResult else findDeployed(resource)
+ }
+
+ def findDeployed(resource : String) = new Enumeration[URL]{
+ private val it = getURLs.flatMap( listClassesInPackage(_,resource) ).iterator
+ def hasMoreElements = it.hasNext
+ def nextElement = it.next
+ }
+
+ def listClassesInPackage(jar : URL, pkg : String) = {
+ val f = new File(jar.getFile)
+ val jf = new JarFile(f)
+ try {
+ val es = jf.entries
+ var result = List[URL]()
+ while(es.hasMoreElements)
+ {
+ val e = es.nextElement
+ if(!e.isDirectory && e.getName.startsWith(pkg) && e.getName.endsWith(".class"))
+ result ::= new URL("jar:" + f.toURI.toURL + "!/" + e)
+ }
+ result
+ } finally {
+ jf.close
+ }
+ }
+}
+
+/**
+ * Handles all modules in the deploy directory (load and unload)
+ */
+trait BootableActorLoaderService extends Bootable with Logging {
+
+ val BOOT_CLASSES = config.getList("akka.boot")
+ lazy val applicationLoader: Option[ClassLoader] = createApplicationClassLoader
+
+ protected def createApplicationClassLoader : Option[ClassLoader] = {
+ Some(
+ if (HOME.isDefined) {
+ val CONFIG = HOME.getOrElse(throwNoAkkaHomeException) + "/config"
+ val DEPLOY = HOME.getOrElse(throwNoAkkaHomeException) + "/deploy"
+ val DEPLOY_DIR = new File(DEPLOY)
+ if (!DEPLOY_DIR.exists) {
+ log.error("Could not find a deploy directory at [%s]", DEPLOY)
+ System.exit(-1)
+ }
+ val filesToDeploy = DEPLOY_DIR.listFiles.toArray.toList
+ .asInstanceOf[List[File]].filter(_.getName.endsWith(".jar"))
+ var dependencyJars: List[URL] = Nil
+ filesToDeploy.map { file =>
+ val jarFile = new JarFile(file)
+ val en = jarFile.entries
+ while (en.hasMoreElements) {
+ val name = en.nextElement.getName
+ if (name.endsWith(".jar")) dependencyJars ::= new File(
+ String.format("jar:file:%s!/%s", jarFile.getName, name)).toURI.toURL
+ }
+ }
+ val toDeploy = filesToDeploy.map(_.toURI.toURL)
+ log.info("Deploying applications from [%s]: [%s]", DEPLOY, toDeploy)
+ log.debug("Loading dependencies [%s]", dependencyJars)
+ val allJars = toDeploy ::: dependencyJars
+
+ new AkkaDeployClassLoader(allJars,Thread.currentThread.getContextClassLoader)
+ } else Thread.currentThread.getContextClassLoader)
+ }
+
+ 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
+ }
+ }
+
+ abstract override def onUnload = {
+ super.onUnload
+ ActorRegistry.shutdownAll
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/actor/FSM.scala b/akka-actor/src/main/scala/akka/actor/FSM.scala
new file mode 100644
index 0000000000..df88db5ade
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/actor/FSM.scala
@@ -0,0 +1,146 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import scala.collection.mutable
+import java.util.concurrent.{ScheduledFuture, TimeUnit}
+
+trait FSM[S, D] {
+ this: Actor =>
+
+ type StateFunction = scala.PartialFunction[Event, State]
+
+ /** DSL */
+ protected final def notifying(transitionHandler: PartialFunction[Transition, Unit]) = {
+ transitionEvent = transitionHandler
+ }
+
+ protected final def when(stateName: S)(stateFunction: StateFunction) = {
+ register(stateName, stateFunction)
+ }
+
+ protected final def startWith(stateName: S, stateData: D, timeout: Option[Long] = None) = {
+ setState(State(stateName, stateData, timeout))
+ }
+
+ protected final def goto(nextStateName: S): State = {
+ State(nextStateName, currentState.stateData)
+ }
+
+ protected final def stay(): State = {
+ goto(currentState.stateName)
+ }
+
+ protected final def stop(): State = {
+ stop(Normal)
+ }
+
+ protected final def stop(reason: Reason): State = {
+ stop(reason, currentState.stateData)
+ }
+
+ protected final def stop(reason: Reason, stateData: D): State = {
+ self ! Stop(reason, stateData)
+ stay
+ }
+
+ def whenUnhandled(stateFunction: StateFunction) = {
+ handleEvent = stateFunction
+ }
+
+ def onTermination(terminationHandler: PartialFunction[Reason, Unit]) = {
+ terminateEvent = terminationHandler
+ }
+
+ /** FSM State data and default handlers */
+ private var currentState: State = _
+ private var timeoutFuture: Option[ScheduledFuture[AnyRef]] = None
+
+ private val transitions = mutable.Map[S, StateFunction]()
+ private def register(name: S, function: StateFunction) {
+ if (transitions contains name) {
+ transitions(name) = transitions(name) orElse function
+ } else {
+ transitions(name) = function
+ }
+ }
+
+ private var handleEvent: StateFunction = {
+ case Event(value, stateData) =>
+ log.warning("Event %s not handled in state %s, staying at current state", value, currentState.stateName)
+ stay
+ }
+
+ private var terminateEvent: PartialFunction[Reason, Unit] = {
+ case failure@Failure(_) => log.error("Stopping because of a %s", failure)
+ case reason => log.info("Stopping because of reason: %s", reason)
+ }
+
+ private var transitionEvent: PartialFunction[Transition, Unit] = {
+ case Transition(from, to) => log.debug("Transitioning from state %s to %s", from, to)
+ }
+
+ override final protected def receive: Receive = {
+ case Stop(reason, stateData) =>
+ terminateEvent.apply(reason)
+ self.stop
+ case StateTimeout if (self.dispatcher.mailboxSize(self) > 0) =>
+ log.trace("Ignoring StateTimeout - ")
+ // state timeout when new message in queue, skip this timeout
+ case value => {
+ timeoutFuture = timeoutFuture.flatMap {ref => ref.cancel(true); None}
+ val event = Event(value, currentState.stateData)
+ val nextState = (transitions(currentState.stateName) orElse handleEvent).apply(event)
+ setState(nextState)
+ }
+ }
+
+ private def setState(nextState: State) = {
+ if (!transitions.contains(nextState.stateName)) {
+ stop(Failure("Next state %s does not exist".format(nextState.stateName)))
+ } else {
+ if (currentState != null && currentState.stateName != nextState.stateName) {
+ transitionEvent.apply(Transition(currentState.stateName, nextState.stateName))
+ }
+ currentState = nextState
+ currentState.timeout.foreach {
+ t =>
+ timeoutFuture = Some(Scheduler.scheduleOnce(self, StateTimeout, t, TimeUnit.MILLISECONDS))
+ }
+ }
+ }
+
+ case class Event(event: Any, stateData: D)
+
+ case class State(stateName: S, stateData: D, timeout: Option[Long] = None) {
+
+ def until(timeout: Long): State = {
+ copy(timeout = Some(timeout))
+ }
+
+ def replying(replyValue:Any): State = {
+ self.sender match {
+ case Some(sender) => sender ! replyValue
+ case None => log.error("Unable to send reply value %s, no sender reference to reply to", replyValue)
+ }
+ this
+ }
+
+ def using(nextStateDate: D): State = {
+ copy(stateData = nextStateDate)
+ }
+ }
+
+ sealed trait Reason
+ case object Normal extends Reason
+ case object Shutdown extends Reason
+ case class Failure(cause: Any) extends Reason
+
+ case object StateTimeout
+
+ case class Transition(from: S, to: S)
+
+ private case class Stop(reason: Reason, stateData: D)
+}
diff --git a/akka-actor/src/main/scala/akka/actor/Implicits.scala b/akka-actor/src/main/scala/akka/actor/Implicits.scala
new file mode 100644
index 0000000000..668d2d8876
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/actor/Implicits.scala
@@ -0,0 +1,22 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka
+
+import actor.{ScalaActorRef, ActorRef}
+
+package object actor {
+ implicit def actorRef2Scala(ref: ActorRef): ScalaActorRef =
+ ref.asInstanceOf[ScalaActorRef]
+
+ implicit def scala2ActorRef(ref: ScalaActorRef): ActorRef =
+ ref.asInstanceOf[ActorRef]
+
+ type Uuid = com.eaio.uuid.UUID
+ def newUuid(): Uuid = new Uuid()
+ def uuidFrom(time: Long, clockSeqAndNode: Long): Uuid = new Uuid(time,clockSeqAndNode)
+ def uuidFrom(uuid: String): Uuid = {
+ new Uuid(uuid)
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/actor/Scheduler.scala b/akka-actor/src/main/scala/akka/actor/Scheduler.scala
new file mode 100644
index 0000000000..ae7f2193ee
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/actor/Scheduler.scala
@@ -0,0 +1,133 @@
+/*
+ * Copyright 2007 WorldWide Conferencing, LLC
+ *
+ * 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.
+ *
+ * Rework of David Pollak's ActorPing class in the Lift Project
+ * which is licensed under the Apache 2 License.
+ */
+package akka.actor
+
+import scala.collection.JavaConversions
+
+import java.util.concurrent._
+
+import akka.util.Logging
+import akka.AkkaException
+
+object Scheduler extends Logging {
+ import Actor._
+
+ case class SchedulerException(msg: String, e: Throwable) extends RuntimeException(msg, e)
+
+ @volatile private var service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory)
+
+ log.info("Starting up Scheduler")
+
+ /**
+ * Schedules to send the specified message to the receiver after initialDelay and then repeated after delay
+ */
+ def schedule(receiver: ActorRef, message: AnyRef, initialDelay: Long, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = {
+ log.trace(
+ "Schedule scheduled event\n\tevent = [%s]\n\treceiver = [%s]\n\tinitialDelay = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]",
+ message, receiver, initialDelay, delay, timeUnit)
+ try {
+ service.scheduleAtFixedRate(
+ new Runnable { def run = receiver ! message },
+ initialDelay, delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]]
+ } catch {
+ case e: Exception => throw SchedulerException(message + " could not be scheduled on " + receiver, e)
+ }
+ }
+
+ /**
+ * Schedules to run specified function to the receiver after initialDelay and then repeated after delay,
+ * avoid blocking operations since this is executed in the schedulers thread
+ */
+ def schedule(f: () => Unit, initialDelay: Long, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] =
+ schedule(new Runnable { def run = f() }, initialDelay, delay, timeUnit)
+
+ /**
+ * Schedules to run specified runnable to the receiver after initialDelay and then repeated after delay,
+ * avoid blocking operations since this is executed in the schedulers thread
+ */
+ def schedule(runnable: Runnable, initialDelay: Long, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = {
+ log.trace(
+ "Schedule scheduled event\n\trunnable = [%s]\n\tinitialDelay = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]",
+ runnable, initialDelay, delay, timeUnit)
+
+ try {
+ service.scheduleAtFixedRate(runnable,initialDelay, delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]]
+ } catch {
+ case e: Exception => throw SchedulerException("Failed to schedule a Runnable", e)
+ }
+ }
+
+ /**
+ * Schedules to send the specified message to the receiver after delay
+ */
+ def scheduleOnce(receiver: ActorRef, message: AnyRef, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = {
+ log.trace(
+ "Schedule one-time event\n\tevent = [%s]\n\treceiver = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]",
+ message, receiver, delay, timeUnit)
+ try {
+ service.schedule(
+ new Runnable { def run = receiver ! message },
+ delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]]
+ } catch {
+ case e: Exception => throw SchedulerException( message + " could not be scheduleOnce'd on " + receiver, e)
+ }
+ }
+
+ /**
+ * Schedules a function to be run after delay,
+ * avoid blocking operations since the runnable is executed in the schedulers thread
+ */
+ def scheduleOnce(f: () => Unit, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] =
+ scheduleOnce(new Runnable { def run = f() }, delay, timeUnit)
+
+ /**
+ * Schedules a runnable to be run after delay,
+ * avoid blocking operations since the runnable is executed in the schedulers thread
+ */
+ def scheduleOnce(runnable: Runnable, delay: Long, timeUnit: TimeUnit): ScheduledFuture[AnyRef] = {
+ log.trace(
+ "Schedule one-time event\n\trunnable = [%s]\n\tdelay = [%s]\n\ttimeUnit = [%s]",
+ runnable, delay, timeUnit)
+ try {
+ service.schedule(runnable,delay, timeUnit).asInstanceOf[ScheduledFuture[AnyRef]]
+ } catch {
+ case e: Exception => throw SchedulerException("Failed to scheduleOnce a Runnable", e)
+ }
+ }
+
+ def shutdown: Unit = synchronized {
+ log.info("Shutting down Scheduler")
+ service.shutdown
+ }
+
+ def restart: Unit = synchronized {
+ log.info("Restarting Scheduler")
+ shutdown
+ service = Executors.newSingleThreadScheduledExecutor(SchedulerThreadFactory)
+ }
+}
+
+private object SchedulerThreadFactory extends ThreadFactory {
+ private var count = 0
+ val threadFactory = Executors.defaultThreadFactory()
+
+ def newThread(r: Runnable): Thread = {
+ val thread = threadFactory.newThread(r)
+ thread.setName("akka:scheduler-" + count)
+ thread.setDaemon(true)
+ thread
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/actor/Supervisor.scala b/akka-actor/src/main/scala/akka/actor/Supervisor.scala
new file mode 100644
index 0000000000..daa0bac6c9
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/actor/Supervisor.scala
@@ -0,0 +1,180 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import akka.config.Supervision._
+import akka.AkkaException
+import akka.util._
+import ReflectiveAccess._
+import Actor._
+
+import java.util.concurrent.{CopyOnWriteArrayList, ConcurrentHashMap}
+import java.net.InetSocketAddress
+import akka.config.Supervision._
+
+class SupervisorException private[akka](message: String) extends AkkaException(message)
+
+/**
+ * Factory object for creating supervisors declarative. It creates instances of the 'Supervisor' class.
+ * These are not actors, if you need a supervisor that is an Actor then you have to use the 'SupervisorActor'
+ * factory object.
+ *
+ *
+ * Here is a sample on how to use it:
+ *
+ *
+ * @author Jonas Bonér
+ */
+object Supervisor {
+ def apply(config: SupervisorConfig): Supervisor = SupervisorFactory(config).newInstance.start
+}
+
+/**
+ * Use this factory instead of the Supervisor factory object if you want to control
+ * instantiation and starting of the Supervisor, if not then it is easier and better
+ * to use the Supervisor factory object.
+ *
+ *
+ * Then create a new Supervisor tree with the concrete Services we have defined.
+ *
+ *
+ * val supervisor = factory.newInstance
+ * supervisor.start // start up all managed servers
+ *
+ *
+ * @author Jonas Bonér
+ */
+case class SupervisorFactory(val config: SupervisorConfig) extends Logging {
+
+ def newInstance: Supervisor = newInstanceFor(config)
+
+ def newInstanceFor(config: SupervisorConfig): Supervisor = {
+ val supervisor = new Supervisor(config.restartStrategy)
+ supervisor.configure(config)
+ supervisor.start
+ supervisor
+ }
+}
+
+/**
+ * NOTE:
+ *
+ * The supervisor class is only used for the configuration system when configuring supervisor
+ * hierarchies declaratively. Should not be used as part of the regular programming API. Instead
+ * wire the children together using 'link', 'spawnLink' etc. and set the 'trapExit' flag in the
+ * children that should trap error signals and trigger restart.
+ *
+ * See the ScalaDoc for the SupervisorFactory for an example on how to declaratively wire up children.
+ *
+ * @author Jonas Bonér
+ */
+sealed class Supervisor(handler: FaultHandlingStrategy) {
+ import Supervisor._
+
+ private val _childActors = new ConcurrentHashMap[String, List[ActorRef]]
+ private val _childSupervisors = new CopyOnWriteArrayList[Supervisor]
+
+ private[akka] val supervisor = actorOf(new SupervisorActor(handler)).start
+
+ def uuid = supervisor.uuid
+
+ def start: Supervisor = {
+ this
+ }
+
+ def shutdown(): Unit = supervisor.stop
+
+ def link(child: ActorRef) = supervisor.link(child)
+
+ def unlink(child: ActorRef) = supervisor.unlink(child)
+
+ def children: List[ActorRef] =
+ _childActors.values.toArray.toList.asInstanceOf[List[List[ActorRef]]].flatten
+
+ def childSupervisors: List[Supervisor] =
+ _childActors.values.toArray.toList.asInstanceOf[List[Supervisor]]
+
+ def configure(config: SupervisorConfig): Unit = config match {
+ case SupervisorConfig(_, servers) =>
+ servers.map(server =>
+ server match {
+ case Supervise(actorRef, lifeCycle, remoteAddress) =>
+ actorRef.start
+ val className = actorRef.actor.getClass.getName
+ val currentActors = {
+ val list = _childActors.get(className)
+ if (list eq null) List[ActorRef]()
+ else list
+ }
+ _childActors.put(className, actorRef :: currentActors)
+ actorRef.lifeCycle = lifeCycle
+ supervisor.link(actorRef)
+ if (remoteAddress.isDefined) {
+ val address = remoteAddress.get
+ RemoteServerModule.registerActor(new InetSocketAddress(address.hostname, address.port), actorRef)
+ }
+ case supervisorConfig @ SupervisorConfig(_, _) => // recursive supervisor configuration
+ val childSupervisor = Supervisor(supervisorConfig)
+ supervisor.link(childSupervisor.supervisor)
+ _childSupervisors.add(childSupervisor)
+ })
+ }
+}
+
+/**
+ * For internal use only.
+ *
+ * @author Jonas Bonér
+ */
+final class SupervisorActor private[akka] (handler: FaultHandlingStrategy) extends Actor {
+ import self._
+ faultHandler = handler
+
+ override def postStop(): Unit = shutdownLinkedActors
+
+ def receive = {
+ // FIXME add a way to respond to MaximumNumberOfRestartsWithinTimeRangeReached in declaratively configured Supervisor
+ case MaximumNumberOfRestartsWithinTimeRangeReached(
+ victim, maxNrOfRetries, withinTimeRange, lastExceptionCausingRestart) =>
+ Actor.log.warning(
+ "Declaratively configured supervisor received a [MaximumNumberOfRestartsWithinTimeRangeReached] notification," +
+ "\n\tbut there is currently no way of handling it in a declaratively configured supervisor." +
+ "\n\tIf you want to be able to handle this error condition then you need to create the supervision tree programatically." +
+ "\n\tThis will be supported in the future.")
+ case unknown => throw new SupervisorException(
+ "SupervisorActor can not respond to messages.\n\tUnknown message [" + unknown + "]")
+ }
+}
+
diff --git a/akka-actor/src/main/scala/akka/actor/UntypedActor.scala b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala
new file mode 100644
index 0000000000..9eec8cbb5d
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/actor/UntypedActor.scala
@@ -0,0 +1,161 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import akka.dispatch._
+import akka.config.Supervision._
+import akka.japi.{Creator, Procedure}
+
+import java.net.InetSocketAddress
+
+import scala.reflect.BeanProperty
+
+/**
+ * Subclass this abstract class to create a MDB-style untyped actor.
+ *
+ * This class is meant to be used from Java.
+ *
+ * Here is an example on how to create and use an UntypedActor:
+ *
+ * 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 logger = log.logger //Give the Java guys a break
+
+ def getContext(): ActorRef = self
+
+ final protected def receive = {
+ case msg => onReceive(msg)
+ }
+
+ /**
+ * Java API for become
+ */
+ def become(behavior: Procedure[Any]):Unit = become(behavior,false)
+
+ /*
+ * Java API for become with optional discardOld
+ */
+ def become(behavior: Procedure[Any], discardOld: Boolean): Unit =
+ super.become({ case msg => behavior.apply(msg) }, discardOld)
+
+ @throws(classOf[Exception])
+ def onReceive(message: Any): Unit
+}
+
+/**
+ * Factory closure for an UntypedActor, to be used with 'UntypedActor.actorOf(factory)'.
+ *
+ * @author Jonas Bonér
+ */
+trait UntypedActorFactory extends Creator[Actor]
+
+/**
+ * Extend this abstract class to create a remote UntypedActor.
+ *
+ * @author Jonas Bonér
+ */
+abstract class RemoteUntypedActor(address: InetSocketAddress) extends UntypedActor {
+ def this(hostname: String, port: Int) = this(new InetSocketAddress(hostname, port))
+ self.makeRemote(address)
+}
+
+/**
+ * Factory object for creating and managing 'UntypedActor's. Meant to be used from Java.
+ *
+ * Example on how to create an actor:
+ *
+ * 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:
+ *
+ * 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:
+ *
+ */
+ def actorOf(factory: UntypedActorFactory): ActorRef = Actor.actorOf(factory.create)
+}
diff --git a/akka-actor/src/main/scala/akka/config/Config.scala b/akka-actor/src/main/scala/akka/config/Config.scala
new file mode 100644
index 0000000000..0571449b67
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/config/Config.scala
@@ -0,0 +1,123 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.config
+
+import akka.AkkaException
+import akka.util.Logging
+import akka.actor.{ActorRef, IllegalActorStateException}
+import akka.dispatch.CompletableFuture
+
+import net.lag.configgy.{Config => CConfig, Configgy, ParseException}
+
+import java.net.InetSocketAddress
+import java.lang.reflect.Method
+
+class ConfigurationException(message: String) extends AkkaException(message)
+class ModuleNotAvailableException(message: String) extends AkkaException(message)
+
+object ConfigLogger extends Logging
+
+/**
+ * Loads up the configuration (from the akka.conf file).
+ *
+ * @author Jonas Bonér
+ */
+object Config {
+ val VERSION = "1.0-SNAPSHOT"
+
+ val HOME = {
+ val envHome = System.getenv("AKKA_HOME") match {
+ case null | "" | "." => None
+ case value => Some(value)
+ }
+
+ val systemHome = System.getProperty("akka.home") match {
+ case null | "" => None
+ case value => Some(value)
+ }
+
+ envHome orElse systemHome
+ }
+
+ val config = {
+
+ val confName = {
+
+ val envConf = System.getenv("AKKA_MODE") match {
+ case null | "" => None
+ case value => Some(value)
+ }
+
+ val systemConf = System.getProperty("akka.mode") match {
+ case null | "" => None
+ case value => Some(value)
+ }
+
+ (envConf orElse systemConf).map("akka." + _ + ".conf").getOrElse("akka.conf")
+ }
+
+ if (System.getProperty("akka.config", "") != "") {
+ val configFile = System.getProperty("akka.config", "")
+ try {
+ Configgy.configure(configFile)
+ ConfigLogger.log.info("Config loaded from -Dakka.config=%s", configFile)
+ } catch {
+ case e: ParseException => throw new ConfigurationException(
+ "Config could not be loaded from -Dakka.config=" + configFile +
+ "\n\tdue to: " + e.toString)
+ }
+ Configgy.config
+ } else if (HOME.isDefined) {
+ try {
+ val configFile = HOME.getOrElse(throwNoAkkaHomeException) + "/config/" + confName
+ Configgy.configure(configFile)
+ ConfigLogger.log.info(
+ "AKKA_HOME is defined as [%s], config loaded from [%s].",
+ HOME.getOrElse(throwNoAkkaHomeException),
+ configFile)
+ } catch {
+ case e: ParseException => throw new ConfigurationException(
+ "AKKA_HOME is defined as [" + HOME.get + "] " +
+ "\n\tbut the 'akka.conf' config file can not be found at [" + HOME.get + "/config/"+ confName + "]," +
+ "\n\tdue to: " + e.toString)
+ }
+ Configgy.config
+ } else if (getClass.getClassLoader.getResource(confName) ne null) {
+ try {
+ Configgy.configureFromResource(confName, getClass.getClassLoader)
+ ConfigLogger.log.info("Config [%s] loaded from the application classpath.",confName)
+ } catch {
+ case e: ParseException => throw new ConfigurationException(
+ "Can't load '" + confName + "' config file from application classpath," +
+ "\n\tdue to: " + e.toString)
+ }
+ Configgy.config
+ } else {
+ ConfigLogger.log.warning(
+ "\nCan't load '" + confName + "'." +
+ "\nOne of the three ways of locating the '" + confName + "' file needs to be defined:" +
+ "\n\t1. Define the '-Dakka.config=...' system property option." +
+ "\n\t2. Put the '" + confName + "' file on the classpath." +
+ "\n\t3. Define 'AKKA_HOME' environment variable pointing to the root of the Akka distribution." +
+ "\nI have no way of finding the '" + confName + "' configuration file." +
+ "\nUsing default values everywhere.")
+ CConfig.fromString("") // default empty config
+ }
+ }
+
+ val CONFIG_VERSION = config.getString("akka.version", VERSION)
+ if (VERSION != CONFIG_VERSION) throw new ConfigurationException(
+ "Akka JAR version [" + VERSION + "] is different than the provided config version [" + CONFIG_VERSION + "]")
+
+ val TIME_UNIT = config.getString("akka.time-unit", "seconds")
+
+ val startTime = System.currentTimeMillis
+ def uptime = (System.currentTimeMillis - startTime) / 1000
+
+ def throwNoAkkaHomeException = throw new ConfigurationException(
+ "Akka home is not defined. Either:" +
+ "\n\t1. Define 'AKKA_HOME' environment variable pointing to the root of the Akka distribution." +
+ "\n\t2. Add the '-Dakka.home=...' option pointing to the root of the Akka distribution.")
+}
diff --git a/akka-actor/src/main/scala/akka/config/Configuration.scala b/akka-actor/src/main/scala/akka/config/Configuration.scala
new file mode 100644
index 0000000000..7e0cb406d4
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/config/Configuration.scala
@@ -0,0 +1,60 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.config
+
+/*
+import akka.kernel.{TypedActor, TypedActorProxy}
+import com.google.inject.{AbstractModule}
+import java.util.{List => JList, ArrayList}
+import scala.reflect.BeanProperty
+
+// ============================================
+// Java version of the configuration API
+
+
+
+sealed abstract class Configuration
+
+class RestartStrategy(@BeanProperty val scheme: FailOverScheme, @BeanProperty val maxNrOfRetries: Int, @BeanProperty val withinTimeRange: Int) extends Configuration {
+ def transform = akka.kernel.RestartStrategy(scheme.transform, maxNrOfRetries, withinTimeRange)
+}
+class LifeCycle(@BeanProperty val scope: Scope, @BeanProperty val shutdownTime: Int) extends Configuration {
+ def transform = akka.kernel.LifeCycle(scope.transform, shutdownTime)
+}
+
+abstract class Scope extends Configuration {
+ def transform: akka.kernel.Scope
+}
+class Permanent extends Scope {
+ override def transform = akka.kernel.Permanent
+}
+class Transient extends Scope {
+ override def transform = akka.kernel.Transient
+}
+class Temporary extends Scope {
+ override def transform = akka.kernel.Temporary
+}
+
+abstract class FailOverScheme extends Configuration {
+ def transform: akka.kernel.FailOverScheme
+}
+class AllForOne extends FailOverScheme {
+ override def transform = akka.kernel.AllForOne
+}
+class OneForOne extends FailOverScheme {
+ override def transform = akka.kernel.OneForOne
+}
+
+abstract class Server extends Configuration
+//class kernelConfig(@BeanProperty val restartStrategy: RestartStrategy, @BeanProperty val servers: JList[Server]) extends Server {
+// def transform = akka.kernel.kernelConfig(restartStrategy.transform, servers.toArray.toList.asInstanceOf[List[Server]].map(_.transform))
+//}
+class Component(@BeanProperty val intf: Class[_],
+ @BeanProperty val target: Class[_],
+ @BeanProperty val lifeCycle: LifeCycle,
+ @BeanProperty val timeout: Int) extends Server {
+ def newWorker(proxy: TypedActorProxy) = akka.kernel.Supervise(proxy.server, lifeCycle.transform)
+}
+*/
diff --git a/akka-actor/src/main/scala/akka/config/Configurator.scala b/akka-actor/src/main/scala/akka/config/Configurator.scala
new file mode 100644
index 0000000000..f63c96b065
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/config/Configurator.scala
@@ -0,0 +1,21 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.config
+
+import akka.config.Supervision. {SuperviseTypedActor, FaultHandlingStrategy}
+
+private[akka] trait TypedActorConfiguratorBase {
+ def getExternalDependency[T](clazz: Class[T]): T
+
+ def configure(restartStrategy: FaultHandlingStrategy, components: List[SuperviseTypedActor]): TypedActorConfiguratorBase
+
+ def inject: TypedActorConfiguratorBase
+
+ def supervise: TypedActorConfiguratorBase
+
+ def reset
+
+ def stop
+}
diff --git a/akka-actor/src/main/scala/akka/config/SupervisionConfig.scala b/akka-actor/src/main/scala/akka/config/SupervisionConfig.scala
new file mode 100644
index 0000000000..f3d6e1ada9
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/config/SupervisionConfig.scala
@@ -0,0 +1,136 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.config
+
+import akka.actor.{ActorRef}
+import akka.dispatch.MessageDispatcher
+
+case class RemoteAddress(val hostname: String, val port: Int)
+
+/**
+ * Configuration classes - not to be used as messages.
+ *
+ * @author Jonas Bonér
+ */
+object Supervision {
+ sealed abstract class ConfigElement
+
+ abstract class Server extends ConfigElement
+ sealed abstract class LifeCycle extends ConfigElement
+ sealed abstract class FaultHandlingStrategy(val trapExit: List[Class[_ <: Throwable]]) extends ConfigElement
+
+ case class SupervisorConfig(restartStrategy: FaultHandlingStrategy, worker: List[Server]) extends Server {
+ //Java API
+ def this(restartStrategy: FaultHandlingStrategy, worker: Array[Server]) = this(restartStrategy,worker.toList)
+ }
+
+ class Supervise(val actorRef: ActorRef, val lifeCycle: LifeCycle, val remoteAddress: Option[RemoteAddress]) extends Server {
+ //Java API
+ def this(actorRef: ActorRef, lifeCycle: LifeCycle, remoteAddress: RemoteAddress) =
+ this(actorRef, lifeCycle, Option(remoteAddress))
+
+ //Java API
+ def this(actorRef: ActorRef, lifeCycle: LifeCycle) =
+ this(actorRef, lifeCycle, None)
+ }
+
+ object Supervise {
+ def apply(actorRef: ActorRef, lifeCycle: LifeCycle, remoteAddress: RemoteAddress) = new Supervise(actorRef, lifeCycle, remoteAddress)
+ def apply(actorRef: ActorRef, lifeCycle: LifeCycle) = new Supervise(actorRef, lifeCycle, None)
+ def unapply(supervise: Supervise) = Some((supervise.actorRef, supervise.lifeCycle, supervise.remoteAddress))
+ }
+
+ object AllForOneStrategy {
+ def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): AllForOneStrategy =
+ new AllForOneStrategy(trapExit,
+ if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
+ }
+
+ case class AllForOneStrategy(override val trapExit: List[Class[_ <: Throwable]],
+ maxNrOfRetries: Option[Int] = None,
+ withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy(trapExit) {
+ def this(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
+ this(trapExit,
+ if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
+
+ def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
+ this(trapExit.toList,
+ if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
+
+ def this(trapExit: java.util.List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
+ this(trapExit.toArray.toList.asInstanceOf[List[Class[_ <: Throwable]]],
+ if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
+ }
+
+ object OneForOneStrategy {
+ def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int): OneForOneStrategy =
+ new OneForOneStrategy(trapExit,
+ if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
+ }
+
+ case class OneForOneStrategy(override val trapExit: List[Class[_ <: Throwable]],
+ maxNrOfRetries: Option[Int] = None,
+ withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy(trapExit) {
+ def this(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
+ this(trapExit,
+ if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
+
+ def this(trapExit: Array[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
+ this(trapExit.toList,
+ if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
+
+ def this(trapExit: java.util.List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
+ this(trapExit.toArray.toList.asInstanceOf[List[Class[_ <: Throwable]]],
+ if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
+ }
+
+ case object NoFaultHandlingStrategy extends FaultHandlingStrategy(Nil)
+
+ //Scala API
+ case object Permanent extends LifeCycle
+ case object Temporary extends LifeCycle
+ case object UndefinedLifeCycle extends LifeCycle
+
+ //Java API (& Scala if you fancy)
+ def permanent(): LifeCycle = Permanent
+ def temporary(): LifeCycle = Temporary
+ def undefinedLifeCycle(): LifeCycle = UndefinedLifeCycle
+
+ //Java API
+ def noFaultHandlingStrategy = NoFaultHandlingStrategy
+
+ case class SuperviseTypedActor(_intf: Class[_],
+ val target: Class[_],
+ val lifeCycle: LifeCycle,
+ val timeout: Long,
+ _dispatcher: MessageDispatcher, // optional
+ _remoteAddress: RemoteAddress // optional
+ ) extends Server {
+ val intf: Option[Class[_]] = Option(_intf)
+ val dispatcher: Option[MessageDispatcher] = Option(_dispatcher)
+ val remoteAddress: Option[RemoteAddress] = Option(_remoteAddress)
+
+ def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long) =
+ this(null: Class[_], target, lifeCycle, timeout, null: MessageDispatcher, null: RemoteAddress)
+
+ def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long) =
+ this(intf, target, lifeCycle, timeout, null: MessageDispatcher, null: RemoteAddress)
+
+ def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher) =
+ this(intf, target, lifeCycle, timeout, dispatcher, null: RemoteAddress)
+
+ def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher) =
+ this(null: Class[_], target, lifeCycle, timeout, dispatcher, null: RemoteAddress)
+
+ def this(intf: Class[_], target: Class[_], lifeCycle: LifeCycle, timeout: Long, remoteAddress: RemoteAddress) =
+ this(intf, target, lifeCycle, timeout, null: MessageDispatcher, remoteAddress)
+
+ def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, remoteAddress: RemoteAddress) =
+ this(null: Class[_], target, lifeCycle, timeout, null: MessageDispatcher, remoteAddress)
+
+ def this(target: Class[_], lifeCycle: LifeCycle, timeout: Long, dispatcher: MessageDispatcher, remoteAddress: RemoteAddress) =
+ this(null: Class[_], target, lifeCycle, timeout, dispatcher, remoteAddress)
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/dataflow/DataFlowVariable.scala b/akka-actor/src/main/scala/akka/dataflow/DataFlowVariable.scala
new file mode 100644
index 0000000000..5b72503c43
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dataflow/DataFlowVariable.scala
@@ -0,0 +1,195 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dataflow
+
+import java.util.concurrent.atomic.AtomicReference
+import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue}
+
+import akka.actor.{Actor, ActorRef}
+import akka.actor.Actor._
+import akka.dispatch.CompletableFuture
+import akka.AkkaException
+import akka.japi.{ Function, SideEffect }
+
+/**
+ * Implements Oz-style dataflow (single assignment) variables.
+ *
+ * @author Jonas Bonér
+ */
+object DataFlow {
+ object Start
+ object Exit
+
+ class DataFlowVariableException(msg: String) extends AkkaException(msg)
+
+ /** Executes the supplied thunk in another thread
+ */
+ def thread(body: => Unit): Unit = spawn(body)
+
+ /** Executes the supplied SideEffect in another thread
+ * JavaAPI
+ */
+ def thread(body: SideEffect): Unit = spawn(body.apply)
+
+ /** Executes the supplied function in another thread
+ */
+ def thread[A <: AnyRef, R <: AnyRef](body: A => R) =
+ actorOf(new ReactiveEventBasedThread(body)).start
+
+ /** Executes the supplied Function in another thread
+ * JavaAPI
+ */
+ def thread[A <: AnyRef, R <: AnyRef](body: Function[A,R]) =
+ actorOf(new ReactiveEventBasedThread(body.apply)).start
+
+ private class ReactiveEventBasedThread[A <: AnyRef, T <: AnyRef](body: A => T)
+ extends Actor {
+ def receive = {
+ case Exit => self.stop
+ case message => self.reply(body(message.asInstanceOf[A]))
+ }
+ }
+
+ private object DataFlowVariable {
+ private sealed abstract class DataFlowVariableMessage
+ private case class Set[T <: Any](value: T) extends DataFlowVariableMessage
+ private object Get extends DataFlowVariableMessage
+ }
+
+ /**
+ * @author Jonas Bonér
+ */
+ sealed class DataFlowVariable[T <: Any](timeoutMs: Long) {
+ import DataFlowVariable._
+
+ def this() = this(1000 * 60)
+
+ private val value = new AtomicReference[Option[T]](None)
+ private val blockedReaders = new ConcurrentLinkedQueue[ActorRef]
+
+ private class In[T <: Any](dataFlow: DataFlowVariable[T]) extends Actor {
+ self.timeout = timeoutMs
+ def receive = {
+ case s@Set(v) =>
+ if (dataFlow.value.compareAndSet(None, Some(v.asInstanceOf[T]))) {
+ while(dataFlow.blockedReaders.peek ne null)
+ dataFlow.blockedReaders.poll ! s
+ } else throw new DataFlowVariableException(
+ "Attempt to change data flow variable (from [" + dataFlow.value.get + "] to [" + v + "])")
+ case Exit => self.stop
+ }
+ }
+
+ private class Out[T <: Any](dataFlow: DataFlowVariable[T]) extends Actor {
+ self.timeout = timeoutMs
+ private var readerFuture: Option[CompletableFuture[Any]] = None
+ def receive = {
+ case Get => dataFlow.value.get match {
+ case Some(value) => self reply value
+ case None => readerFuture = self.senderFuture
+ }
+ case Set(v:T) => readerFuture.map(_ completeWithResult v)
+ case Exit => self.stop
+ }
+ }
+
+ private[this] val in = actorOf(new In(this)).start
+
+ /** Sets the value of this variable (if unset) with the value of the supplied variable
+ */
+ def <<(ref: DataFlowVariable[T]) {
+ if (this.value.get.isEmpty) in ! Set(ref())
+ else throw new DataFlowVariableException(
+ "Attempt to change data flow variable (from [" + this.value.get + "] to [" + ref() + "])")
+ }
+
+ /** Sets the value of this variable (if unset) with the value of the supplied variable
+ * JavaAPI
+ */
+ def set(ref: DataFlowVariable[T]) { this << ref }
+
+ /** Sets the value of this variable (if unset)
+ */
+ def <<(value: T) {
+ if (this.value.get.isEmpty) in ! Set(value)
+ else throw new DataFlowVariableException(
+ "Attempt to change data flow variable (from [" + this.value.get + "] to [" + value + "])")
+ }
+
+ /** Sets the value of this variable (if unset) with the value of the supplied variable
+ * JavaAPI
+ */
+ def set(value: T) { this << value }
+
+ /** Retrieves the value of variable
+ * throws a DataFlowVariableException if it times out
+ */
+ def get(): T = this()
+
+ /** Retrieves the value of variable
+ * throws a DataFlowVariableException if it times out
+ */
+ def apply(): T = {
+ value.get getOrElse {
+ val out = actorOf(new Out(this)).start
+
+ val result = try {
+ blockedReaders offer out
+ (out !! Get).as[T]
+ } catch {
+ case e: Exception =>
+ out ! Exit
+ throw e
+ }
+
+ result.getOrElse(throw new DataFlowVariableException("Timed out (after " + timeoutMs + " milliseconds) while waiting for result"))
+ }
+ }
+
+ def shutdown = in ! Exit
+ }
+
+ /**
+ * @author Jonas Bonér
+ */
+ class DataFlowStream[T <: Any] extends Seq[T] {
+ private[this] val queue = new LinkedBlockingQueue[DataFlowVariable[T]]
+
+ def <<<(ref: DataFlowVariable[T]) = queue.offer(ref)
+
+ def <<<(value: T) = {
+ val ref = new DataFlowVariable[T]
+ ref << value
+ queue.offer(ref)
+ }
+
+ def apply(): T = {
+ val ref = queue.take
+ val result = ref()
+ ref.shutdown
+ result
+ }
+
+ def take: DataFlowVariable[T] = queue.take
+
+ //==== For Seq ====
+
+ def length: Int = queue.size
+
+ def apply(i: Int): T = {
+ if (i == 0) apply()
+ else throw new UnsupportedOperationException(
+ "Access by index other than '0' is not supported by DataFlowStream")
+ }
+
+ def iterator: Iterator[T] = new Iterator[T] {
+ private val iter = queue.iterator
+ def hasNext: Boolean = iter.hasNext
+ def next: T = { val ref = iter.next; ref() }
+ }
+
+ override def toList: List[T] = queue.toArray.toList.asInstanceOf[List[T]]
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
new file mode 100644
index 0000000000..4d33bf03ce
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dispatch/Dispatchers.scala
@@ -0,0 +1,222 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dispatch
+
+import akka.actor.{Actor, ActorRef}
+import akka.config.Config._
+import akka.util.{Duration, Logging}
+import akka.actor.newUuid
+
+import net.lag.configgy.ConfigMap
+
+import java.util.concurrent.ThreadPoolExecutor.{AbortPolicy, CallerRunsPolicy, DiscardOldestPolicy, DiscardPolicy}
+import java.util.concurrent.TimeUnit
+
+/**
+ * Scala API. Dispatcher factory.
+ *
+ * Example usage:
+ *
+ * val dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("name")
+ * dispatcher
+ * .withNewThreadPoolWithBoundedBlockingQueue(100)
+ * .setCorePoolSize(16)
+ * .setMaxPoolSize(128)
+ * .setKeepAliveTimeInMillis(60000)
+ * .setRejectionPolicy(new CallerRunsPolicy)
+ * .buildThreadPool
+ *
+ *
+ * Java API. Dispatcher factory.
+ *
+ * Example usage:
+ *
+ * MessageDispatcher dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("name");
+ * dispatcher
+ * .withNewThreadPoolWithBoundedBlockingQueue(100)
+ * .setCorePoolSize(16)
+ * .setMaxPoolSize(128)
+ * .setKeepAliveTimeInMillis(60000)
+ * .setRejectionPolicy(new CallerRunsPolicy)
+ * .buildThreadPool();
+ *
+ *
+ *
+ * @author Jonas Bonér
+ */
+object Dispatchers extends Logging {
+ val THROUGHPUT = config.getInt("akka.actor.throughput", 5)
+ val DEFAULT_SHUTDOWN_TIMEOUT = config.getLong("akka.actor.dispatcher-shutdown-timeout").
+ map(time => Duration(time, TIME_UNIT)).
+ getOrElse(Duration(1000,TimeUnit.MILLISECONDS))
+ val MAILBOX_CAPACITY = config.getInt("akka.actor.default-dispatcher.mailbox-capacity", -1)
+ val MAILBOX_PUSH_TIME_OUT = Duration(config.getInt("akka.actor.default-dispatcher.mailbox-push-timeout-time", 10), TIME_UNIT)
+ val THROUGHPUT_DEADLINE_TIME = Duration(config.getInt("akka.actor.throughput-deadline-time",-1), TIME_UNIT)
+ val THROUGHPUT_DEADLINE_TIME_MILLIS = THROUGHPUT_DEADLINE_TIME.toMillis.toInt
+ val MAILBOX_TYPE: MailboxType = if (MAILBOX_CAPACITY < 0) UnboundedMailbox() else BoundedMailbox()
+
+ lazy val defaultGlobalDispatcher = {
+ config.getConfigMap("akka.actor.default-dispatcher").flatMap(from).getOrElse(globalExecutorBasedEventDrivenDispatcher)
+ }
+
+ object globalHawtDispatcher extends HawtDispatcher
+
+ object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher("global", THROUGHPUT, THROUGHPUT_DEADLINE_TIME_MILLIS, MAILBOX_TYPE)
+
+ /**
+ * Creates an event-driven dispatcher based on the excellent HawtDispatch library.
+ *
+ * Can be beneficial to use the HawtDispatcher.pin(self) to "pin" an actor to a specific thread.
+ *
+ * See the ScalaDoc for the {@link akka.dispatch.HawtDispatcher} for details.
+ */
+ def newHawtDispatcher(aggregate: Boolean) = new HawtDispatcher(aggregate)
+
+ /**
+ * Creates an thread based dispatcher serving a single actor through the same single thread.
+ * Uses the default timeout
+ *
+ * E.g. each actor consumes its own thread.
+ */
+ def newThreadBasedDispatcher(actor: ActorRef) = new ThreadBasedDispatcher(actor)
+
+ /**
+ * Creates an thread based dispatcher serving a single actor through the same single thread.
+ * Uses the default timeout
+ * If capacity is negative, it's Integer.MAX_VALUE
+ *
+ * E.g. each actor consumes its own thread.
+ */
+ def newThreadBasedDispatcher(actor: ActorRef, mailboxCapacity: Int) = new ThreadBasedDispatcher(actor, mailboxCapacity)
+
+ /**
+ * Creates an thread based dispatcher serving a single actor through the same single thread.
+ * If capacity is negative, it's Integer.MAX_VALUE
+ *
+ * E.g. each actor consumes its own thread.
+ */
+ def newThreadBasedDispatcher(actor: ActorRef, mailboxCapacity: Int, pushTimeOut: Duration) =
+ new ThreadBasedDispatcher(actor, mailboxCapacity, pushTimeOut)
+
+ /**
+ * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
+ *
+ * Has a fluent builder interface for configuring its semantics.
+ */
+ def newExecutorBasedEventDrivenDispatcher(name: String) =
+ ThreadPoolConfigDispatcherBuilder(config => new ExecutorBasedEventDrivenDispatcher(name,config),ThreadPoolConfig())
+
+ /**
+ * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
+ *
+ * Has a fluent builder interface for configuring its semantics.
+ */
+ def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, mailboxType: MailboxType) =
+ ThreadPoolConfigDispatcherBuilder(config =>
+ new ExecutorBasedEventDrivenDispatcher(name, throughput, THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType, config),ThreadPoolConfig())
+
+
+ /**
+ * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
+ *
+ * Has a fluent builder interface for configuring its semantics.
+ */
+ def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxType: MailboxType) =
+ ThreadPoolConfigDispatcherBuilder(config =>
+ new ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineMs, mailboxType, config),ThreadPoolConfig())
+
+ /**
+ * Creates a executor-based event-driven dispatcher with work stealing (TODO: better doc) serving multiple (millions) of actors through a thread pool.
+ *
+ * Has a fluent builder interface for configuring its semantics.
+ */
+ def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String): ThreadPoolConfigDispatcherBuilder =
+ newExecutorBasedEventDrivenWorkStealingDispatcher(name,MAILBOX_TYPE)
+
+ /**
+ * Creates a executor-based event-driven dispatcher with work stealing (TODO: better doc) serving multiple (millions) of actors through a thread pool.
+ *
+ * Has a fluent builder interface for configuring its semantics.
+ */
+ def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String, mailboxType: MailboxType) =
+ ThreadPoolConfigDispatcherBuilder(config => new ExecutorBasedEventDrivenWorkStealingDispatcher(name,mailboxType,config),ThreadPoolConfig())
+
+ /**
+ * 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
+ * # (ExecutorBasedEventDrivenWorkStealing), ExecutorBasedEventDriven,
+ * # Hawt, GlobalExecutorBasedEventDriven, GlobalHawt
+ * keep-alive-time = 60 # 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", newUuid.toString)
+
+ def configureThreadPool(createDispatcher: => (ThreadPoolConfig) => MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
+ import ThreadPoolConfigDispatcherBuilder.conf_?
+
+ //Apply the following options to the config if they are present in the cfg
+ ThreadPoolConfigDispatcherBuilder(createDispatcher,ThreadPoolConfig()).configure(
+ conf_?(cfg getInt "keep-alive-time" )(time => _.setKeepAliveTime(Duration(time, TIME_UNIT))),
+ conf_?(cfg getDouble "core-pool-size-factor")(factor => _.setCorePoolSizeFromFactor(factor)),
+ conf_?(cfg getDouble "max-pool-size-factor" )(factor => _.setMaxPoolSizeFromFactor(factor)),
+ conf_?(cfg getInt "executor-bounds" )(bounds => _.setExecutorBounds(bounds)),
+ conf_?(cfg getBool "allow-core-timeout" )(allow => _.setAllowCoreThreadTimeout(allow)),
+ conf_?(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)
+ })(policy => _.setRejectionPolicy(policy)))
+ }
+
+ lazy val mailboxType: MailboxType = {
+ val capacity = cfg.getInt("mailbox-capacity", MAILBOX_CAPACITY)
+ // FIXME how do we read in isBlocking for mailbox? Now set to 'false'.
+ if (capacity < 0) UnboundedMailbox()
+ else BoundedMailbox(false, capacity, Duration(cfg.getInt("mailbox-push-timeout", MAILBOX_PUSH_TIME_OUT.toMillis.toInt), TIME_UNIT))
+ }
+
+ cfg.getString("type") map {
+ case "ExecutorBasedEventDriven" =>
+ configureThreadPool(threadPoolConfig => new ExecutorBasedEventDrivenDispatcher(
+ name,
+ cfg.getInt("throughput", THROUGHPUT),
+ cfg.getInt("throughput-deadline", THROUGHPUT_DEADLINE_TIME_MILLIS),
+ mailboxType,
+ threadPoolConfig)).build
+
+ case "ExecutorBasedEventDrivenWorkStealing" =>
+ configureThreadPool(poolCfg => new ExecutorBasedEventDrivenWorkStealingDispatcher(name, mailboxType,poolCfg)).build
+
+ case "Hawt" => new HawtDispatcher(cfg.getBool("aggregate",true))
+ case "GlobalExecutorBasedEventDriven" => globalExecutorBasedEventDrivenDispatcher
+ case "GlobalHawt" => globalHawtDispatcher
+ case unknown => throw new IllegalArgumentException("Unknown dispatcher type [%s]" format unknown)
+ }
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala
new file mode 100644
index 0000000000..83ff50427a
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcher.scala
@@ -0,0 +1,224 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dispatch
+
+import akka.actor.{ActorRef, IllegalActorStateException}
+import akka.util.ReflectiveAccess.EnterpriseModule
+
+import java.util.Queue
+import akka.util.Switch
+import java.util.concurrent.atomic.AtomicReference
+import java.util.concurrent. {ExecutorService, RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue}
+
+/**
+ * Default settings are:
+ *
+ * - withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
+ * - NR_START_THREADS = 16
+ * - NR_MAX_THREADS = 128
+ * - KEEP_ALIVE_TIME = 60000L // one minute
+ *
+ *
+ *
+ * The dispatcher has a fluent builder interface to build up a thread pool to suite your use-case.
+ * There is a default thread pool defined but make use of the builder if you need it. Here are some examples.
+ *
+ *
+ * Scala API.
+ *
+ * Example usage:
+ *
+ * val dispatcher = new ExecutorBasedEventDrivenDispatcher("name")
+ * dispatcher
+ * .withNewThreadPoolWithBoundedBlockingQueue(100)
+ * .setCorePoolSize(16)
+ * .setMaxPoolSize(128)
+ * .setKeepAliveTimeInMillis(60000)
+ * .setRejectionPolicy(new CallerRunsPolicy)
+ * .buildThreadPool
+ *
+ *
+ *
+ * Java API.
+ *
+ * Example usage:
+ *
+ * ExecutorBasedEventDrivenDispatcher dispatcher = new ExecutorBasedEventDrivenDispatcher("name");
+ * dispatcher
+ * .withNewThreadPoolWithBoundedBlockingQueue(100)
+ * .setCorePoolSize(16)
+ * .setMaxPoolSize(128)
+ * .setKeepAliveTimeInMillis(60000)
+ * .setRejectionPolicy(new CallerRunsPolicy())
+ * .buildThreadPool();
+ *
+ *
+ *
+ * But the preferred way of creating dispatchers is to use
+ * the {@link akka.dispatch.Dispatchers} factory object.
+ *
+ * @author Jonas Bonér
+ * @param throughput positive integer indicates the dispatcher will only process so much messages at a time from the
+ * mailbox, without checking the mailboxes of other actors. Zero or negative means the dispatcher
+ * always continues until the mailbox is empty.
+ * Larger values (or zero or negative) increase througput, smaller values increase fairness
+ */
+class ExecutorBasedEventDrivenDispatcher(
+ _name: String,
+ val throughput: Int = Dispatchers.THROUGHPUT,
+ val throughputDeadlineTime: Int = Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS,
+ _mailboxType: MailboxType = Dispatchers.MAILBOX_TYPE,
+ val config: ThreadPoolConfig = ThreadPoolConfig())
+ extends MessageDispatcher {
+
+ def this(_name: String, throughput: Int, throughputDeadlineTime: Int, mailboxType: MailboxType) =
+ this(_name, throughput, throughputDeadlineTime, mailboxType,ThreadPoolConfig()) // Needed for Java API usage
+
+ def this(_name: String, throughput: Int, mailboxType: MailboxType) =
+ this(_name, throughput, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, mailboxType) // Needed for Java API usage
+
+ def this(_name: String, throughput: Int) =
+ this(_name, throughput, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, Dispatchers.MAILBOX_TYPE) // Needed for Java API usage
+
+ def this(_name: String, _config: ThreadPoolConfig) =
+ this(_name, Dispatchers.THROUGHPUT, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, Dispatchers.MAILBOX_TYPE, _config)
+
+ def this(_name: String) =
+ this(_name, Dispatchers.THROUGHPUT, Dispatchers.THROUGHPUT_DEADLINE_TIME_MILLIS, Dispatchers.MAILBOX_TYPE) // Needed for Java API usage
+
+ val name = "akka:event-driven:dispatcher:" + _name
+ val mailboxType = Some(_mailboxType)
+
+ private[akka] val threadFactory = new MonitorableThreadFactory(name)
+ private[akka] val executorService = new AtomicReference[ExecutorService](config.createLazyExecutorService(threadFactory))
+
+ private[akka] def dispatch(invocation: MessageInvocation) = {
+ val mbox = getMailbox(invocation.receiver)
+ mbox enqueue invocation
+ registerForExecution(mbox)
+ }
+
+ /**
+ * @return the mailbox associated with the actor
+ */
+ private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox]
+
+ override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
+
+ def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef = mailboxType match {
+ case UnboundedMailbox(blocking) => new DefaultUnboundedMessageQueue(blocking) with ExecutableMailbox {
+ def dispatcher = ExecutorBasedEventDrivenDispatcher.this
+ }
+
+ case BoundedMailbox(blocking, capacity, pushTimeOut) =>
+ new DefaultBoundedMessageQueue(capacity, pushTimeOut, blocking) with ExecutableMailbox {
+ def dispatcher = ExecutorBasedEventDrivenDispatcher.this
+ }
+ }
+
+ /**
+ * Creates and returns a durable mailbox for the given actor.
+ */
+ def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = mailboxType match {
+ // FIXME make generic (work for TypedActor as well)
+ case FileBasedDurableMailbox(serializer) => EnterpriseModule.createFileBasedMailbox(actorRef).asInstanceOf[MessageQueue]
+ case ZooKeeperBasedDurableMailbox(serializer) => EnterpriseModule.createZooKeeperBasedMailbox(actorRef).asInstanceOf[MessageQueue]
+ case BeanstalkBasedDurableMailbox(serializer) => EnterpriseModule.createBeanstalkBasedMailbox(actorRef).asInstanceOf[MessageQueue]
+ case RedisBasedDurableMailbox(serializer) => EnterpriseModule.createRedisBasedMailbox(actorRef).asInstanceOf[MessageQueue]
+ case AMQPBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("AMQPBasedDurableMailbox is not yet supported")
+ case JMSBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("JMSBasedDurableMailbox is not yet supported")
+ }
+
+ private[akka] def start = log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput)
+
+ private[akka] def shutdown {
+ val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory))
+ if (old ne null) {
+ log.debug("Shutting down %s", toString)
+ old.shutdownNow()
+ }
+ }
+
+
+ private[akka] def registerForExecution(mbox: MessageQueue with ExecutableMailbox): Unit = if (active.isOn) {
+ if (mbox.suspended.isOff && mbox.dispatcherLock.tryLock()) {
+ try {
+ executorService.get() execute mbox
+ } catch {
+ case e: RejectedExecutionException =>
+ mbox.dispatcherLock.unlock()
+ throw e
+ }
+ }
+ } else log.warning("%s is shut down,\n\tignoring the rest of the messages in the mailbox of\n\t%s", this, mbox)
+
+ override val toString = getClass.getSimpleName + "[" + name + "]"
+
+ def suspend(actorRef: ActorRef) {
+ log.debug("Suspending %s",actorRef.uuid)
+ getMailbox(actorRef).suspended.switchOn
+ }
+
+ def resume(actorRef: ActorRef) {
+ log.debug("Resuming %s",actorRef.uuid)
+ val mbox = getMailbox(actorRef)
+ mbox.suspended.switchOff
+ registerForExecution(mbox)
+ }
+}
+
+/**
+ * This is the behavior of an ExecutorBasedEventDrivenDispatchers mailbox.
+ */
+trait ExecutableMailbox extends Runnable { self: MessageQueue =>
+
+ def dispatcher: ExecutorBasedEventDrivenDispatcher
+
+ final def run = {
+ val reschedule = try {
+ try { processMailbox() } catch { case ie: InterruptedException => true }
+ } finally {
+ dispatcherLock.unlock()
+ }
+ if (reschedule || !self.isEmpty)
+ dispatcher.registerForExecution(this)
+ }
+
+ /**
+ * Process the messages in the mailbox
+ *
+ * @return true if the processing finished before the mailbox was empty, due to the throughput constraint
+ */
+ final def processMailbox(): Boolean = {
+ if (self.suspended.isOn)
+ true
+ else {
+ var nextMessage = self.dequeue
+ if (nextMessage ne null) {
+ val throttle = dispatcher.throughput > 0
+ var processedMessages = 0
+ val isDeadlineEnabled = throttle && dispatcher.throughputDeadlineTime > 0
+ val started = if (isDeadlineEnabled) System.currentTimeMillis else 0
+ do {
+ nextMessage.invoke
+
+ if (throttle) { // Will be elided when false
+ processedMessages += 1
+ if ((processedMessages >= dispatcher.throughput) ||
+ (isDeadlineEnabled && (System.currentTimeMillis - started) >= dispatcher.throughputDeadlineTime)) // If we're throttled, break out
+ return !self.isEmpty
+ }
+
+ if (self.suspended.isOn)
+ return true
+
+ nextMessage = self.dequeue
+ } while (nextMessage ne null)
+ }
+ false
+ }
+ }
+}
+
diff --git a/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
new file mode 100644
index 0000000000..69aa5d9365
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
@@ -0,0 +1,256 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dispatch
+
+import jsr166x.{Deque, ConcurrentLinkedDeque, LinkedBlockingDeque}
+
+import akka.actor.{Actor, ActorRef, IllegalActorStateException}
+import akka.util.Switch
+import java.util.concurrent. {ExecutorService, CopyOnWriteArrayList}
+import java.util.concurrent.atomic.AtomicReference
+
+/**
+ * An executor based event driven dispatcher which will try to redistribute work from busy actors to idle actors. It is assumed
+ * that all actors using the same instance of this dispatcher can process all messages that have been sent to one of the actors. I.e. the
+ * actors belong to a pool of actors, and to the client there is no guarantee about which actor instance actually processes a given message.
+ *
+ * Although the technique used in this implementation is commonly known as "work stealing", the actual implementation is probably
+ * best described as "work donating" because the actor of which work is being stolen takes the initiative.
+ *
+ * This dispatcher attempts to redistribute work between actors each time a message is dispatched on a busy actor. Work
+ * will not be redistributed when actors are busy, but no new messages are dispatched.
+ * TODO: it would be nice to be able to redistribute work even when no new messages are being dispatched, without impacting dispatching performance ?!
+ *
+ * The preferred way of creating dispatchers is to use
+ * the {@link akka.dispatch.Dispatchers} factory object.
+ *
+ * @see akka.dispatch.ExecutorBasedEventDrivenWorkStealingDispatcher
+ * @see akka.dispatch.Dispatchers
+ *
+ * @author Jan Van Besien
+ */
+class ExecutorBasedEventDrivenWorkStealingDispatcher(
+ _name: String,
+ _mailboxType: MailboxType = Dispatchers.MAILBOX_TYPE,
+ config: ThreadPoolConfig = ThreadPoolConfig()) extends MessageDispatcher {
+
+ def this(_name: String, mailboxType: MailboxType) = this(_name, mailboxType,ThreadPoolConfig())
+
+ def this(_name: String) = this(_name, Dispatchers.MAILBOX_TYPE,ThreadPoolConfig())
+
+ //implicit def actorRef2actor(actorRef: ActorRef): Actor = actorRef.actor
+
+ val mailboxType = Some(_mailboxType)
+ val name = "akka:event-driven-work-stealing:dispatcher:" + _name
+
+ /** Type of the actors registered in this dispatcher. */
+ @volatile private var actorType: Option[Class[_]] = None
+ private val pooledActors = new CopyOnWriteArrayList[ActorRef]
+ private[akka] val threadFactory = new MonitorableThreadFactory(name)
+ private[akka] val executorService = new AtomicReference[ExecutorService](config.createLazyExecutorService(threadFactory))
+
+ /** The index in the pooled actors list which was last used to steal work */
+ @volatile private var lastThiefIndex = 0
+
+ /**
+ * @return the mailbox associated with the actor
+ */
+ private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[Deque[MessageInvocation] with MessageQueue with Runnable]
+
+ override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
+
+ private[akka] def dispatch(invocation: MessageInvocation) {
+ val mbox = getMailbox(invocation.receiver)
+ mbox enqueue invocation
+ executorService.get() execute mbox
+ }
+
+ /**
+ * Try processing the mailbox of the given actor. Fails if the dispatching lock on the actor is already held by
+ * another thread (because then that thread is already processing the mailbox).
+ *
+ * @return true if the mailbox was processed, false otherwise
+ */
+ private def tryProcessMailbox(mailbox: MessageQueue): Boolean = {
+ var mailboxWasProcessed = false
+
+ // this do-wile loop is required to prevent missing new messages between the end of processing
+ // the mailbox and releasing the lock
+ do {
+ if (mailbox.dispatcherLock.tryLock) {
+ try {
+ mailboxWasProcessed = processMailbox(mailbox)
+ } finally {
+ mailbox.dispatcherLock.unlock
+ }
+ }
+ } while ((mailboxWasProcessed && !mailbox.isEmpty))
+
+ mailboxWasProcessed
+ }
+
+ /**
+ * Process the messages in the mailbox of the given actor.
+ * @return
+ */
+ private def processMailbox(mailbox: MessageQueue): Boolean = try {
+ if (mailbox.suspended.isOn)
+ return false
+
+ var messageInvocation = mailbox.dequeue
+ while (messageInvocation ne null) {
+ messageInvocation.invoke
+ if (mailbox.suspended.isOn)
+ return false
+ messageInvocation = mailbox.dequeue
+ }
+ true
+ } catch {
+ case ie: InterruptedException => false
+ }
+
+ private def findThief(receiver: ActorRef): Option[ActorRef] = {
+ // copy to prevent concurrent modifications having any impact
+ val actors = pooledActors.toArray(new Array[ActorRef](pooledActors.size))
+ val i = if ( lastThiefIndex > actors.size ) 0 else lastThiefIndex
+
+ // we risk to pick a thief which is unregistered from the dispatcher in the meantime, but that typically means
+ // the dispatcher is being shut down...
+ val (thief: Option[ActorRef], index: Int) = doFindThief(receiver, actors, i)
+ lastThiefIndex = (index + 1) % actors.size
+ thief
+ }
+
+ /**
+ * Find a thief to process the receivers messages from the given list of actors.
+ *
+ * @param receiver original receiver of the message
+ * @param actors list of actors to find a thief in
+ * @param startIndex first index to start looking in the list (i.e. for round robin)
+ * @return the thief (or None) and the new index to start searching next time
+ */
+ private def doFindThief(receiver: ActorRef, actors: Array[ActorRef], startIndex: Int): (Option[ActorRef], Int) = {
+ for (i <- 0 to actors.length) {
+ val index = (i + startIndex) % actors.length
+ val actor = actors(index)
+ if (actor != receiver && getMailbox(actor).isEmpty) return (Some(actor), index)
+ }
+ (None, startIndex) // nothing found, reuse same start index next time
+ }
+
+ /**
+ * Try donating messages to the thief and processing the thiefs mailbox. Doesn't do anything if we can not acquire
+ * the thiefs dispatching lock, because in that case another thread is already processing the thiefs mailbox.
+ */
+ private def tryDonateAndProcessMessages(receiver: ActorRef, thief: ActorRef) = {
+ val mailbox = getMailbox(thief)
+ if (mailbox.dispatcherLock.tryLock) {
+ try {
+ while(donateMessage(receiver, thief)) processMailbox(mailbox)
+ } finally {
+ mailbox.dispatcherLock.unlock
+ }
+ }
+ }
+
+ /**
+ * Steal a message from the receiver and give it to the thief.
+ */
+ private def donateMessage(receiver: ActorRef, thief: ActorRef): Boolean = {
+ val donated = getMailbox(receiver).pollLast
+ if (donated ne null) {
+ if (donated.senderFuture.isDefined) thief.postMessageToMailboxAndCreateFutureResultWithTimeout[Any](
+ donated.message, receiver.timeout, donated.sender, donated.senderFuture)
+ else if (donated.sender.isDefined) thief.postMessageToMailbox(donated.message, donated.sender)
+ else thief.postMessageToMailbox(donated.message, None)
+ true
+ } else false
+ }
+
+ private[akka] def start = log.debug("Starting up %s",toString)
+
+ private[akka] def shutdown {
+ val old = executorService.getAndSet(config.createLazyExecutorService(threadFactory))
+ if (old ne null) {
+ log.debug("Shutting down %s", toString)
+ old.shutdownNow()
+ }
+ }
+
+
+ def suspend(actorRef: ActorRef) {
+ getMailbox(actorRef).suspended.switchOn
+ }
+
+ def resume(actorRef: ActorRef) {
+ val mbox = getMailbox(actorRef)
+ mbox.suspended.switchOff
+ executorService.get() execute mbox
+ }
+
+ override val toString = "ExecutorBasedEventDrivenWorkStealingDispatcher[" + name + "]"
+
+ private[akka] def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef = mailboxType match {
+ case UnboundedMailbox(blocking) => // FIXME make use of 'blocking' in work stealer ConcurrentLinkedDeque
+ new ConcurrentLinkedDeque[MessageInvocation] with MessageQueue with Runnable {
+ def enqueue(handle: MessageInvocation): Unit = this.add(handle)
+
+ def dequeue: MessageInvocation = this.poll()
+
+ def run = if (!tryProcessMailbox(this)) {
+ // we are not able to process our mailbox (another thread is busy with it), so lets donate some of our mailbox
+ // to another actor and then process his mailbox in stead.
+ findThief(actorRef).foreach( tryDonateAndProcessMessages(actorRef,_) )
+ }
+ }
+ case BoundedMailbox(blocking, capacity, pushTimeOut) =>
+ new LinkedBlockingDeque[MessageInvocation](capacity) with MessageQueue with Runnable {
+ def enqueue(handle: MessageInvocation): Unit = this.add(handle)
+
+ def dequeue: MessageInvocation = this.poll()
+
+ def run = if (!tryProcessMailbox(this)) {
+ // we are not able to process our mailbox (another thread is busy with it), so lets donate some of our mailbox
+ // to another actor and then process his mailbox in stead.
+ findThief(actorRef).foreach( tryDonateAndProcessMessages(actorRef, _) )
+ }
+ }
+ }
+
+ /**
+ * Creates and returns a durable mailbox for the given actor.
+ */
+ private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = mailboxType match {
+ // FIXME make generic (work for TypedActor as well)
+ case FileBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("FileBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
+ case ZooKeeperBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("ZooKeeperBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
+ case BeanstalkBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("BeanstalkBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
+ case RedisBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("RedisBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
+ case AMQPBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("AMQPBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
+ case JMSBasedDurableMailbox(serializer) => throw new UnsupportedOperationException("JMSBasedDurableMailbox is not yet supported for ExecutorBasedEventDrivenWorkStealingDispatcher")
+ }
+
+ private[akka] override def register(actorRef: ActorRef) = {
+ verifyActorsAreOfSameType(actorRef)
+ pooledActors add actorRef
+ super.register(actorRef)
+ }
+
+ private[akka] override def unregister(actorRef: ActorRef) = {
+ pooledActors remove actorRef
+ super.unregister(actorRef)
+ }
+
+ private def verifyActorsAreOfSameType(actorOfId: ActorRef) = {
+ actorType match {
+ case None => actorType = Some(actorOfId.actor.getClass)
+ case Some(aType) =>
+ if (aType != actorOfId.actor.getClass)
+ throw new IllegalActorStateException(String.format(
+ "Can't register actor %s in a work stealing dispatcher which already knows actors of type %s",
+ actorOfId.actor, aType))
+ }
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/dispatch/Future.scala b/akka-actor/src/main/scala/akka/dispatch/Future.scala
new file mode 100644
index 0000000000..2ac412d36d
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dispatch/Future.scala
@@ -0,0 +1,264 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dispatch
+
+import akka.AkkaException
+import akka.actor.Actor.spawn
+import akka.routing.Dispatcher
+
+import java.util.concurrent.locks.ReentrantLock
+import java.util.concurrent.TimeUnit
+import akka.japi.Procedure
+
+class FutureTimeoutException(message: String) extends AkkaException(message)
+
+object Futures {
+
+ /**
+ * Module with utility methods for working with Futures.
+ *
+ * val future = Futures.future(1000) {
+ * ... // do stuff
+ * }
+ *
+ */
+ def future[T](timeout: Long,
+ dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher)
+ (body: => T): Future[T] = {
+ val f = new DefaultCompletableFuture[T](timeout)
+ spawn({
+ try { f completeWithResult body }
+ catch { case e => f completeWithException e}
+ })(dispatcher)
+ f
+ }
+
+ /**
+ * (Blocking!)
+ */
+ def awaitAll(futures: List[Future[_]]): Unit = futures.foreach(_.await)
+
+ /**
+ * Returns the First Future that is completed (blocking!)
+ */
+ def awaitOne(futures: List[Future[_]], timeout: Long = Long.MaxValue): Future[_] = firstCompletedOf(futures).await
+
+ /**
+ * Returns a Future to the result of the first future in the list that is completed
+ */
+ def firstCompletedOf(futures: List[Future[_]], timeout: Long = Long.MaxValue): Future[_] = {
+ val futureResult = new DefaultCompletableFuture[Any](timeout)
+ val fun = (f: Future[_]) => futureResult completeWith f.asInstanceOf[Future[Any]]
+ for(f <- futures) f onComplete fun
+ futureResult
+ }
+
+ /**
+ * Applies the supplied function to the specified collection of Futures after awaiting each future to be completed
+ */
+ def awaitMap[A,B](in: Traversable[Future[A]])(fun: (Future[A]) => B): Traversable[B] =
+ in map { f => fun(f.await) }
+
+ /**
+ * Returns Future.resultOrException of the first completed of the 2 Futures provided (blocking!)
+ */
+ def awaitEither[T](f1: Future[T], f2: Future[T]): Option[T] = awaitOne(List(f1,f2)).asInstanceOf[Future[T]].resultOrException
+}
+
+sealed trait Future[T] {
+ def await : Future[T]
+
+ def awaitBlocking : Future[T]
+
+ def isCompleted: Boolean
+
+ def isExpired: Boolean
+
+ def timeoutInNanos: Long
+
+ def result: Option[T]
+
+ def exception: Option[Throwable]
+
+ def onComplete(func: Future[T] => Unit): Future[T]
+
+ /**
+ * Returns the current result, throws the exception is one has been raised, else returns None
+ */
+ def resultOrException: Option[T] = {
+ val r = result
+ if (r.isDefined) result
+ else {
+ val problem = exception
+ if (problem.isDefined) throw problem.get
+ else None
+ }
+ }
+
+ /* Java API */
+ def onComplete(proc: Procedure[Future[T]]): Future[T] = onComplete(f => proc(f))
+
+ def map[O](f: (T) => O): Future[O] = {
+ val wrapped = this
+ new Future[O] {
+ def await = { wrapped.await; this }
+ def awaitBlocking = { wrapped.awaitBlocking; this }
+ def isCompleted = wrapped.isCompleted
+ def isExpired = wrapped.isExpired
+ def timeoutInNanos = wrapped.timeoutInNanos
+ def result: Option[O] = { wrapped.result map f }
+ def exception: Option[Throwable] = wrapped.exception
+ def onComplete(func: Future[O] => Unit): Future[O] = { wrapped.onComplete(_ => func(this)); this }
+ }
+ }
+}
+
+trait CompletableFuture[T] extends Future[T] {
+ def completeWithResult(result: T)
+ def completeWithException(exception: Throwable)
+ def completeWith(other: Future[T]) {
+ val result = other.result
+ val exception = other.exception
+ if (result.isDefined) completeWithResult(result.get)
+ else if (exception.isDefined) completeWithException(exception.get)
+ //else TODO how to handle this case?
+ }
+}
+
+// Based on code from the actorom actor framework by Sergio Bossa [http://code.google.com/p/actorom/].
+class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] {
+ import TimeUnit.{MILLISECONDS => TIME_UNIT}
+
+ def this() = this(0)
+
+ val timeoutInNanos = TIME_UNIT.toNanos(timeout)
+ private val _startTimeInNanos = currentTimeInNanos
+ private val _lock = new ReentrantLock
+ private val _signal = _lock.newCondition
+ private var _completed: Boolean = _
+ private var _result: Option[T] = None
+ private var _exception: Option[Throwable] = None
+ private var _listeners: List[Future[T] => Unit] = Nil
+
+ def await = try {
+ _lock.lock
+ var wait = timeoutInNanos - (currentTimeInNanos - _startTimeInNanos)
+ while (!_completed && wait > 0) {
+ var start = currentTimeInNanos
+ try {
+ wait = _signal.awaitNanos(wait)
+ if (wait <= 0) throw new FutureTimeoutException("Futures timed out after [" + timeout + "] milliseconds")
+ } catch {
+ case e: InterruptedException =>
+ wait = wait - (currentTimeInNanos - start)
+ }
+ }
+ this
+ } finally {
+ _lock.unlock
+ }
+
+ def awaitBlocking = try {
+ _lock.lock
+ while (!_completed) {
+ _signal.await
+ }
+ this
+ } finally {
+ _lock.unlock
+ }
+
+ def isCompleted: Boolean = try {
+ _lock.lock
+ _completed
+ } finally {
+ _lock.unlock
+ }
+
+ def isExpired: Boolean = try {
+ _lock.lock
+ timeoutInNanos - (currentTimeInNanos - _startTimeInNanos) <= 0
+ } finally {
+ _lock.unlock
+ }
+
+ def result: Option[T] = try {
+ _lock.lock
+ _result
+ } finally {
+ _lock.unlock
+ }
+
+ def exception: Option[Throwable] = try {
+ _lock.lock
+ _exception
+ } finally {
+ _lock.unlock
+ }
+
+ def completeWithResult(result: T) {
+ val notify = try {
+ _lock.lock
+ if (!_completed) {
+ _completed = true
+ _result = Some(result)
+ true
+ } else false
+ } finally {
+ _signal.signalAll
+ _lock.unlock
+ }
+
+ if (notify)
+ notifyListeners
+ }
+
+ def completeWithException(exception: Throwable) {
+ val notify = try {
+ _lock.lock
+ if (!_completed) {
+ _completed = true
+ _exception = Some(exception)
+ true
+ } else false
+ } finally {
+ _signal.signalAll
+ _lock.unlock
+ }
+
+ if (notify)
+ notifyListeners
+ }
+
+ def onComplete(func: Future[T] => Unit): CompletableFuture[T] = {
+ val notifyNow = try {
+ _lock.lock
+ if (!_completed) {
+ _listeners ::= func
+ false
+ }
+ else
+ true
+ } finally {
+ _lock.unlock
+ }
+
+ if (notifyNow)
+ notifyListener(func)
+
+ this
+ }
+
+ private def notifyListeners() {
+ for(l <- _listeners)
+ notifyListener(l)
+ }
+
+ private def notifyListener(func: Future[T] => Unit) {
+ func(this)
+ }
+
+ private def currentTimeInNanos: Long = TIME_UNIT.toNanos(System.currentTimeMillis)
+}
diff --git a/akka-actor/src/main/scala/akka/dispatch/HawtDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/HawtDispatcher.scala
new file mode 100644
index 0000000000..1031ae4c9a
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dispatch/HawtDispatcher.scala
@@ -0,0 +1,212 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dispatch
+
+import akka.actor.ActorRef
+
+import org.fusesource.hawtdispatch.DispatchQueue
+import org.fusesource.hawtdispatch.ScalaDispatch._
+import org.fusesource.hawtdispatch.DispatchQueue.QueueType
+import org.fusesource.hawtdispatch.ListEventAggregator
+
+import java.util.concurrent.atomic.{AtomicInteger, AtomicBoolean}
+import java.util.concurrent.CountDownLatch
+import akka.util.Switch
+
+/**
+ * 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 _ =>
+ }
+ }
+ }.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.
+ *
+ *
+ * @author Hiram Chirino
+ */
+class HawtDispatcher(val aggregate: Boolean = true, val parent: DispatchQueue = globalQueue) extends MessageDispatcher {
+ import HawtDispatcher._
+
+ val mailboxType: Option[MailboxType] = None
+
+ private[akka] def start { retainNonDaemon }
+
+ private[akka] def shutdown { releaseNonDaemon }
+
+ private[akka] def dispatch(invocation: MessageInvocation){
+ mailbox(invocation.receiver).dispatch(invocation)
+ }
+
+ // hawtdispatch does not have a way to get queue sizes, getting an accurate
+ // size can cause extra contention.. is this really needed?
+ // TODO: figure out if this can be optional in akka
+ override def mailboxSize(actorRef: ActorRef) = 0
+
+ override def createMailbox(actorRef: ActorRef): AnyRef = {
+ val queue = parent.createSerialQueue(actorRef.toString)
+ if (aggregate) new AggregatingHawtDispatcherMailbox(queue)
+ else new HawtDispatcherMailbox(queue)
+ }
+
+ def suspend(actorRef: ActorRef) = mailbox(actorRef).suspend
+ def resume(actorRef:ActorRef) = mailbox(actorRef).resume
+
+ private[akka] def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef = null.asInstanceOf[AnyRef]
+
+ /**
+ * Creates and returns a durable mailbox for the given actor.
+ */
+ private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef = null.asInstanceOf[AnyRef]
+
+ override def toString = "HawtDispatcher"
+}
+
+class HawtDispatcherMailbox(val queue: DispatchQueue) {
+ def dispatch(invocation: MessageInvocation) {
+ queue {
+ invocation.invoke
+ }
+ }
+
+ def suspend = queue.suspend
+ def resume = queue.resume
+}
+
+class AggregatingHawtDispatcherMailbox(queue:DispatchQueue) extends HawtDispatcherMailbox(queue) {
+ private val source = createSource(new ListEventAggregator[MessageInvocation](), queue)
+ source.setEventHandler (^{drain_source} )
+ source.resume
+
+ private def drain_source = source.getData.foreach(_.invoke)
+
+ override def suspend = source.suspend
+ override def resume = source.resume
+
+ override def dispatch(invocation: MessageInvocation) {
+ if (getCurrentQueue eq null) {
+ // we are being call from a non hawtdispatch thread, can't aggregate
+ // it's events
+ super.dispatch(invocation)
+ } else {
+ // we are being call from a hawtdispatch thread, use the dispatch source
+ // so that multiple invocations issues on this thread will aggregate and then once
+ // the thread runs out of work, they get transferred as a batch to the other thread.
+ source.merge(invocation)
+ }
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/dispatch/MailboxHandling.scala b/akka-actor/src/main/scala/akka/dispatch/MailboxHandling.scala
new file mode 100644
index 0000000000..ff71b607ce
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dispatch/MailboxHandling.scala
@@ -0,0 +1,108 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dispatch
+
+import akka.actor.{Actor, ActorType, ActorRef, ActorInitializationException}
+import akka.util.ReflectiveAccess.EnterpriseModule
+import akka.AkkaException
+
+import java.util.{Queue, List}
+import java.util.concurrent._
+import concurrent.forkjoin.LinkedTransferQueue
+import akka.util._
+
+class MessageQueueAppendFailedException(message: String) extends AkkaException(message)
+
+/**
+ * @author Jonas Bonér
+ */
+trait MessageQueue {
+ val dispatcherLock = new SimpleLock
+ val suspended = new Switch(false)
+ def enqueue(handle: MessageInvocation)
+ def dequeue(): MessageInvocation
+ def size: Int
+ def isEmpty: Boolean
+}
+
+/**
+ * Mailbox configuration.
+ */
+sealed trait MailboxType
+
+abstract class TransientMailboxType(val blocking: Boolean = false) extends MailboxType
+case class UnboundedMailbox(block: Boolean = false) extends TransientMailboxType(block)
+case class BoundedMailbox(
+ block: Boolean = false,
+ val capacity: Int = { if (Dispatchers.MAILBOX_CAPACITY < 0) Int.MaxValue else Dispatchers.MAILBOX_CAPACITY },
+ val pushTimeOut: Duration = Dispatchers.MAILBOX_PUSH_TIME_OUT) extends TransientMailboxType(block) {
+ if (capacity < 0) throw new IllegalArgumentException("The capacity for BoundedMailbox can not be negative")
+ if (pushTimeOut eq null) throw new IllegalArgumentException("The push time-out for BoundedMailbox can not be null")
+}
+
+abstract class DurableMailboxType(val serializer: EnterpriseModule.Serializer) extends MailboxType {
+ if (serializer eq null) throw new IllegalArgumentException("The serializer for DurableMailboxType can not be null")
+}
+case class FileBasedDurableMailbox(ser: EnterpriseModule.Serializer) extends DurableMailboxType(ser)
+case class RedisBasedDurableMailbox(ser: EnterpriseModule.Serializer) extends DurableMailboxType(ser)
+case class BeanstalkBasedDurableMailbox(ser: EnterpriseModule.Serializer) extends DurableMailboxType(ser)
+case class ZooKeeperBasedDurableMailbox(ser: EnterpriseModule.Serializer) extends DurableMailboxType(ser)
+case class AMQPBasedDurableMailbox(ser: EnterpriseModule.Serializer) extends DurableMailboxType(ser)
+case class JMSBasedDurableMailbox(ser: EnterpriseModule.Serializer) extends DurableMailboxType(ser)
+
+class DefaultUnboundedMessageQueue(blockDequeue: Boolean)
+ extends LinkedBlockingQueue[MessageInvocation] with MessageQueue {
+
+ final def enqueue(handle: MessageInvocation) {
+ this add handle
+ }
+
+ final def dequeue(): MessageInvocation = {
+ if (blockDequeue) this.take()
+ else this.poll()
+ }
+}
+
+class DefaultBoundedMessageQueue(capacity: Int, pushTimeOut: Duration, blockDequeue: Boolean)
+ extends LinkedBlockingQueue[MessageInvocation](capacity) with MessageQueue {
+
+ final def enqueue(handle: MessageInvocation) {
+ if (pushTimeOut.toMillis > 0) {
+ if (!this.offer(handle, pushTimeOut.length, pushTimeOut.unit))
+ throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + toString)
+ } else this put handle
+ }
+
+ final def dequeue(): MessageInvocation =
+ if (blockDequeue) this.take()
+ else this.poll()
+}
+
+/**
+ * @author Jonas Bonér
+ */
+trait MailboxFactory {
+
+ val mailboxType: Option[MailboxType]
+
+ /**
+ * Creates a MessageQueue (Mailbox) with the specified properties.
+ */
+ private[akka] def createMailbox(actorRef: ActorRef): AnyRef =
+ mailboxType.getOrElse(throw new IllegalStateException("No mailbox type defined")) match {
+ case mb: TransientMailboxType => createTransientMailbox(actorRef, mb)
+ case mb: DurableMailboxType => createDurableMailbox(actorRef, mb)
+ }
+
+ /**
+ * Creates and returns a transient mailbox for the given actor.
+ */
+ private[akka] def createTransientMailbox(actorRef: ActorRef, mailboxType: TransientMailboxType): AnyRef
+
+ /**
+ * Creates and returns a durable mailbox for the given actor.
+ */
+ private[akka] def createDurableMailbox(actorRef: ActorRef, mailboxType: DurableMailboxType): AnyRef
+}
diff --git a/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala
new file mode 100644
index 0000000000..467bccd13e
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dispatch/MessageHandling.scala
@@ -0,0 +1,179 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dispatch
+
+import java.util.concurrent._
+import atomic. {AtomicInteger, AtomicBoolean, AtomicReference, AtomicLong}
+import akka.util. {Switch, ReentrantGuard, Logging, HashCode}
+import akka.actor._
+
+/**
+ * @author Jonas Bonér
+ */
+final class MessageInvocation(val receiver: ActorRef,
+ val message: Any,
+ val sender: Option[ActorRef],
+ val senderFuture: Option[CompletableFuture[Any]]) {
+ if (receiver eq null) throw new IllegalArgumentException("Receiver can't be null")
+
+ def invoke = try {
+ receiver.invoke(this)
+ } catch {
+ case e: NullPointerException => throw new ActorInitializationException(
+ "Don't call 'self ! message' in the Actor's constructor (in Scala this means in the body of the class).")
+ }
+
+ override def hashCode(): Int = {
+ var result = HashCode.SEED
+ result = HashCode.hash(result, receiver.actor)
+ result = HashCode.hash(result, message.asInstanceOf[AnyRef])
+ result
+ }
+
+ override def equals(that: Any): Boolean = {
+ that.isInstanceOf[MessageInvocation] &&
+ that.asInstanceOf[MessageInvocation].receiver.actor == receiver.actor &&
+ that.asInstanceOf[MessageInvocation].message == message
+ }
+
+ override def toString = {
+ "MessageInvocation[" +
+ "\n\tmessage = " + message +
+ "\n\treceiver = " + receiver +
+ "\n\tsender = " + sender +
+ "\n\tsenderFuture = " + senderFuture +
+ "]"
+ }
+}
+
+object MessageDispatcher {
+ val UNSCHEDULED = 0
+ val SCHEDULED = 1
+ val RESCHEDULED = 2
+}
+
+/**
+ * @author Jonas Bonér
+ */
+trait MessageDispatcher extends MailboxFactory with Logging {
+ import MessageDispatcher._
+ protected val uuids = new ConcurrentSkipListSet[Uuid]
+ protected val guard = new ReentrantGuard
+ private var shutdownSchedule = UNSCHEDULED //This can be non-volatile since it is protected by guard withGuard
+ protected val active = new Switch(false)
+
+ /**
+ * Attaches the specified actorRef to this dispatcher
+ */
+ final def attach(actorRef: ActorRef): Unit = guard withGuard {
+ register(actorRef)
+ }
+
+ /**
+ * Detaches the specified actorRef from this dispatcher
+ */
+ final def detach(actorRef: ActorRef): Unit = guard withGuard {
+ unregister(actorRef)
+ }
+
+ private[akka] final def dispatchMessage(invocation: MessageInvocation): Unit = if (active.isOn) {
+ dispatch(invocation)
+ } else throw new IllegalActorStateException("Can't submit invocations to dispatcher since it's not started")
+
+ private[akka] def register(actorRef: ActorRef) {
+ if (actorRef.mailbox eq null) actorRef.mailbox = createMailbox(actorRef)
+ uuids add actorRef.uuid
+ if (active.isOff) {
+ active.switchOn {
+ start
+ }
+ }
+ }
+
+ private[akka] def unregister(actorRef: ActorRef) = {
+ if (uuids remove actorRef.uuid) {
+ actorRef.mailbox = null
+ if (uuids.isEmpty){
+ shutdownSchedule match {
+ case UNSCHEDULED =>
+ shutdownSchedule = SCHEDULED
+ Scheduler.scheduleOnce(shutdownAction, timeoutMs, TimeUnit.MILLISECONDS)
+ case SCHEDULED =>
+ shutdownSchedule = RESCHEDULED
+ case RESCHEDULED => //Already marked for reschedule
+ }
+ }
+ }
+ }
+
+ /**
+ * Traverses the list of actors (uuids) currently being attached to this dispatcher and stops those actors
+ */
+ def stopAllAttachedActors {
+ val i = uuids.iterator
+ while(i.hasNext()) {
+ val uuid = i.next()
+ ActorRegistry.actorFor(uuid) match {
+ case Some(actor) => actor.stop
+ case None =>
+ log.error("stopAllLinkedActors couldn't find linked actor: " + uuid)
+ }
+ }
+ }
+
+ private val shutdownAction = new Runnable {
+ def run = guard withGuard {
+ shutdownSchedule match {
+ case RESCHEDULED =>
+ shutdownSchedule = SCHEDULED
+ Scheduler.scheduleOnce(this, timeoutMs, TimeUnit.MILLISECONDS)
+ case SCHEDULED =>
+ if (uuids.isEmpty()) {
+ active switchOff {
+ shutdown // shut down in the dispatcher's references is zero
+ }
+ }
+ shutdownSchedule = UNSCHEDULED
+ case UNSCHEDULED => //Do nothing
+ }
+ }
+ }
+
+ /**
+ * When the dispatcher no longer has any actors registered, how long will it wait until it shuts itself down, in Ms
+ * defaulting to your akka configs "akka.actor.dispatcher-shutdown-timeout" or otherwise, 1 Second
+ */
+ private[akka] def timeoutMs: Long = Dispatchers.DEFAULT_SHUTDOWN_TIMEOUT.toMillis
+
+ /**
+ * After the call to this method, the dispatcher mustn't begin any new message processing for the specified reference
+ */
+ def suspend(actorRef: ActorRef): Unit
+
+ /*
+ * After the call to this method, the dispatcher must begin any new message processing for the specified reference
+ */
+ def resume(actorRef: ActorRef): Unit
+
+ /**
+ * Will be called when the dispatcher is to queue an invocation for execution
+ */
+ private[akka] def dispatch(invocation: MessageInvocation): Unit
+
+ /**
+ * Called one time every time an actor is attached to this dispatcher and this dispatcher was previously shutdown
+ */
+ private[akka] def start: Unit
+
+ /**
+ * Called one time every time an actor is detached from this dispatcher and this dispatcher has no actors left attached
+ */
+ private[akka] def shutdown: Unit
+
+ /**
+ * Returns the size of the mailbox for the specified actor
+ */
+ def mailboxSize(actorRef: ActorRef): Int
+}
diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadBasedDispatcher.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadBasedDispatcher.scala
new file mode 100644
index 0000000000..ab7541e65b
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dispatch/ThreadBasedDispatcher.scala
@@ -0,0 +1,52 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dispatch
+
+import akka.actor.{Actor, ActorRef}
+import akka.config.Config.config
+import akka.util.Duration
+
+import java.util.Queue
+import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, LinkedBlockingQueue}
+import akka.actor
+import java.util.concurrent.atomic.AtomicReference
+
+/**
+ * Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
+ *
+ * @author Jonas Bonér
+ */
+class ThreadBasedDispatcher(_actor: ActorRef, _mailboxType: MailboxType)
+ extends ExecutorBasedEventDrivenDispatcher(
+ _actor.uuid.toString,Dispatchers.THROUGHPUT,-1,_mailboxType,ThreadBasedDispatcher.oneThread) {
+
+ private[akka] val owner = new AtomicReference[ActorRef](_actor)
+
+ def this(actor: ActorRef) =
+ this(actor, UnboundedMailbox(true)) // For Java API
+
+ def this(actor: ActorRef, capacity: Int) =
+ this(actor, BoundedMailbox(true, capacity)) //For Java API
+
+ def this(actor: ActorRef, capacity: Int, pushTimeOut: Duration) = //For Java API
+ this(actor, BoundedMailbox(true, capacity, pushTimeOut))
+
+ override def register(actorRef: ActorRef) = {
+ val actor = owner.get()
+ if ((actor ne null) && actorRef != actor) throw new IllegalArgumentException("Cannot register to anyone but " + actor)
+ owner.compareAndSet(null,actorRef) //Register if unregistered
+ super.register(actorRef)
+ }
+
+ override def unregister(actorRef: ActorRef) = {
+ super.unregister(actorRef)
+ owner.compareAndSet(actorRef,null) //Unregister (prevent memory leak)
+ }
+}
+
+object ThreadBasedDispatcher {
+ val oneThread: ThreadPoolConfig = ThreadPoolConfig(allowCorePoolTimeout = true, corePoolSize = 1, maxPoolSize = 1)
+}
+
diff --git a/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala
new file mode 100644
index 0000000000..f6e92892d0
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/dispatch/ThreadPoolBuilder.scala
@@ -0,0 +1,264 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dispatch
+
+import java.util.Collection
+import java.util.concurrent._
+import atomic.{AtomicLong, AtomicInteger}
+import ThreadPoolExecutor.CallerRunsPolicy
+
+import akka.util. {Duration, Logging}
+
+object ThreadPoolConfig {
+ type Bounds = Int
+ type FlowHandler = Either[RejectedExecutionHandler,Bounds]
+ type QueueFactory = () => BlockingQueue[Runnable]
+
+ val defaultAllowCoreThreadTimeout: Boolean = false
+ val defaultCorePoolSize: Int = 16
+ val defaultMaxPoolSize: Int = 128
+ val defaultTimeout: Duration = Duration(60000L,TimeUnit.MILLISECONDS)
+ def defaultFlowHandler: FlowHandler = flowHandler(new CallerRunsPolicy)
+
+ def flowHandler(rejectionHandler: RejectedExecutionHandler): FlowHandler = Left(rejectionHandler)
+ def flowHandler(bounds: Int): FlowHandler = Right(bounds)
+
+ def fixedPoolSize(size: Int): Int = size
+ def scaledPoolSize(multiplier: Double): Int =
+ (Runtime.getRuntime.availableProcessors * multiplier).ceil.toInt
+
+ def arrayBlockingQueue(capacity: Int, fair: Boolean): QueueFactory =
+ () => new ArrayBlockingQueue[Runnable](capacity,fair)
+
+ def synchronousQueue(fair: Boolean): QueueFactory =
+ () => new SynchronousQueue[Runnable](fair)
+
+ def linkedBlockingQueue(): QueueFactory =
+ () => new LinkedBlockingQueue[Runnable]()
+
+ def linkedBlockingQueue(capacity: Int): QueueFactory =
+ () => new LinkedBlockingQueue[Runnable](capacity)
+
+ def reusableQueue(queue: BlockingQueue[Runnable]): QueueFactory =
+ () => queue
+
+ def reusableQueue(queueFactory: QueueFactory): QueueFactory = {
+ val queue = queueFactory()
+ () => queue
+ }
+}
+
+case class ThreadPoolConfig(allowCorePoolTimeout: Boolean = ThreadPoolConfig.defaultAllowCoreThreadTimeout,
+ corePoolSize: Int = ThreadPoolConfig.defaultCorePoolSize,
+ maxPoolSize: Int = ThreadPoolConfig.defaultMaxPoolSize,
+ threadTimeout: Duration = ThreadPoolConfig.defaultTimeout,
+ flowHandler: ThreadPoolConfig.FlowHandler = ThreadPoolConfig.defaultFlowHandler,
+ queueFactory: ThreadPoolConfig.QueueFactory = ThreadPoolConfig.linkedBlockingQueue()) {
+
+ final def createLazyExecutorService(threadFactory: ThreadFactory): ExecutorService =
+ new LazyExecutorServiceWrapper(createExecutorService(threadFactory))
+
+ final def createExecutorService(threadFactory: ThreadFactory): ExecutorService = {
+ flowHandler match {
+ case Left(rejectHandler) =>
+ val service = new ThreadPoolExecutor(corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory, rejectHandler)
+ service.allowCoreThreadTimeOut(allowCorePoolTimeout)
+ service
+ case Right(bounds) =>
+ val service = new ThreadPoolExecutor(corePoolSize, maxPoolSize, threadTimeout.length, threadTimeout.unit, queueFactory(), threadFactory)
+ service.allowCoreThreadTimeOut(allowCorePoolTimeout)
+ new BoundedExecutorDecorator(service,bounds)
+ }
+ }
+}
+
+trait DispatcherBuilder {
+ def build: MessageDispatcher
+}
+
+object ThreadPoolConfigDispatcherBuilder {
+ def conf_?[T](opt: Option[T])(fun: (T) => ThreadPoolConfigDispatcherBuilder => ThreadPoolConfigDispatcherBuilder):
+ Option[(ThreadPoolConfigDispatcherBuilder) => ThreadPoolConfigDispatcherBuilder] = opt map fun
+}
+
+case class ThreadPoolConfigDispatcherBuilder(dispatcherFactory: (ThreadPoolConfig) => MessageDispatcher, config: ThreadPoolConfig) extends DispatcherBuilder {
+ import ThreadPoolConfig._
+ def build = dispatcherFactory(config)
+
+ //TODO remove this, for backwards compat only
+ def buildThreadPool = build
+
+ def withNewBoundedThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity(bounds: Int): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(flowHandler = flowHandler(bounds), queueFactory = linkedBlockingQueue()))
+
+ def withNewThreadPoolWithCustomBlockingQueue(newQueueFactory: QueueFactory): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(flowHandler = defaultFlowHandler, queueFactory = newQueueFactory))
+
+ def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolConfigDispatcherBuilder =
+ withNewThreadPoolWithCustomBlockingQueue(reusableQueue(queue))
+
+ def withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity: ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(queueFactory = linkedBlockingQueue(), flowHandler = defaultFlowHandler))
+
+ def withNewThreadPoolWithLinkedBlockingQueueWithCapacity(capacity: Int): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(queueFactory = linkedBlockingQueue(capacity), flowHandler = defaultFlowHandler))
+
+ def withNewThreadPoolWithSynchronousQueueWithFairness(fair: Boolean): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(queueFactory = synchronousQueue(fair), flowHandler = defaultFlowHandler))
+
+ def withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(capacity: Int, fair: Boolean): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(queueFactory = arrayBlockingQueue(capacity,fair), flowHandler = defaultFlowHandler))
+
+ def setCorePoolSize(size: Int): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(corePoolSize = size))
+
+ def setMaxPoolSize(size: Int): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(maxPoolSize = size))
+
+ def setCorePoolSizeFromFactor(multiplier: Double): ThreadPoolConfigDispatcherBuilder =
+ setCorePoolSize(scaledPoolSize(multiplier))
+
+ def setMaxPoolSizeFromFactor(multiplier: Double): ThreadPoolConfigDispatcherBuilder =
+ setMaxPoolSize(scaledPoolSize(multiplier))
+
+ def setExecutorBounds(bounds: Int): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(flowHandler = flowHandler(bounds)))
+
+ def setKeepAliveTimeInMillis(time: Long): ThreadPoolConfigDispatcherBuilder =
+ setKeepAliveTime(Duration(time,TimeUnit.MILLISECONDS))
+
+ def setKeepAliveTime(time: Duration): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(threadTimeout = time))
+
+ def setRejectionPolicy(policy: RejectedExecutionHandler): ThreadPoolConfigDispatcherBuilder =
+ setFlowHandler(flowHandler(policy))
+
+ def setFlowHandler(newFlowHandler: FlowHandler): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(flowHandler = newFlowHandler))
+
+ def setAllowCoreThreadTimeout(allow: Boolean): ThreadPoolConfigDispatcherBuilder =
+ this.copy(config = config.copy(allowCorePoolTimeout = allow))
+
+ def configure(fs: Option[Function[ThreadPoolConfigDispatcherBuilder,ThreadPoolConfigDispatcherBuilder]]*):
+ ThreadPoolConfigDispatcherBuilder = fs.foldLeft(this)( (c,f) => f.map( _(c) ).getOrElse(c))
+}
+
+/**
+ * @author Jonas Bonér
+ */
+class MonitorableThreadFactory(val name: String) extends ThreadFactory {
+ protected val counter = new AtomicLong
+
+ def newThread(runnable: Runnable) = new MonitorableThread(runnable, name)
+}
+
+/**
+ * @author Jonas Bonér
+ */
+object MonitorableThread {
+ val DEFAULT_NAME = "MonitorableThread"
+ val created = new AtomicInteger
+ val alive = new AtomicInteger
+ @volatile var debugLifecycle = false
+}
+
+// FIXME fix the issues with using the monitoring in MonitorableThread
+
+/**
+ * @author Jonas Bonér
+ */
+class MonitorableThread(runnable: Runnable, name: String)
+ extends Thread(runnable, name + "-" + MonitorableThread.created.incrementAndGet) with Logging {
+
+ setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+ def uncaughtException(thread: Thread, cause: Throwable) =
+ log.error(cause, "UNCAUGHT in thread [%s]", thread.getName)
+ })
+
+ override def run = {
+ val debug = MonitorableThread.debugLifecycle
+ log.debug("Created thread %s", getName)
+ try {
+ MonitorableThread.alive.incrementAndGet
+ super.run
+ } finally {
+ MonitorableThread.alive.decrementAndGet
+ log.debug("Exiting thread %s", getName)
+ }
+ }
+}
+
+/**
+ * @author Jonas Bonér
+ */
+class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorServiceDelegate {
+ protected val semaphore = new Semaphore(bound)
+
+ override def execute(command: Runnable) = {
+ semaphore.acquire
+ try {
+ executor.execute(new Runnable() {
+ def run = {
+ try {
+ command.run
+ } finally {
+ semaphore.release
+ }
+ }
+ })
+ } catch {
+ case e: RejectedExecutionException =>
+ semaphore.release
+ case e =>
+ log.error(e,"Unexpected exception")
+ throw e
+ }
+ }
+}
+
+trait ExecutorServiceDelegate extends ExecutorService with Logging {
+
+ def executor: ExecutorService
+
+ def execute(command: Runnable) = executor.execute(command)
+
+ def shutdown = executor.shutdown
+
+ def shutdownNow = executor.shutdownNow
+
+ def isShutdown = executor.isShutdown
+
+ def isTerminated = executor.isTerminated
+
+ def awaitTermination(l: Long, timeUnit: TimeUnit) = executor.awaitTermination(l, timeUnit)
+
+ def submit[T](callable: Callable[T]) = executor.submit(callable)
+
+ def submit[T](runnable: Runnable, t: T) = executor.submit(runnable, t)
+
+ def submit(runnable: Runnable) = executor.submit(runnable)
+
+ def invokeAll[T](callables: Collection[_ <: Callable[T]]) = executor.invokeAll(callables)
+
+ def invokeAll[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAll(callables, l, timeUnit)
+
+ def invokeAny[T](callables: Collection[_ <: Callable[T]]) = executor.invokeAny(callables)
+
+ def invokeAny[T](callables: Collection[_ <: Callable[T]], l: Long, timeUnit: TimeUnit) = executor.invokeAny(callables, l, timeUnit)
+}
+
+trait LazyExecutorService extends ExecutorServiceDelegate {
+
+ def createExecutor: ExecutorService
+
+ lazy val executor = {
+ log.info("Lazily initializing ExecutorService for ",this)
+ createExecutor
+ }
+}
+
+class LazyExecutorServiceWrapper(executorFactory: => ExecutorService) extends LazyExecutorService {
+ def createExecutor = executorFactory
+}
diff --git a/akka-actor/src/main/scala/akka/japi/JavaAPI.scala b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala
new file mode 100644
index 0000000000..88732a4efc
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/japi/JavaAPI.scala
@@ -0,0 +1,86 @@
+package akka.japi
+
+/**
+ * A Function interface. Used to create first-class-functions is Java (sort of).
+ */
+trait Function[T,R] {
+ def apply(param: T): R
+}
+
+/** A Procedure is like a Function, but it doesn't produce a return value
+ */
+trait Procedure[T] {
+ def apply(param: T): Unit
+}
+
+/**
+ * An executable piece of code that takes no parameters and doesn't return any value.
+ */
+trait SideEffect {
+ def apply: Unit
+}
+
+/**
+ + * A constructor/factory, takes no parameters but creates a new value of type T every call
+ + */
+trait Creator[T] {
+ def create: T
+}
+
+
+/**
+ * This class represents optional values. Instances of Option
+ * are either instances of case class Some or it is case
+ * object None.
+ *
+ * Java API
+ */
+sealed abstract class Option[A] extends java.lang.Iterable[A] {
+ import scala.collection.JavaConversions._
+
+ def get: A
+ def isEmpty: Boolean
+ def isDefined = !isEmpty
+ def asScala: scala.Option[A]
+ def iterator = if (isEmpty) Iterator.empty else Iterator.single(get)
+}
+
+object Option {
+ /**
+ * Option factory that creates Some
+ */
+ def some[A](v: A): Option[A] = Some(v)
+
+ /**
+ * Option factory that creates None
+ */
+ def none[A] = None.asInstanceOf[Option[A]]
+
+ /**
+ * Option factory that creates None if
+ * v is null, Some(v) otherwise.
+ */
+ def option[A](v: A): Option[A] = if (v == null) none else some(v)
+
+ /**
+ * Class Some[A] represents existing values of type
+ * A.
+ */
+ final case class Some[A](v: A) extends Option[A] {
+ def get = v
+ def isEmpty = false
+ def asScala = scala.Some(v)
+ }
+
+ /**
+ * This case object represents non-existent values.
+ */
+ private case object None extends Option[Nothing] {
+ def get = throw new NoSuchElementException("None.get")
+ def isEmpty = true
+ def asScala = scala.None
+ }
+
+ implicit def java2ScalaOption[A](o: Option[A]): scala.Option[A] = o.asScala
+ implicit def scala2JavaOption[A](o: scala.Option[A]): Option[A] = option(o.get)
+}
diff --git a/akka-actor/src/main/scala/akka/routing/Iterators.scala b/akka-actor/src/main/scala/akka/routing/Iterators.scala
new file mode 100644
index 0000000000..c7a6f245c4
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/routing/Iterators.scala
@@ -0,0 +1,46 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.routing
+
+import akka.actor.ActorRef
+import scala.collection.JavaConversions._
+
+/**
+ * An Iterator that is either always empty or yields an infinite number of Ts.
+ */
+trait InfiniteIterator[T] extends Iterator[T]
+
+/**
+ * CyclicIterator is a round-robin style InfiniteIterator that cycles the supplied List.
+ */
+class CyclicIterator[T](items: List[T]) extends InfiniteIterator[T] {
+ def this(items: java.util.List[T]) = this(items.toList)
+
+ @volatile private[this] var current: List[T] = items
+
+ def hasNext = items != Nil
+
+ def next = {
+ val nc = if (current == Nil) items else current
+ current = nc.tail
+ nc.head
+ }
+
+ override def exists(f: T => Boolean): Boolean = items.exists(f)
+
+}
+
+/**
+ * This InfiniteIterator always returns the Actor that has the currently smallest mailbox
+ * useful for work-stealing.
+ */
+class SmallestMailboxFirstIterator(items : List[ActorRef]) extends InfiniteIterator[ActorRef] {
+ def this(items: java.util.List[ActorRef]) = this(items.toList)
+ def hasNext = items != Nil
+
+ def next = items.reduceLeft((a1, a2) => if (a1.mailboxSize < a2.mailboxSize) a1 else a2)
+
+ override def exists(f: ActorRef => Boolean): Boolean = items.exists(f)
+}
diff --git a/akka-actor/src/main/scala/akka/routing/Listeners.scala b/akka-actor/src/main/scala/akka/routing/Listeners.scala
new file mode 100644
index 0000000000..c80ff21353
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/routing/Listeners.scala
@@ -0,0 +1,37 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.routing
+
+import akka.actor.{Actor, ActorRef}
+import java.util.concurrent.ConcurrentSkipListSet
+import scala.collection.JavaConversions._
+
+sealed trait ListenerMessage
+case class Listen(listener: ActorRef) extends ListenerMessage
+case class Deafen(listener: ActorRef) extends ListenerMessage
+case class WithListeners(f: (ActorRef) => Unit) extends ListenerMessage
+
+/**
+ * Listeners is a generic trait to implement listening capability on an Actor.
+ *
+ * Use the gossip(msg) method to have it sent to the listeners.
+ *
+ * Send Listen(self) to start listening.
+ *
+ * Send Deafen(self) to stop listening.
+ *
+ * Send WithListeners(fun) to traverse the current listeners.
+ */
+trait Listeners { self: Actor =>
+ private val listeners = new ConcurrentSkipListSet[ActorRef]
+
+ protected def listenerManagement: Receive = {
+ case Listen(l) => listeners add l
+ case Deafen(l) => listeners remove l
+ case WithListeners(f) => listeners foreach f
+ }
+
+ protected def gossip(msg: Any) = listeners foreach (_ ! msg)
+}
diff --git a/akka-actor/src/main/scala/akka/routing/Routers.scala b/akka-actor/src/main/scala/akka/routing/Routers.scala
new file mode 100644
index 0000000000..e770159b2b
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/routing/Routers.scala
@@ -0,0 +1,73 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.routing
+
+import akka.actor.{UntypedActor, Actor, ActorRef}
+
+/**
+ * A Dispatcher is a trait whose purpose is to route incoming messages to actors.
+ */
+trait Dispatcher { this: Actor =>
+
+ protected def transform(msg: Any): Any = msg
+
+ protected def routes: PartialFunction[Any, ActorRef]
+
+ protected def dispatch: Receive = {
+ case a if routes.isDefinedAt(a) =>
+ if (isSenderDefined) routes(a).forward(transform(a))(someSelf)
+ else routes(a).!(transform(a))(None)
+ }
+
+ def receive = dispatch
+
+ private def isSenderDefined = self.senderFuture.isDefined || self.sender.isDefined
+}
+
+/**
+ * An UntypedDispatcher is an abstract class whose purpose is to route incoming messages to actors.
+ */
+abstract class UntypedDispatcher extends UntypedActor {
+ protected def transform(msg: Any): Any = msg
+
+ protected def route(msg: Any): ActorRef
+
+ private def isSenderDefined = self.senderFuture.isDefined || self.sender.isDefined
+
+ @throws(classOf[Exception])
+ def onReceive(msg: Any): Unit = {
+ val r = route(msg)
+ if(r eq null)
+ throw new IllegalStateException("No route for " + msg + " defined!")
+ if (isSenderDefined) r.forward(transform(msg))(someSelf)
+ else r.!(transform(msg))(None)
+ }
+}
+
+/**
+ * A LoadBalancer is a specialized kind of Dispatcher, that is supplied an InfiniteIterator of targets
+ * to dispatch incoming messages to.
+ */
+trait LoadBalancer extends Dispatcher { self: Actor =>
+ protected def seq: InfiniteIterator[ActorRef]
+
+ protected def routes = { case x if seq.hasNext => seq.next }
+
+ override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) )
+}
+
+/**
+ * A UntypedLoadBalancer is a specialized kind of UntypedDispatcher, that is supplied an InfiniteIterator of targets
+ * to dispatch incoming messages to.
+ */
+abstract class UntypedLoadBalancer extends UntypedDispatcher {
+ protected def seq: InfiniteIterator[ActorRef]
+
+ protected def route(msg: Any) =
+ if (seq.hasNext) seq.next
+ else null
+
+ override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) )
+}
diff --git a/akka-actor/src/main/scala/akka/routing/Routing.scala b/akka-actor/src/main/scala/akka/routing/Routing.scala
new file mode 100644
index 0000000000..4bbb51ddab
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/routing/Routing.scala
@@ -0,0 +1,61 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.routing
+
+import akka.actor.{Actor, ActorRef}
+import akka.actor.Actor._
+
+object Routing {
+
+ type PF[A, B] = PartialFunction[A, B]
+
+ /**
+ * Creates a new PartialFunction whose isDefinedAt is a combination
+ * of the two parameters, and whose apply is first to call filter.apply
+ * and then filtered.apply.
+ */
+ def filter[A, B](filter: PF[A, Unit], filtered: PF[A, B]): PF[A, B] = {
+ case a: A if filtered.isDefinedAt(a) && filter.isDefinedAt(a) =>
+ filter(a)
+ filtered(a)
+ }
+
+ /**
+ * Interceptor is a filter(x,y) where x.isDefinedAt is considered to be always true.
+ */
+ def intercept[A, B](interceptor: (A) => Unit, interceptee: PF[A, B]): PF[A, B] =
+ filter({case a if a.isInstanceOf[A] => interceptor(a)}, interceptee)
+
+ /**
+ * Creates a LoadBalancer from the thunk-supplied InfiniteIterator.
+ */
+ def loadBalancerActor(actors: => InfiniteIterator[ActorRef]): ActorRef =
+ actorOf(new Actor with LoadBalancer {
+ val seq = actors
+ }).start
+
+ /**
+ * Creates a Dispatcher given a routing and a message-transforming function.
+ */
+ def dispatcherActor(routing: PF[Any, ActorRef], msgTransformer: (Any) => Any): ActorRef =
+ actorOf(new Actor with Dispatcher {
+ override def transform(msg: Any) = msgTransformer(msg)
+ def routes = routing
+ }).start
+
+ /**
+ * Creates a Dispatcher given a routing.
+ */
+ def dispatcherActor(routing: PF[Any, ActorRef]): ActorRef = actorOf(new Actor with Dispatcher {
+ def routes = routing
+ }).start
+
+ /**
+ * Creates an actor that pipes all incoming messages to
+ * both another actor and through the supplied function
+ */
+ def loggerActor(actorToLog: ActorRef, logger: (Any) => Unit): ActorRef =
+ dispatcherActor({case _ => actorToLog}, logger)
+}
diff --git a/akka-actor/src/main/scala/akka/util/Address.scala b/akka-actor/src/main/scala/akka/util/Address.scala
new file mode 100644
index 0000000000..8ee1693553
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/util/Address.scala
@@ -0,0 +1,23 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.util
+
+object Address {
+ def apply(hostname: String, port: Int) = new Address(hostname, port)
+}
+
+class Address(val hostname: String, val port: Int) {
+ override def hashCode: Int = {
+ var result = HashCode.SEED
+ result = HashCode.hash(result, hostname)
+ result = HashCode.hash(result, port)
+ result
+ }
+
+ override def equals(that: Any): Boolean = {
+ that.isInstanceOf[Address] &&
+ that.asInstanceOf[Address].hostname == hostname &&
+ that.asInstanceOf[Address].port == port
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/util/AkkaException.scala b/akka-actor/src/main/scala/akka/util/AkkaException.scala
new file mode 100644
index 0000000000..3ab80a2512
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/util/AkkaException.scala
@@ -0,0 +1,52 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka
+
+import akka.util.Logging
+import akka.actor.newUuid
+
+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
+ * override def hashCode: Int = {
+ * var result = HashCode.SEED
+ * //collect the contributions of various fields
+ * result = HashCode.hash(result, fPrimitive)
+ * result = HashCode.hash(result, fObject)
+ * result = HashCode.hash(result, fArray)
+ * result
+ * }
+ *
+ *
+ * @author Jonas Bonér
+ */
+object HashCode {
+ val SEED = 23
+
+ def hash(seed: Int, any: Any): Int = any match {
+ case value: Boolean => hash(seed, value)
+ case value: Char => hash(seed, value)
+ case value: Short => hash(seed, value)
+ case value: Int => hash(seed, value)
+ case value: Long => hash(seed, value)
+ case value: Float => hash(seed, value)
+ case value: Double => hash(seed, value)
+ case value: Byte => hash(seed, value)
+ case value: AnyRef =>
+ var result = seed
+ if (value eq null) result = hash(result, 0)
+ else if (!isArray(value)) result = hash(result, value.hashCode())
+ else for (id <- 0 until JArray.getLength(value)) result = hash(result, JArray.get(value, id)) // is an array
+ result
+ }
+ def hash(seed: Int, value: Boolean): Int = firstTerm(seed) + (if (value) 1 else 0)
+ def hash(seed: Int, value: Char): Int = firstTerm(seed) + value.asInstanceOf[Int]
+ def hash(seed: Int, value: Int): Int = firstTerm(seed) + value
+ def hash(seed: Int, value: Long): Int = firstTerm(seed) + (value ^ (value >>> 32) ).asInstanceOf[Int]
+ def hash(seed: Int, value: Float): Int = hash(seed, JFloat.floatToIntBits(value))
+ def hash(seed: Int, value: Double): Int = hash(seed, JDouble.doubleToLongBits(value))
+
+ private def firstTerm(seed: Int): Int = PRIME * seed
+ private def isArray(anyRef: AnyRef): Boolean = anyRef.getClass.isArray
+ private val PRIME = 37
+}
+
diff --git a/akka-actor/src/main/scala/akka/util/Helpers.scala b/akka-actor/src/main/scala/akka/util/Helpers.scala
new file mode 100644
index 0000000000..25180a68d7
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/util/Helpers.scala
@@ -0,0 +1,100 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.util
+
+/**
+ * @author Jonas Bonér
+ */
+object Helpers extends Logging {
+
+ implicit def null2Option[T](t: T): Option[T] = Option(t)
+
+ def intToBytes(value: Int): Array[Byte] = {
+ val bytes = new Array[Byte](4)
+ bytes(0) = (value >>> 24).asInstanceOf[Byte]
+ bytes(1) = (value >>> 16).asInstanceOf[Byte]
+ bytes(2) = (value >>> 8).asInstanceOf[Byte]
+ bytes(3) = value.asInstanceOf[Byte]
+ bytes
+ }
+
+ def bytesToInt(bytes: Array[Byte], offset: Int): Int = {
+ (0 until 4).foldLeft(0)((value, index) => value + ((bytes(index + offset) & 0x000000FF) << ((4 - 1 - index) * 8)))
+ }
+
+ /**
+ * Convenience helper to cast the given Option of Any to an Option of the given type. Will throw a ClassCastException
+ * if the actual type is not assignable from the given one.
+ */
+ def narrow[T](o: Option[Any]): Option[T] = {
+ require((o ne null), "Option to be narrowed must not be null!")
+ o.asInstanceOf[Option[T]]
+ }
+
+ /**
+ * Convenience helper to cast the given Option of Any to an Option of the given type. Will swallow a possible
+ * ClassCastException and return None in that case.
+ */
+ def narrowSilently[T: Manifest](o: Option[Any]): Option[T] =
+ try {
+ narrow(o)
+ } catch {
+ case e: ClassCastException =>
+ log.warning(e, "Cannot narrow %s to expected type %s!", o, implicitly[Manifest[T]].erasure.getName)
+ None
+ }
+
+ /**
+ * Reference that can hold either a typed value or an exception.
+ *
+ * Usage:
+ *
+ * scala> ResultOrError(1)
+ * res0: ResultOrError[Int] = ResultOrError@a96606
+ *
+ * scala> res0()
+ res1: Int = 1
+ *
+ * scala> res0() = 3
+ *
+ * scala> res0()
+ * res3: Int = 3
+ *
+ * scala> res0() = { println("Hello world"); 3}
+ * Hello world
+ *
+ * scala> res0()
+ * res5: Int = 3
+ *
+ * scala> res0() = error("Lets see what happens here...")
+ *
+ * scala> res0()
+ * java.lang.RuntimeException: Lets see what happens here...
+ * at ResultOrError.apply(Helper.scala:11)
+ * at .(:6)
+ * at .()
+ * at Re...
+ *
+ */
+ class ResultOrError[R](result: R){
+ private[this] var contents: Either[R, Throwable] = Left(result)
+
+ def update(value: => R) = {
+ contents = try {
+ Left(value)
+ } catch {
+ case (error : Throwable) => Right(error)
+ }
+ }
+
+ def apply() = contents match {
+ case Left(result) => result
+ case Right(error) => throw error.fillInStackTrace
+ }
+ }
+ object ResultOrError {
+ def apply[R](result: R) = new ResultOrError(result)
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/util/ListenerManagement.scala b/akka-actor/src/main/scala/akka/util/ListenerManagement.scala
new file mode 100644
index 0000000000..9166680873
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/util/ListenerManagement.scala
@@ -0,0 +1,76 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.util
+
+import java.util.concurrent.ConcurrentSkipListSet
+
+import akka.actor.ActorRef
+
+/**
+ * A manager for listener actors. Intended for mixin by observables.
+ *
+ * @author Martin Krasser
+ */
+trait ListenerManagement extends Logging {
+
+ private val listeners = new ConcurrentSkipListSet[ActorRef]
+
+ /**
+ * Specifies whether listeners should be started when added and stopped when removed or not
+ */
+ protected def manageLifeCycleOfListeners: Boolean = true
+
+ /**
+ * Adds the listener this this registry's listener list.
+ * The listener is started by this method if manageLifeCycleOfListeners yields true.
+ */
+ def addListener(listener: ActorRef) {
+ if (manageLifeCycleOfListeners) listener.start
+ listeners add listener
+ }
+
+ /**
+ * Removes the listener this this registry's listener list.
+ * The listener is stopped by this method if manageLifeCycleOfListeners yields true.
+ */
+ def removeListener(listener: ActorRef) {
+ listeners remove listener
+ if (manageLifeCycleOfListeners) listener.stop
+ }
+
+ /*
+ * Returns whether there are any listeners currently
+ */
+ def hasListeners: Boolean = !listeners.isEmpty
+
+ /**
+ * Checks if a specfic listener is registered.
+ */
+ def hasListener(listener: ActorRef): Boolean = listeners.contains(listener)
+
+ protected def notifyListeners(message: => Any) {
+ if (hasListeners) {
+ val msg = message
+ val iterator = listeners.iterator
+ while (iterator.hasNext) {
+ val listener = iterator.next
+ if (listener.isRunning) listener ! msg
+ else log.warning("Can't notify [%s] since it is not running.", listener)
+ }
+ }
+ }
+
+ /**
+ * Execute f with each listener as argument.
+ */
+ protected def foreachListener(f: (ActorRef) => Unit) {
+ val iterator = listeners.iterator
+ while (iterator.hasNext) {
+ val listener = iterator.next
+ if (listener.isRunning) f(listener)
+ else log.warning("Can't notify [%s] since it is not running.", listener)
+ }
+ }
+}
diff --git a/akka-actor/src/main/scala/akka/util/LockUtil.scala b/akka-actor/src/main/scala/akka/util/LockUtil.scala
new file mode 100644
index 0000000000..977383e9ff
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/util/LockUtil.scala
@@ -0,0 +1,167 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.util
+
+import java.util.concurrent.locks.{ReentrantReadWriteLock, ReentrantLock}
+import java.util.concurrent.atomic. {AtomicBoolean}
+
+/**
+ * @author Jonas Bonér
+ */
+class ReentrantGuard {
+ val lock = new ReentrantLock
+
+ def withGuard[T](body: => T): T = {
+ lock.lock
+ try {
+ body
+ } finally {
+ lock.unlock
+ }
+ }
+
+ def tryWithGuard[T](body: => T): T = {
+ while(!lock.tryLock) { Thread.sleep(10) } // wait on the monitor to be unlocked
+ try {
+ body
+ } finally {
+ lock.unlock
+ }
+ }
+}
+
+/**
+ * @author Jonas Bonér
+ */
+class ReadWriteGuard {
+ private val rwl = new ReentrantReadWriteLock
+ private val readLock = rwl.readLock
+ private val writeLock = rwl.writeLock
+
+ def withWriteGuard[T](body: => T): T = {
+ writeLock.lock
+ try {
+ body
+ } finally {
+ writeLock.unlock
+ }
+ }
+
+ def withReadGuard[T](body: => T): T = {
+ readLock.lock
+ try {
+ body
+ } finally {
+ readLock.unlock
+ }
+ }
+}
+
+/**
+ * A very simple lock that uses CCAS (Compare Compare-And-Swap)
+ * Does not keep track of the owner and isn't Reentrant, so don't nest and try to stick to the if*-methods
+ */
+class SimpleLock {
+ val acquired = new AtomicBoolean(false)
+
+ def ifPossible(perform: () => Unit): Boolean = {
+ if (tryLock()) {
+ try {
+ perform
+ } finally {
+ unlock()
+ }
+ true
+ } else false
+ }
+
+ def ifPossibleYield[T](perform: () => T): Option[T] = {
+ if (tryLock()) {
+ try {
+ Some(perform())
+ } finally {
+ unlock()
+ }
+ } else None
+ }
+
+ def ifPossibleApply[T,R](value: T)(function: (T) => R): Option[R] = {
+ if (tryLock()) {
+ try {
+ Some(function(value))
+ } finally {
+ unlock()
+ }
+ } else None
+ }
+
+ def tryLock() = {
+ if (acquired.get) false
+ else acquired.compareAndSet(false,true)
+ }
+
+ def tryUnlock() = {
+ acquired.compareAndSet(true,false)
+ }
+
+ def locked = acquired.get
+
+ def unlock() {
+ acquired.set(false)
+ }
+}
+
+/**
+ * An atomic switch that can be either on or off
+ */
+class Switch(startAsOn: Boolean = false) {
+ private val switch = new AtomicBoolean(startAsOn)
+
+ protected def transcend(from: Boolean,action: => Unit): Boolean = synchronized {
+ if (switch.compareAndSet(from, !from)) {
+ try {
+ action
+ } catch {
+ case t =>
+ switch.compareAndSet(!from,from) //Revert status
+ throw t
+ }
+ true
+ } else false
+ }
+
+ def switchOff(action: => Unit): Boolean = transcend(from = true, action)
+ def switchOn(action: => Unit): Boolean = transcend(from = false, action)
+
+ def switchOff: Boolean = synchronized { switch.compareAndSet(true, false) }
+ def switchOn: Boolean = synchronized { switch.compareAndSet(false, true) }
+
+ def ifOnYield[T](action: => T): Option[T] = {
+ if (switch.get) Some(action)
+ else None
+ }
+
+ def ifOffYield[T](action: => T): Option[T] = {
+ if (switch.get) Some(action)
+ else None
+ }
+
+ def ifOn(action: => Unit): Boolean = {
+ if (switch.get) {
+ action
+ true
+ } else false
+ }
+
+ def ifOff(action: => Unit): Boolean = {
+ if (!switch.get) {
+ action
+ true
+ } else false
+ }
+
+ def isOn = switch.get
+ def isOff = !isOn
+}
diff --git a/akka-actor/src/main/scala/akka/util/Logging.scala b/akka-actor/src/main/scala/akka/util/Logging.scala
new file mode 100644
index 0000000000..02ef6f3136
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/util/Logging.scala
@@ -0,0 +1,170 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.util
+
+import org.slf4j.{Logger => SLFLogger,LoggerFactory => SLFLoggerFactory}
+
+import java.io.StringWriter
+import java.io.PrintWriter
+import java.net.InetAddress
+import java.net.UnknownHostException
+
+/**
+ * Base trait for all classes that wants to be able use the logging infrastructure.
+ *
+ * @author Jonas Bonér
+ */
+trait Logging {
+ @transient val log = Logger(this.getClass.getName)
+}
+
+/**
+ * Scala SLF4J wrapper
+ *
+ * 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...)
+ */
+class Logger(val logger: SLFLogger) {
+ def name = logger.getName
+
+ 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 trace(t: Throwable, msg: => String) {
+ if (trace_?) logger.trace(msg,t)
+ }
+
+ def trace(fmt: => String, arg: Any, argN: Any*) {
+ trace(message(fmt,arg,argN:_*))
+ }
+
+ 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 warn(t: Throwable, fmt: => String, arg: Any, argN: Any*) = warning(t, fmt, arg, argN)
+
+ def warning(t: Throwable, msg: => String) {
+ if (warning_?) logger.warn(msg,t)
+ }
+
+ def warn(t: Throwable, msg: => String) = warning(t, msg)
+
+ def warning(fmt: => String, arg: Any, argN: Any*) {
+ warning(message(fmt,arg,argN:_*))
+ }
+
+ def warn(fmt: => String, arg: Any, argN: Any*) = warning(fmt, arg, argN:_*)
+
+ def warning(msg: => String) {
+ if (warning_?) logger warn msg
+ }
+
+ def warn(msg: => String) = warning(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-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala
new file mode 100644
index 0000000000..5257d596f0
--- /dev/null
+++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala
@@ -0,0 +1,272 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.util
+
+import akka.actor.{ActorRef, IllegalActorStateException, ActorType, Uuid}
+import akka.dispatch.{Future, CompletableFuture, MessageInvocation}
+import akka.config.{Config, ModuleNotAvailableException}
+import akka.AkkaException
+
+import java.net.InetSocketAddress
+
+/**
+ * Helper class for reflective access to different modules in order to allow optional loading of modules.
+ *
+ * @author Jonas Bonér
+ */
+object ReflectiveAccess extends Logging {
+
+ val loader = getClass.getClassLoader
+
+ lazy val isRemotingEnabled = RemoteClientModule.isRemotingEnabled
+ lazy val isTypedActorEnabled = TypedActorModule.isTypedActorEnabled
+ lazy val isEnterpriseEnabled = EnterpriseModule.isEnterpriseEnabled
+
+ def ensureRemotingEnabled = RemoteClientModule.ensureRemotingEnabled
+ def ensureTypedActorEnabled = TypedActorModule.ensureTypedActorEnabled
+ def ensureEnterpriseEnabled = EnterpriseModule.ensureEnterpriseEnabled
+
+ /**
+ * Reflective access to the RemoteClient module.
+ *
+ * @author Jonas Bonér
+ */
+ object RemoteClientModule {
+
+ type RemoteClient = {
+ def send[T](
+ message: Any,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[_]],
+ remoteAddress: InetSocketAddress,
+ timeout: Long,
+ isOneWay: Boolean,
+ actorRef: ActorRef,
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: ActorType): Option[CompletableFuture[T]]
+ def registerSupervisorForActor(actorRef: ActorRef)
+ }
+
+ type RemoteClientObject = {
+ def register(hostname: String, port: Int, uuid: Uuid): Unit
+ def unregister(hostname: String, port: Int, uuid: Uuid): Unit
+ def clientFor(address: InetSocketAddress): RemoteClient
+ def clientFor(hostname: String, port: Int, loader: Option[ClassLoader]): RemoteClient
+ }
+
+ lazy val isRemotingEnabled = remoteClientObjectInstance.isDefined
+
+ def ensureRemotingEnabled = if (!isRemotingEnabled) throw new ModuleNotAvailableException(
+ "Can't load the remoting module, make sure that akka-remote.jar is on the classpath")
+
+ val remoteClientObjectInstance: Option[RemoteClientObject] =
+ getObjectFor("akka.remote.RemoteClient$")
+
+ def register(address: InetSocketAddress, uuid: Uuid) = {
+ ensureRemotingEnabled
+ remoteClientObjectInstance.get.register(address.getHostName, address.getPort, uuid)
+ }
+
+ def unregister(address: InetSocketAddress, uuid: Uuid) = {
+ ensureRemotingEnabled
+ remoteClientObjectInstance.get.unregister(address.getHostName, address.getPort, uuid)
+ }
+
+ def registerSupervisorForActor(remoteAddress: InetSocketAddress, actorRef: ActorRef) = {
+ ensureRemotingEnabled
+ val remoteClient = remoteClientObjectInstance.get.clientFor(remoteAddress)
+ remoteClient.registerSupervisorForActor(actorRef)
+ }
+
+ def clientFor(hostname: String, port: Int, loader: Option[ClassLoader]): RemoteClient = {
+ ensureRemotingEnabled
+ remoteClientObjectInstance.get.clientFor(hostname, port, loader)
+ }
+
+ def send[T](
+ message: Any,
+ senderOption: Option[ActorRef],
+ senderFuture: Option[CompletableFuture[_]],
+ remoteAddress: InetSocketAddress,
+ timeout: Long,
+ isOneWay: Boolean,
+ actorRef: ActorRef,
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: ActorType): Option[CompletableFuture[T]] = {
+ ensureRemotingEnabled
+ clientFor(remoteAddress.getHostName, remoteAddress.getPort, None).send[T](
+ message, senderOption, senderFuture, remoteAddress, timeout, isOneWay, actorRef, typedActorInfo, actorType)
+ }
+ }
+
+ /**
+ * Reflective access to the RemoteServer module.
+ *
+ * @author Jonas Bonér
+ */
+ object RemoteServerModule {
+ val HOSTNAME = Config.config.getString("akka.remote.server.hostname", "localhost")
+ val PORT = Config.config.getInt("akka.remote.server.port", 2552)
+
+ type RemoteServerObject = {
+ def registerActor(address: InetSocketAddress, actor: ActorRef): Unit
+ def registerTypedActor(address: InetSocketAddress, name: String, typedActor: AnyRef): Unit
+ }
+
+ type RemoteNodeObject = {
+ def unregister(actorRef: ActorRef): Unit
+ }
+
+ val remoteServerObjectInstance: Option[RemoteServerObject] =
+ getObjectFor("akka.remote.RemoteServer$")
+
+ val remoteNodeObjectInstance: Option[RemoteNodeObject] =
+ getObjectFor("akka.remote.RemoteNode$")
+
+ def registerActor(address: InetSocketAddress, actorRef: ActorRef) = {
+ ensureRemotingEnabled
+ remoteServerObjectInstance.get.registerActor(address, actorRef)
+ }
+
+ def registerTypedActor(address: InetSocketAddress, implementationClassName: String, proxy: AnyRef) = {
+ ensureRemotingEnabled
+ remoteServerObjectInstance.get.registerTypedActor(address, implementationClassName, proxy)
+ }
+
+ def unregister(actorRef: ActorRef) = {
+ ensureRemotingEnabled
+ remoteNodeObjectInstance.get.unregister(actorRef)
+ }
+ }
+
+ /**
+ * Reflective access to the TypedActors module.
+ *
+ * @author Jonas Bonér
+ */
+ object TypedActorModule {
+
+ type TypedActorObject = {
+ def isJoinPoint(message: Any): Boolean
+ def isJoinPointAndOneWay(message: Any): Boolean
+ def actorFor(proxy: AnyRef): Option[ActorRef]
+ def proxyFor(actorRef: ActorRef): Option[AnyRef]
+ def stop(anyRef: AnyRef) : Unit
+ }
+
+ lazy val isTypedActorEnabled = typedActorObjectInstance.isDefined
+
+ def ensureTypedActorEnabled = if (!isTypedActorEnabled) throw new ModuleNotAvailableException(
+ "Can't load the typed actor module, make sure that akka-typed-actor.jar is on the classpath")
+
+ val typedActorObjectInstance: Option[TypedActorObject] =
+ getObjectFor("akka.actor.TypedActor$")
+
+ def resolveFutureIfMessageIsJoinPoint(message: Any, future: Future[_]): Boolean = {
+ ensureTypedActorEnabled
+ if (typedActorObjectInstance.get.isJoinPointAndOneWay(message)) {
+ future.asInstanceOf[CompletableFuture[Option[_]]].completeWithResult(None)
+ }
+ typedActorObjectInstance.get.isJoinPoint(message)
+ }
+ }
+
+ object EnterpriseModule {
+
+ type Mailbox = {
+ def enqueue(message: MessageInvocation)
+ def dequeue: MessageInvocation
+ }
+
+ type Serializer = {
+ def toBinary(obj: AnyRef): Array[Byte]
+ def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef
+ }
+
+ lazy val isEnterpriseEnabled = clusterObjectInstance.isDefined
+
+ val clusterObjectInstance: Option[AnyRef] =
+ getObjectFor("akka.cluster.Cluster$")
+
+ val serializerClass: Option[Class[_]] =
+ getClassFor("akka.serialization.Serializer")
+
+ def ensureEnterpriseEnabled = if (!isEnterpriseEnabled) throw new ModuleNotAvailableException(
+ "Feature is only available in Akka Enterprise edition")
+
+ def createFileBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.actor.mailbox.FileBasedMailbox", actorRef)
+
+ def createZooKeeperBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.actor.mailbox.ZooKeeperBasedMailbox", actorRef)
+
+ def createBeanstalkBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.actor.mailbox.BeanstalkBasedMailbox", actorRef)
+
+ def createRedisBasedMailbox(actorRef: ActorRef): Mailbox = createMailbox("akka.actor.mailbox.RedisBasedMailbox", actorRef)
+
+ private def createMailbox(mailboxClassname: String, actorRef: ActorRef): Mailbox = {
+ ensureEnterpriseEnabled
+ createInstance(
+ mailboxClassname,
+ Array(classOf[ActorRef]),
+ Array(actorRef).asInstanceOf[Array[AnyRef]],
+ loader)
+ .getOrElse(throw new IllegalActorStateException("Could not create durable mailbox [" + mailboxClassname + "] for actor [" + actorRef + "]"))
+ .asInstanceOf[Mailbox]
+ }
+ }
+
+ val noParams = Array[Class[_]]()
+ val noArgs = Array[AnyRef]()
+
+ def createInstance[T](clazz: Class[_],
+ params: Array[Class[_]],
+ args: Array[AnyRef]): Option[T] = try {
+ assert(clazz ne null)
+ assert(params ne null)
+ assert(args ne null)
+ val ctor = clazz.getDeclaredConstructor(params: _*)
+ ctor.setAccessible(true)
+ Some(ctor.newInstance(args: _*).asInstanceOf[T])
+ } catch {
+ case e =>
+ log.warning("Could not instantiate class [%s] due to [%s]", clazz.getName, e.getCause)
+ None
+ }
+
+ def createInstance[T](fqn: String,
+ params: Array[Class[_]],
+ args: Array[AnyRef],
+ classloader: ClassLoader = loader): Option[T] = try {
+ assert(fqn ne null)
+ assert(params ne null)
+ assert(args ne null)
+ val clazz = classloader.loadClass(fqn)
+ val ctor = clazz.getDeclaredConstructor(params: _*)
+ ctor.setAccessible(true)
+ Some(ctor.newInstance(args: _*).asInstanceOf[T])
+ } catch {
+ case e =>
+ log.warning("Could not instantiate class [%s] due to [%s]", fqn, e.getCause)
+ None
+ }
+
+ def getObjectFor[T](fqn: String, classloader: ClassLoader = loader): Option[T] = try {//Obtains a reference to $MODULE$
+ assert(fqn ne null)
+ val clazz = classloader.loadClass(fqn)
+ val instance = clazz.getDeclaredField("MODULE$")
+ instance.setAccessible(true)
+ Option(instance.get(null).asInstanceOf[T])
+ } catch {
+ case e: ClassNotFoundException =>
+ log.debug("Could not get object [%s] due to [%s]", fqn, e)
+ None
+ }
+
+ def getClassFor[T](fqn: String, classloader: ClassLoader = loader): Option[Class[T]] = try {
+ assert(fqn ne null)
+ Some(classloader.loadClass(fqn).asInstanceOf[Class[T]])
+ } catch {
+ case e => None
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/Messages.scala b/akka-actor/src/test/scala/akka/Messages.scala
new file mode 100644
index 0000000000..fda3ba5b91
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/Messages.scala
@@ -0,0 +1,13 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka
+
+abstract class TestMessage
+
+case object Ping extends TestMessage
+case object Pong extends TestMessage
+case object OneWay extends TestMessage
+case object Die extends TestMessage
+case object NotifySupervisorExit extends TestMessage
diff --git a/akka-actor/src/test/scala/akka/actor/actor/ActorFireForgetRequestReplySpec.scala b/akka-actor/src/test/scala/akka/actor/actor/ActorFireForgetRequestReplySpec.scala
new file mode 100644
index 0000000000..0f9debe5b0
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/actor/ActorFireForgetRequestReplySpec.scala
@@ -0,0 +1,91 @@
+package akka.actor
+
+import java.util.concurrent.{TimeUnit, CyclicBarrier, TimeoutException}
+import akka.config.Supervision._
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+import akka.dispatch.Dispatchers
+import Actor._
+
+object ActorFireForgetRequestReplySpec {
+ class ReplyActor extends Actor {
+
+ def receive = {
+ case "Send" =>
+ self.reply("Reply")
+ case "SendImplicit" =>
+ self.sender.get ! "ReplyImplicit"
+ }
+ }
+
+ class CrashingTemporaryActor extends Actor {
+ self.lifeCycle = Temporary
+
+ def receive = {
+ case "Die" =>
+ state.finished.await
+ throw new Exception("Expected exception")
+ }
+ }
+
+ class SenderActor(replyActor: ActorRef) extends Actor {
+
+ def receive = {
+ case "Init" =>
+ replyActor ! "Send"
+ case "Reply" => {
+ state.s = "Reply"
+ state.finished.await
+ }
+ case "InitImplicit" => replyActor ! "SendImplicit"
+ case "ReplyImplicit" => {
+ state.s = "ReplyImplicit"
+ state.finished.await
+ }
+ }
+ }
+
+ object state {
+ var s = "NIL"
+ val finished = new CyclicBarrier(2)
+ }
+}
+
+class ActorFireForgetRequestReplySpec extends JUnitSuite {
+ import ActorFireForgetRequestReplySpec._
+
+ @Test
+ def shouldReplyToBangMessageUsingReply = {
+ state.finished.reset
+ val replyActor = actorOf[ReplyActor].start
+ val senderActor = actorOf(new SenderActor(replyActor)).start
+ senderActor ! "Init"
+ try { state.finished.await(1L, TimeUnit.SECONDS) }
+ catch { case e: TimeoutException => fail("Never got the message") }
+ assert("Reply" === state.s)
+ }
+
+ @Test
+ def shouldReplyToBangMessageUsingImplicitSender = {
+ state.finished.reset
+ val replyActor = actorOf[ReplyActor].start
+ val senderActor = actorOf(new SenderActor(replyActor)).start
+ senderActor ! "InitImplicit"
+ try { state.finished.await(1L, TimeUnit.SECONDS) }
+ catch { case e: TimeoutException => fail("Never got the message") }
+ assert("ReplyImplicit" === state.s)
+ }
+
+ @Test
+ def shouldShutdownCrashedTemporaryActor = {
+ state.finished.reset
+ val actor = actorOf[CrashingTemporaryActor].start
+ assert(actor.isRunning)
+ actor ! "Die"
+ try { state.finished.await(10L, TimeUnit.SECONDS) }
+ catch { case e: TimeoutException => fail("Never got the message") }
+ Thread.sleep(100)
+ assert(actor.isShutdown)
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/actor/actor/ActorRefSpec.scala b/akka-actor/src/test/scala/akka/actor/actor/ActorRefSpec.scala
new file mode 100644
index 0000000000..35922670f3
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/actor/ActorRefSpec.scala
@@ -0,0 +1,101 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import org.scalatest.Spec
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
+
+import akka.actor._
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+
+object ActorRefSpec {
+
+ var latch = new CountDownLatch(4)
+
+ class ReplyActor extends Actor {
+ var replyTo: Channel[Any] = null
+
+ def receive = {
+ case "complexRequest" => {
+ replyTo = self.channel
+ val worker = Actor.actorOf[WorkerActor].start
+ worker ! "work"
+ }
+ case "complexRequest2" =>
+ val worker = Actor.actorOf[WorkerActor].start
+ worker ! self.channel
+ case "workDone" => replyTo ! "complexReply"
+ case "simpleRequest" => self.reply("simpleReply")
+ }
+ }
+
+ class WorkerActor() extends Actor {
+ def receive = {
+ case "work" => {
+ work
+ self.reply("workDone")
+ self.stop
+ }
+ case replyTo: Channel[Any] => {
+ work
+ replyTo ! "complexReply"
+ }
+ }
+
+ private def work {
+ Thread.sleep(1000)
+ }
+ }
+
+ class SenderActor(replyActor: ActorRef) extends Actor {
+
+ def receive = {
+ case "complex" => replyActor ! "complexRequest"
+ case "complex2" => replyActor ! "complexRequest2"
+ case "simple" => replyActor ! "simpleRequest"
+ case "complexReply" => {
+ println("got complex reply")
+ latch.countDown
+ }
+ case "simpleReply" => {
+ println("got simple reply")
+ latch.countDown
+ }
+ }
+ }
+}
+
+@RunWith(classOf[JUnitRunner])
+class ActorRefSpec extends
+ Spec with
+ ShouldMatchers with
+ BeforeAndAfterAll {
+
+ import ActorRefSpec._
+
+ describe("ActorRef") {
+ it("should support to reply via channel") {
+ val serverRef = Actor.actorOf[ReplyActor].start
+ val clientRef = Actor.actorOf(new SenderActor(serverRef)).start
+
+ clientRef ! "complex"
+ clientRef ! "simple"
+ clientRef ! "simple"
+ clientRef ! "simple"
+ assert(latch.await(4L, TimeUnit.SECONDS))
+ latch = new CountDownLatch(4)
+ clientRef ! "complex2"
+ clientRef ! "simple"
+ clientRef ! "simple"
+ clientRef ! "simple"
+ assert(latch.await(4L, TimeUnit.SECONDS))
+ clientRef.stop
+ serverRef.stop
+ }
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/actor/actor/Bench.scala b/akka-actor/src/test/scala/akka/actor/actor/Bench.scala
new file mode 100644
index 0000000000..f043f5c92e
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/actor/Bench.scala
@@ -0,0 +1,119 @@
+/* The Computer Language Benchmarks Game
+ http://shootout.alioth.debian.org/
+ contributed by Julien Gaugaz
+ inspired by the version contributed by Yura Taras and modified by Isaac Gouy
+*/
+package akka.actor
+
+import akka.actor.Actor._
+
+object Chameneos {
+
+ sealed trait ChameneosEvent
+ case class Meet(from: ActorRef, colour: Colour) extends ChameneosEvent
+ case class Change(colour: Colour) extends ChameneosEvent
+ case class MeetingCount(count: Int) extends ChameneosEvent
+ case object Exit extends ChameneosEvent
+
+ abstract class Colour
+ case object RED extends Colour
+ case object YELLOW extends Colour
+ case object BLUE extends Colour
+ case object FADED extends Colour
+
+ val colours = Array[Colour](BLUE, RED, YELLOW)
+
+ var start = 0L
+ var end = 0L
+
+ class Chameneo(var mall: ActorRef, var colour: Colour, cid: Int) extends Actor {
+ var meetings = 0
+ self.start
+ mall ! Meet(self, colour)
+
+ def receive = {
+ case Meet(from, otherColour) =>
+ colour = complement(otherColour)
+ meetings = meetings +1
+ from ! Change(colour)
+ mall ! Meet(self, colour)
+
+ case Change(newColour) =>
+ colour = newColour
+ meetings = meetings +1
+ mall ! Meet(self, colour)
+
+ case Exit =>
+ colour = FADED
+ self.sender.get ! MeetingCount(meetings)
+ }
+
+ def complement(otherColour: Colour): Colour = colour match {
+ case RED => otherColour match {
+ case RED => RED
+ case YELLOW => BLUE
+ case BLUE => YELLOW
+ case FADED => FADED
+ }
+ case YELLOW => otherColour match {
+ case RED => BLUE
+ case YELLOW => YELLOW
+ case BLUE => RED
+ case FADED => FADED
+ }
+ case BLUE => otherColour match {
+ case RED => YELLOW
+ case YELLOW => RED
+ case BLUE => BLUE
+ case FADED => FADED
+ }
+ case FADED => FADED
+ }
+
+ override def toString = cid + "(" + colour + ")"
+ }
+
+ class Mall(var n: Int, numChameneos: Int) extends Actor {
+ var waitingChameneo: Option[ActorRef] = None
+ var sumMeetings = 0
+ var numFaded = 0
+
+ override def preStart = {
+ for (i <- 0 until numChameneos) actorOf(new Chameneo(self, colours(i % 3), i))
+ }
+
+ def receive = {
+ case MeetingCount(i) =>
+ numFaded += 1
+ sumMeetings += i
+ if (numFaded == numChameneos) {
+ Chameneos.end = System.currentTimeMillis
+ self.stop
+ }
+
+ case msg @ Meet(a, c) =>
+ if (n > 0) {
+ waitingChameneo match {
+ case Some(chameneo) =>
+ n -= 1
+ chameneo ! msg
+ waitingChameneo = None
+ case None => waitingChameneo = self.sender
+ }
+ } else {
+ waitingChameneo.foreach(_ ! Exit)
+ self.sender.get ! Exit
+ }
+ }
+ }
+
+ def run {
+// System.setProperty("akka.config", "akka.conf")
+ Chameneos.start = System.currentTimeMillis
+ actorOf(new Mall(1000000, 4)).start
+ Thread.sleep(10000)
+ println("Elapsed: " + (end - start))
+ }
+
+ def main(args : Array[String]): Unit = run
+}
diff --git a/akka-actor/src/test/scala/akka/actor/actor/FSMActorSpec.scala b/akka-actor/src/test/scala/akka/actor/actor/FSMActorSpec.scala
new file mode 100644
index 0000000000..146c57759c
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/actor/FSMActorSpec.scala
@@ -0,0 +1,126 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+import org.multiverse.api.latches.StandardLatch
+
+import java.util.concurrent.TimeUnit
+
+object FSMActorSpec {
+
+ val unlockedLatch = new StandardLatch
+ val lockedLatch = new StandardLatch
+ val unhandledLatch = new StandardLatch
+ val terminatedLatch = new StandardLatch
+ val transitionLatch = new StandardLatch
+
+ sealed trait LockState
+ case object Locked extends LockState
+ case object Open extends LockState
+
+ class Lock(code: String, timeout: Int) extends Actor with FSM[LockState, CodeState] {
+
+ notifying {
+ case Transition(Locked, Open) => transitionLatch.open
+ case Transition(_, _) => ()
+ }
+
+ when(Locked) {
+ case Event(digit: Char, CodeState(soFar, code)) => {
+ soFar + digit match {
+ case incomplete if incomplete.length < code.length =>
+ stay using CodeState(incomplete, code)
+ case codeTry if (codeTry == code) => {
+ doUnlock
+ goto(Open) using CodeState("", code) until timeout
+ }
+ case wrong => {
+ log.error("Wrong code %s", wrong)
+ stay using CodeState("", code)
+ }
+ }
+ }
+ case Event("hello", _) => stay replying "world"
+ case Event("bye", _) => stop(Shutdown)
+ }
+
+ when(Open) {
+ case Event(StateTimeout, stateData) => {
+ doLock
+ goto(Locked)
+ }
+ }
+
+ startWith(Locked, CodeState("", code))
+
+ whenUnhandled {
+ case Event(_, stateData) => {
+ log.info("Unhandled")
+ unhandledLatch.open
+ stay
+ }
+ }
+
+ onTermination {
+ case reason => terminatedLatch.open
+ }
+
+ private def doLock() {
+ log.info("Locked")
+ lockedLatch.open
+ }
+
+ private def doUnlock = {
+ log.info("Unlocked")
+ unlockedLatch.open
+ }
+ }
+
+ case class CodeState(soFar: String, code: String)
+}
+
+class FSMActorSpec extends JUnitSuite {
+ import FSMActorSpec._
+
+ @Test
+ def unlockTheLock = {
+
+ // lock that locked after being open for 1 sec
+ val lock = Actor.actorOf(new Lock("33221", 1000)).start
+
+ lock ! '3'
+ lock ! '3'
+ lock ! '2'
+ lock ! '2'
+ lock ! '1'
+
+ assert(unlockedLatch.tryAwait(1, TimeUnit.SECONDS))
+ assert(transitionLatch.tryAwait(1, TimeUnit.SECONDS))
+ assert(lockedLatch.tryAwait(2, TimeUnit.SECONDS))
+
+ lock ! "not_handled"
+ assert(unhandledLatch.tryAwait(2, TimeUnit.SECONDS))
+
+ val answerLatch = new StandardLatch
+ object Hello
+ object Bye
+ val tester = Actor.actorOf(new Actor {
+ protected def receive = {
+ case Hello => lock ! "hello"
+ case "world" => answerLatch.open
+ case Bye => lock ! "bye"
+ }
+ }).start
+ tester ! Hello
+ assert(answerLatch.tryAwait(2, TimeUnit.SECONDS))
+
+ tester ! Bye
+ assert(terminatedLatch.tryAwait(2, TimeUnit.SECONDS))
+ }
+}
+
diff --git a/akka-actor/src/test/scala/akka/actor/actor/ForwardActorSpec.scala b/akka-actor/src/test/scala/akka/actor/actor/ForwardActorSpec.scala
new file mode 100644
index 0000000000..3a1efe1fe8
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/actor/ForwardActorSpec.scala
@@ -0,0 +1,81 @@
+package akka.actor
+
+import java.util.concurrent.{TimeUnit, CountDownLatch}
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+import Actor._
+
+object ForwardActorSpec {
+ object ForwardState {
+ var sender: Option[ActorRef] = None
+ }
+
+ class ReceiverActor extends Actor {
+ val latch = new CountDownLatch(1)
+ def receive = {
+ case "SendBang" => {
+ ForwardState.sender = self.sender
+ latch.countDown
+ }
+ case "SendBangBang" => self.reply("SendBangBang")
+ }
+ }
+
+
+ class ForwardActor extends Actor {
+ val receiverActor = actorOf[ReceiverActor]
+ receiverActor.start
+ def receive = {
+ case "SendBang" => receiverActor.forward("SendBang")
+ case "SendBangBang" => receiverActor.forward("SendBangBang")
+ }
+ }
+
+ class BangSenderActor extends Actor {
+ val forwardActor = actorOf[ForwardActor]
+ forwardActor.start
+ forwardActor ! "SendBang"
+ def receive = {
+ case _ => {}
+ }
+ }
+
+ class BangBangSenderActor extends Actor {
+ val latch = new CountDownLatch(1)
+ val forwardActor = actorOf[ForwardActor]
+ forwardActor.start
+ (forwardActor !! "SendBangBang") match {
+ case Some(_) => latch.countDown
+ case None => {}
+ }
+ def receive = {
+ case _ => {}
+ }
+ }
+}
+
+class ForwardActorSpec extends JUnitSuite {
+ import ForwardActorSpec._
+
+ @Test
+ def shouldForwardActorReferenceWhenInvokingForwardOnBang {
+ val senderActor = actorOf[BangSenderActor]
+ val latch = senderActor.actor.asInstanceOf[BangSenderActor]
+ .forwardActor.actor.asInstanceOf[ForwardActor]
+ .receiverActor.actor.asInstanceOf[ReceiverActor]
+ .latch
+ senderActor.start
+ assert(latch.await(1L, TimeUnit.SECONDS))
+ assert(ForwardState.sender ne null)
+ assert(senderActor.toString === ForwardState.sender.get.toString)
+ }
+
+ @Test
+ def shouldForwardActorReferenceWhenInvokingForwardOnBangBang {
+ val senderActor = actorOf[BangBangSenderActor]
+ senderActor.start
+ val latch = senderActor.actor.asInstanceOf[BangBangSenderActor].latch
+ assert(latch.await(1L, TimeUnit.SECONDS))
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/actor/actor/HotSwapSpec.scala b/akka-actor/src/test/scala/akka/actor/actor/HotSwapSpec.scala
new file mode 100644
index 0000000000..011141c746
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/actor/HotSwapSpec.scala
@@ -0,0 +1,145 @@
+package akka.actor
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+import Actor._
+
+import java.util.concurrent.CyclicBarrier
+
+class HotSwapSpec extends WordSpec with MustMatchers {
+
+ "An Actor" should {
+
+ "be able to hotswap its behavior with HotSwap(..)" in {
+ val barrier = new CyclicBarrier(2)
+ @volatile var _log = ""
+ val a = actorOf( new Actor {
+ def receive = { case _ => _log += "default" }
+ }).start
+ a ! HotSwap( self => {
+ case _ =>
+ _log += "swapped"
+ barrier.await
+ })
+ a ! "swapped"
+ barrier.await
+ _log must be ("swapped")
+ }
+
+ "be able to hotswap its behavior with become(..)" in {
+ val barrier = new CyclicBarrier(2)
+ @volatile var _log = ""
+ val a = actorOf(new Actor {
+ def receive = {
+ case "init" =>
+ _log += "init"
+ barrier.await
+ case "swap" => become({
+ case _ =>
+ _log += "swapped"
+ barrier.await
+ })
+ }
+ }).start
+
+ a ! "init"
+ barrier.await
+ _log must be ("init")
+
+ barrier.reset
+ _log = ""
+ a ! "swap"
+ a ! "swapped"
+ barrier.await
+ _log must be ("swapped")
+ }
+
+ "be able to revert hotswap its behavior with RevertHotSwap(..)" in {
+ val barrier = new CyclicBarrier(2)
+ @volatile var _log = ""
+ val a = actorOf( new Actor {
+ def receive = {
+ case "init" =>
+ _log += "init"
+ barrier.await
+ }
+ }).start
+
+ a ! "init"
+ barrier.await
+ _log must be ("init")
+
+ barrier.reset
+ _log = ""
+ a ! HotSwap(self => {
+ case "swapped" =>
+ _log += "swapped"
+ barrier.await
+ })
+
+ a ! "swapped"
+ barrier.await
+ _log must be ("swapped")
+
+ barrier.reset
+ _log = ""
+ a ! RevertHotSwap
+
+ a ! "init"
+ barrier.await
+ _log must be ("init")
+
+ // try to revert hotswap below the bottom of the stack
+ barrier.reset
+ _log = ""
+ a ! RevertHotSwap
+
+ a ! "init"
+ barrier.await
+ _log must be ("init")
+ }
+
+ "be able to revert hotswap its behavior with unbecome" in {
+ val barrier = new CyclicBarrier(2)
+ @volatile var _log = ""
+ val a = actorOf(new Actor {
+ def receive = {
+ case "init" =>
+ _log += "init"
+ barrier.await
+ case "swap" =>
+ become({
+ case "swapped" =>
+ _log += "swapped"
+ barrier.await
+ case "revert" =>
+ unbecome
+ })
+ barrier.await
+ }
+ }).start
+
+ a ! "init"
+ barrier.await
+ _log must be ("init")
+
+ barrier.reset
+ _log = ""
+ a ! "swap"
+ barrier.await
+
+ barrier.reset
+ _log = ""
+ a ! "swapped"
+ barrier.await
+ _log must be ("swapped")
+
+ barrier.reset
+ _log = ""
+ a ! "revert"
+ a ! "init"
+ barrier.await
+ _log must be ("init")
+ }
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/actor/actor/ReceiveTimeoutSpec.scala b/akka-actor/src/test/scala/akka/actor/actor/ReceiveTimeoutSpec.scala
new file mode 100644
index 0000000000..e469ab43d1
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/actor/ReceiveTimeoutSpec.scala
@@ -0,0 +1,104 @@
+package akka.actor
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+import java.util.concurrent.TimeUnit
+import org.multiverse.api.latches.StandardLatch
+import Actor._
+import java.util.concurrent.atomic.AtomicInteger
+
+class ReceiveTimeoutSpec extends JUnitSuite {
+
+ @Test def receiveShouldGetTimeout= {
+
+ val timeoutLatch = new StandardLatch
+
+ val timeoutActor = actorOf(new Actor {
+ self.receiveTimeout = Some(500L)
+
+ protected def receive = {
+ case ReceiveTimeout => timeoutLatch.open
+ }
+ }).start
+
+ assert(timeoutLatch.tryAwait(3, TimeUnit.SECONDS))
+ timeoutActor.stop
+ }
+
+ @Test def swappedReceiveShouldAlsoGetTimout = {
+ val timeoutLatch = new StandardLatch
+
+ val timeoutActor = actorOf(new Actor {
+ self.receiveTimeout = Some(500L)
+
+ protected def receive = {
+ case ReceiveTimeout => timeoutLatch.open
+ }
+ }).start
+
+ // after max 1 second the timeout should already been sent
+ assert(timeoutLatch.tryAwait(3, TimeUnit.SECONDS))
+
+ val swappedLatch = new StandardLatch
+ timeoutActor ! HotSwap(self => {
+ case ReceiveTimeout => swappedLatch.open
+ })
+
+ assert(swappedLatch.tryAwait(3, TimeUnit.SECONDS))
+ timeoutActor.stop
+ }
+
+ @Test def timeoutShouldBeRescheduledAfterRegularReceive = {
+
+ val timeoutLatch = new StandardLatch
+ case object Tick
+ val timeoutActor = actorOf(new Actor {
+ self.receiveTimeout = Some(500L)
+
+ protected def receive = {
+ case Tick => ()
+ case ReceiveTimeout => timeoutLatch.open
+ }
+ }).start
+ timeoutActor ! Tick
+
+ assert(timeoutLatch.tryAwait(2, TimeUnit.SECONDS) == true)
+ timeoutActor.stop
+ }
+
+ @Test def timeoutShouldBeTurnedOffIfDesired = {
+ val count = new AtomicInteger(0)
+ val timeoutLatch = new StandardLatch
+ case object Tick
+ val timeoutActor = actorOf(new Actor {
+ self.receiveTimeout = Some(500L)
+
+ protected def receive = {
+ case Tick => ()
+ case ReceiveTimeout =>
+ timeoutLatch.open
+ count.incrementAndGet
+ self.receiveTimeout = None
+ }
+ }).start
+ timeoutActor ! Tick
+
+ assert(timeoutLatch.tryAwait(2, TimeUnit.SECONDS) == true)
+ assert(count.get === 1)
+ timeoutActor.stop
+ }
+
+ @Test def timeoutShouldNotBeSentWhenNotSpecified = {
+ val timeoutLatch = new StandardLatch
+ val timeoutActor = actorOf(new Actor {
+
+ protected def receive = {
+ case ReceiveTimeout => timeoutLatch.open
+ }
+ }).start
+
+ assert(timeoutLatch.tryAwait(1, TimeUnit.SECONDS) == false)
+ timeoutActor.stop
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/actor/supervisor/RestartStrategySpec.scala b/akka-actor/src/test/scala/akka/actor/supervisor/RestartStrategySpec.scala
new file mode 100644
index 0000000000..59bf800909
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/supervisor/RestartStrategySpec.scala
@@ -0,0 +1,275 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import java.lang.Thread.sleep
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+import Actor._
+import java.util.concurrent.{TimeUnit, CountDownLatch}
+import akka.config.Supervision.{Permanent, LifeCycle, OneForOneStrategy}
+import org.multiverse.api.latches.StandardLatch
+
+class RestartStrategySpec extends JUnitSuite {
+
+ object Ping
+ object Crash
+
+ @Test
+ def slaveShouldStayDeadAfterMaxRestartsWithinTimeRange = {
+
+ val boss = actorOf(new Actor{
+ self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), Some(1000))
+ protected def receive = { case _ => () }
+ }).start
+
+ val restartLatch = new StandardLatch
+ val secondRestartLatch = new StandardLatch
+ val countDownLatch = new CountDownLatch(3)
+ val stopLatch = new StandardLatch
+
+
+ val slave = actorOf(new Actor{
+
+ protected def receive = {
+ case Ping => countDownLatch.countDown
+ case Crash => throw new Exception("Crashing...")
+ }
+ override def postRestart(reason: Throwable) = {
+ if (!restartLatch.isOpen)
+ restartLatch.open
+ else
+ secondRestartLatch.open
+ }
+
+ override def postStop = {
+ stopLatch.open
+ }
+ })
+ boss.startLink(slave)
+
+ slave ! Ping
+ slave ! Crash
+ slave ! Ping
+
+ // test restart and post restart ping
+ assert(restartLatch.tryAwait(1, TimeUnit.SECONDS))
+
+ assert(slave.isRunning)
+
+ // now crash again... should not restart
+ slave ! Crash
+ slave ! Ping
+
+ assert(secondRestartLatch.tryAwait(1, TimeUnit.SECONDS))
+ assert(countDownLatch.await(1, TimeUnit.SECONDS))
+
+ slave ! Crash
+ assert(stopLatch.tryAwait(1, TimeUnit.SECONDS))
+
+ assert(!slave.isRunning)
+ }
+
+ @Test
+ def slaveShouldBeImmortalWithoutMaxRestartsAndTimeRange = {
+
+ val boss = actorOf(new Actor{
+ self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, None)
+ protected def receive = { case _ => () }
+ }).start
+
+ val countDownLatch = new CountDownLatch(100)
+
+ val slave = actorOf(new Actor{
+
+ protected def receive = {
+ case Crash => throw new Exception("Crashing...")
+ }
+
+ override def postRestart(reason: Throwable) = {
+ countDownLatch.countDown
+ }
+ })
+
+ boss.startLink(slave)
+ (1 to 100) foreach { _ => slave ! Crash }
+ assert(countDownLatch.await(120, TimeUnit.SECONDS))
+ assert(slave.isRunning)
+ }
+
+ @Test
+ def slaveShouldRestartAfterNumberOfCrashesNotWithinTimeRange = {
+
+ val boss = actorOf(new Actor{
+ self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), Some(500))
+ protected def receive = { case _ => () }
+ }).start
+
+ val restartLatch = new StandardLatch
+ val secondRestartLatch = new StandardLatch
+ val thirdRestartLatch = new StandardLatch
+ val pingLatch = new StandardLatch
+ val secondPingLatch = new StandardLatch
+
+ val slave = actorOf(new Actor{
+
+ protected def receive = {
+ case Ping =>
+ if (!pingLatch.isOpen) pingLatch.open else secondPingLatch.open
+ case Crash => throw new Exception("Crashing...")
+ }
+ override def postRestart(reason: Throwable) = {
+ if (!restartLatch.isOpen)
+ restartLatch.open
+ else if (!secondRestartLatch.isOpen)
+ secondRestartLatch.open
+ else
+ thirdRestartLatch.open
+ }
+
+ override def postStop = {
+ if (restartLatch.isOpen) {
+ secondRestartLatch.open
+ }
+ }
+ })
+ boss.startLink(slave)
+
+ slave ! Ping
+ slave ! Crash
+
+ assert(restartLatch.tryAwait(1, TimeUnit.SECONDS))
+ assert(pingLatch.tryAwait(1, TimeUnit.SECONDS))
+
+ slave ! Ping
+ slave ! Crash
+
+ assert(secondRestartLatch.tryAwait(1, TimeUnit.SECONDS))
+ assert(secondPingLatch.tryAwait(1, TimeUnit.SECONDS))
+
+ // sleep to go out of the restart strategy's time range
+ sleep(700L)
+
+ // now crash again... should and post restart ping
+ slave ! Crash
+ slave ! Ping
+
+ assert(thirdRestartLatch.tryAwait(1, TimeUnit.SECONDS))
+
+ assert(slave.isRunning)
+ }
+
+ @Test
+ def slaveShouldNotRestartAfterMaxRetries = {
+ val boss = actorOf(new Actor{
+ self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), Some(2), None)
+ protected def receive = { case _ => () }
+ }).start
+
+ val restartLatch = new StandardLatch
+ val secondRestartLatch = new StandardLatch
+ val countDownLatch = new CountDownLatch(3)
+ val stopLatch = new StandardLatch
+
+
+ val slave = actorOf(new Actor{
+
+ protected def receive = {
+ case Ping => countDownLatch.countDown
+ case Crash => throw new Exception("Crashing...")
+ }
+ override def postRestart(reason: Throwable) = {
+ if (!restartLatch.isOpen)
+ restartLatch.open
+ else
+ secondRestartLatch.open
+ }
+
+ override def postStop = {
+ stopLatch.open
+ }
+ })
+ boss.startLink(slave)
+
+ slave ! Ping
+ slave ! Crash
+ slave ! Ping
+
+ // test restart and post restart ping
+ assert(restartLatch.tryAwait(1, TimeUnit.SECONDS))
+
+ assert(slave.isRunning)
+
+ // now crash again... should not restart
+ slave ! Crash
+ slave ! Ping
+
+ assert(secondRestartLatch.tryAwait(1, TimeUnit.SECONDS))
+ assert(countDownLatch.await(1, TimeUnit.SECONDS))
+
+ sleep(700L)
+
+ slave ! Crash
+ assert(stopLatch.tryAwait(1, TimeUnit.SECONDS))
+
+ assert(!slave.isRunning)
+ }
+
+ @Test
+ def slaveShouldNotRestartWithinsTimeRange = {
+
+ val restartLatch,stopLatch,maxNoOfRestartsLatch = new StandardLatch
+ val countDownLatch = new CountDownLatch(2)
+
+ val boss = actorOf(new Actor{
+ self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, Some(1000))
+ protected def receive = {
+ case m:MaximumNumberOfRestartsWithinTimeRangeReached => maxNoOfRestartsLatch.open
+ }
+ }).start
+
+ val slave = actorOf(new Actor{
+
+ protected def receive = {
+ case Ping => countDownLatch.countDown
+ case Crash => throw new Exception("Crashing...")
+ }
+
+ override def postRestart(reason: Throwable) = {
+ restartLatch.open
+ }
+
+ override def postStop = {
+ stopLatch.open
+ }
+ })
+ boss.startLink(slave)
+
+ slave ! Ping
+ slave ! Crash
+ slave ! Ping
+
+ // test restart and post restart ping
+ assert(restartLatch.tryAwait(1, TimeUnit.SECONDS))
+
+ assert(slave.isRunning)
+
+ // now crash again... should not restart
+ slave ! Crash
+ slave ! Ping
+
+ assert(countDownLatch.await(1, TimeUnit.SECONDS))
+
+ slave ! Crash
+ assert(stopLatch.tryAwait(1, TimeUnit.SECONDS))
+
+ assert(maxNoOfRestartsLatch.tryAwait(1,TimeUnit.SECONDS))
+
+ assert(!slave.isRunning)
+ }
+}
+
diff --git a/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorHierarchySpec.scala b/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorHierarchySpec.scala
new file mode 100644
index 0000000000..bfdf9b600a
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorHierarchySpec.scala
@@ -0,0 +1,79 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+import Actor._
+import akka.config.Supervision.OneForOneStrategy
+
+import java.util.concurrent.{TimeUnit, CountDownLatch}
+
+object SupervisorHierarchySpec {
+ class FireWorkerException(msg: String) extends Exception(msg)
+
+ class CountDownActor(countDown: CountDownLatch) extends Actor {
+ protected def receive = { case _ => () }
+ override def postRestart(reason: Throwable) = countDown.countDown
+ }
+
+ class CrasherActor extends Actor {
+ protected def receive = { case _ => () }
+ }
+}
+
+class SupervisorHierarchySpec extends JUnitSuite {
+ import SupervisorHierarchySpec._
+
+ @Test
+ def killWorkerShouldRestartMangerAndOtherWorkers = {
+ val countDown = new CountDownLatch(4)
+
+ val workerOne = actorOf(new CountDownActor(countDown))
+ val workerTwo = actorOf(new CountDownActor(countDown))
+ val workerThree = actorOf(new CountDownActor(countDown))
+
+ val boss = actorOf(new Actor{
+ self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 5, 1000)
+
+ protected def receive = { case _ => () }
+ }).start
+
+ val manager = actorOf(new CountDownActor(countDown))
+ boss.startLink(manager)
+
+ manager.startLink(workerOne)
+ manager.startLink(workerTwo)
+ manager.startLink(workerThree)
+
+ workerOne ! Exit(workerOne, new FireWorkerException("Fire the worker!"))
+
+ // manager + all workers should be restarted by only killing a worker
+ // manager doesn't trap exits, so boss will restart manager
+
+ assert(countDown.await(2, TimeUnit.SECONDS))
+ }
+
+ @Test
+ def supervisorShouldReceiveNotificationMessageWhenMaximumNumberOfRestartsWithinTimeRangeIsReached = {
+ val countDown = new CountDownLatch(2)
+ val crasher = actorOf(new CountDownActor(countDown))
+ val boss = actorOf(new Actor{
+ self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 1, 5000)
+ protected def receive = {
+ case MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _) =>
+ countDown.countDown
+ }
+ }).start
+ boss.startLink(crasher)
+
+ crasher ! Exit(crasher, new FireWorkerException("Fire the worker!"))
+ crasher ! Exit(crasher, new FireWorkerException("Fire the worker!"))
+
+ assert(countDown.await(2, TimeUnit.SECONDS))
+ }
+}
+
diff --git a/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala b/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala
new file mode 100644
index 0000000000..08de79c400
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorMiscSpec.scala
@@ -0,0 +1,79 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.actor
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+import akka.dispatch.Dispatchers
+import akka.config.Supervision.{SupervisorConfig, OneForOneStrategy, Supervise, Permanent}
+import java.util.concurrent.CountDownLatch
+
+class SupervisorMiscSpec extends WordSpec with MustMatchers {
+ "A Supervisor" should {
+
+ "restart a crashing actor and its dispatcher for any dispatcher" in {
+ val countDownLatch = new CountDownLatch(4)
+
+ val actor1 = Actor.actorOf(new Actor {
+ self.dispatcher = Dispatchers.newThreadBasedDispatcher(self)
+ override def postRestart(cause: Throwable) {countDownLatch.countDown}
+
+ protected def receive = {
+ case "kill" => throw new Exception("killed")
+ case _ => println("received unknown message")
+ }
+ }).start
+
+ val actor2 = Actor.actorOf(new Actor {
+ self.dispatcher = Dispatchers.newThreadBasedDispatcher(self)
+ override def postRestart(cause: Throwable) {countDownLatch.countDown}
+
+ protected def receive = {
+ case "kill" => throw new Exception("killed")
+ case _ => println("received unknown message")
+ }
+ }).start
+
+ val actor3 = Actor.actorOf(new Actor {
+ self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher("test").build
+ override def postRestart(cause: Throwable) {countDownLatch.countDown}
+
+ protected def receive = {
+ case "kill" => throw new Exception("killed")
+ case _ => println("received unknown message")
+ }
+ }).start
+
+ val actor4 = Actor.actorOf(new Actor {
+ self.dispatcher = Dispatchers.newHawtDispatcher(true)
+ override def postRestart(cause: Throwable) {countDownLatch.countDown}
+
+ protected def receive = {
+ case "kill" => throw new Exception("killed")
+ case _ => println("received unknown message")
+ }
+ }).start
+
+ val sup = Supervisor(
+ SupervisorConfig(
+ OneForOneStrategy(List(classOf[Exception]),3, 5000),
+ Supervise(actor1, Permanent) ::
+ Supervise(actor2, Permanent) ::
+ Supervise(actor3, Permanent) ::
+ Supervise(actor4, Permanent) ::
+ Nil))
+
+ actor1 ! "kill"
+ actor2 ! "kill"
+ actor3 ! "kill"
+ actor4 ! "kill"
+
+ countDownLatch.await()
+ assert(!actor1.isShutdown, "actor1 is shutdown")
+ assert(!actor2.isShutdown, "actor2 is shutdown")
+ assert(!actor3.isShutdown, "actor3 is shutdown")
+ assert(!actor4.isShutdown, "actor4 is shutdown")
+ }
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala b/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala
new file mode 100644
index 0000000000..de06871323
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/actor/supervisor/SupervisorSpec.scala
@@ -0,0 +1,638 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import akka.config.Supervision._
+import akka.{OneWay, Die, Ping}
+import Actor._
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.concurrent. {CountDownLatch, TimeUnit, LinkedBlockingQueue}
+
+object SupervisorSpec {
+ var messageLog = new LinkedBlockingQueue[String]
+ var oneWayLog = new LinkedBlockingQueue[String]
+
+ def clearMessageLogs {
+ messageLog.clear
+ oneWayLog.clear
+ }
+
+ class PingPong1Actor extends Actor {
+ import self._
+ //dispatcher = Dispatchers.newThreadBasedDispatcher(self)
+ def receive = {
+ case Ping =>
+ messageLog.put("ping")
+ reply("pong")
+
+ case OneWay =>
+ oneWayLog.put("oneway")
+
+ case Die =>
+ println("******************** GOT DIE 1")
+ throw new RuntimeException("Expected exception; to test fault-tolerance")
+ }
+ override def postRestart(reason: Throwable) {
+ println("******************** restart 1")
+ messageLog.put(reason.getMessage)
+ }
+ }
+
+ class PingPong2Actor extends Actor {
+ import self._
+ def receive = {
+ case Ping =>
+ messageLog.put("ping")
+ reply("pong")
+ case Die =>
+ println("******************** GOT DIE 2")
+ throw new RuntimeException("Expected exception; to test fault-tolerance")
+ }
+ override def postRestart(reason: Throwable) {
+ println("******************** restart 2")
+ messageLog.put(reason.getMessage)
+ }
+ }
+
+ class PingPong3Actor extends Actor {
+ import self._
+ def receive = {
+ case Ping =>
+ messageLog.put("ping")
+ reply("pong")
+ case Die =>
+ println("******************** GOT DIE 3")
+ throw new RuntimeException("Expected exception; to test fault-tolerance")
+ }
+
+ override def postRestart(reason: Throwable) {
+ println("******************** restart 3")
+ messageLog.put(reason.getMessage)
+ }
+ }
+
+ class TemporaryActor extends Actor {
+ import self._
+ lifeCycle = Temporary
+ def receive = {
+ case Ping =>
+ messageLog.put("ping")
+ reply("pong")
+ case Die =>
+ println("******************** GOT DIE 3")
+ throw new RuntimeException("Expected exception; to test fault-tolerance")
+ }
+
+ override def postRestart(reason: Throwable) {
+ println("******************** restart temporary")
+ messageLog.put(reason.getMessage)
+ }
+ }
+
+ class Master extends Actor {
+ self.faultHandler = OneForOneStrategy(List(classOf[Exception]), 5, 1000)
+ val temp = self.spawnLink[TemporaryActor]
+ override def receive = {
+ case Die => temp !! (Die, 5000)
+ }
+ }
+}
+
+/**
+ * @author Jonas Bonér
+ */
+class SupervisorSpec extends JUnitSuite {
+ import SupervisorSpec._
+
+ var pingpong1: ActorRef = _
+ var pingpong2: ActorRef = _
+ var pingpong3: ActorRef = _
+ var temporaryActor: ActorRef = _
+
+/*
+ @Test def shouldStartServer = {
+ clearMessageLogs
+ val sup = getSingleActorAllForOneSupervisor
+ sup.start
+
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+ }
+*/
+ @Test def shoulNotRestartProgrammaticallyLinkedTemporaryActor = {
+ clearMessageLogs
+ val master = actorOf[Master].start
+
+ intercept[RuntimeException] {
+ master !! (Die, 5000)
+ }
+
+ Thread.sleep(1000)
+ assert(messageLog.size === 0)
+ }
+
+ @Test def shoulNotRestartTemporaryActor = {
+ clearMessageLogs
+ val sup = getTemporaryActorAllForOneSupervisor
+
+ intercept[RuntimeException] {
+ temporaryActor !! (Die, 5000)
+ }
+
+ Thread.sleep(1000)
+ assert(messageLog.size === 0)
+ }
+
+ @Test def shouldStartServerForNestedSupervisorHierarchy = {
+ clearMessageLogs
+ val sup = getNestedSupervisorsAllForOneConf
+ sup.start
+
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+ }
+
+ @Test def shouldKillSingleActorOneForOne = {
+ clearMessageLogs
+ val sup = getSingleActorOneForOneSupervisor
+
+ intercept[RuntimeException] {
+ pingpong1 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldCallKillCallSingleActorOneForOne = {
+ clearMessageLogs
+ val sup = getSingleActorOneForOneSupervisor
+
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ intercept[RuntimeException] {
+ pingpong1 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldKillSingleActorAllForOne = {
+ clearMessageLogs
+ val sup = getSingleActorAllForOneSupervisor
+
+ intercept[RuntimeException] {
+ pingpong1 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldCallKillCallSingleActorAllForOne = {
+ clearMessageLogs
+ val sup = getSingleActorAllForOneSupervisor
+
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ intercept[RuntimeException] {
+ pingpong1 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldKillMultipleActorsOneForOne1 = {
+ clearMessageLogs
+ val sup = getMultipleActorsOneForOneConf
+
+ intercept[RuntimeException] {
+ pingpong1 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldKillMultipleActorsOneForOne2 = {
+ clearMessageLogs
+ val sup = getMultipleActorsOneForOneConf
+
+ intercept[RuntimeException] {
+ pingpong3 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldKillCallMultipleActorsOneForOne = {
+ clearMessageLogs
+ val sup = getMultipleActorsOneForOneConf
+
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong2 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong3 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ intercept[RuntimeException] {
+ pingpong2 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong2 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong3 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldKillMultipleActorsAllForOne = {
+ clearMessageLogs
+ val sup = getMultipleActorsAllForOneConf
+
+ intercept[RuntimeException] {
+ pingpong2 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldCallKillCallMultipleActorsAllForOne = {
+ clearMessageLogs
+ val sup = getMultipleActorsAllForOneConf
+
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong2 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong3 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ intercept[RuntimeException] {
+ pingpong2 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong2 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong3 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldOneWayKillSingleActorOneForOne = {
+ clearMessageLogs
+ val sup = getSingleActorOneForOneSupervisor
+
+ pingpong1 ! Die
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldOneWayCallKillCallSingleActorOneForOne = {
+ clearMessageLogs
+ val sup = getSingleActorOneForOneSupervisor
+
+ pingpong1 ! OneWay
+
+ expect("oneway") {
+ oneWayLog.poll(5, TimeUnit.SECONDS)
+ }
+ pingpong1 ! Die
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ pingpong1 ! OneWay
+
+ expect("oneway") {
+ oneWayLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldRestartKilledActorsForNestedSupervisorHierarchy = {
+ clearMessageLogs
+ val sup = getNestedSupervisorsAllForOneConf
+
+
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong2 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong3 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ intercept[RuntimeException] {
+ pingpong2 !! (Die, 5000)
+ }
+
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5 , TimeUnit.SECONDS)
+ }
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("Expected exception; to test fault-tolerance") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("pong") {
+ (pingpong1 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong2 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("pong") {
+ (pingpong3 !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ expect("ping") {
+ messageLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ @Test def shouldAttemptRestartWhenExceptionDuringRestart {
+ val inits = new AtomicInteger(0)
+ val dyingActor = actorOf(new Actor {
+ self.lifeCycle = Permanent
+ log.debug("Creating dying actor, attempt: " + inits.incrementAndGet)
+
+ if (!(inits.get % 2 != 0))
+ throw new IllegalStateException("Don't wanna!")
+
+
+ def receive = {
+ case Ping => self.reply_?("pong")
+ case Die => throw new Exception("expected")
+ }
+ })
+
+ val supervisor =
+ Supervisor(
+ SupervisorConfig(
+ OneForOneStrategy(classOf[Exception] :: Nil,3,10000),
+ Supervise(dyingActor,Permanent) :: Nil))
+
+ intercept[Exception] {
+ dyingActor !! (Die, 5000)
+ }
+
+ expect("pong") {
+ (dyingActor !! (Ping, 5000)).getOrElse("nil")
+ }
+
+ expect(3) { inits.get }
+ supervisor.shutdown
+ }
+
+ // =============================================
+ // Create some supervisors with different configurations
+
+ def getTemporaryActorAllForOneSupervisor: Supervisor = {
+ temporaryActor = actorOf[TemporaryActor].start
+
+ Supervisor(
+ SupervisorConfig(
+ AllForOneStrategy(List(classOf[Exception]), 3, 5000),
+ Supervise(
+ temporaryActor,
+ Temporary)
+ :: Nil))
+ }
+
+ def getSingleActorAllForOneSupervisor: Supervisor = {
+ pingpong1 = actorOf[PingPong1Actor].start
+
+ Supervisor(
+ SupervisorConfig(
+ AllForOneStrategy(List(classOf[Exception]), 3, 5000),
+ Supervise(
+ pingpong1,
+ Permanent)
+ :: Nil))
+ }
+
+ def getSingleActorOneForOneSupervisor: Supervisor = {
+ pingpong1 = actorOf[PingPong1Actor].start
+
+ Supervisor(
+ SupervisorConfig(
+ OneForOneStrategy(List(classOf[Exception]), 3, 5000),
+ Supervise(
+ pingpong1,
+ Permanent)
+ :: Nil))
+ }
+
+ def getMultipleActorsAllForOneConf: Supervisor = {
+ pingpong1 = actorOf[PingPong1Actor].start
+ pingpong2 = actorOf[PingPong2Actor].start
+ pingpong3 = actorOf[PingPong3Actor].start
+
+ Supervisor(
+ SupervisorConfig(
+ AllForOneStrategy(List(classOf[Exception]), 3, 5000),
+ Supervise(
+ pingpong1,
+ Permanent)
+ ::
+ Supervise(
+ pingpong2,
+ Permanent)
+ ::
+ Supervise(
+ pingpong3,
+ Permanent)
+ :: Nil))
+ }
+
+ def getMultipleActorsOneForOneConf: Supervisor = {
+ pingpong1 = actorOf[PingPong1Actor].start
+ pingpong2 = actorOf[PingPong2Actor].start
+ pingpong3 = actorOf[PingPong3Actor].start
+
+ Supervisor(
+ SupervisorConfig(
+ OneForOneStrategy(List(classOf[Exception]), 3, 5000),
+ Supervise(
+ pingpong1,
+ Permanent)
+ ::
+ Supervise(
+ pingpong2,
+ Permanent)
+ ::
+ Supervise(
+ pingpong3,
+ Permanent)
+ :: Nil))
+ }
+
+ def getNestedSupervisorsAllForOneConf: Supervisor = {
+ pingpong1 = actorOf[PingPong1Actor].start
+ pingpong2 = actorOf[PingPong2Actor].start
+ pingpong3 = actorOf[PingPong3Actor].start
+
+ Supervisor(
+ SupervisorConfig(
+ AllForOneStrategy(List(classOf[Exception]), 3, 5000),
+ Supervise(
+ pingpong1,
+ Permanent)
+ ::
+ SupervisorConfig(
+ AllForOneStrategy(Nil, 3, 5000),
+ Supervise(
+ pingpong2,
+ Permanent)
+ ::
+ Supervise(
+ pingpong3,
+ Permanent)
+ :: Nil)
+ :: Nil))
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/dataflow/DataFlowSpec.scala b/akka-actor/src/test/scala/akka/dataflow/DataFlowSpec.scala
new file mode 100644
index 0000000000..f787060fc8
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dataflow/DataFlowSpec.scala
@@ -0,0 +1,165 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.dataflow
+
+import org.scalatest.Spec
+import org.scalatest.Assertions
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
+
+import akka.dispatch.DefaultCompletableFuture
+import java.util.concurrent.{TimeUnit, CountDownLatch}
+import annotation.tailrec
+import java.util.concurrent.atomic.{AtomicLong, AtomicReference, AtomicInteger}
+import akka.actor.ActorRegistry
+
+@RunWith(classOf[JUnitRunner])
+class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll {
+ describe("DataflowVariable") {
+ it("should be able to set the value of one variable from other variables") {
+ import DataFlow._
+
+ val latch = new CountDownLatch(1)
+ val result = new AtomicInteger(0)
+ val x, y, z = new DataFlowVariable[Int]
+ thread {
+ z << x() + y()
+ result.set(z())
+ latch.countDown
+ }
+ thread { x << 40 }
+ thread { y << 2 }
+
+ latch.await(10,TimeUnit.SECONDS) should equal (true)
+ result.get should equal (42)
+ List(x,y,z).foreach(_.shutdown)
+ }
+
+ it("should be able to sum a sequence of ints") {
+ import DataFlow._
+
+ def ints(n: Int, max: Int): List[Int] =
+ if (n == max) Nil
+ else n :: ints(n + 1, max)
+
+ def sum(s: Int, stream: List[Int]): List[Int] = stream match {
+ case Nil => s :: Nil
+ case h :: t => s :: sum(h + s, t)
+ }
+
+ val latch = new CountDownLatch(1)
+ val result = new AtomicReference[List[Int]](Nil)
+ val x = new DataFlowVariable[List[Int]]
+ val y = new DataFlowVariable[List[Int]]
+ val z = new DataFlowVariable[List[Int]]
+
+ thread { x << ints(0, 1000) }
+ thread { y << sum(0, x()) }
+
+ thread { z << y()
+ result.set(z())
+ latch.countDown
+ }
+
+ latch.await(10,TimeUnit.SECONDS) should equal (true)
+ result.get should equal (sum(0,ints(0,1000)))
+ List(x,y,z).foreach(_.shutdown)
+ }
+
+ /*it("should be able to join streams") {
+ import DataFlow._
+ ActorRegistry.shutdownAll
+
+ def ints(n: Int, max: Int, stream: DataFlowStream[Int]): Unit = if (n != max) {
+ stream <<< n
+ ints(n + 1, max, stream)
+ }
+
+ def sum(s: Int, in: DataFlowStream[Int], out: DataFlowStream[Int]): Unit = {
+ out <<< s
+ sum(in() + s, in, out)
+ }
+
+ val producer = new DataFlowStream[Int]
+ val consumer = new DataFlowStream[Int]
+ val latch = new CountDownLatch(1)
+ val result = new AtomicInteger(0)
+
+ val t1 = thread { ints(0, 1000, producer) }
+ val t2 = thread {
+ Thread.sleep(1000)
+ result.set(producer.map(x => x * x).foldLeft(0)(_ + _))
+ latch.countDown
+ }
+
+ latch.await(3,TimeUnit.SECONDS) should equal (true)
+ result.get should equal (332833500)
+ }
+
+ it("should be able to sum streams recursively") {
+ import DataFlow._
+
+ def ints(n: Int, max: Int, stream: DataFlowStream[Int]): Unit = if (n != max) {
+ stream <<< n
+ ints(n + 1, max, stream)
+ }
+
+ def sum(s: Int, in: DataFlowStream[Int], out: DataFlowStream[Int]): Unit = {
+ out <<< s
+ sum(in() + s, in, out)
+ }
+
+ val result = new AtomicLong(0)
+
+ val producer = new DataFlowStream[Int]
+ val consumer = new DataFlowStream[Int]
+ val latch = new CountDownLatch(1)
+
+ @tailrec def recurseSum(stream: DataFlowStream[Int]): Unit = {
+ val x = stream()
+
+ if(result.addAndGet(x) == 166666500)
+ latch.countDown
+
+ recurseSum(stream)
+ }
+
+ thread { ints(0, 1000, producer) }
+ thread { sum(0, producer, consumer) }
+ thread { recurseSum(consumer) }
+
+ latch.await(15,TimeUnit.SECONDS) should equal (true)
+ }*/
+
+ /* Test not ready for prime time, causes some sort of deadlock */
+ /* it("should be able to conditionally set variables") {
+
+ import DataFlow._
+ ActorRegistry.shutdownAll
+
+ val latch = new CountDownLatch(1)
+ val x, y, z, v = new DataFlowVariable[Int]
+
+ val main = thread {
+ x << 1
+ z << Math.max(x(),y())
+ latch.countDown
+ }
+
+ val setY = thread {
+ // Thread.sleep(2000)
+ y << 2
+ }
+
+ val setV = thread {
+ v << y
+ }
+ List(x,y,z,v) foreach (_.shutdown)
+ latch.await(2,TimeUnit.SECONDS) should equal (true)
+ }*/
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/ActorModelSpec.scala b/akka-actor/src/test/scala/akka/dispatch/ActorModelSpec.scala
new file mode 100644
index 0000000000..df088ce89c
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/ActorModelSpec.scala
@@ -0,0 +1,312 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.actor.dispatch
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+import org.scalatest.Assertions._
+import akka.dispatch._
+import akka.actor.{ActorRef, Actor}
+import akka.actor.Actor._
+import java.util.concurrent.atomic.AtomicLong
+import java.util.concurrent. {ConcurrentHashMap, CountDownLatch, TimeUnit}
+import akka.actor.dispatch.ActorModelSpec.MessageDispatcherInterceptor
+import akka.util.Duration
+
+object ActorModelSpec {
+
+ sealed trait ActorModelMessage
+ case class Reply_?(expect: Any) extends ActorModelMessage
+ case class Reply(expect: Any) extends ActorModelMessage
+ case class Forward(to: ActorRef,msg: Any) extends ActorModelMessage
+ case class CountDown(latch: CountDownLatch) extends ActorModelMessage
+ case class Increment(counter: AtomicLong) extends ActorModelMessage
+ case class Await(latch: CountDownLatch) extends ActorModelMessage
+ case class Meet(acknowledge: CountDownLatch, waitFor: CountDownLatch) extends ActorModelMessage
+ case class CountDownNStop(latch: CountDownLatch) extends ActorModelMessage
+ case object Restart extends ActorModelMessage
+
+ val Ping = "Ping"
+ val Pong = "Pong"
+
+ class DispatcherActor(dispatcher: MessageDispatcherInterceptor) extends Actor {
+ self.dispatcher = dispatcher.asInstanceOf[MessageDispatcher]
+
+ def ack { dispatcher.getStats(self).msgsProcessed.incrementAndGet() }
+
+ override def postRestart(reason: Throwable) {
+ dispatcher.getStats(self).restarts.incrementAndGet()
+ }
+
+ def receive = {
+ case Await(latch) => ack; latch.await()
+ case Meet(sign, wait) => ack; sign.countDown(); wait.await()
+ case Reply(msg) => ack; self.reply(msg)
+ case Reply_?(msg) => ack; self.reply_?(msg)
+ case Forward(to,msg) => ack; to.forward(msg)
+ case CountDown(latch) => ack; latch.countDown()
+ case Increment(count) => ack; count.incrementAndGet()
+ case CountDownNStop(l)=> ack; l.countDown; self.stop
+ case Restart => ack; throw new Exception("Restart requested")
+ }
+ }
+
+ class InterceptorStats {
+ val suspensions = new AtomicLong(0)
+ val resumes = new AtomicLong(0)
+ val registers = new AtomicLong(0)
+ val unregisters = new AtomicLong(0)
+ val msgsReceived = new AtomicLong(0)
+ val msgsProcessed = new AtomicLong(0)
+ val restarts = new AtomicLong(0)
+ }
+
+ trait MessageDispatcherInterceptor extends MessageDispatcher {
+ val stats = new ConcurrentHashMap[ActorRef,InterceptorStats]
+ val starts = new AtomicLong(0)
+ val stops = new AtomicLong(0)
+
+ def getStats(actorRef: ActorRef) = {
+ stats.putIfAbsent(actorRef,new InterceptorStats)
+ stats.get(actorRef)
+ }
+
+ abstract override def suspend(actorRef: ActorRef) {
+ super.suspend(actorRef)
+ getStats(actorRef).suspensions.incrementAndGet()
+ }
+
+ abstract override def resume(actorRef: ActorRef) {
+ super.resume(actorRef)
+ getStats(actorRef).resumes.incrementAndGet()
+ }
+
+ private[akka] abstract override def register(actorRef: ActorRef) {
+ super.register(actorRef)
+ getStats(actorRef).registers.incrementAndGet()
+ }
+
+ private[akka] abstract override def unregister(actorRef: ActorRef) {
+ super.unregister(actorRef)
+ getStats(actorRef).unregisters.incrementAndGet()
+ }
+
+ private[akka] abstract override def dispatch(invocation: MessageInvocation) {
+ super.dispatch(invocation)
+ getStats(invocation.receiver).msgsReceived.incrementAndGet()
+ }
+
+ private[akka] abstract override def start {
+ super.start
+ starts.incrementAndGet()
+ }
+
+ private[akka] abstract override def shutdown {
+ super.shutdown
+ stops.incrementAndGet()
+ }
+ }
+
+ def assertDispatcher(dispatcher: MessageDispatcherInterceptor)(
+ starts: Long = dispatcher.starts.get(),
+ stops: Long = dispatcher.stops.get()
+ ) {
+ assert(starts === dispatcher.starts.get(), "Dispatcher starts")
+ assert(stops === dispatcher.stops.get(), "Dispatcher stops")
+ }
+
+ def assertCountDown(latch: CountDownLatch,wait: Long,hint: AnyRef){
+ assert(latch.await(wait,TimeUnit.MILLISECONDS) === true)
+ }
+
+ def assertNoCountDown(latch: CountDownLatch,wait: Long,hint: AnyRef){
+ assert(latch.await(wait,TimeUnit.MILLISECONDS) === false)
+ }
+
+ def statsFor(actorRef: ActorRef, dispatcher: MessageDispatcher = null) =
+ dispatcher.asInstanceOf[MessageDispatcherInterceptor].getStats(actorRef)
+
+ def assertRefDefaultZero(actorRef: ActorRef,dispatcher: MessageDispatcher = null)(
+ suspensions: Long = 0,
+ resumes: Long = 0,
+ registers: Long = 0,
+ unregisters: Long = 0,
+ msgsReceived: Long = 0,
+ msgsProcessed: Long = 0,
+ restarts: Long = 0) {
+ assertRef(actorRef,dispatcher)(
+ suspensions,
+ resumes,
+ registers,
+ unregisters,
+ msgsReceived,
+ msgsProcessed,
+ restarts
+ )
+ }
+
+ def assertRef(actorRef: ActorRef,dispatcher: MessageDispatcher = null)(
+ suspensions: Long = statsFor(actorRef).suspensions.get(),
+ resumes: Long = statsFor(actorRef).resumes.get(),
+ registers: Long = statsFor(actorRef).registers.get(),
+ unregisters: Long = statsFor(actorRef).unregisters.get(),
+ msgsReceived: Long = statsFor(actorRef).msgsReceived.get(),
+ msgsProcessed: Long = statsFor(actorRef).msgsProcessed.get(),
+ restarts: Long = statsFor(actorRef).restarts.get()
+ ) {
+ val stats = statsFor(actorRef,if (dispatcher eq null) actorRef.dispatcher else dispatcher)
+ assert(stats.suspensions.get() === suspensions, "Suspensions")
+ assert(stats.resumes.get() === resumes, "Resumes")
+ assert(stats.registers.get() === registers, "Registers")
+ assert(stats.unregisters.get() === unregisters, "Unregisters")
+ assert(stats.msgsReceived.get() === msgsReceived, "Received")
+ assert(stats.msgsProcessed.get() === msgsProcessed, "Processed")
+ assert(stats.restarts.get() === restarts, "Restarts")
+ }
+
+ def await(condition: => Boolean)(withinMs: Long, intervalMs: Long = 25): Boolean = try {
+ val until = System.currentTimeMillis() + withinMs
+ while(System.currentTimeMillis() <= until) {
+ try {
+ if (condition) return true
+
+ Thread.sleep(intervalMs)
+ } catch { case e: InterruptedException => }
+ }
+ false
+ }
+
+ def newTestActor(implicit d: MessageDispatcherInterceptor) = actorOf(new DispatcherActor(d))
+}
+
+abstract class ActorModelSpec extends JUnitSuite {
+ import ActorModelSpec._
+
+ protected def newInterceptedDispatcher: MessageDispatcherInterceptor
+
+ @Test def dispatcherShouldDynamicallyHandleItsOwnLifeCycle {
+ implicit val dispatcher = newInterceptedDispatcher
+ val a = newTestActor
+ assertDispatcher(dispatcher)(starts = 0, stops = 0)
+ a.start
+ assertDispatcher(dispatcher)(starts = 1, stops = 0)
+ a.stop
+ await(dispatcher.stops.get == 1)(withinMs = dispatcher.timeoutMs * 5)
+ assertDispatcher(dispatcher)(starts = 1, stops = 1)
+ assertRef(a,dispatcher)(
+ suspensions = 0,
+ resumes = 0,
+ registers = 1,
+ unregisters = 1,
+ msgsReceived = 0,
+ msgsProcessed = 0,
+ restarts = 0
+ )
+ }
+
+ @Test def dispatcherShouldProcessMessagesOneAtATime {
+ implicit val dispatcher = newInterceptedDispatcher
+ val a = newTestActor
+ val start,step1,step2,oneAtATime = new CountDownLatch(1)
+ a.start
+
+ a ! CountDown(start)
+ assertCountDown(start,3000, "Should process first message within 3 seconds")
+ assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, msgsProcessed = 1)
+
+ a ! Meet(step1,step2)
+ assertCountDown(step1,3000, "Didn't process the Meet message in 3 seocnds")
+ assertRefDefaultZero(a)(registers = 1, msgsReceived = 2, msgsProcessed = 2)
+
+ a ! CountDown(oneAtATime)
+ assertNoCountDown(oneAtATime,500,"Processed message when not allowed to")
+ assertRefDefaultZero(a)(registers = 1, msgsReceived = 3, msgsProcessed = 2)
+
+ step2.countDown()
+ assertCountDown(oneAtATime,500,"Processed message when allowed")
+ assertRefDefaultZero(a)(registers = 1, msgsReceived = 3, msgsProcessed = 3)
+
+ a.stop
+ assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 3, msgsProcessed = 3)
+ }
+
+ @Test def dispatcherShouldProcessMessagesInParallel: Unit = {
+ implicit val dispatcher = newInterceptedDispatcher
+ val a, b = newTestActor.start
+ val aStart,aStop,bParallel = new CountDownLatch(1)
+
+ a ! Meet(aStart,aStop)
+ assertCountDown(aStart,3000, "Should process first message within 3 seconds")
+
+ b ! CountDown(bParallel)
+ assertCountDown(bParallel, 3000, "Should process other actors in parallel")
+
+ aStop.countDown()
+ a.stop
+ b.stop
+ assertRefDefaultZero(a)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1)
+ assertRefDefaultZero(b)(registers = 1, unregisters = 1, msgsReceived = 1, msgsProcessed = 1)
+ }
+
+ @Test def dispatcherShouldSuspendAndResumeAFailingNonSupervisedPermanentActor {
+ implicit val dispatcher = newInterceptedDispatcher
+ val a = newTestActor.start
+ val done = new CountDownLatch(1)
+ a ! Restart
+ a ! CountDown(done)
+ assertCountDown(done, 3000, "Should be suspended+resumed and done with next message within 3 seconds")
+ a.stop
+ assertRefDefaultZero(a)(registers = 1,unregisters = 1, msgsReceived = 2,
+ msgsProcessed = 2, suspensions = 1, resumes = 1)
+ }
+
+ @Test def dispatcherShouldNotProcessMessagesForASuspendedActor {
+ implicit val dispatcher = newInterceptedDispatcher
+ val a = newTestActor.start
+ val done = new CountDownLatch(1)
+ dispatcher.suspend(a)
+ a ! CountDown(done)
+ assertNoCountDown(done, 1000, "Should not process messages while suspended")
+ assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, suspensions = 1)
+
+ dispatcher.resume(a)
+ assertCountDown(done, 3000, "Should resume processing of messages when resumed")
+ assertRefDefaultZero(a)(registers = 1, msgsReceived = 1, msgsProcessed = 1,
+ suspensions = 1, resumes = 1)
+
+ a.stop
+ assertRefDefaultZero(a)(registers = 1,unregisters = 1, msgsReceived = 1, msgsProcessed = 1,
+ suspensions = 1, resumes = 1)
+ }
+
+ @Test def dispatcherShouldHandleWavesOfActors {
+ implicit val dispatcher = newInterceptedDispatcher
+
+ def flood(num: Int) {
+ val cachedMessage = CountDownNStop(new CountDownLatch(num))
+ (1 to num) foreach {
+ _ => newTestActor.start ! cachedMessage
+ }
+ assertCountDown(cachedMessage.latch,10000, "Should process " + num + " countdowns")
+ }
+ for(run <- 1 to 3) {
+ flood(10000)
+ await(dispatcher.stops.get == run)(withinMs = 10000)
+ assertDispatcher(dispatcher)(starts = run, stops = run)
+ }
+ }
+}
+
+class ExecutorBasedEventDrivenDispatcherModelTest extends ActorModelSpec {
+ def newInterceptedDispatcher =
+ new ExecutorBasedEventDrivenDispatcher("foo") with MessageDispatcherInterceptor
+}
+
+class HawtDispatcherModelTest extends ActorModelSpec {
+ def newInterceptedDispatcher = new HawtDispatcher(false) with MessageDispatcherInterceptor
+}
+
+class ExecutorBasedEventDrivenWorkStealingDispatcherModelTest extends ActorModelSpec {
+ def newInterceptedDispatcher = new ExecutorBasedEventDrivenWorkStealingDispatcher("foo") with MessageDispatcherInterceptor
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/DispatchersSpec.scala b/akka-actor/src/test/scala/akka/dispatch/DispatchersSpec.scala
new file mode 100644
index 0000000000..ba9398de2c
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/DispatchersSpec.scala
@@ -0,0 +1,70 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.actor.dispatch
+
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+import net.lag.configgy.Config
+import scala.reflect.{Manifest}
+import akka.dispatch._
+
+object DispatchersSpec {
+ import Dispatchers._
+ //
+ val tipe = "type"
+ val keepalivems = "keep-alive-time"
+ val corepoolsizefactor = "core-pool-size-factor"
+ val maxpoolsizefactor = "max-pool-size-factor"
+ val executorbounds = "executor-bounds"
+ val allowcoretimeout = "allow-core-timeout"
+ val rejectionpolicy = "rejection-policy" // abort, caller-runs, discard-oldest, discard
+ val throughput = "throughput" // Throughput for ExecutorBasedEventDrivenDispatcher
+ val aggregate = "aggregate" // Aggregate on/off for HawtDispatchers
+
+ def instance(dispatcher: MessageDispatcher): (MessageDispatcher) => Boolean = _ == dispatcher
+ def ofType[T <: MessageDispatcher : Manifest]: (MessageDispatcher) => Boolean = _.getClass == manifest[T].erasure
+
+ def typesAndValidators: Map[String,(MessageDispatcher) => Boolean] = Map(
+ "ExecutorBasedEventDrivenWorkStealing" -> ofType[ExecutorBasedEventDrivenWorkStealingDispatcher],
+ "ExecutorBasedEventDriven" -> ofType[ExecutorBasedEventDrivenDispatcher],
+ "Hawt" -> ofType[HawtDispatcher],
+ "GlobalExecutorBasedEventDriven" -> instance(globalExecutorBasedEventDrivenDispatcher),
+ "GlobalHawt" -> instance(globalHawtDispatcher)
+ )
+
+ def validTypes = typesAndValidators.keys.toList
+
+ lazy val allDispatchers: Map[String,Option[MessageDispatcher]] = {
+ validTypes.map(t => (t,from(Config.fromMap(Map(tipe -> t))))).toMap
+ }
+}
+
+class DispatchersSpec extends JUnitSuite {
+
+ import Dispatchers._
+ import DispatchersSpec._
+
+ @Test def shouldYieldNoneIfTypeIsMissing {
+ assert(from(Config.fromMap(Map())) === None)
+ }
+
+ @Test(expected = classOf[IllegalArgumentException])
+ def shouldThrowIllegalArgumentExceptionIfTypeDoesntExist {
+ from(Config.fromMap(Map(tipe -> "typedoesntexist")))
+ }
+
+ @Test def shouldGetTheCorrectTypesOfDispatchers {
+ //It can create/obtain all defined types
+ assert(allDispatchers.values.forall(_.isDefined))
+ //All created/obtained dispatchers are of the expeced type/instance
+ assert(typesAndValidators.forall( tuple => tuple._2(allDispatchers(tuple._1).get) ))
+ }
+
+ @Test def defaultingToDefaultWhileLoadingTheDefaultShouldWork {
+ assert(from(Config.fromMap(Map())).getOrElse(defaultGlobalDispatcher) == defaultGlobalDispatcher)
+ }
+
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
new file mode 100644
index 0000000000..e9b34c17d3
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
@@ -0,0 +1,138 @@
+package akka.actor.dispatch
+
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+import akka.dispatch.{Dispatchers,ExecutorBasedEventDrivenDispatcher}
+import akka.actor.Actor
+import Actor._
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
+
+object ExecutorBasedEventDrivenDispatcherActorSpec {
+ class TestActor extends Actor {
+ self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString).build
+ def receive = {
+ case "Hello" =>
+ self.reply("World")
+ case "Failure" =>
+ throw new RuntimeException("Expected exception; to test fault-tolerance")
+ }
+ }
+
+ object OneWayTestActor {
+ val oneWay = new CountDownLatch(1)
+ }
+ class OneWayTestActor extends Actor {
+ self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid.toString).build
+ def receive = {
+ case "OneWay" => OneWayTestActor.oneWay.countDown
+ }
+ }
+}
+class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite {
+ import ExecutorBasedEventDrivenDispatcherActorSpec._
+
+ private val unit = TimeUnit.MILLISECONDS
+
+ @Test def shouldSendOneWay = {
+ val actor = actorOf[OneWayTestActor].start
+ val result = actor ! "OneWay"
+ assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS))
+ actor.stop
+ }
+
+ @Test def shouldSendReplySync = {
+ val actor = actorOf[TestActor].start
+ val result = (actor !! ("Hello", 10000)).as[String]
+ assert("World" === result.get)
+ actor.stop
+ }
+
+ @Test def shouldSendReplyAsync = {
+ val actor = actorOf[TestActor].start
+ val result = actor !! "Hello"
+ assert("World" === result.get.asInstanceOf[String])
+ actor.stop
+ }
+
+ @Test def shouldSendReceiveException = {
+ val actor = actorOf[TestActor].start
+ try {
+ actor !! "Failure"
+ fail("Should have thrown an exception")
+ } catch {
+ case e =>
+ assert("Expected exception; to test fault-tolerance" === e.getMessage())
+ }
+ actor.stop
+ }
+
+ @Test def shouldRespectThroughput {
+ val throughputDispatcher = Dispatchers.
+ newExecutorBasedEventDrivenDispatcher("THROUGHPUT",101,0,Dispatchers.MAILBOX_TYPE).
+ setCorePoolSize(1).
+ build
+
+ val works = new AtomicBoolean(true)
+ val latch = new CountDownLatch(100)
+ val start = new CountDownLatch(1)
+ val fastOne = actorOf(
+ new Actor {
+ self.dispatcher = throughputDispatcher
+ def receive = { case "sabotage" => works.set(false) }
+ }).start
+
+ val slowOne = actorOf(
+ new Actor {
+ self.dispatcher = throughputDispatcher
+ def receive = {
+ case "hogexecutor" => start.await
+ case "ping" => if (works.get) latch.countDown
+ }
+ }).start
+
+ slowOne ! "hogexecutor"
+ (1 to 100) foreach { _ => slowOne ! "ping"}
+ fastOne ! "sabotage"
+ start.countDown
+ val result = latch.await(3,TimeUnit.SECONDS)
+ fastOne.stop
+ slowOne.stop
+ assert(result === true)
+ }
+
+ @Test def shouldRespectThroughputDeadline {
+ val deadlineMs = 100
+ val throughputDispatcher = Dispatchers.
+ newExecutorBasedEventDrivenDispatcher("THROUGHPUT",2,deadlineMs,Dispatchers.MAILBOX_TYPE).
+ setCorePoolSize(1).
+ build
+ val works = new AtomicBoolean(true)
+ val latch = new CountDownLatch(1)
+ val start = new CountDownLatch(1)
+ val ready = new CountDownLatch(1)
+
+ val fastOne = actorOf(
+ new Actor {
+ self.dispatcher = throughputDispatcher
+ def receive = { case "ping" => if(works.get) latch.countDown; self.stop }
+ }).start
+
+ val slowOne = actorOf(
+ new Actor {
+ self.dispatcher = throughputDispatcher
+ def receive = {
+ case "hogexecutor" => ready.countDown; start.await
+ case "ping" => works.set(false); self.stop
+ }
+ }).start
+
+ slowOne ! "hogexecutor"
+ slowOne ! "ping"
+ fastOne ! "ping"
+ assert(ready.await(2,TimeUnit.SECONDS) === true)
+ Thread.sleep(deadlineMs+10) // wait just a bit more than the deadline
+ start.countDown
+ assert(latch.await(2,TimeUnit.SECONDS) === true)
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala b/akka-actor/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala
new file mode 100644
index 0000000000..66a02e0d33
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenDispatcherActorsSpec.scala
@@ -0,0 +1,61 @@
+package akka.actor.dispatch
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+import org.scalatest.matchers.MustMatchers
+import java.util.concurrent.CountDownLatch
+import akka.actor.Actor
+import Actor._
+
+/**
+ * Tests the behaviour of the executor based event driven dispatcher when multiple actors are being dispatched on it.
+ *
+ * @author Jan Van Besien
+ */
+class ExecutorBasedEventDrivenDispatcherActorsSpec extends JUnitSuite with MustMatchers {
+ class SlowActor(finishedCounter: CountDownLatch) extends Actor {
+ self.id = "SlowActor"
+
+ def receive = {
+ case x: Int => {
+ Thread.sleep(50) // slow actor
+ finishedCounter.countDown
+ }
+ }
+ }
+
+ class FastActor(finishedCounter: CountDownLatch) extends Actor {
+ self.id = "FastActor"
+
+ def receive = {
+ case x: Int => {
+ finishedCounter.countDown
+ }
+ }
+ }
+
+ @Test def slowActorShouldntBlockFastActor {
+ val sFinished = new CountDownLatch(50)
+ val fFinished = new CountDownLatch(10)
+ val s = actorOf(new SlowActor(sFinished)).start
+ val f = actorOf(new FastActor(fFinished)).start
+
+ // send a lot of stuff to s
+ for (i <- 1 to 50) {
+ s ! i
+ }
+
+ // send some messages to f
+ for (i <- 1 to 10) {
+ f ! i
+ }
+
+ // now assert that f is finished while s is still busy
+ fFinished.await
+ assert(sFinished.getCount > 0)
+ sFinished.await
+ assert(sFinished.getCount === 0)
+ f.stop
+ s.stop
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala b/akka-actor/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala
new file mode 100644
index 0000000000..698ffe8063
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala
@@ -0,0 +1,115 @@
+package akka.actor.dispatch
+
+import org.scalatest.matchers.MustMatchers
+import org.scalatest.junit.JUnitSuite
+
+import org.junit.Test
+
+import java.util.concurrent.{TimeUnit, CountDownLatch}
+import akka.actor.{IllegalActorStateException, Actor}
+import Actor._
+import akka.dispatch.{MessageQueue, Dispatchers}
+
+object ExecutorBasedEventDrivenWorkStealingDispatcherSpec {
+ val delayableActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher").build
+ val sharedActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher").build
+ val parentActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher").build
+
+ class DelayableActor(name: String, delay: Int, finishedCounter: CountDownLatch) extends Actor {
+ self.dispatcher = delayableActorDispatcher
+ @volatile var invocationCount = 0
+ self.id = name
+
+ def receive = {
+ case x: Int => {
+ Thread.sleep(delay)
+ invocationCount += 1
+ finishedCounter.countDown
+ }
+ }
+ }
+
+ class FirstActor extends Actor {
+ self.dispatcher = sharedActorDispatcher
+ def receive = {case _ => {}}
+ }
+
+ class SecondActor extends Actor {
+ self.dispatcher = sharedActorDispatcher
+ def receive = {case _ => {}}
+ }
+
+ class ParentActor extends Actor {
+ self.dispatcher = parentActorDispatcher
+ def receive = {case _ => {}}
+ }
+
+ class ChildActor extends ParentActor {
+ }
+}
+
+/**
+ * @author Jan Van Besien
+ */
+class ExecutorBasedEventDrivenWorkStealingDispatcherSpec extends JUnitSuite with MustMatchers {
+ import ExecutorBasedEventDrivenWorkStealingDispatcherSpec._
+
+ @Test def fastActorShouldStealWorkFromSlowActor {
+ val finishedCounter = new CountDownLatch(110)
+
+ val slow = actorOf(new DelayableActor("slow", 50, finishedCounter)).start
+ val fast = actorOf(new DelayableActor("fast", 10, finishedCounter)).start
+
+ var sentToFast = 0
+
+ for (i <- 1 to 100) {
+ // send most work to slow actor
+ if (i % 20 == 0) {
+ fast ! i
+ sentToFast += 1
+ }
+ else
+ slow ! i
+ }
+
+ // now send some messages to actors to keep the dispatcher dispatching messages
+ for (i <- 1 to 10) {
+ Thread.sleep(150)
+ if (i % 2 == 0) {
+ fast ! i
+ sentToFast += 1
+ }
+ else
+ slow ! i
+ }
+
+ finishedCounter.await(5, TimeUnit.SECONDS)
+ fast.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true)
+ slow.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true)
+ fast.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast
+ fast.actor.asInstanceOf[DelayableActor].invocationCount must be >
+ (slow.actor.asInstanceOf[DelayableActor].invocationCount)
+ slow.stop
+ fast.stop
+ }
+
+ @Test def canNotUseActorsOfDifferentTypesInSameDispatcher(): Unit = {
+ val first = actorOf[FirstActor]
+ val second = actorOf[SecondActor]
+
+ first.start
+ intercept[IllegalActorStateException] {
+ second.start
+ }
+ }
+
+ @Test def canNotUseActorsOfDifferentSubTypesInSameDispatcher(): Unit = {
+ val parent = actorOf[ParentActor]
+ val child = actorOf[ChildActor]
+
+ parent.start
+ intercept[IllegalActorStateException] {
+ child.start
+ }
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/FutureSpec.scala b/akka-actor/src/test/scala/akka/dispatch/FutureSpec.scala
new file mode 100644
index 0000000000..2b456a7cd3
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/FutureSpec.scala
@@ -0,0 +1,146 @@
+package akka.actor
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+import akka.dispatch.Futures
+import Actor._
+import org.multiverse.api.latches.StandardLatch
+import java.util.concurrent.CountDownLatch
+
+object FutureSpec {
+ class TestActor extends Actor {
+ def receive = {
+ case "Hello" =>
+ self.reply("World")
+ case "NoReply" => {}
+ case "Failure" =>
+ throw new RuntimeException("Expected exception; to test fault-tolerance")
+ }
+ }
+
+ class TestDelayActor(await: StandardLatch) extends Actor {
+ def receive = {
+ case "Hello" =>
+ await.await
+ self.reply("World")
+ case "NoReply" => { await.await }
+ case "Failure" =>
+ await.await
+ throw new RuntimeException("Expected exception; to test fault-tolerance")
+ }
+ }
+}
+
+class FutureSpec extends JUnitSuite {
+ import FutureSpec._
+
+ @Test def shouldActorReplyResultThroughExplicitFuture {
+ val actor = actorOf[TestActor]
+ actor.start
+ val future = actor !!! "Hello"
+ future.await
+ assert(future.result.isDefined)
+ assert("World" === future.result.get)
+ actor.stop
+ }
+
+ @Test def shouldActorReplyExceptionThroughExplicitFuture {
+ val actor = actorOf[TestActor]
+ actor.start
+ val future = actor !!! "Failure"
+ future.await
+ assert(future.exception.isDefined)
+ assert("Expected exception; to test fault-tolerance" === future.exception.get.getMessage)
+ actor.stop
+ }
+
+ // FIXME: implement Futures.awaitEither, and uncomment these two tests
+ @Test def shouldFutureAwaitEitherLeft = {
+ val actor1 = actorOf[TestActor].start
+ val actor2 = actorOf[TestActor].start
+ val future1 = actor1 !!! "Hello"
+ val future2 = actor2 !!! "NoReply"
+ val result = Futures.awaitEither(future1, future2)
+ assert(result.isDefined)
+ assert("World" === result.get)
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldFutureAwaitEitherRight = {
+ val actor1 = actorOf[TestActor].start
+ val actor2 = actorOf[TestActor].start
+ val future1 = actor1 !!! "NoReply"
+ val future2 = actor2 !!! "Hello"
+ val result = Futures.awaitEither(future1, future2)
+ assert(result.isDefined)
+ assert("World" === result.get)
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldFutureAwaitOneLeft = {
+ val actor1 = actorOf[TestActor].start
+ val actor2 = actorOf[TestActor].start
+ val future1 = actor1 !!! "NoReply"
+ val future2 = actor2 !!! "Hello"
+ val result = Futures.awaitOne(List(future1, future2))
+ assert(result.result.isDefined)
+ assert("World" === result.result.get)
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldFutureAwaitOneRight = {
+ val actor1 = actorOf[TestActor].start
+ val actor2 = actorOf[TestActor].start
+ val future1 = actor1 !!! "Hello"
+ val future2 = actor2 !!! "NoReply"
+ val result = Futures.awaitOne(List(future1, future2))
+ assert(result.result.isDefined)
+ assert("World" === result.result.get)
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldFutureAwaitAll = {
+ val actor1 = actorOf[TestActor].start
+ val actor2 = actorOf[TestActor].start
+ val future1 = actor1 !!! "Hello"
+ val future2 = actor2 !!! "Hello"
+ Futures.awaitAll(List(future1, future2))
+ assert(future1.result.isDefined)
+ assert("World" === future1.result.get)
+ assert(future2.result.isDefined)
+ assert("World" === future2.result.get)
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldFutureMapBeDeferred {
+ val latch = new StandardLatch
+ val actor1 = actorOf(new TestDelayActor(latch)).start
+
+ val mappedFuture = (actor1.!!).map(x => 5)
+ assert(mappedFuture.isCompleted === false)
+ assert(mappedFuture.isExpired === false)
+ latch.open
+ mappedFuture.await
+ assert(mappedFuture.isCompleted === true)
+ assert(mappedFuture.isExpired === false)
+ assert(mappedFuture.result === Some(5))
+ }
+
+ @Test def shouldFuturesAwaitMapHandleEmptySequence {
+ assert(Futures.awaitMap[Nothing,Unit](Nil)(x => ()) === Nil)
+ }
+
+ @Test def shouldFuturesAwaitMapHandleNonEmptySequence {
+ val latches = (1 to 3) map (_ => new StandardLatch)
+ val actors = latches map (latch => actorOf(new TestDelayActor(latch)).start)
+ val futures = actors map (actor => (actor.!!))
+ latches foreach { _.open }
+
+ assert(Futures.awaitMap(futures)(_.result.map(_.length).getOrElse(0)).sum === (latches.size * "World".length))
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/HawtDispatcherActorSpec.scala b/akka-actor/src/test/scala/akka/dispatch/HawtDispatcherActorSpec.scala
new file mode 100644
index 0000000000..1d1b2c1e24
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/HawtDispatcherActorSpec.scala
@@ -0,0 +1,71 @@
+package akka.actor.dispatch
+
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+import akka.dispatch.{HawtDispatcher, Dispatchers}
+import akka.actor.Actor
+import Actor._
+
+object HawtDispatcherActorSpec {
+ class TestActor extends Actor {
+ self.dispatcher = new HawtDispatcher()
+ def receive = {
+ case "Hello" =>
+ self.reply("World")
+ case "Failure" =>
+ throw new RuntimeException("Expected exception; to test fault-tolerance")
+ }
+ }
+
+ object OneWayTestActor {
+ val oneWay = new CountDownLatch(1)
+ }
+ class OneWayTestActor extends Actor {
+ self.dispatcher = new HawtDispatcher()
+ def receive = {
+ case "OneWay" => OneWayTestActor.oneWay.countDown
+ }
+ }
+}
+
+class HawtDispatcherActorSpec extends JUnitSuite {
+ import HawtDispatcherActorSpec._
+
+ private val unit = TimeUnit.MILLISECONDS
+
+ @Test def shouldSendOneWay = {
+ val actor = actorOf[OneWayTestActor].start
+ val result = actor ! "OneWay"
+ assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS))
+ actor.stop
+ }
+
+ @Test def shouldSendReplySync = {
+ val actor = actorOf[TestActor].start
+ val result = (actor !! ("Hello", 10000)).as[String]
+ assert("World" === result.get)
+ actor.stop
+ }
+
+ @Test def shouldSendReplyAsync = {
+ val actor = actorOf[TestActor].start
+ val result = actor !! "Hello"
+ assert("World" === result.get.asInstanceOf[String])
+ actor.stop
+ }
+
+ @Test def shouldSendReceiveException = {
+ val actor = actorOf[TestActor].start
+ try {
+ actor !! "Failure"
+ fail("Should have thrown an exception")
+ } catch {
+ case e =>
+ assert("Expected exception; to test fault-tolerance" === e.getMessage())
+ }
+ actor.stop
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/HawtDispatcherEchoServer.scala b/akka-actor/src/test/scala/akka/dispatch/HawtDispatcherEchoServer.scala
new file mode 100644
index 0000000000..a052ca690d
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/HawtDispatcherEchoServer.scala
@@ -0,0 +1,207 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor.dispatch
+
+import scala.collection.mutable.ListBuffer
+
+import java.util.concurrent.TimeUnit
+import java.net.InetSocketAddress
+import java.io.IOException
+import java.nio.ByteBuffer
+import java.nio.channels.{SocketChannel, SelectionKey, ServerSocketChannel}
+
+import akka.actor._
+import akka.actor.Actor._
+import akka.dispatch.HawtDispatcher
+
+import org.fusesource.hawtdispatch.DispatchSource
+import org.fusesource.hawtdispatch.ScalaDispatch._
+
+/**
+ * This is an example of how to crate an Akka actor based TCP echo server using
+ * the HawtDispatch dispatcher and NIO event sources.
+ */
+object HawtDispatcherEchoServer {
+
+ private val hawt = new HawtDispatcher
+ var port=4444;
+ var useReactorPattern=true
+
+ def main(args:Array[String]):Unit = run
+
+ def run() = {
+ val server = actorOf(new Server(port))
+ server.start
+ Scheduler.schedule(server, DisplayStats, 1, 5, TimeUnit.SECONDS)
+
+ println("Press enter to shutdown.");
+ System.in.read
+ server ! Shutdown
+ }
+
+ case object Shutdown
+ case object DisplayStats
+ case class SessionClosed(session:ActorRef)
+
+ class Server(val port: Int) extends Actor {
+
+ self.dispatcher = hawt
+
+ var channel:ServerSocketChannel = _
+ var accept_source:DispatchSource = _
+ var sessions = ListBuffer[ActorRef]()
+
+ override def preStart = {
+ channel = ServerSocketChannel.open();
+ channel.socket().bind(new InetSocketAddress(port));
+ channel.configureBlocking(false);
+
+ // Setup the accept source, it will callback to the handler methods
+ // via the actor's mailbox so you don't need to worry about
+ // synchronizing with the local variables
+ accept_source = createSource(channel, SelectionKey.OP_ACCEPT, HawtDispatcher.queue(self));
+ accept_source.setEventHandler(^{ accept });
+ accept_source.setDisposer(^{
+ channel.close();
+ println("Closed port: "+port);
+ });
+
+ accept_source.resume
+
+ println("Listening on port: "+port);
+ }
+
+
+ private def accept() = {
+ var socket = channel.accept();
+ while( socket!=null ) {
+ try {
+ socket.configureBlocking(false);
+ val session = actorOf(new Session(self, socket))
+ session.start()
+ sessions += session
+ } catch {
+ case e: Exception =>
+ socket.close
+ }
+ socket = channel.accept();
+ }
+ }
+
+ def receive = {
+ case SessionClosed(session) =>
+ sessions = sessions.filterNot( _ == session )
+ session.stop
+ case DisplayStats =>
+ sessions.foreach { session=>
+ session ! DisplayStats
+ }
+ case Shutdown =>
+ sessions.foreach { session=>
+ session.stop
+ }
+ sessions.clear
+ accept_source.release
+ self.stop
+ }
+ }
+
+ class Session(val server:ActorRef, val channel: SocketChannel) extends Actor {
+
+ self.dispatcher = hawt
+
+ val buffer = ByteBuffer.allocate(1024);
+ val remote_address = channel.socket.getRemoteSocketAddress.toString
+
+ var read_source:DispatchSource = _
+ var write_source:DispatchSource = _
+
+ var readCounter = 0L
+ var writeCounter = 0L
+ var closed = false
+
+ override def preStart = {
+
+ if(useReactorPattern) {
+ // Then we will be using the reactor pattern for handling IO:
+ // Pin this actor to a single thread. The read/write event sources will poll
+ // a Selector on the pinned thread. Since the IO events are generated on the same
+ // thread as where the Actor is pinned to, it can avoid a substantial amount
+ // thread synchronization. Plus your GC will perform better since all the IO
+ // processing is done on a single thread.
+ HawtDispatcher.pin(self)
+ } else {
+ // Then we will be using sing the proactor pattern for handling IO:
+ // Then the actor will not be pinned to a specific thread. The read/write
+ // event sources will poll a Selector and then asynchronously dispatch the
+ // event's to the actor via the thread pool.
+ }
+
+ // Setup the sources, they will callback to the handler methods
+ // via the actor's mailbox so you don't need to worry about
+ // synchronizing with the local variables
+ read_source = createSource(channel, SelectionKey.OP_READ, HawtDispatcher.queue(self));
+ read_source.setEventHandler(^{ read })
+ read_source.setCancelHandler(^{ close })
+
+ write_source = createSource(channel, SelectionKey.OP_WRITE, HawtDispatcher.queue(self));
+ write_source.setEventHandler(^{ write })
+ write_source.setCancelHandler(^{ close })
+
+ read_source.resume
+ println("Accepted connection from: "+remote_address);
+ }
+
+ override def postStop = {
+ closed = true
+ read_source.release
+ write_source.release
+ channel.close
+ }
+
+ private def catchio(func: =>Unit):Unit = {
+ try {
+ func
+ } catch {
+ case e:IOException => close
+ }
+ }
+
+ def read():Unit = catchio {
+ channel.read(buffer) match {
+ case -1 =>
+ close // peer disconnected.
+ case 0 =>
+ case count:Int =>
+ readCounter += count
+ buffer.flip;
+ read_source.suspend
+ write_source.resume
+ write()
+ }
+ }
+
+ def write() = catchio {
+ writeCounter += channel.write(buffer)
+ if (buffer.remaining == 0) {
+ buffer.clear
+ write_source.suspend
+ read_source.resume
+ }
+ }
+
+ def close() = {
+ if( !closed ) {
+ closed = true
+ server ! SessionClosed(self)
+ }
+ }
+
+ def receive = {
+ case DisplayStats =>
+ println("connection to %s reads: %,d bytes, writes: %,d".format(remote_address, readCounter, writeCounter))
+ }
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/MailboxConfigSpec.scala b/akka-actor/src/test/scala/akka/dispatch/MailboxConfigSpec.scala
new file mode 100644
index 0000000000..5dd0dfbe6d
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/MailboxConfigSpec.scala
@@ -0,0 +1,54 @@
+package akka.actor.dispatch
+
+import org.scalatest.junit.JUnitSuite
+
+import org.junit.Test
+
+import akka.actor.Actor
+import akka.util.Duration
+import akka.dispatch._
+import Actor._
+
+import java.util.concurrent.{BlockingQueue, CountDownLatch, TimeUnit}
+import java.util.concurrent.atomic.AtomicReference
+
+class MailboxTypeSpec extends JUnitSuite {
+ @Test def shouldDoNothing = assert(true)
+
+/*
+ private val unit = TimeUnit.MILLISECONDS
+
+ @Test def shouldCreateUnboundedQueue = {
+ val m = UnboundedMailbox(false)
+ assert(m.newMailbox("uuid").asInstanceOf[BlockingQueue[MessageInvocation]].remainingCapacity === Integer.MAX_VALUE)
+ }
+
+ @Test def shouldCreateBoundedQueue = {
+ val m = BoundedMailbox(blocking = false, capacity = 1)
+ assert(m.newMailbox("uuid").asInstanceOf[BlockingQueue[MessageInvocation]].remainingCapacity === 1)
+ }
+
+ @Test(expected = classOf[MessageQueueAppendFailedException]) def shouldThrowMessageQueueAppendFailedExceptionWhenTimeOutEnqueue = {
+ val m = BoundedMailbox(false, 1, Duration(1, unit))
+ val testActor = actorOf( new Actor { def receive = { case _ => }} )
+ val mbox = m.newMailbox("uuid")
+ (1 to 10000) foreach { i => mbox.enqueue(new MessageInvocation(testActor, i, None, None, None)) }
+ }
+
+
+ @Test def shouldBeAbleToDequeueUnblocking = {
+ val m = BoundedMailbox(false, 1, Duration(1, unit))
+ val mbox = m.newMailbox("uuid")
+ val latch = new CountDownLatch(1)
+ val t = new Thread { override def run = {
+ mbox.dequeue
+ latch.countDown
+ }}
+ t.start
+ val result = latch.await(5000,unit)
+ if (!result)
+ t.interrupt
+ assert(result === true)
+ }
+ */
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/ThreadBasedActorSpec.scala b/akka-actor/src/test/scala/akka/dispatch/ThreadBasedActorSpec.scala
new file mode 100644
index 0000000000..eee135ebab
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/ThreadBasedActorSpec.scala
@@ -0,0 +1,67 @@
+package akka.actor.dispatch
+
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+import akka.dispatch.Dispatchers
+import akka.actor.Actor
+import Actor._
+
+object ThreadBasedActorSpec {
+ class TestActor extends Actor {
+ self.dispatcher = Dispatchers.newThreadBasedDispatcher(self)
+
+ def receive = {
+ case "Hello" =>
+ self.reply("World")
+ case "Failure" =>
+ throw new RuntimeException("Expected exception; to test fault-tolerance")
+ }
+ }
+}
+
+class ThreadBasedActorSpec extends JUnitSuite {
+ import ThreadBasedActorSpec._
+
+ private val unit = TimeUnit.MILLISECONDS
+
+ @Test def shouldSendOneWay {
+ var oneWay = new CountDownLatch(1)
+ val actor = actorOf(new Actor {
+ self.dispatcher = Dispatchers.newThreadBasedDispatcher(self)
+ def receive = {
+ case "OneWay" => oneWay.countDown
+ }
+ }).start
+ val result = actor ! "OneWay"
+ assert(oneWay.await(1, TimeUnit.SECONDS))
+ actor.stop
+ }
+
+ @Test def shouldSendReplySync = {
+ val actor = actorOf[TestActor].start
+ val result = (actor !! ("Hello", 10000)).as[String]
+ assert("World" === result.get)
+ actor.stop
+ }
+
+ @Test def shouldSendReplyAsync = {
+ val actor = actorOf[TestActor].start
+ val result = actor !! "Hello"
+ assert("World" === result.get.asInstanceOf[String])
+ actor.stop
+ }
+
+ @Test def shouldSendReceiveException = {
+ val actor = actorOf[TestActor].start
+ try {
+ actor !! "Failure"
+ fail("Should have thrown an exception")
+ } catch {
+ case e =>
+ assert("Expected exception; to test fault-tolerance" === e.getMessage())
+ }
+ actor.stop
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/dispatch/ThreadBasedDispatcherSpec.scala b/akka-actor/src/test/scala/akka/dispatch/ThreadBasedDispatcherSpec.scala
new file mode 100644
index 0000000000..603b17e336
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/dispatch/ThreadBasedDispatcherSpec.scala
@@ -0,0 +1,91 @@
+package akka.dispatch
+
+import java.util.concurrent.CountDownLatch
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicBoolean
+import java.util.concurrent.locks.Lock
+import java.util.concurrent.locks.ReentrantLock
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.{Test, Before}
+
+import akka.actor.Actor
+import Actor._
+
+// FIXME use this test when we have removed the MessageInvoker classes
+/*
+class ThreadBasedDispatcherSpec extends JUnitSuite {
+ private var threadingIssueDetected: AtomicBoolean = null
+ val key1 = actorOf(new Actor { def receive = { case _ => {}} })
+ val key2 = actorOf(new Actor { def receive = { case _ => {}} })
+ val key3 = actorOf(new Actor { def receive = { case _ => {}} })
+
+ class TestMessageHandle(handleLatch: CountDownLatch) extends MessageInvoker {
+ val guardLock: Lock = new ReentrantLock
+
+ def invoke(message: MessageInvocation) {
+ try {
+ if (threadingIssueDetected.get) return
+ if (guardLock.tryLock) {
+ handleLatch.countDown
+ } else {
+ threadingIssueDetected.set(true)
+ }
+ } catch {
+ case e: Exception => threadingIssueDetected.set(true)
+ } finally {
+ guardLock.unlock
+ }
+ }
+ }
+
+ @Before
+ def setUp = {
+ threadingIssueDetected = new AtomicBoolean(false)
+ }
+
+ @Test
+ def shouldMessagesDispatchedToTheSameHandlerAreExecutedSequentially = {
+ internalTestMessagesDispatchedToTheSameHandlerAreExecutedSequentially
+ }
+
+ @Test
+ def shouldMessagesDispatchedToHandlersAreExecutedInFIFOOrder = {
+ internalTestMessagesDispatchedToHandlersAreExecutedInFIFOOrder
+ }
+
+ private def internalTestMessagesDispatchedToTheSameHandlerAreExecutedSequentially(): Unit = {
+ val guardLock = new ReentrantLock
+ val handleLatch = new CountDownLatch(100)
+ val dispatcher = new ThreadBasedDispatcher("name", new TestMessageHandle(handleLatch))
+ dispatcher.start
+ for (i <- 0 until 100) {
+ dispatcher.dispatch(new MessageInvocation(key1, new Object, None, None))
+ }
+ assert(handleLatch.await(5, TimeUnit.SECONDS))
+ assert(!threadingIssueDetected.get)
+ }
+
+ private def internalTestMessagesDispatchedToHandlersAreExecutedInFIFOOrder(): Unit = {
+ val handleLatch = new CountDownLatch(100)
+ val dispatcher = new ThreadBasedDispatcher("name", new MessageInvoker {
+ var currentValue = -1;
+ def invoke(message: MessageInvocation) {
+ if (threadingIssueDetected.get) return
+ val messageValue = message.message.asInstanceOf[Int]
+ if (messageValue.intValue == currentValue + 1) {
+ currentValue = messageValue.intValue
+ handleLatch.countDown
+ } else threadingIssueDetected.set(true)
+ }
+ })
+ dispatcher.start
+ for (i <- 0 until 100) {
+ dispatcher.dispatch(new MessageInvocation(key1, i, None, None))
+ }
+ assert(handleLatch.await(5, TimeUnit.SECONDS))
+ assert(!threadingIssueDetected.get)
+ dispatcher.postStop
+ }
+}
+*/
diff --git a/akka-actor/src/test/scala/akka/japi/JavaAPITest.scala b/akka-actor/src/test/scala/akka/japi/JavaAPITest.scala
new file mode 100644
index 0000000000..30aa4f4499
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/japi/JavaAPITest.scala
@@ -0,0 +1,5 @@
+package akka.japi
+
+import org.scalatest.junit.JUnitSuite
+
+class JavaAPITest extends JavaAPITestBase with JUnitSuite
diff --git a/akka-actor/src/test/scala/akka/misc/ActorRegistrySpec.scala b/akka-actor/src/test/scala/akka/misc/ActorRegistrySpec.scala
new file mode 100644
index 0000000000..6148b04f53
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/misc/ActorRegistrySpec.scala
@@ -0,0 +1,253 @@
+package akka.actor
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+import Actor._
+import java.util.concurrent.{CyclicBarrier, TimeUnit, CountDownLatch}
+
+object ActorRegistrySpec {
+ var record = ""
+ class TestActor extends Actor {
+ self.id = "MyID"
+ def receive = {
+ case "ping" =>
+ record = "pong" + record
+ self.reply("got ping")
+ }
+ }
+
+ class TestActor2 extends Actor {
+ self.id = "MyID2"
+ def receive = {
+ case "ping" =>
+ record = "pong" + record
+ self.reply("got ping")
+ case "ping2" =>
+ record = "pong" + record
+ self.reply("got ping")
+ }
+ }
+
+}
+
+class ActorRegistrySpec extends JUnitSuite {
+ import ActorRegistrySpec._
+
+ @Test def shouldGetActorByIdFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor = actorOf[TestActor]
+ actor.start
+ val actors = ActorRegistry.actorsFor("MyID")
+ assert(actors.size === 1)
+ assert(actors.head.actor.isInstanceOf[TestActor])
+ assert(actors.head.id === "MyID")
+ actor.stop
+ }
+
+ @Test def shouldGetActorByUUIDFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor = actorOf[TestActor]
+ val uuid = actor.uuid
+ actor.start
+ val actorOrNone = ActorRegistry.actorFor(uuid)
+ assert(actorOrNone.isDefined)
+ assert(actorOrNone.get.uuid === uuid)
+ actor.stop
+ }
+
+ @Test def shouldGetActorByClassFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor = actorOf[TestActor]
+ actor.start
+ val actors = ActorRegistry.actorsFor(classOf[TestActor])
+ assert(actors.size === 1)
+ assert(actors.head.actor.isInstanceOf[TestActor])
+ assert(actors.head.id === "MyID")
+ actor.stop
+ }
+
+ @Test def shouldGetActorByManifestFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor = actorOf[TestActor]
+ actor.start
+ val actors = ActorRegistry.actorsFor[TestActor]
+ assert(actors.size === 1)
+ assert(actors.head.actor.isInstanceOf[TestActor])
+ assert(actors.head.id === "MyID")
+ actor.stop
+ }
+
+ @Test def shouldFindThingsFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor = actorOf[TestActor]
+ actor.start
+ val found = ActorRegistry.find({ case a: ActorRef if a.actor.isInstanceOf[TestActor] => a })
+ assert(found.isDefined)
+ assert(found.get.actor.isInstanceOf[TestActor])
+ assert(found.get.id === "MyID")
+ actor.stop
+ }
+
+ @Test def shouldGetActorsByIdFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor1 = actorOf[TestActor]
+ actor1.start
+ val actor2 = actorOf[TestActor]
+ actor2.start
+ val actors = ActorRegistry.actorsFor("MyID")
+ assert(actors.size === 2)
+ assert(actors.head.actor.isInstanceOf[TestActor])
+ assert(actors.head.id === "MyID")
+ assert(actors.last.actor.isInstanceOf[TestActor])
+ assert(actors.last.id === "MyID")
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldGetActorsByClassFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor1 = actorOf[TestActor]
+ actor1.start
+ val actor2 = actorOf[TestActor]
+ actor2.start
+ val actors = ActorRegistry.actorsFor(classOf[TestActor])
+ assert(actors.size === 2)
+ assert(actors.head.actor.isInstanceOf[TestActor])
+ assert(actors.head.id === "MyID")
+ assert(actors.last.actor.isInstanceOf[TestActor])
+ assert(actors.last.id === "MyID")
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldGetActorsByManifestFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor1 = actorOf[TestActor]
+ actor1.start
+ val actor2 = actorOf[TestActor]
+ actor2.start
+ val actors = ActorRegistry.actorsFor[TestActor]
+ assert(actors.size === 2)
+ assert(actors.head.actor.isInstanceOf[TestActor])
+ assert(actors.head.id === "MyID")
+ assert(actors.last.actor.isInstanceOf[TestActor])
+ assert(actors.last.id === "MyID")
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldGetActorsByMessageFromActorRegistry {
+
+ ActorRegistry.shutdownAll
+ val actor1 = actorOf[TestActor]
+ actor1.start
+ val actor2 = actorOf[TestActor2]
+ actor2.start
+
+ val actorsForAcotrTestActor = ActorRegistry.actorsFor[TestActor]
+ assert(actorsForAcotrTestActor.size === 1)
+
+ val actorsForAcotrTestActor2 = ActorRegistry.actorsFor[TestActor2]
+ assert(actorsForAcotrTestActor2.size === 1)
+
+ val actorsForAcotr = ActorRegistry.actorsFor[Actor]
+ assert(actorsForAcotr.size === 2)
+
+
+ val actorsForMessagePing2 = ActorRegistry.actorsFor[Actor]("ping2")
+ assert(actorsForMessagePing2.size === 1)
+
+ val actorsForMessagePing = ActorRegistry.actorsFor[Actor]("ping")
+ assert(actorsForMessagePing.size === 2)
+
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldGetAllActorsFromActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor1 = actorOf[TestActor]
+ actor1.start
+ val actor2 = actorOf[TestActor]
+ actor2.start
+ val actors = ActorRegistry.actors
+ assert(actors.size === 2)
+ assert(actors.head.actor.isInstanceOf[TestActor])
+ assert(actors.head.id === "MyID")
+ assert(actors.last.actor.isInstanceOf[TestActor])
+ assert(actors.last.id === "MyID")
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldGetResponseByAllActorsInActorRegistryWhenInvokingForeach {
+ ActorRegistry.shutdownAll
+ val actor1 = actorOf[TestActor]
+ actor1.start
+ val actor2 = actorOf[TestActor]
+ actor2.start
+ record = ""
+ ActorRegistry.foreach(actor => actor !! "ping")
+ assert(record === "pongpong")
+ actor1.stop
+ actor2.stop
+ }
+
+ @Test def shouldShutdownAllActorsInActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor1 = actorOf[TestActor]
+ actor1.start
+ val actor2 = actorOf[TestActor]
+ actor2.start
+ ActorRegistry.shutdownAll
+ assert(ActorRegistry.actors.size === 0)
+ }
+
+ @Test def shouldRemoveUnregisterActorInActorRegistry {
+ ActorRegistry.shutdownAll
+ val actor1 = actorOf[TestActor]
+ actor1.start
+ val actor2 = actorOf[TestActor]
+ actor2.start
+ assert(ActorRegistry.actors.size === 2)
+ ActorRegistry.unregister(actor1)
+ assert(ActorRegistry.actors.size === 1)
+ ActorRegistry.unregister(actor2)
+ assert(ActorRegistry.actors.size === 0)
+ }
+
+ @Test def shouldBeAbleToRegisterActorsConcurrently {
+ ActorRegistry.shutdownAll
+
+ def mkTestActors = for(i <- (1 to 10).toList;j <- 1 to 3000) yield actorOf( new Actor {
+ self.id = i.toString
+ def receive = { case _ => }
+ })
+
+ val latch = new CountDownLatch(3)
+ val barrier = new CyclicBarrier(3)
+
+ def mkThread(actors: Iterable[ActorRef]) = new Thread {
+ this.start
+ override def run {
+ barrier.await
+ actors foreach { _.start }
+ latch.countDown
+ }
+ }
+ val a1,a2,a3 = mkTestActors
+ val t1 = mkThread(a1)
+ val t2 = mkThread(a2)
+ val t3 = mkThread(a3)
+
+
+ assert(latch.await(30,TimeUnit.SECONDS) === true)
+
+ for(i <- 1 to 10) {
+ val theId = i.toString
+ val actors = ActorRegistry.actorsFor(theId).toSet
+ for(a <- actors if a.id == theId) assert(actors contains a)
+ assert(actors.size === 9000)
+ }
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/misc/SchedulerSpec.scala b/akka-actor/src/test/scala/akka/misc/SchedulerSpec.scala
new file mode 100644
index 0000000000..a4471503da
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/misc/SchedulerSpec.scala
@@ -0,0 +1,127 @@
+package akka.actor
+
+import org.scalatest.junit.JUnitSuite
+import Actor._
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+import akka.config.Supervision._
+import org.multiverse.api.latches.StandardLatch
+import org.junit.Test
+
+class SchedulerSpec extends JUnitSuite {
+
+ def withCleanEndState(action: => Unit) {
+ action
+ Scheduler.restart
+ ActorRegistry.shutdownAll
+ }
+
+
+ @Test def schedulerShouldScheduleMoreThanOnce = withCleanEndState {
+
+ case object Tick
+ val countDownLatch = new CountDownLatch(3)
+ val tickActor = actorOf(new Actor {
+ def receive = { case Tick => countDownLatch.countDown }
+ }).start
+ // run every 50 millisec
+ Scheduler.schedule(tickActor, Tick, 0, 50, TimeUnit.MILLISECONDS)
+
+ // after max 1 second it should be executed at least the 3 times already
+ assert(countDownLatch.await(1, TimeUnit.SECONDS))
+
+ val countDownLatch2 = new CountDownLatch(3)
+
+ Scheduler.schedule( () => countDownLatch2.countDown, 0, 50, TimeUnit.MILLISECONDS)
+
+ // after max 1 second it should be executed at least the 3 times already
+ assert(countDownLatch2.await(1, TimeUnit.SECONDS))
+ }
+
+ @Test def schedulerShouldScheduleOnce = withCleanEndState {
+ case object Tick
+ val countDownLatch = new CountDownLatch(3)
+ val tickActor = actorOf(new Actor {
+ def receive = { case Tick => countDownLatch.countDown }
+ }).start
+ // run every 50 millisec
+ Scheduler.scheduleOnce(tickActor, Tick, 50, TimeUnit.MILLISECONDS)
+ Scheduler.scheduleOnce( () => countDownLatch.countDown, 50, TimeUnit.MILLISECONDS)
+
+ // after 1 second the wait should fail
+ assert(countDownLatch.await(1, TimeUnit.SECONDS) == false)
+ // should still be 1 left
+ assert(countDownLatch.getCount == 1)
+ }
+
+ /**
+ * ticket #372
+ */
+ @Test def schedulerShouldntCreateActors = withCleanEndState {
+ object Ping
+ val ticks = new CountDownLatch(1000)
+ val actor = actorOf(new Actor {
+ def receive = { case Ping => ticks.countDown }
+ }).start
+ val numActors = ActorRegistry.actors.length
+ (1 to 1000).foreach( _ => Scheduler.scheduleOnce(actor,Ping,1,TimeUnit.MILLISECONDS) )
+ assert(ticks.await(10,TimeUnit.SECONDS))
+ assert(ActorRegistry.actors.length === numActors)
+ }
+
+ /**
+ * ticket #372
+ */
+ @Test def schedulerShouldBeCancellable = withCleanEndState {
+ object Ping
+ val ticks = new CountDownLatch(1)
+
+ val actor = actorOf(new Actor {
+ def receive = { case Ping => ticks.countDown }
+ }).start
+
+ (1 to 10).foreach { i =>
+ val future = Scheduler.scheduleOnce(actor,Ping,1,TimeUnit.SECONDS)
+ future.cancel(true)
+ }
+ assert(ticks.await(3,TimeUnit.SECONDS) == false) //No counting down should've been made
+ }
+
+ /**
+ * ticket #307
+ */
+ @Test def actorRestartShouldPickUpScheduleAgain = withCleanEndState {
+
+ object Ping
+ object Crash
+
+ val restartLatch = new StandardLatch
+ val pingLatch = new CountDownLatch(6)
+
+ val actor = actorOf(new Actor {
+ self.lifeCycle = Permanent
+
+ def receive = {
+ case Ping => pingLatch.countDown
+ case Crash => throw new Exception("CRASH")
+ }
+
+ override def postRestart(reason: Throwable) = restartLatch.open
+ })
+
+ Supervisor(
+ SupervisorConfig(
+ AllForOneStrategy(List(classOf[Exception]), 3, 1000),
+ Supervise(
+ actor,
+ Permanent)
+ :: Nil)).start
+
+ Scheduler.schedule(actor, Ping, 500, 500, TimeUnit.MILLISECONDS)
+ // appx 2 pings before crash
+ Scheduler.scheduleOnce(actor, Crash, 1000, TimeUnit.MILLISECONDS)
+
+ assert(restartLatch.tryAwait(2, TimeUnit.SECONDS))
+ // should be enough time for the ping countdown to recover and reach 6 pings
+ assert(pingLatch.await(4, TimeUnit.SECONDS))
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/routing/RoutingSpec.scala b/akka-actor/src/test/scala/akka/routing/RoutingSpec.scala
new file mode 100644
index 0000000000..0f46b2340c
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/routing/RoutingSpec.scala
@@ -0,0 +1,183 @@
+package akka.actor.routing
+
+import akka.actor.Actor
+import akka.actor.Actor._
+import akka.util.Logging
+
+import org.scalatest.Suite
+import org.junit.runner.RunWith
+import org.scalatest.junit.JUnitRunner
+import org.scalatest.matchers.MustMatchers
+import org.junit.Test
+
+import java.util.concurrent.atomic.AtomicInteger
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+import akka.routing._
+
+@RunWith(classOf[JUnitRunner])
+class RoutingSpec extends junit.framework.TestCase with Suite with MustMatchers with Logging {
+ import Routing._
+
+ @Test def testDispatcher = {
+ val (testMsg1,testMsg2,testMsg3,testMsg4) = ("test1","test2","test3","test4")
+ val targetOk = new AtomicInteger(0)
+ val t1 = actorOf( new Actor() {
+ def receive = {
+ case `testMsg1` => self.reply(3)
+ case `testMsg2` => self.reply(7)
+ }
+ } ).start
+
+ val t2 = actorOf( new Actor() {
+ def receive = {
+ case `testMsg3` => self.reply(11)
+ }
+ }).start
+
+ val d = dispatcherActor {
+ case `testMsg1`|`testMsg2` => t1
+ case `testMsg3` => t2
+ }.start
+
+ val result = for {
+ a <- (d !! (testMsg1, 5000)).as[Int]
+ b <- (d !! (testMsg2, 5000)).as[Int]
+ c <- (d !! (testMsg3, 5000)).as[Int]
+ } yield a + b + c
+
+ result.isDefined must be (true)
+ result.get must be(21)
+
+ for(a <- List(t1,t2,d)) a.stop
+ }
+
+ @Test def testLogger = {
+ val msgs = new java.util.concurrent.ConcurrentSkipListSet[Any]
+ val latch = new CountDownLatch(2)
+ val t1 = actorOf(new Actor { def receive = { case _ => } }).start
+ val l = loggerActor(t1,(x) => { msgs.add(x); latch.countDown }).start
+ val foo : Any = "foo"
+ val bar : Any = "bar"
+ l ! foo
+ l ! bar
+ val done = latch.await(5,TimeUnit.SECONDS)
+ done must be (true)
+ msgs must ( have size (2) and contain (foo) and contain (bar) )
+ t1.stop
+ l.stop
+ }
+
+ @Test def testSmallestMailboxFirstDispatcher = {
+ val t1ProcessedCount = new AtomicInteger(0)
+ val latch = new CountDownLatch(500)
+ val t1 = actorOf(new Actor {
+ def receive = {
+ case x =>
+ Thread.sleep(50) // slow actor
+ t1ProcessedCount.incrementAndGet
+ latch.countDown
+ }
+ }).start
+
+ val t2ProcessedCount = new AtomicInteger(0)
+ val t2 = actorOf(new Actor {
+ def receive = {
+ case x => t2ProcessedCount.incrementAndGet
+ latch.countDown
+ }
+ }).start
+ val d = loadBalancerActor(new SmallestMailboxFirstIterator(t1 :: t2 :: Nil))
+ for (i <- 1 to 500) d ! i
+ val done = latch.await(10,TimeUnit.SECONDS)
+ done must be (true)
+ t1ProcessedCount.get must be < (t2ProcessedCount.get) // because t1 is much slower and thus has a bigger mailbox all the time
+ for(a <- List(t1,t2,d)) a.stop
+ }
+
+ @Test def testListener = {
+ val latch = new CountDownLatch(2)
+ val foreachListener = new CountDownLatch(2)
+ val num = new AtomicInteger(0)
+ val i = actorOf(new Actor with Listeners {
+ def receive = listenerManagement orElse {
+ case "foo" => gossip("bar")
+ }
+ })
+ i.start
+
+ def newListener = actorOf(new Actor {
+ def receive = {
+ case "bar" =>
+ num.incrementAndGet
+ latch.countDown
+ case "foo" => foreachListener.countDown
+ }
+ }).start
+
+ val a1 = newListener
+ val a2 = newListener
+ val a3 = newListener
+
+ i ! Listen(a1)
+ i ! Listen(a2)
+ i ! Listen(a3)
+ i ! Deafen(a3)
+ i ! WithListeners(_ ! "foo")
+ i ! "foo"
+
+ val done = latch.await(5,TimeUnit.SECONDS)
+ done must be (true)
+ num.get must be (2)
+ val withListeners = foreachListener.await(5,TimeUnit.SECONDS)
+ withListeners must be (true)
+ for(a <- List(i,a1,a2,a3)) a.stop
+ }
+
+ @Test def testIsDefinedAt = {
+ import akka.actor.ActorRef
+
+ val (testMsg1,testMsg2,testMsg3,testMsg4) = ("test1","test2","test3","test4")
+
+ val t1 = actorOf( new Actor() {
+ def receive = {
+ case `testMsg1` => self.reply(3)
+ case `testMsg2` => self.reply(7)
+ }
+ } ).start
+
+ val t2 = actorOf( new Actor() {
+ def receive = {
+ case `testMsg1` => self.reply(3)
+ case `testMsg2` => self.reply(7)
+ }
+ } ).start
+
+ val t3 = actorOf( new Actor() {
+ def receive = {
+ case `testMsg1` => self.reply(3)
+ case `testMsg2` => self.reply(7)
+ }
+ } ).start
+
+ val t4 = actorOf( new Actor() {
+ def receive = {
+ case `testMsg1` => self.reply(3)
+ case `testMsg2` => self.reply(7)
+ }
+ } ).start
+
+ val d1 = loadBalancerActor(new SmallestMailboxFirstIterator(t1 :: t2 :: Nil))
+ val d2 = loadBalancerActor(new CyclicIterator[ActorRef](t3 :: t4 :: Nil))
+
+ t1.isDefinedAt(testMsg1) must be (true)
+ t1.isDefinedAt(testMsg3) must be (false)
+ t2.isDefinedAt(testMsg1) must be (true)
+ t2.isDefinedAt(testMsg3) must be (false)
+ d1.isDefinedAt(testMsg1) must be (true)
+ d1.isDefinedAt(testMsg3) must be (false)
+ d2.isDefinedAt(testMsg1) must be (true)
+ d2.isDefinedAt(testMsg3) must be (false)
+
+ for(a <- List(t1,t2,d1,d2)) a.stop
+ }
+}
diff --git a/akka-actor/src/test/scala/akka/ticket/Ticket001Spec.scala b/akka-actor/src/test/scala/akka/ticket/Ticket001Spec.scala
new file mode 100644
index 0000000000..d4de2675fb
--- /dev/null
+++ b/akka-actor/src/test/scala/akka/ticket/Ticket001Spec.scala
@@ -0,0 +1,13 @@
+package akka.actor.ticket
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+
+class Ticket001Spec extends WordSpec with MustMatchers {
+
+ "An XXX" should {
+ "do YYY" in {
+ 1 must be (1)
+ }
+ }
+}
diff --git a/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerPrivateQueueTestIntegration.scala b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerPrivateQueueTestIntegration.scala
new file mode 100644
index 0000000000..6b03b6ded8
--- /dev/null
+++ b/akka-amqp/src/test/scala/akka/amqp/test/AMQPConsumerPrivateQueueTestIntegration.scala
@@ -0,0 +1,45 @@
+package akka.amqp.test
+
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+import akka.amqp._
+import org.multiverse.api.latches.StandardLatch
+import akka.actor.Actor._
+import org.scalatest.matchers.MustMatchers
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+import akka.amqp.AMQP.{ConsumerParameters, ChannelParameters, ProducerParameters}
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+import akka.actor.Actor
+
+class AMQPConsumerPrivateQueueTestIntegration extends JUnitSuite with MustMatchers {
+
+ @Test
+ def consumerMessage = AMQPTest.withCleanEndState {
+ val connection = AMQP.newConnection()
+ val countDown = new CountDownLatch(2)
+ val channelCallback = actorOf(new Actor {
+ def receive = {
+ case Started => countDown.countDown
+ case Restarting => ()
+ case Stopped => ()
+ }
+ }).start
+
+ val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
+
+ val payloadLatch = new StandardLatch
+ val consumer = AMQP.newConsumer(connection, ConsumerParameters("my.private.routing.key", actorOf(new Actor {
+ def receive = { case Delivery(payload, _, _, _, _, _) => payloadLatch.open }
+ }), channelParameters = Some(channelParameters)))
+
+ val producer = AMQP.newProducer(connection,
+ ProducerParameters(channelParameters = Some(channelParameters)))
+
+ countDown.await(2, TimeUnit.SECONDS) must be (true)
+ producer ! Message("some_payload".getBytes, "my.private.routing.key")
+ payloadLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
+ }
+}
\ No newline at end of file
diff --git a/akka-camel/src/main/scala/akka/CamelContextLifecycle.scala b/akka-camel/src/main/scala/akka/CamelContextLifecycle.scala
new file mode 100644
index 0000000000..93375131d2
--- /dev/null
+++ b/akka-camel/src/main/scala/akka/CamelContextLifecycle.scala
@@ -0,0 +1,202 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.camel
+
+import java.util.Map
+
+import org.apache.camel.{ProducerTemplate, CamelContext}
+import org.apache.camel.impl.DefaultCamelContext
+
+import akka.camel.component.TypedActorComponent
+import akka.japi.{Option => JOption}
+import akka.util.Logging
+
+/**
+ * Manages the lifecycle of a CamelContext. Allowed transitions are
+ * init -> start -> stop -> init -> ... etc.
+ *
+ * @author Martin Krasser
+ */
+trait CamelContextLifecycle extends Logging {
+ // TODO: enforce correct state transitions
+ // valid: init -> start -> stop -> init ...
+
+ private var _context: Option[CamelContext] = None
+ private var _template: Option[ProducerTemplate] = None
+
+ private var _initialized = false
+ private var _started = false
+
+ /**
+ * Camel component for accessing typed actors.
+ */
+ private[camel] var typedActorComponent: TypedActorComponent = _
+
+ /**
+ * Registry in which typed actors are TEMPORARILY registered during
+ * creation of Camel routes to these actors.
+ */
+ private[camel] var typedActorRegistry: Map[String, AnyRef] = _
+
+ /**
+ * Returns Some(CamelContext) (containing the current CamelContext)
+ * if CamelContextLifecycle has been initialized, otherwise None.
+ */
+ def context: Option[CamelContext] = _context
+
+ /**
+ * Returns Some(ProducerTemplate) (containing the current ProducerTemplate)
+ * if CamelContextLifecycle has been initialized, otherwise None.
+ */
+ def template: Option[ProducerTemplate] = _template
+
+ /**
+ * Returns Some(CamelContext) (containing the current CamelContext)
+ * if CamelContextLifecycle has been initialized, otherwise None.
+ *
+ * Java API.
+ */
+ def getContext: JOption[CamelContext] = context
+
+ /**
+ * Returns Some(ProducerTemplate) (containing the current ProducerTemplate)
+ * if CamelContextLifecycle has been initialized, otherwise None.
+ *
+ * Java API.
+ */
+ def getTemplate: JOption[ProducerTemplate] = template
+
+ /**
+ * Returns the current CamelContext if this CamelContextLifecycle
+ * has been initialized, otherwise throws an IllegalStateException.
+ */
+ def mandatoryContext =
+ if (context.isDefined) context.get
+ else throw new IllegalStateException("no current CamelContext")
+
+ /**
+ * Returns the current ProducerTemplate if this CamelContextLifecycle
+ * has been initialized, otherwise throws an IllegalStateException.
+ */
+ def mandatoryTemplate =
+ if (template.isDefined) template.get
+ else throw new IllegalStateException("no current ProducerTemplate")
+
+ /**
+ * Returns the current CamelContext if this CamelContextLifecycle
+ * has been initialized, otherwise throws an IllegalStateException.
+ *
+ * Java API.
+ */
+ def getMandatoryContext = mandatoryContext
+
+ /**
+ * Returns the current ProducerTemplate if this CamelContextLifecycle
+ * has been initialized, otherwise throws an IllegalStateException.
+ *
+ * Java API.
+ */
+ def getMandatoryTemplate = mandatoryTemplate
+
+ def initialized = _initialized
+ def started = _started
+
+ /**
+ * Starts the CamelContext and an associated ProducerTemplate.
+ */
+ def start = {
+ for {
+ c <- context
+ t <- template
+ } {
+ c.start
+ t.start
+ _started = true
+ log.info("Camel context started")
+ }
+ }
+
+ /**
+ * Stops the CamelContext and the associated ProducerTemplate.
+ */
+ def stop = {
+ for {
+ t <- template
+ c <- context
+ } {
+ t.stop
+ c.stop
+ _started = false
+ _initialized = false
+ log.info("Camel context stopped")
+ }
+ }
+
+ /**
+ * Initializes this lifecycle object with the a DefaultCamelContext.
+ */
+ def init(): Unit = init(new DefaultCamelContext)
+
+ /**
+ * Initializes this lifecycle object with the given CamelContext. For the passed
+ * CamelContext, stream-caching is enabled. If applications want to disable stream-
+ * caching they can do so after this method returned and prior to calling start.
+ * This method also registers a new TypedActorComponent at the passes CamelContext
+ * under a name defined by TypedActorComponent.InternalSchema.
+ */
+ def init(context: CamelContext) {
+ this.typedActorComponent = new TypedActorComponent
+ this.typedActorRegistry = typedActorComponent.typedActorRegistry
+
+ context.setStreamCaching(true)
+ context.addComponent(TypedActorComponent.InternalSchema, typedActorComponent)
+
+ this._context = Some(context)
+ this._template = Some(context.createProducerTemplate)
+
+ _initialized = true
+ log.info("Camel context initialized")
+ }
+}
+
+/**
+ * Manages a global CamelContext and an associated ProducerTemplate.
+ */
+object CamelContextManager extends CamelContextLifecycle {
+
+ // -----------------------------------------------------
+ // The inherited getters aren't statically accessible
+ // from Java. Therefore, they are redefined here.
+ // TODO: investigate if this is a Scala bug.
+ // -----------------------------------------------------
+
+ /**
+ * see CamelContextLifecycle.getContext
+ *
+ * Java API.
+ */
+ override def getMandatoryTemplate = super.getMandatoryTemplate
+}
diff --git a/akka-camel/src/main/scala/akka/CamelService.scala b/akka-camel/src/main/scala/akka/CamelService.scala
new file mode 100644
index 0000000000..b546636610
--- /dev/null
+++ b/akka-camel/src/main/scala/akka/CamelService.scala
@@ -0,0 +1,275 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.camel
+
+import java.util.concurrent.CountDownLatch
+import java.util.concurrent.TimeUnit
+
+import org.apache.camel.CamelContext
+
+import akka.actor.Actor._
+import akka.actor.{AspectInitRegistry, ActorRegistry}
+import akka.config.Config._
+import akka.japi.{SideEffect, Option => JOption}
+import akka.util.{Logging, Bootable}
+
+/**
+ * Publishes (untyped) consumer actors and typed consumer actors via Camel endpoints. Actors
+ * are published (asynchronously) when they are started and unpublished (asynchronously) when
+ * they are stopped. The CamelService is notified about actor start- and stop-events by
+ * registering listeners at ActorRegistry and AspectInitRegistry.
+ *
+ * @author Martin Krasser
+ */
+trait CamelService extends Bootable with Logging {
+ private[camel] val consumerPublisher = actorOf[ConsumerPublisher]
+ private[camel] val publishRequestor = actorOf[PublishRequestor]
+
+ private val serviceEnabled = config.getBool("akka.camel.service", true)
+
+ /**
+ * Starts this CamelService unless akka.camel.service is set to false.
+ */
+ abstract override def onLoad = {
+ if (serviceEnabled) registerPublishRequestor
+ super.onLoad
+ if (serviceEnabled) start
+ }
+
+ /**
+ * Stops this CamelService unless akka.camel.service is set to false.
+ */
+ abstract override def onUnload = {
+ if (serviceEnabled) stop
+ super.onUnload
+ }
+
+ @deprecated("use start() instead")
+ def load = start
+
+ @deprecated("use stop() instead")
+ def unload = stop
+
+ /**
+ * Starts this 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 typed actors that have been created
+ * with TypedActor.newInstance(..) (and TypedActor.newRemoteInstance(..)
+ * on a remote node).
+ */
+ def start: CamelService = {
+ if (!publishRequestorRegistered) registerPublishRequestor
+
+ // Only init and start if not already done by application
+ if (!CamelContextManager.initialized) CamelContextManager.init
+ if (!CamelContextManager.started) CamelContextManager.start
+
+ // 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 and return it
+ CamelServiceManager.register(this)
+ CamelServiceManager.mandatoryService
+ }
+
+ /**
+ * Stops this CamelService. All published consumer actors and typed consumer actor methods will be
+ * unpublished asynchronously.
+ */
+ def stop = {
+ // Unregister this instance as current CamelService
+ CamelServiceManager.unregister(this)
+
+ // Remove related listeners from registry
+ unregisterPublishRequestor
+
+ // Stop related services
+ consumerPublisher.stop
+ CamelContextManager.stop
+ }
+
+ /**
+ * Waits for an expected number (count) of endpoints to be activated
+ * during execution of f. The wait-timeout is by default 10 seconds.
+ * Other timeout values can be set via the timeout and timeUnit
+ * parameters.
+ */
+ def awaitEndpointActivation(count: Int, timeout: Long = 10, timeUnit: TimeUnit = TimeUnit.SECONDS)(f: => Unit): Boolean = {
+ val activation = expectEndpointActivationCount(count)
+ f; activation.await(timeout, timeUnit)
+ }
+
+ /**
+ * Waits for an expected number (count) of endpoints to be de-activated
+ * during execution of f. The wait-timeout is by default 10 seconds.
+ * Other timeout values can be set via the timeout and timeUnit
+ * parameters.
+ */
+ def awaitEndpointDeactivation(count: Int, timeout: Long = 10, timeUnit: TimeUnit = TimeUnit.SECONDS)(f: => Unit): Boolean = {
+ val activation = expectEndpointDeactivationCount(count)
+ f; activation.await(timeout, timeUnit)
+ }
+
+ /**
+ * Waits for an expected number (count) of endpoints to be activated
+ * during execution of p. The wait timeout is 10 seconds.
+ *
+ * Java API
+ */
+ def awaitEndpointActivation(count: Int, p: SideEffect): Boolean = {
+ awaitEndpointActivation(count, 10, TimeUnit.SECONDS, p)
+ }
+
+ /**
+ * Waits for an expected number (count) of endpoints to be activated
+ * during execution of p. Timeout values can be set via the
+ * timeout and timeUnit parameters.
+ *
+ * Java API
+ */
+ def awaitEndpointActivation(count: Int, timeout: Long, timeUnit: TimeUnit, p: SideEffect): Boolean = {
+ awaitEndpointActivation(count, timeout, timeUnit) { p.apply }
+ }
+
+ /**
+ * Waits for an expected number (count) of endpoints to be de-activated
+ * during execution of p. The wait timeout is 10 seconds.
+ *
+ * Java API
+ */
+ def awaitEndpointDeactivation(count: Int, p: SideEffect): Boolean = {
+ awaitEndpointDeactivation(count, 10, TimeUnit.SECONDS, p)
+ }
+
+ /**
+ * Waits for an expected number (count) of endpoints to be de-activated
+ * during execution of p. Timeout values can be set via the
+ * timeout and timeUnit parameters.
+ *
+ * Java API
+ */
+ def awaitEndpointDeactivation(count: Int, timeout: Long, timeUnit: TimeUnit, p: SideEffect): Boolean = {
+ awaitEndpointDeactivation(count, timeout, timeUnit) { p.apply }
+ }
+
+ /**
+ * Sets an expectation on the number of upcoming endpoint activations and returns
+ * a CountDownLatch that can be used to wait for the activations to occur. Endpoint
+ * activations that occurred in the past are not considered.
+ */
+ private def expectEndpointActivationCount(count: Int): CountDownLatch =
+ (consumerPublisher !! SetExpectedRegistrationCount(count)).as[CountDownLatch].get
+
+ /**
+ * Sets an expectation on the number of upcoming endpoint de-activations and returns
+ * a CountDownLatch that can be used to wait for the de-activations to occur. Endpoint
+ * de-activations that occurred in the past are not considered.
+ */
+ private def expectEndpointDeactivationCount(count: Int): CountDownLatch =
+ (consumerPublisher !! SetExpectedUnregistrationCount(count)).as[CountDownLatch].get
+
+ private[camel] def publishRequestorRegistered: Boolean = {
+ ActorRegistry.hasListener(publishRequestor) ||
+ AspectInitRegistry.hasListener(publishRequestor)
+ }
+
+ private[camel] def registerPublishRequestor: Unit = {
+ ActorRegistry.addListener(publishRequestor)
+ AspectInitRegistry.addListener(publishRequestor)
+ }
+
+ private[camel] def unregisterPublishRequestor: Unit = {
+ ActorRegistry.removeListener(publishRequestor)
+ AspectInitRegistry.removeListener(publishRequestor)
+ }
+}
+
+/**
+ * Manages a global CamelService (the 'current' CamelService).
+ *
+ * @author Martin Krasser
+ */
+object CamelServiceManager {
+
+ /**
+ * The current (optional) CamelService. Is defined when a CamelService has been started.
+ */
+ private var _current: Option[CamelService] = None
+
+ /**
+ * Starts a new CamelService and makes it the current CamelService.
+ *
+ * @see CamelService#start
+ * @see CamelService#onLoad
+ */
+ def startCamelService = CamelServiceFactory.createCamelService.start
+
+ /**
+ * Stops the current CamelService.
+ *
+ * @see CamelService#stop
+ * @see CamelService#onUnload
+ */
+ def stopCamelService = for (s <- service) s.stop
+
+ /**
+ * Returns Some(CamelService) if this CamelService
+ * has been started, None otherwise.
+ */
+ def service = _current
+
+ /**
+ * Returns the current CamelService if CamelService
+ * has been started, otherwise throws an IllegalStateException.
+ *
+ * Java API
+ */
+ def getService: JOption[CamelService] = CamelServiceManager.service
+
+ /**
+ * Returns Some(CamelService) (containing the current CamelService)
+ * if this CamelServicehas been started, None otherwise.
+ */
+ def mandatoryService =
+ if (_current.isDefined) _current.get
+ else throw new IllegalStateException("co current CamelService")
+
+ /**
+ * Returns Some(CamelService) (containing the current CamelService)
+ * if this CamelServicehas been started, None otherwise.
+ *
+ * Java API
+ */
+ def getMandatoryService = mandatoryService
+
+ 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")
+}
+
+/**
+ * @author Martin Krasser
+ */
+object CamelServiceFactory {
+ /**
+ * Creates a new CamelService instance.
+ */
+ def createCamelService: CamelService = new CamelService { }
+
+ /**
+ * Creates a new CamelService instance and initializes it with the given CamelContext.
+ */
+ def createCamelService(camelContext: CamelContext): CamelService = {
+ CamelContextManager.init(camelContext)
+ createCamelService
+ }
+}
diff --git a/akka-camel/src/main/scala/akka/Consumer.scala b/akka-camel/src/main/scala/akka/Consumer.scala
new file mode 100644
index 0000000000..a6323c3bae
--- /dev/null
+++ b/akka-camel/src/main/scala/akka/Consumer.scala
@@ -0,0 +1,145 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.camel
+
+import java.net.InetSocketAddress
+
+import org.apache.camel.{Exchange, Processor}
+import org.apache.camel.model.{RouteDefinition, ProcessorDefinition}
+
+import akka.actor._
+import akka.japi.{Function => JFunction}
+
+/**
+ * Mixed in by Actor implementations that consume message from Camel endpoints.
+ *
+ * @author Martin Krasser
+ */
+trait Consumer { self: Actor =>
+ import RouteDefinitionHandler._
+
+ /**
+ * The default route definition handler is the identity function
+ */
+ private[camel] var routeDefinitionHandler: RouteDefinitionHandler = identity
+
+ /**
+ * Returns the Camel endpoint URI to consume messages from.
+ */
+ def endpointUri: String
+
+ /**
+ * Determines whether two-way communications between an endpoint and this consumer actor
+ * should be done in blocking or non-blocking mode (default is non-blocking). This method
+ * doesn't have any effect on one-way communications (they'll never block).
+ */
+ def blocking = false
+
+ /**
+ * Sets the route definition handler for creating a custom route to this consumer instance.
+ */
+ def onRouteDefinition(h: RouteDefinition => ProcessorDefinition[_]): Unit = onRouteDefinition(from(h))
+
+ /**
+ * Sets the route definition handler for creating a custom route to this consumer instance.
+ *
+ * Java API.
+ */
+ def onRouteDefinition(h: RouteDefinitionHandler): Unit = routeDefinitionHandler = h
+}
+
+/**
+ * Java-friendly Consumer.
+ *
+ * @see UntypedConsumerActor
+ * @see RemoteUntypedConsumerActor
+ *
+ * @author Martin Krasser
+ */
+trait UntypedConsumer extends Consumer { self: UntypedActor =>
+ final override def endpointUri = getEndpointUri
+ final override def blocking = isBlocking
+
+ /**
+ * Returns the Camel endpoint URI to consume messages from.
+ */
+ def getEndpointUri(): String
+
+ /**
+ * Determines whether two-way communications between an endpoint and this consumer actor
+ * should be done in blocking or non-blocking mode (default is non-blocking). This method
+ * doesn't have any effect on one-way communications (they'll never block).
+ */
+ def isBlocking() = super.blocking
+}
+
+/**
+ * Subclass this abstract class to create an MDB-style untyped consumer actor. This
+ * class is meant to be used from Java.
+ */
+abstract class UntypedConsumerActor extends UntypedActor with UntypedConsumer
+
+/**
+ * Subclass this abstract class to create an MDB-style remote untyped consumer
+ * actor. This class is meant to be used from Java.
+ */
+abstract class RemoteUntypedConsumerActor(address: InetSocketAddress) extends RemoteUntypedActor(address) with UntypedConsumer {
+ def this(host: String, port: Int) = this(new InetSocketAddress(host, port))
+}
+
+/**
+ * A callback handler for route definitions to consumer actors.
+ *
+ * @author Martin Krasser
+ */
+trait RouteDefinitionHandler {
+ def onRouteDefinition(rd: RouteDefinition): ProcessorDefinition[_]
+}
+
+/**
+ * The identity route definition handler.
+ *
+ * @author Martin Krasser
+ *
+ */
+class RouteDefinitionIdentity extends RouteDefinitionHandler {
+ def onRouteDefinition(rd: RouteDefinition) = rd
+}
+
+/**
+ * @author Martin Krasser
+ */
+object RouteDefinitionHandler {
+ /**
+ * Returns the identity route definition handler
+ */
+ val identity = new RouteDefinitionIdentity
+
+ /**
+ * Created a route definition handler from the given function.
+ */
+ def from(f: RouteDefinition => ProcessorDefinition[_]) = new RouteDefinitionHandler {
+ def onRouteDefinition(rd: RouteDefinition) = f(rd)
+ }
+}
+
+/**
+ * @author Martin Krasser
+ */
+private[camel] object Consumer {
+ /**
+ * Applies a function f to actorRef if actorRef
+ * references a consumer actor. A valid reference to a consumer actor is a local actor
+ * reference with a target actor that implements the Consumer trait. The
+ * target Consumer object is passed as argument to f. This
+ * method returns None if actorRef is not a valid reference
+ * to a consumer actor, Some consumer actor otherwise.
+ */
+ def forConsumer[T](actorRef: ActorRef)(f: Consumer => T): Option[T] = {
+ if (!actorRef.actor.isInstanceOf[Consumer]) None
+ else if (actorRef.remoteAddress.isDefined) None
+ else Some(f(actorRef.actor.asInstanceOf[Consumer]))
+ }
+}
diff --git a/akka-camel/src/main/scala/akka/ConsumerPublisher.scala b/akka-camel/src/main/scala/akka/ConsumerPublisher.scala
new file mode 100644
index 0000000000..39c4e0bb2f
--- /dev/null
+++ b/akka-camel/src/main/scala/akka/ConsumerPublisher.scala
@@ -0,0 +1,351 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.camel
+
+import collection.mutable.ListBuffer
+
+import java.io.InputStream
+import java.lang.reflect.Method
+import java.util.concurrent.CountDownLatch
+
+import org.apache.camel.builder.RouteBuilder
+import org.apache.camel.model.{ProcessorDefinition, RouteDefinition}
+
+import akka.actor._
+import akka.camel.component.TypedActorComponent
+import akka.util.Logging
+
+/**
+ * @author Martin Krasser
+ */
+private[camel] object ConsumerPublisher extends Logging {
+ /**
+ * Creates a route to the registered consumer actor.
+ */
+ def handleConsumerActorRegistered(event: ConsumerActorRegistered) {
+ CamelContextManager.mandatoryContext.addRoutes(new ConsumerActorRouteBuilder(event))
+ log.info("published actor %s at endpoint %s" format (event.actorRef, event.endpointUri))
+ }
+
+ /**
+ * Stops the route to the already un-registered consumer actor.
+ */
+ def handleConsumerActorUnregistered(event: ConsumerActorUnregistered) {
+ CamelContextManager.mandatoryContext.stopRoute(event.uuid)
+ log.info("unpublished actor %s from endpoint %s" format (event.actorRef, event.endpointUri))
+ }
+
+ /**
+ * Creates a route to an typed actor method.
+ */
+ def handleConsumerMethodRegistered(event: ConsumerMethodRegistered) {
+ CamelContextManager.typedActorRegistry.put(event.methodUuid, event.typedActor)
+ CamelContextManager.mandatoryContext.addRoutes(new ConsumerMethodRouteBuilder(event))
+ log.info("published method %s of %s at endpoint %s" format (event.methodName, event.typedActor, event.endpointUri))
+ }
+
+ /**
+ * Stops the route to the already un-registered consumer actor method.
+ */
+ def handleConsumerMethodUnregistered(event: ConsumerMethodUnregistered) {
+ CamelContextManager.typedActorRegistry.remove(event.methodUuid)
+ CamelContextManager.mandatoryContext.stopRoute(event.methodUuid)
+ log.info("unpublished method %s of %s from endpoint %s" format (event.methodName, event.typedActor, event.endpointUri))
+ }
+}
+
+/**
+ * Actor that publishes consumer actors and typed actor methods at Camel endpoints.
+ * The Camel context used for publishing is obtained via CamelContextManager.context.
+ * This actor accepts messages of type
+ * akka.camel.ConsumerActorRegistered,
+ * akka.camel.ConsumerActorUnregistered,
+ * akka.camel.ConsumerMethodRegistered and
+ * akka.camel.ConsumerMethodUnregistered.
+ *
+ * @author Martin Krasser
+ */
+private[camel] class ConsumerPublisher extends Actor {
+ import ConsumerPublisher._
+
+ @volatile private var registrationLatch = new CountDownLatch(0)
+ @volatile private var unregistrationLatch = new CountDownLatch(0)
+
+ protected def receive = {
+ case r: ConsumerActorRegistered => {
+ handleConsumerActorRegistered(r)
+ registrationLatch.countDown
+ }
+ case u: ConsumerActorUnregistered => {
+ handleConsumerActorUnregistered(u)
+ unregistrationLatch.countDown
+ }
+ case mr: ConsumerMethodRegistered => {
+ handleConsumerMethodRegistered(mr)
+ registrationLatch.countDown
+ }
+ case mu: ConsumerMethodUnregistered => {
+ handleConsumerMethodUnregistered(mu)
+ unregistrationLatch.countDown
+ }
+ case SetExpectedRegistrationCount(num) => {
+ registrationLatch = new CountDownLatch(num)
+ self.reply(registrationLatch)
+ }
+ case SetExpectedUnregistrationCount(num) => {
+ unregistrationLatch = new CountDownLatch(num)
+ self.reply(unregistrationLatch)
+ }
+ case _ => { /* ignore */}
+ }
+}
+
+private[camel] case class SetExpectedRegistrationCount(num: Int)
+private[camel] case class SetExpectedUnregistrationCount(num: Int)
+
+/**
+ * Abstract route to a target which is either an actor or an typed actor method.
+ *
+ * @param endpointUri endpoint URI of the consumer actor or typed actor method.
+ * @param id actor identifier or typed actor identifier (registry key).
+ *
+ * @author Martin Krasser
+ */
+private[camel] abstract class ConsumerRouteBuilder(endpointUri: String, id: String) extends RouteBuilder {
+ // TODO: make conversions configurable
+ private val bodyConversions = Map(
+ "file" -> classOf[InputStream]
+ )
+
+ def configure = {
+ val schema = endpointUri take endpointUri.indexOf(":") // e.g. "http" from "http://whatever/..."
+ val cnvopt = bodyConversions.get(schema)
+
+ onRouteDefinition(startRouteDefinition(cnvopt)).to(targetUri)
+ }
+
+ protected def routeDefinitionHandler: RouteDefinitionHandler
+ protected def targetUri: String
+
+ private def onRouteDefinition(rd: RouteDefinition) = routeDefinitionHandler.onRouteDefinition(rd)
+ private def startRouteDefinition(bodyConversion: Option[Class[_]]): RouteDefinition = bodyConversion match {
+ case Some(clazz) => from(endpointUri).routeId(id).convertBodyTo(clazz)
+ case None => from(endpointUri).routeId(id)
+ }
+}
+
+/**
+ * Defines the route to a (untyped) consumer actor.
+ *
+ * @author Martin Krasser
+ */
+private[camel] class ConsumerActorRouteBuilder(event: ConsumerActorRegistered) extends ConsumerRouteBuilder(event.endpointUri, event.uuid) {
+ protected def routeDefinitionHandler: RouteDefinitionHandler = event.routeDefinitionHandler
+ protected def targetUri = "actor:uuid:%s?blocking=%s" format (event.uuid, event.blocking)
+}
+
+/**
+ * Defines the route to a typed actor method.
+ *
+ * @author Martin Krasser
+ */
+private[camel] class ConsumerMethodRouteBuilder(event: ConsumerMethodRegistered) extends ConsumerRouteBuilder(event.endpointUri, event.methodUuid) {
+ protected def routeDefinitionHandler: RouteDefinitionHandler = event.routeDefinitionHandler
+ protected def targetUri = "%s:%s?method=%s" format (TypedActorComponent.InternalSchema, event.methodUuid, event.methodName)
+}
+
+/**
+ * A registration listener that triggers publication of consumer actors and typed actor
+ * methods as well as un-publication of consumer actors and typed actor methods. This actor
+ * needs to be initialized with a PublishRequestorInit command message for
+ * obtaining a reference to a publisher actor. Before initialization it buffers
+ * all outbound messages and delivers them to the publisher when receiving a
+ * PublishRequestorInit message. After initialization, outbound messages are
+ * delivered directly without buffering.
+ *
+ * @see PublishRequestorInit
+ *
+ * @author Martin Krasser
+ */
+private[camel] class PublishRequestor extends Actor {
+ private val events = ListBuffer[ConsumerEvent]()
+ private var publisher: Option[ActorRef] = None
+
+ protected def receive = {
+ case ActorRegistered(actor) =>
+ for (event <- ConsumerActorRegistered.forConsumer(actor)) deliverCurrentEvent(event)
+ case ActorUnregistered(actor) =>
+ for (event <- ConsumerActorUnregistered.forConsumer(actor)) deliverCurrentEvent(event)
+ case AspectInitRegistered(proxy, init) =>
+ for (event <- ConsumerMethodRegistered.forConsumer(proxy, init)) deliverCurrentEvent(event)
+ case AspectInitUnregistered(proxy, init) =>
+ for (event <- ConsumerMethodUnregistered.forConsumer(proxy, init)) deliverCurrentEvent(event)
+ case PublishRequestorInit(pub) => {
+ publisher = Some(pub)
+ deliverBufferedEvents
+ }
+ case _ => { /* ignore */ }
+ }
+
+ private def deliverCurrentEvent(event: ConsumerEvent) = {
+ publisher match {
+ case Some(pub) => pub ! event
+ case None => events += event
+ }
+ }
+
+ private def deliverBufferedEvents = {
+ for (event <- events) deliverCurrentEvent(event)
+ events.clear
+ }
+}
+
+/**
+ * Command message to initialize a PublishRequestor to use consumerPublisher
+ * for publishing actors or typed actor methods.
+ */
+private[camel] case class PublishRequestorInit(consumerPublisher: ActorRef)
+
+/**
+ * A consumer (un)registration event.
+ */
+private[camel] sealed trait ConsumerEvent
+
+/**
+ * A consumer actor (un)registration event.
+ */
+private[camel] trait ConsumerActorEvent extends ConsumerEvent {
+ val actorRef: ActorRef
+ val actor: Consumer
+
+ val uuid = actorRef.uuid.toString
+ val endpointUri = actor.endpointUri
+ val blocking = actor.blocking
+ val routeDefinitionHandler = actor.routeDefinitionHandler
+}
+
+/**
+ * A consumer method (un)registration event.
+ */
+private[camel] trait ConsumerMethodEvent extends ConsumerEvent {
+ val typedActor: AnyRef
+ val init: AspectInit
+ val method: Method
+
+ val uuid = init.actorRef.uuid.toString
+ val methodName = method.getName
+ val methodUuid = "%s_%s" format (uuid, methodName)
+
+ lazy val routeDefinitionHandler = consumeAnnotation.routeDefinitionHandler.newInstance
+ lazy val consumeAnnotation = method.getAnnotation(classOf[consume])
+ lazy val endpointUri = consumeAnnotation.value
+}
+
+/**
+ * Event indicating that a consumer actor has been registered at the actor registry.
+ */
+private[camel] case class ConsumerActorRegistered(actorRef: ActorRef, actor: Consumer) extends ConsumerActorEvent
+
+/**
+ * Event indicating that a consumer actor has been unregistered from the actor registry.
+ */
+private[camel] case class ConsumerActorUnregistered(actorRef: ActorRef, actor: Consumer) extends ConsumerActorEvent
+
+/**
+ * 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.
+ */
+private[camel] case class ConsumerMethodRegistered(typedActor: AnyRef, init: AspectInit, method: Method) extends ConsumerMethodEvent
+
+/**
+ * Event indicating that an typed actor has been stopped. For each @consume
+ * annotated typed object method a separate instance of this class is created.
+ */
+private[camel] case class ConsumerMethodUnregistered(typedActor: AnyRef, init: AspectInit, method: Method) extends ConsumerMethodEvent
+
+/**
+ * @author Martin Krasser
+ */
+private[camel] object ConsumerActorRegistered {
+ /**
+ * Creates an ConsumerActorRegistered event message for a consumer actor or None if
+ * actorRef is not a consumer actor.
+ */
+ def forConsumer(actorRef: ActorRef): Option[ConsumerActorRegistered] = {
+ Consumer.forConsumer[ConsumerActorRegistered](actorRef) {
+ actor => ConsumerActorRegistered(actorRef, actor)
+ }
+ }
+}
+
+/**
+ * @author Martin Krasser
+ */
+private[camel] object ConsumerActorUnregistered {
+ /**
+ * Creates an ConsumerActorUnregistered event message for a consumer actor or None if
+ * actorRef is not a consumer actor.
+ */
+ def forConsumer(actorRef: ActorRef): Option[ConsumerActorUnregistered] = {
+ Consumer.forConsumer[ConsumerActorUnregistered](actorRef) {
+ actor => ConsumerActorUnregistered(actorRef, actor)
+ }
+ }
+}
+
+/**
+ * @author Martin Krasser
+ */
+private[camel] object ConsumerMethod {
+ /**
+ * 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 typedActor is a proxy for a remote typed actor
+ * f is never called and Nil is returned.
+ */
+ 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))
+ }
+ }
+}
+
+/**
+ * @author Martin Krasser
+ */
+private[camel] object ConsumerMethodRegistered {
+ /**
+ * Creates a list of ConsumerMethodRegistered event messages for a typed actor or an empty
+ * list if the typed actor is a proxy for a remote typed actor or the typed actor doesn't
+ * have any @consume annotated methods.
+ */
+ def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodRegistered] = {
+ ConsumerMethod.forConsumer(typedActor, init) {
+ m => ConsumerMethodRegistered(typedActor, init, m)
+ }
+ }
+}
+
+/**
+ * @author Martin Krasser
+ */
+private[camel] object ConsumerMethodUnregistered {
+ /**
+ * Creates a list of ConsumerMethodUnregistered event messages for a typed actor or an empty
+ * list if the typed actor is a proxy for a remote typed actor or the typed actor doesn't
+ * have any @consume annotated methods.
+ */
+ def forConsumer(typedActor: AnyRef, init: AspectInit): List[ConsumerMethodUnregistered] = {
+ ConsumerMethod.forConsumer(typedActor, init) {
+ m => ConsumerMethodUnregistered(typedActor, init, m)
+ }
+ }
+}
diff --git a/akka-camel/src/main/scala/akka/Message.scala b/akka-camel/src/main/scala/akka/Message.scala
new file mode 100644
index 0000000000..aa1fcbd083
--- /dev/null
+++ b/akka-camel/src/main/scala/akka/Message.scala
@@ -0,0 +1,380 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.camel
+
+import java.util.{Map => JMap, Set => JSet}
+
+import scala.collection.JavaConversions._
+
+import org.apache.camel.{Exchange, Message => CamelMessage}
+import org.apache.camel.util.ExchangeHelper
+
+import akka.japi.{Function => JFunction}
+
+/**
+ * An immutable representation of a Camel message.
+ *
+ * @author Martin Krasser
+ */
+case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
+
+ /**
+ * Creates a Message with given body and empty headers map.
+ */
+ def this(body: Any) = this(body, Map.empty[String, Any])
+
+ /**
+ * Creates a Message with given body and headers map. A copy of the headers map is made.
+ *
+ * Java API
+ */
+ def this(body: Any, headers: JMap[String, Any]) = this(body, headers.toMap)
+
+ /**
+ * Returns the body of the message converted to the type T. Conversion is done
+ * using Camel's type converter. The type converter is obtained from the CamelContext managed
+ * by CamelContextManager. Applications have to ensure proper initialization of
+ * CamelContextManager.
+ *
+ * @see CamelContextManager.
+ */
+ def bodyAs[T](implicit m: Manifest[T]): T = getBodyAs(m.erasure.asInstanceOf[Class[T]])
+
+ /**
+ * Returns the body of the message converted to the type as given by the clazz
+ * parameter. Conversion is done using Camel's type converter. The type converter is obtained
+ * from the CamelContext managed by CamelContextManager. Applications have to ensure proper
+ * initialization of CamelContextManager.
+ *
+ * Java API
+ *
+ * @see CamelContextManager.
+ */
+ def getBodyAs[T](clazz: Class[T]): T =
+ CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](clazz, body)
+
+ /**
+ * Returns those headers from this message whose name is contained in names.
+ */
+ def headers(names: Set[String]): Map[String, Any] = headers.filter(names contains _._1)
+
+ /**
+ * Returns those headers from this message whose name is contained in names.
+ * The returned headers map is backed up by an immutable headers map. Any attempt to modify
+ * the returned map will throw an exception.
+ *
+ * Java API
+ */
+ def getHeaders(names: JSet[String]): JMap[String, Any] = headers.filter(names contains _._1)
+
+ /**
+ * Returns all headers from this message. The returned headers map is backed up by this
+ * message's immutable headers map. Any attempt to modify the returned map will throw an
+ * exception.
+ *
+ * Java API
+ */
+ def getHeaders: JMap[String, Any] = headers
+
+ /**
+ * 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. Throws NoSuchElementException
+ * if the header doesn't exist.
+ *
+ * Java API
+ */
+ def getHeader(name: String): Any = header(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 =
+ getHeaderAs(name, m.erasure.asInstanceOf[Class[T]])
+
+ /**
+ * Returns the header with given name converted to type as given by the clazz
+ * parameter. Throws NoSuchElementException if the header doesn't exist.
+ *
+ * Java API
+ */
+ def getHeaderAs[T](name: String, clazz: Class[T]): T =
+ CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](clazz, header(name))
+
+ /**
+ * Creates a Message with a transformed body using a transformer function.
+ */
+ def transformBody[A](transformer: A => Any): Message = setBody(transformer(body.asInstanceOf[A]))
+
+ /**
+ * Creates a Message with a transformed body using a transformer function.
+ *
+ * Java API
+ */
+ def transformBody[A](transformer: JFunction[A, Any]): Message = setBody(transformer(body.asInstanceOf[A]))
+
+ /**
+ * Creates a Message with current body converted to type T.
+ */
+ def setBodyAs[T](implicit m: Manifest[T]): Message = setBodyAs(m.erasure.asInstanceOf[Class[T]])
+
+ /**
+ * Creates a Message with current body converted to type clazz.
+ *
+ * Java API
+ */
+ def setBodyAs[T](clazz: Class[T]): Message = setBody(getBodyAs(clazz))
+
+ /**
+ * Creates a Message with a given body.
+ */
+ def setBody(body: Any) = new Message(body, this.headers)
+
+ /**
+ * Creates a new Message with given headers.
+ */
+ def setHeaders(headers: Map[String, Any]): Message = copy(this.body, headers)
+
+ /**
+ * Creates a new Message with given headers. A copy of the headers map is made.
+ *
+ * Java API
+ */
+ def setHeaders(headers: JMap[String, Any]): Message = setHeaders(headers.toMap)
+
+ /**
+ * Creates a new Message with given headers added to the current headers.
+ */
+ def addHeaders(headers: Map[String, Any]): Message = copy(this.body, this.headers ++ headers)
+
+ /**
+ * Creates a new Message with given headers added to the current headers.
+ * A copy of the headers map is made.
+ *
+ * Java API
+ */
+ def addHeaders(headers: JMap[String, Any]): Message = addHeaders(headers.toMap)
+
+ /**
+ * Creates a new Message with the given header added to the current headers.
+ */
+ def addHeader(header: (String, Any)): Message = copy(this.body, this.headers + header)
+
+ /**
+ * Creates a new Message with the given header, represented by name and
+ * value added to the existing headers.
+ *
+ * Java API
+ */
+ def addHeader(name: String, value: Any): Message = addHeader((name, value))
+
+ /**
+ * Creates a new Message where the header with given headerName is removed from
+ * the existing headers.
+ */
+ def removeHeader(headerName: String) = copy(this.body, this.headers - headerName)
+}
+
+/**
+ * Companion object of Message class.
+ *
+ * @author Martin Krasser
+ */
+object Message {
+
+ /**
+ * Message header to correlate request with response messages. Applications that send
+ * messages to a Producer actor may want to set this header on the request message
+ * so that it can be correlated with an asynchronous response. Messages send to Consumer
+ * actors have this header already set.
+ */
+ val MessageExchangeId = "MessageExchangeId".intern
+
+ /**
+ * Creates a new Message with body as message body and an empty header map.
+ */
+ //def apply(body: Any) = new Message(body)
+
+ /**
+ * Creates a canonical form of the given message msg. If msg of type
+ * Message then msg is returned, otherwise msg is set as body of a
+ * newly created Message object.
+ */
+ def canonicalize(msg: Any) = msg match {
+ case mobj: Message => mobj
+ case body => new Message(body)
+ }
+}
+
+/**
+ * An immutable representation of a failed Camel exchange. It contains the failure cause
+ * obtained from Exchange.getException and the headers from either the Exchange.getIn
+ * message or Exchange.getOut message, depending on the exchange pattern.
+ *
+ * @author Martin Krasser
+ */
+case class Failure(val cause: Exception, val headers: Map[String, Any] = Map.empty) {
+
+ /**
+ * Creates a Failure with cause body and empty headers map.
+ */
+ def this(cause: Exception) = this(cause, Map.empty[String, Any])
+
+ /**
+ * Creates a Failure with given cause and headers map. A copy of the headers map is made.
+ *
+ * Java API
+ */
+ def this(cause: Exception, headers: JMap[String, Any]) = this(cause, headers.toMap)
+
+ /**
+ * Returns the cause of this Failure.
+ *
+ * Java API.
+ */
+ def getCause = cause
+
+ /**
+ * Returns all headers from this failure message. The returned headers map is backed up by
+ * this message's immutable headers map. Any attempt to modify the returned map will throw
+ * an exception.
+ *
+ * Java API
+ */
+ def getHeaders: JMap[String, Any] = headers
+}
+
+/**
+ * Adapter for converting an org.apache.camel.Exchange to and from Message and Failure objects.
+ *
+ * @author Martin Krasser
+ */
+class CamelExchangeAdapter(exchange: Exchange) {
+ import CamelMessageConversion.toMessageAdapter
+
+ /**
+ * Sets Exchange.getIn from the given Message object.
+ */
+ def fromRequestMessage(msg: Message): Exchange = { requestMessage.fromMessage(msg); exchange }
+
+ /**
+ * Depending on the exchange pattern, sets Exchange.getIn or Exchange.getOut from the given
+ * Message object. If the exchange is out-capable then the Exchange.getOut is set, otherwise
+ * Exchange.getIn.
+ */
+ def fromResponseMessage(msg: Message): Exchange = { responseMessage.fromMessage(msg); exchange }
+
+ /**
+ * Sets Exchange.getException from the given Failure message. Headers of the Failure message
+ * are ignored.
+ */
+ def fromFailureMessage(msg: Failure): Exchange = { exchange.setException(msg.cause); exchange }
+
+ /**
+ * Creates a Message object from Exchange.getIn.
+ */
+ def toRequestMessage: Message = toRequestMessage(Map.empty)
+
+ /**
+ * Depending on the exchange pattern, creates a Message object from Exchange.getIn or Exchange.getOut.
+ * If the exchange is out-capable then the Exchange.getOut is set, otherwise Exchange.getIn.
+ */
+ def toResponseMessage: Message = toResponseMessage(Map.empty)
+
+ /**
+ * Creates a Failure object from the adapted Exchange.
+ *
+ * @see Failure
+ */
+ def toFailureMessage: Failure = toFailureMessage(Map.empty)
+
+ /**
+ * Creates a Message object from Exchange.getIn.
+ *
+ * @param headers additional headers to set on the created Message in addition to those
+ * in the Camel message.
+ */
+ def toRequestMessage(headers: Map[String, Any]): Message = requestMessage.toMessage(headers)
+
+ /**
+ * Depending on the exchange pattern, creates a Message object from Exchange.getIn or Exchange.getOut.
+ * If the exchange is out-capable then the Exchange.getOut is set, otherwise Exchange.getIn.
+ *
+ * @param headers additional headers to set on the created Message in addition to those
+ * in the Camel message.
+ */
+ def toResponseMessage(headers: Map[String, Any]): Message = responseMessage.toMessage(headers)
+
+ /**
+ * Creates a Failure object from the adapted Exchange.
+ *
+ * @param headers additional headers to set on the created Message in addition to those
+ * in the Camel message.
+ *
+ * @see Failure
+ */
+ def toFailureMessage(headers: Map[String, Any]): Failure =
+ Failure(exchange.getException, headers ++ responseMessage.toMessage.headers)
+
+ private def requestMessage = exchange.getIn
+
+ private def responseMessage = ExchangeHelper.getResultMessage(exchange)
+
+}
+
+/**
+ * Adapter for converting an org.apache.camel.Message to and from Message objects.
+ *
+ * @author Martin Krasser
+ */
+class CamelMessageAdapter(val cm: CamelMessage) {
+ /**
+ * Set the adapted Camel message from the given Message object.
+ */
+ def fromMessage(m: Message): CamelMessage = {
+ cm.setBody(m.body)
+ for (h <- m.headers) cm.getHeaders.put(h._1, h._2.asInstanceOf[AnyRef])
+ cm
+ }
+
+ /**
+ * Creates a new Message object from the adapted Camel message.
+ */
+ def toMessage: Message = toMessage(Map.empty)
+
+ /**
+ * Creates a new Message object from the adapted Camel message.
+ *
+ * @param headers additional headers to set on the created Message in addition to those
+ * in the Camel message.
+ */
+ def toMessage(headers: Map[String, Any]): Message = Message(cm.getBody, cmHeaders(headers, cm))
+
+ private def cmHeaders(headers: Map[String, Any], cm: CamelMessage) = headers ++ cm.getHeaders
+}
+
+/**
+ * Defines conversion methods to CamelExchangeAdapter and CamelMessageAdapter.
+ * Imported by applications that implicitly want to use conversion methods of
+ * CamelExchangeAdapter and CamelMessageAdapter.
+ */
+object CamelMessageConversion {
+
+ /**
+ * Creates an CamelExchangeAdapter for the given Camel exchange.
+ */
+ implicit def toExchangeAdapter(ce: Exchange): CamelExchangeAdapter =
+ new CamelExchangeAdapter(ce)
+
+ /**
+ * Creates an CamelMessageAdapter for the given Camel message.
+ */
+ implicit def toMessageAdapter(cm: CamelMessage): CamelMessageAdapter =
+ new CamelMessageAdapter(cm)
+}
diff --git a/akka-camel/src/main/scala/akka/Producer.scala b/akka-camel/src/main/scala/akka/Producer.scala
new file mode 100644
index 0000000000..ae23ae8c4e
--- /dev/null
+++ b/akka-camel/src/main/scala/akka/Producer.scala
@@ -0,0 +1,256 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.camel
+
+import CamelMessageConversion.toExchangeAdapter
+
+import org.apache.camel._
+import org.apache.camel.processor.SendProcessor
+
+import akka.actor.{Actor, ActorRef, UntypedActor}
+
+/**
+ * Support trait for producing messages to Camel endpoints.
+ *
+ * @author Martin Krasser
+ */
+trait ProducerSupport { this: Actor =>
+
+ /**
+ * Message headers to copy by default from request message to response-message.
+ */
+ private val headersToCopyDefault = Set(Message.MessageExchangeId)
+
+ /**
+ * Endpoint object resolved from the current CamelContext with
+ * endpointUri.
+ */
+ private lazy val endpoint = CamelContextManager.mandatoryContext.getEndpoint(endpointUri)
+
+ /**
+ * SendProcessor for producing messages to endpoint.
+ */
+ private lazy val processor = createSendProcessor
+
+ /**
+ * If set to false (default), this producer expects a response message from the Camel endpoint.
+ * If set to true, this producer initiates an in-only message exchange with the Camel endpoint
+ * (fire and forget).
+ */
+ def oneway: Boolean = false
+
+ /**
+ * Returns the Camel endpoint URI to produce messages to.
+ */
+ def endpointUri: String
+
+ /**
+ * Returns the names of message headers to copy from a request message to a response message.
+ * By default only the Message.MessageExchangeId is copied. Applications may override this to
+ * define an application-specific set of message headers to copy.
+ */
+ def headersToCopy: Set[String] = headersToCopyDefault
+
+ /**
+ * Default implementation of Actor.postStop for freeing resources needed
+ * to actually send messages to endpointUri.
+ */
+ override def postStop {
+ processor.stop
+ }
+
+ /**
+ * Initiates a message exchange of given pattern with the endpoint specified by
+ * endpointUri. The in-message of the initiated exchange is the canonical form
+ * of msg. After sending the in-message, the processing result (response) is passed
+ * as argument to receiveAfterProduce. If the response is received synchronously from
+ * the endpoint then receiveAfterProduce is called synchronously as well. If the
+ * response is received asynchronously, the receiveAfterProduce is called
+ * asynchronously. This is done by wrapping the response, adding it to this producers
+ * mailbox, unwrapping it and calling receiveAfterProduce. The original
+ * sender and senderFuture are thereby preserved.
+ *
+ * @see Message#canonicalize(Any)
+ *
+ * @param msg message to produce
+ * @param pattern exchange pattern
+ */
+ protected def produce(msg: Any, pattern: ExchangePattern): Unit = {
+ val cmsg = Message.canonicalize(msg)
+ val exchange = createExchange(pattern).fromRequestMessage(cmsg)
+ processor.process(exchange, new AsyncCallback {
+ val producer = self
+ // Need copies of sender and senderFuture references here
+ // since the callback could be done later by another thread.
+ val sender = self.sender
+ val senderFuture = self.senderFuture
+
+ def done(doneSync: Boolean): Unit = {
+ (doneSync, exchange.isFailed) match {
+ case (true, true) => dispatchSync(exchange.toFailureMessage(cmsg.headers(headersToCopy)))
+ case (true, false) => dispatchSync(exchange.toResponseMessage(cmsg.headers(headersToCopy)))
+ case (false, true) => dispatchAsync(FailureResult(exchange.toFailureMessage(cmsg.headers(headersToCopy))))
+ case (false, false) => dispatchAsync(MessageResult(exchange.toResponseMessage(cmsg.headers(headersToCopy))))
+ }
+ }
+
+ private def dispatchSync(result: Any) =
+ receiveAfterProduce(result)
+
+ private def dispatchAsync(result: Any) = {
+ if (senderFuture.isDefined)
+ producer.postMessageToMailboxAndCreateFutureResultWithTimeout(result, producer.timeout, sender, senderFuture)
+ else
+ producer.postMessageToMailbox(result, sender)
+ }
+ })
+ }
+
+ /**
+ * Produces msg to the endpoint specified by endpointUri. Before the message is
+ * actually sent it is pre-processed by calling receiveBeforeProduce. If oneway
+ * is true, an in-only message exchange is initiated, otherwise an in-out message exchange.
+ *
+ * @see Producer#produce(Any, ExchangePattern)
+ */
+ protected def produce: Receive = {
+ case res: MessageResult => receiveAfterProduce(res.message)
+ case res: FailureResult => receiveAfterProduce(res.failure)
+ case msg => {
+ if (oneway)
+ produce(receiveBeforeProduce(msg), ExchangePattern.InOnly)
+ else
+ produce(receiveBeforeProduce(msg), ExchangePattern.InOut)
+ }
+ }
+
+ /**
+ * Called before the message is sent to the endpoint specified by endpointUri. The original
+ * message is passed as argument. By default, this method simply returns the argument but may be overridden
+ * by subtraits or subclasses.
+ */
+ protected def receiveBeforeProduce: PartialFunction[Any, Any] = {
+ case msg => msg
+ }
+
+ /**
+ * Called after a response was received from the endpoint specified by endpointUri. The
+ * response is passed as argument. By default, this method sends the response back to the original sender
+ * if oneway is false. If oneway is true, nothing is
+ * done. This method may be overridden by subtraits or subclasses (e.g. to forward responses to another
+ * actor).
+ */
+ protected def receiveAfterProduce: Receive = {
+ case msg => if (!oneway) self.reply(msg)
+ }
+
+ /**
+ * Creates a new Exchange of given pattern from the endpoint specified by
+ * endpointUri.
+ */
+ private def createExchange(pattern: ExchangePattern): Exchange = endpoint.createExchange(pattern)
+
+ /**
+ * Creates a new SendProcessor for endpoint.
+ */
+ private def createSendProcessor = {
+ val sendProcessor = new SendProcessor(endpoint)
+ sendProcessor.start
+ sendProcessor
+ }
+}
+
+/**
+ * Mixed in by Actor implementations to produce messages to Camel endpoints.
+ */
+trait Producer extends ProducerSupport { this: Actor =>
+
+ /**
+ * Default implementation of Actor.receive. Any messages received by this actors
+ * will be produced to the endpoint specified by endpointUri.
+ */
+ protected def receive = produce
+}
+
+/**
+ * Java-friendly ProducerSupport.
+ *
+ * @see UntypedProducerActor
+ *
+ * @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 a response was received from the endpoint specified by endpointUri. The
+ * response is passed as argument. By default, this method sends the response back to the original sender
+ * if oneway is false. If oneway is true, nothing is
+ * done. This method may be overridden by subclasses (e.g. to forward responses to another actor).
+ */
+ @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
+ */
+private[camel] case class MessageResult(message: Message)
+
+/**
+ * @author Martin Krasser
+ */
+private[camel] case class FailureResult(failure: Failure)
+
+/**
+ * A one-way producer.
+ *
+ * @author Martin Krasser
+ */
+trait Oneway extends Producer { this: Actor =>
+ override def oneway = true
+}
+
diff --git a/akka-camel/src/main/scala/akka/component/ActorComponent.scala b/akka-camel/src/main/scala/akka/component/ActorComponent.scala
new file mode 100644
index 0000000000..e84a894ee3
--- /dev/null
+++ b/akka-camel/src/main/scala/akka/component/ActorComponent.scala
@@ -0,0 +1,305 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.camel.component
+
+import java.net.InetSocketAddress
+import java.util.{Map => JMap}
+import java.util.concurrent.TimeoutException
+import java.util.concurrent.atomic.AtomicReference
+
+import org.apache.camel._
+import org.apache.camel.impl.{DefaultProducer, DefaultEndpoint, DefaultComponent}
+
+import akka.actor._
+import akka.camel.{Failure, Message}
+import akka.camel.CamelMessageConversion.toExchangeAdapter
+import akka.dispatch.{CompletableFuture, MessageInvocation, MessageDispatcher}
+
+import scala.reflect.BeanProperty
+
+/**
+ * @author Martin Krasser
+ */
+object ActorComponent {
+ /**
+ * Name of the message header containing the actor id or uuid.
+ */
+ val ActorIdentifier = "CamelActorIdentifier"
+}
+
+/**
+ * Camel component for sending messages to and receiving replies from (untyped) actors.
+ *
+ * @see akka.camel.component.ActorEndpoint
+ * @see akka.camel.component.ActorProducer
+ *
+ * @author Martin Krasser
+ */
+class ActorComponent extends DefaultComponent {
+ def createEndpoint(uri: String, remaining: String, parameters: JMap[String, Object]): ActorEndpoint = {
+ val (idType, idValue) = parsePath(remaining)
+ new ActorEndpoint(uri, this, idType, idValue)
+ }
+
+ private def parsePath(remaining: String): Tuple2[String, Option[String]] = remaining match {
+ case null | "" => throw new IllegalArgumentException("invalid path: [%s] - should be or id: or uuid:" format remaining)
+ case id if id startsWith "id:" => ("id", parseIdentifier(id substring 3))
+ case uuid if uuid startsWith "uuid:" => ("uuid", parseIdentifier(uuid substring 5))
+ case id => ("id", parseIdentifier(id))
+ }
+
+ private def parseIdentifier(identifier: String): Option[String] =
+ if (identifier.length > 0) Some(identifier) else None
+}
+
+/**
+ * Camel endpoint for sending messages to and receiving replies from (untyped) actors. Actors
+ * are referenced using actor endpoint URIs of the following format:
+ * actor:,
+ * actor:id:[] and
+ * actor:uuid:[],
+ * where refers to ActorRef.id and
+ * refers to the String-representation od ActorRef.uuid. In URIs that contain
+ * id: or uuid:, an actor identifier (id or uuid) is optional. In this
+ * case, the in-message of an exchange produced to this endpoint must contain a message header
+ * with name CamelActorIdentifier and a value that is the target actor's identifier.
+ * If the URI contains an actor identifier, a message with a CamelActorIdentifier
+ * header overrides the identifier in the endpoint URI.
+ *
+ * @see akka.camel.component.ActorComponent
+ * @see akka.camel.component.ActorProducer
+
+ * @author Martin Krasser
+ */
+class ActorEndpoint(uri: String,
+ comp: ActorComponent,
+ val idType: String,
+ val idValue: Option[String]) extends DefaultEndpoint(uri, comp) {
+
+ /**
+ * Whether to block caller thread during two-way message exchanges with (untyped) actors. This is
+ * set via the blocking=true|false endpoint URI parameter. Default value is
+ * false.
+ */
+ @BeanProperty var blocking: Boolean = false
+
+ /**
+ * @throws UnsupportedOperationException
+ */
+ def createConsumer(processor: Processor): Consumer =
+ throw new UnsupportedOperationException("actor consumer not supported yet")
+
+ /**
+ * Creates a new ActorProducer instance initialized with this endpoint.
+ */
+ def createProducer: ActorProducer = new ActorProducer(this)
+
+ /**
+ * Returns true.
+ */
+ def isSingleton: Boolean = true
+}
+
+/**
+ * Sends the in-message of an exchange to an (untyped) actor, identified by an
+ * actor endpoint URI or by a CamelActorIdentifier message header.
+ *
+ *
If the exchange pattern is out-capable and blocking is set to
+ * true then the producer waits for a reply, using the !! operator.
+ *
If the exchange pattern is out-capable and blocking is set to
+ * false then the producer sends the message using the ! operator, together
+ * with a callback handler. The callback handler is an ActorRef that can be
+ * used by the receiving actor to asynchronously reply to the route that is sending the
+ * message.
+ *
If the exchange pattern is in-only then the producer sends the message using the
+ * ! operator.
+ *
+ *
+ * @see akka.camel.component.ActorComponent
+ * @see akka.camel.component.ActorEndpoint
+ *
+ * @author Martin Krasser
+ */
+class ActorProducer(val ep: ActorEndpoint) extends DefaultProducer(ep) with AsyncProcessor {
+ import ActorProducer._
+
+ private lazy val uuid = uuidFrom(ep.idValue.getOrElse(throw new ActorIdentifierNotSetException))
+
+ def process(exchange: Exchange) =
+ if (exchange.getPattern.isOutCapable) sendSync(exchange) else sendAsync(exchange)
+
+ def process(exchange: Exchange, callback: AsyncCallback): Boolean = {
+ (exchange.getPattern.isOutCapable, ep.blocking) match {
+ case (true, true) => {
+ sendSync(exchange)
+ callback.done(true)
+ true
+ }
+ case (true, false) => {
+ sendAsync(exchange, Some(AsyncCallbackAdapter(exchange, callback)))
+ false
+ }
+ case (false, _) => {
+ sendAsync(exchange)
+ callback.done(true)
+ true
+ }
+ }
+ }
+
+ private def sendSync(exchange: Exchange) = {
+ val actor = target(exchange)
+ val result: Any = actor !! requestFor(exchange)
+
+ 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))
+ }
+ }
+
+ private def sendAsync(exchange: Exchange, sender: Option[ActorRef] = None) =
+ target(exchange).!(requestFor(exchange))(sender)
+
+ private def target(exchange: Exchange) =
+ targetOption(exchange) getOrElse (throw new ActorNotRegisteredException(ep.getEndpointUri))
+
+ private def targetOption(exchange: Exchange): Option[ActorRef] = ep.idType match {
+ case "id" => targetById(targetId(exchange))
+ case "uuid" => targetByUuid(targetUuid(exchange))
+ }
+
+ private def targetId(exchange: Exchange) = exchange.getIn.getHeader(ActorComponent.ActorIdentifier) match {
+ case id: String => id
+ case null => ep.idValue.getOrElse(throw new ActorIdentifierNotSetException)
+ }
+
+ private def targetUuid(exchange: Exchange) = exchange.getIn.getHeader(ActorComponent.ActorIdentifier) match {
+ case uuid: Uuid => uuid
+ case uuid: String => uuidFrom(uuid)
+ case null => uuid
+ }
+
+ private def targetById(id: String) = ActorRegistry.actorsFor(id) match {
+ case actors if actors.length == 0 => None
+ case actors => Some(actors(0))
+ }
+
+ private def targetByUuid(uuid: Uuid) = ActorRegistry.actorFor(uuid)
+}
+
+/**
+ * @author Martin Krasser
+ */
+private[camel] object ActorProducer {
+ def requestFor(exchange: Exchange) =
+ exchange.toRequestMessage(Map(Message.MessageExchangeId -> exchange.getExchangeId))
+}
+
+/**
+ * Thrown to indicate that an actor referenced by an endpoint URI cannot be
+ * found in the ActorRegistry.
+ *
+ * @author Martin Krasser
+ */
+class ActorNotRegisteredException(uri: String) extends RuntimeException {
+ override def getMessage = "%s not registered" format uri
+}
+
+/**
+ * Thrown to indicate that no actor identifier has been set.
+ *
+ * @author Martin Krasser
+ */
+class ActorIdentifierNotSetException extends RuntimeException {
+ override def getMessage = "actor identifier not set"
+}
+
+/**
+ * @author Martin Krasser
+ */
+private[akka] object AsyncCallbackAdapter {
+ /**
+ * Creates and starts an AsyncCallbackAdapter.
+ *
+ * @param exchange message exchange to write results to.
+ * @param callback callback object to generate completion notifications.
+ */
+ def apply(exchange: Exchange, callback: AsyncCallback) =
+ new AsyncCallbackAdapter(exchange, callback).start
+}
+
+/**
+ * Adapts an ActorRef to a Camel AsyncCallback. Used by receiving actors to reply
+ * asynchronously to Camel routes with ActorRef.reply.
+ *
+ * Please note that this adapter can only be used locally at the moment which should not
+ * be a problem is most situations since Camel endpoints are only activated for local actor references,
+ * never for remote references.
+ *
+ * @author Martin Krasser
+ */
+private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCallback) extends ActorRef with ScalaActorRef {
+
+ def start = {
+ _status = ActorRefInternals.RUNNING
+ this
+ }
+
+ def stop() = {
+ _status = ActorRefInternals.SHUTDOWN
+ }
+
+ /**
+ * Populates the initial exchange with the reply message and uses the
+ * callback handler to notify Camel about the asynchronous completion of the message
+ * exchange.
+ *
+ * @param message reply message
+ * @param sender ignored
+ */
+ protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]) = {
+ message match {
+ case msg: Failure => exchange.fromFailureMessage(msg)
+ case msg => exchange.fromResponseMessage(Message.canonicalize(msg))
+ }
+ callback.done(false)
+ }
+
+ def actorClass: Class[_ <: Actor] = unsupported
+ def actorClassName = unsupported
+ def dispatcher_=(md: MessageDispatcher): Unit = unsupported
+ def dispatcher: MessageDispatcher = unsupported
+ def makeRemote(hostname: String, port: Int): Unit = unsupported
+ def makeRemote(address: InetSocketAddress): Unit = unsupported
+ def homeAddress_=(address: InetSocketAddress): Unit = unsupported
+ def remoteAddress: Option[InetSocketAddress] = unsupported
+ def link(actorRef: ActorRef): Unit = unsupported
+ def unlink(actorRef: ActorRef): Unit = unsupported
+ def startLink(actorRef: ActorRef): Unit = unsupported
+ def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = unsupported
+ def spawn(clazz: Class[_ <: Actor]): ActorRef = unsupported
+ def spawnRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
+ def spawnLink(clazz: Class[_ <: Actor]): ActorRef = unsupported
+ def spawnLinkRemote(clazz: Class[_ <: Actor], hostname: String, port: Int): ActorRef = unsupported
+ def shutdownLinkedActors: Unit = 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: AnyRef = unsupported
+ protected[akka] def mailbox_=(msg: AnyRef):AnyRef = unsupported
+ protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
+ protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
+ protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
+ protected[akka] def linkedActors: JMap[Uuid, ActorRef] = unsupported
+ protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported
+ protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
+ protected[akka] def remoteAddress_=(addr: Option[InetSocketAddress]): Unit = unsupported
+ protected[akka] def registerSupervisorAsRemoteActor = unsupported
+ protected[akka] def supervisor_=(sup: Option[ActorRef]): Unit = unsupported
+ protected[akka] def actorInstance: AtomicReference[Actor] = unsupported
+
+ private def unsupported = throw new UnsupportedOperationException("Not supported for %s" format classOf[AsyncCallbackAdapter].getName)
+}
diff --git a/akka-camel/src/main/scala/akka/component/TypedActorComponent.scala b/akka-camel/src/main/scala/akka/component/TypedActorComponent.scala
new file mode 100644
index 0000000000..f4a7f1b099
--- /dev/null
+++ b/akka-camel/src/main/scala/akka/component/TypedActorComponent.scala
@@ -0,0 +1,111 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.camel.component
+
+import java.util.Map
+import java.util.concurrent.ConcurrentHashMap
+import org.apache.camel.CamelContext
+import org.apache.camel.component.bean._
+
+/**
+ * @author Martin Krasser
+ */
+object TypedActorComponent {
+ /**
+ * Default schema name for typed actor endpoint URIs.
+ */
+ val InternalSchema = "typed-actor-internal"
+}
+
+/**
+ * Camel component for exchanging messages with typed actors. This component
+ * tries to obtain the typed actor from its 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 TypedActorComponent extends BeanComponent {
+ val typedActorRegistry = new ConcurrentHashMap[String, AnyRef]
+
+ /**
+ * Creates an org.apache.camel.component.bean.BeanEndpoint with a custom
+ * bean holder that uses typedActorRegistry for getting access to typed
+ * actors (beans).
+ *
+ * @see akka.camel.component.TypedActorHolder
+ */
+ override def createEndpoint(uri: String, remaining: String, parameters: Map[String, AnyRef]) = {
+ val endpoint = new BeanEndpoint(uri, this)
+ endpoint.setBeanName(remaining)
+ endpoint.setBeanHolder(createBeanHolder(remaining))
+ setProperties(endpoint.getProcessor, parameters)
+ endpoint
+ }
+
+ private def createBeanHolder(beanName: String) =
+ new TypedActorHolder(typedActorRegistry, getCamelContext, beanName).createCacheHolder
+}
+
+/**
+ * org.apache.camel.component.bean.BeanHolder implementation that uses a custom
+ * registry for getting access to typed actors.
+ *
+ * @author Martin Krasser
+ */
+class TypedActorHolder(typedActorRegistry: Map[String, AnyRef], context: CamelContext, name: String)
+ extends RegistryBean(context, name) {
+
+ /**
+ * Returns an akka.camel.component.TypedActorInfo instance.
+ */
+ override def getBeanInfo: BeanInfo =
+ new TypedActorInfo(getContext, getBean.getClass, getParameterMappingStrategy)
+
+ /**
+ * Obtains a typed actor from typedActorRegistry. If the typed actor cannot
+ * be found then this method tries to obtain the actor from the CamelContext's registry.
+ *
+ * @return a typed actor or null.
+ */
+ override def getBean: AnyRef = {
+ val bean = typedActorRegistry.get(getName)
+ if (bean eq null) super.getBean else bean
+ }
+}
+
+/**
+ * Typed actor meta information.
+ *
+ * @author Martin Krasser
+ */
+class TypedActorInfo(context: CamelContext, clazz: Class[_], strategy: ParameterMappingStrategy)
+ extends BeanInfo(context, clazz, strategy) {
+
+ /**
+ * Introspects AspectWerkz proxy classes.
+ *
+ * @param clazz AspectWerkz proxy class.
+ */
+ protected override def introspect(clazz: Class[_]): Unit = {
+
+ // TODO: fix target class detection in BeanInfo.introspect(Class)
+ // Camel assumes that classes containing a '$$' in the class name
+ // are classes generated with CGLIB. This conflicts with proxies
+ // created from interfaces with AspectWerkz. Once the fix is in
+ // place this method can be removed.
+
+ for (method <- clazz.getDeclaredMethods) {
+ if (isValidMethod(clazz, method)) {
+ introspect(clazz, method)
+ }
+ }
+ val superclass = clazz.getSuperclass
+ if ((superclass ne null) && !superclass.equals(classOf[AnyRef])) {
+ introspect(superclass)
+ }
+ }
+}
diff --git a/akka-camel/src/test/java/akka/camel/ConsumerJavaTestBase.java b/akka-camel/src/test/java/akka/camel/ConsumerJavaTestBase.java
new file mode 100644
index 0000000000..c34ce0cc2e
--- /dev/null
+++ b/akka-camel/src/test/java/akka/camel/ConsumerJavaTestBase.java
@@ -0,0 +1,59 @@
+package akka.camel;
+
+import akka.actor.ActorRegistry;
+import akka.actor.TypedActor;
+import akka.actor.UntypedActor;
+import akka.japi.SideEffect;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static akka.camel.CamelContextManager.*;
+import static akka.camel.CamelServiceManager.*;
+
+import static org.junit.Assert.*;
+
+/**
+ * @author Martin Krasser
+ */
+public class ConsumerJavaTestBase {
+
+ private SampleErrorHandlingTypedConsumer consumer;
+
+ @BeforeClass
+ public static void setUpBeforeClass() {
+ startCamelService();
+ }
+
+ @AfterClass
+ public static void tearDownAfterClass() {
+ stopCamelService();
+ ActorRegistry.shutdownAll();
+ }
+
+ @Test
+ public void shouldHandleExceptionThrownByActorAndGenerateCustomResponse() {
+ getMandatoryService().awaitEndpointActivation(1, new SideEffect() {
+ public void apply() {
+ UntypedActor.actorOf(SampleErrorHandlingConsumer.class).start();
+ }
+ });
+ String result = getMandatoryTemplate().requestBody("direct:error-handler-test-java", "hello", String.class);
+ assertEquals("error: hello", result);
+ }
+
+ @Test
+ public void shouldHandleExceptionThrownByTypedActorAndGenerateCustomResponse() {
+ getMandatoryService().awaitEndpointActivation(1, new SideEffect() {
+ public void apply() {
+ consumer = TypedActor.newInstance(
+ SampleErrorHandlingTypedConsumer.class,
+ SampleErrorHandlingTypedConsumerImpl.class);
+ }
+ });
+ String result = getMandatoryTemplate().requestBody("direct:error-handler-test-java-typed", "hello", String.class);
+ assertEquals("error: hello", result);
+ }
+
+}
diff --git a/akka-camel/src/test/java/akka/camel/SampleErrorHandlingConsumer.java b/akka-camel/src/test/java/akka/camel/SampleErrorHandlingConsumer.java
new file mode 100644
index 0000000000..4e35d4e6ab
--- /dev/null
+++ b/akka-camel/src/test/java/akka/camel/SampleErrorHandlingConsumer.java
@@ -0,0 +1,34 @@
+package akka.camel;
+
+import org.apache.camel.builder.Builder;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.RouteDefinition;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleErrorHandlingConsumer extends UntypedConsumerActor {
+
+ public String getEndpointUri() {
+ return "direct:error-handler-test-java";
+ }
+
+ public boolean isBlocking() {
+ return true;
+ }
+
+ public void preStart() {
+ onRouteDefinition(new RouteDefinitionHandler() {
+ public ProcessorDefinition> onRouteDefinition(RouteDefinition rd) {
+ return rd.onException(Exception.class).handled(true).transform(Builder.exceptionMessage()).end();
+ }
+ });
+ }
+
+ public void onReceive(Object message) throws Exception {
+ Message msg = (Message)message;
+ String body = msg.getBodyAs(String.class);
+ throw new Exception(String.format("error: %s", body));
+ }
+
+}
diff --git a/akka-camel/src/test/java/akka/camel/SampleErrorHandlingTypedConsumer.java b/akka-camel/src/test/java/akka/camel/SampleErrorHandlingTypedConsumer.java
new file mode 100644
index 0000000000..d8a8c79440
--- /dev/null
+++ b/akka-camel/src/test/java/akka/camel/SampleErrorHandlingTypedConsumer.java
@@ -0,0 +1,11 @@
+package akka.camel;
+
+/**
+ * @author Martin Krasser
+ */
+public interface SampleErrorHandlingTypedConsumer {
+
+ @consume(value="direct:error-handler-test-java-typed", routeDefinitionHandler=SampleRouteDefinitionHandler.class)
+ String willFail(String s);
+
+}
diff --git a/akka-camel/src/test/java/akka/camel/SampleErrorHandlingTypedConsumerImpl.java b/akka-camel/src/test/java/akka/camel/SampleErrorHandlingTypedConsumerImpl.java
new file mode 100644
index 0000000000..cfa42a7521
--- /dev/null
+++ b/akka-camel/src/test/java/akka/camel/SampleErrorHandlingTypedConsumerImpl.java
@@ -0,0 +1,14 @@
+package akka.camel;
+
+import akka.actor.TypedActor;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleErrorHandlingTypedConsumerImpl extends TypedActor implements SampleErrorHandlingTypedConsumer {
+
+ public String willFail(String s) {
+ throw new RuntimeException(String.format("error: %s", s));
+ }
+
+}
diff --git a/akka-camel/src/test/java/akka/camel/SampleRouteDefinitionHandler.java b/akka-camel/src/test/java/akka/camel/SampleRouteDefinitionHandler.java
new file mode 100644
index 0000000000..f1a99aa7d4
--- /dev/null
+++ b/akka-camel/src/test/java/akka/camel/SampleRouteDefinitionHandler.java
@@ -0,0 +1,14 @@
+package akka.camel;
+
+import org.apache.camel.builder.Builder;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.RouteDefinition;
+
+/**
+ * @author Martin Krasser
+ */
+public class SampleRouteDefinitionHandler implements RouteDefinitionHandler {
+ public ProcessorDefinition> onRouteDefinition(RouteDefinition rd) {
+ return rd.onException(Exception.class).handled(true).transform(Builder.exceptionMessage()).end();
+ }
+}
diff --git a/akka-camel/src/test/scala/akka/CamelContextLifecycleTest.scala b/akka-camel/src/test/scala/akka/CamelContextLifecycleTest.scala
new file mode 100644
index 0000000000..910373738f
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/CamelContextLifecycleTest.scala
@@ -0,0 +1,36 @@
+package akka.camel
+
+import org.apache.camel.impl.{DefaultProducerTemplate, DefaultCamelContext}
+import org.junit.Test
+import org.scalatest.junit.JUnitSuite
+
+class CamelContextLifecycleTest extends JUnitSuite with CamelContextLifecycle {
+ @Test def shouldManageCustomCamelContext {
+ assert(context === None)
+ assert(template === None)
+
+ intercept[IllegalStateException] { mandatoryContext }
+ intercept[IllegalStateException] { mandatoryTemplate }
+
+ val ctx = new TestCamelContext
+ assert(ctx.isStreamCaching === false)
+
+ init(ctx)
+
+ assert(mandatoryContext.isStreamCaching === true)
+ assert(!mandatoryContext.asInstanceOf[TestCamelContext].isStarted)
+ assert(mandatoryTemplate.asInstanceOf[DefaultProducerTemplate].isStarted)
+
+ start
+
+ assert(mandatoryContext.asInstanceOf[TestCamelContext].isStarted)
+ assert(mandatoryTemplate.asInstanceOf[DefaultProducerTemplate].isStarted)
+
+ stop
+
+ assert(!mandatoryContext.asInstanceOf[TestCamelContext].isStarted)
+ assert(!mandatoryTemplate.asInstanceOf[DefaultProducerTemplate].isStarted)
+ }
+
+ class TestCamelContext extends DefaultCamelContext
+}
diff --git a/akka-camel/src/test/scala/akka/CamelExchangeAdapterTest.scala b/akka-camel/src/test/scala/akka/CamelExchangeAdapterTest.scala
new file mode 100644
index 0000000000..3996179b5b
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/CamelExchangeAdapterTest.scala
@@ -0,0 +1,109 @@
+package akka.camel
+
+import org.apache.camel.impl.{DefaultCamelContext, DefaultExchange}
+import org.apache.camel.ExchangePattern
+import org.junit.Test
+import org.scalatest.junit.JUnitSuite
+
+class CamelExchangeAdapterTest extends JUnitSuite {
+ import CamelMessageConversion.toExchangeAdapter
+
+ @Test def shouldSetInMessageFromRequestMessage = {
+ val e1 = sampleInOnly.fromRequestMessage(Message("x"))
+ assert(e1.getIn.getBody === "x")
+ val e2 = sampleInOut.fromRequestMessage(Message("y"))
+ assert(e2.getIn.getBody === "y")
+ }
+
+ @Test def shouldSetOutMessageFromResponseMessage = {
+ val e1 = sampleInOut.fromResponseMessage(Message("y"))
+ assert(e1.getOut.getBody === "y")
+ }
+
+ @Test def shouldSetInMessageFromResponseMessage = {
+ val e1 = sampleInOnly.fromResponseMessage(Message("x"))
+ assert(e1.getIn.getBody === "x")
+ }
+
+ @Test def shouldSetExceptionFromFailureMessage = {
+ val e1 = sampleInOnly.fromFailureMessage(Failure(new Exception("test1")))
+ assert(e1.getException.getMessage === "test1")
+ val e2 = sampleInOut.fromFailureMessage(Failure(new Exception("test2")))
+ assert(e2.getException.getMessage === "test2")
+ }
+
+ @Test def shouldCreateRequestMessageFromInMessage = {
+ val m = sampleInOnly.toRequestMessage
+ assert(m === Message("test-in", Map("key-in" -> "val-in")))
+ }
+
+ @Test def shouldCreateResponseMessageFromInMessage = {
+ val m = sampleInOnly.toResponseMessage
+ assert(m === Message("test-in", Map("key-in" -> "val-in")))
+ }
+
+ @Test def shouldCreateResponseMessageFromOutMessage = {
+ val m = sampleInOut.toResponseMessage
+ assert(m === Message("test-out", Map("key-out" -> "val-out")))
+ }
+
+ @Test def shouldCreateFailureMessageFromExceptionAndInMessage = {
+ val e1 = sampleInOnly
+ e1.setException(new Exception("test1"))
+ assert(e1.toFailureMessage.cause.getMessage === "test1")
+ assert(e1.toFailureMessage.headers("key-in") === "val-in")
+ }
+
+ @Test def shouldCreateFailureMessageFromExceptionAndOutMessage = {
+ val e1 = sampleInOut
+ e1.setException(new Exception("test2"))
+ assert(e1.toFailureMessage.cause.getMessage === "test2")
+ assert(e1.toFailureMessage.headers("key-out") === "val-out")
+ }
+
+ @Test def shouldCreateRequestMessageFromInMessageWithAdditionalHeader = {
+ val m = sampleInOnly.toRequestMessage(Map("x" -> "y"))
+ assert(m === Message("test-in", Map("key-in" -> "val-in", "x" -> "y")))
+ }
+
+ @Test def shouldCreateResponseMessageFromInMessageWithAdditionalHeader = {
+ val m = sampleInOnly.toResponseMessage(Map("x" -> "y"))
+ assert(m === Message("test-in", Map("key-in" -> "val-in", "x" -> "y")))
+ }
+
+ @Test def shouldCreateResponseMessageFromOutMessageWithAdditionalHeader = {
+ val m = sampleInOut.toResponseMessage(Map("x" -> "y"))
+ assert(m === Message("test-out", Map("key-out" -> "val-out", "x" -> "y")))
+ }
+
+ @Test def shouldCreateFailureMessageFromExceptionAndInMessageWithAdditionalHeader = {
+ val e1 = sampleInOnly
+ e1.setException(new Exception("test1"))
+ assert(e1.toFailureMessage.cause.getMessage === "test1")
+ val headers = e1.toFailureMessage(Map("x" -> "y")).headers
+ assert(headers("key-in") === "val-in")
+ assert(headers("x") === "y")
+ }
+
+ @Test def shouldCreateFailureMessageFromExceptionAndOutMessageWithAdditionalHeader = {
+ val e1 = sampleInOut
+ e1.setException(new Exception("test2"))
+ assert(e1.toFailureMessage.cause.getMessage === "test2")
+ val headers = e1.toFailureMessage(Map("x" -> "y")).headers
+ assert(headers("key-out") === "val-out")
+ assert(headers("x") === "y")
+ }
+
+ private def sampleInOnly = sampleExchange(ExchangePattern.InOnly)
+ private def sampleInOut = sampleExchange(ExchangePattern.InOut)
+
+ private def sampleExchange(pattern: ExchangePattern) = {
+ val exchange = new DefaultExchange(new DefaultCamelContext)
+ exchange.getIn.setBody("test-in")
+ exchange.getOut.setBody("test-out")
+ exchange.getIn.setHeader("key-in", "val-in")
+ exchange.getOut.setHeader("key-out", "val-out")
+ exchange.setPattern(pattern)
+ exchange
+ }
+}
diff --git a/akka-camel/src/test/scala/akka/CamelMessageAdapterTest.scala b/akka-camel/src/test/scala/akka/CamelMessageAdapterTest.scala
new file mode 100644
index 0000000000..0c20ae1c29
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/CamelMessageAdapterTest.scala
@@ -0,0 +1,38 @@
+package akka.camel
+
+import org.apache.camel.impl.DefaultMessage
+import org.junit.Test
+import org.scalatest.junit.JUnitSuite
+
+class CamelMessageAdapterTest extends JUnitSuite {
+ import CamelMessageConversion.toMessageAdapter
+
+ @Test def shouldOverwriteBodyAndAddHeader = {
+ val cm = sampleMessage.fromMessage(Message("blah", Map("key" -> "baz")))
+ assert(cm.getBody === "blah")
+ assert(cm.getHeader("foo") === "bar")
+ assert(cm.getHeader("key") === "baz")
+ }
+
+ @Test def shouldCreateMessageWithBodyAndHeader = {
+ val m = sampleMessage.toMessage
+ assert(m.body === "test")
+ assert(m.headers("foo") === "bar")
+ }
+
+ @Test def shouldCreateMessageWithBodyAndHeaderAndCustomHeader = {
+ val m = sampleMessage.toMessage(Map("key" -> "baz"))
+ assert(m.body === "test")
+ assert(m.headers("foo") === "bar")
+ assert(m.headers("key") === "baz")
+ }
+
+ private[camel] def sampleMessage = {
+ val message = new DefaultMessage
+ message.setBody("test")
+ message.setHeader("foo", "bar")
+ message
+ }
+
+
+}
diff --git a/akka-camel/src/test/scala/akka/CamelServiceManagerTest.scala b/akka-camel/src/test/scala/akka/CamelServiceManagerTest.scala
new file mode 100644
index 0000000000..48ab29c6b8
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/CamelServiceManagerTest.scala
@@ -0,0 +1,62 @@
+package akka.camel
+
+import org.scalatest.{BeforeAndAfterAll, WordSpec}
+import org.scalatest.matchers.MustMatchers
+
+import akka.actor.ActorRegistry
+
+/**
+ * @author Martin Krasser
+ */
+class CamelServiceManagerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
+
+ override def afterAll = {
+ CamelServiceManager.stopCamelService
+ 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.mandatoryService must be theSameInstanceAs (service)
+ }
+ }
+ "the stopCamelService method been has been called" must {
+ "have unregistered the current CamelService instance" in {
+ val service = CamelServiceManager.stopCamelService
+ CamelServiceManager.service must be (None)
+ }
+ }
+ }
+
+ "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.mandatoryService must be theSameInstanceAs (service)
+ }
+ }
+ "the current CamelService instance has been stopped externally" must {
+ "have unregistered the current CamelService instance" in {
+ service.stop
+ CamelServiceManager.service must be (None)
+ }
+ }
+ }
+
+ "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 }
+ }
+ }
+ }
+}
diff --git a/akka-camel/src/test/scala/akka/ConsumerJavaTest.scala b/akka-camel/src/test/scala/akka/ConsumerJavaTest.scala
new file mode 100644
index 0000000000..48741dda96
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/ConsumerJavaTest.scala
@@ -0,0 +1,5 @@
+package akka.camel
+
+import org.scalatest.junit.JUnitSuite
+
+class ConsumerJavaTest extends ConsumerJavaTestBase with JUnitSuite
\ No newline at end of file
diff --git a/akka-camel/src/test/scala/akka/ConsumerRegisteredTest.scala b/akka-camel/src/test/scala/akka/ConsumerRegisteredTest.scala
new file mode 100644
index 0000000000..e85c5f905c
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/ConsumerRegisteredTest.scala
@@ -0,0 +1,63 @@
+package akka.camel
+
+import org.junit.Test
+import org.scalatest.junit.JUnitSuite
+import akka.actor.{ActorRef, Actor, UntypedActor}
+
+class ConsumerRegisteredTest extends JUnitSuite {
+ import ConsumerRegisteredTest._
+
+ @Test def shouldCreateSomeNonBlockingPublishRequestFromConsumer = {
+ val c = Actor.actorOf[ConsumerActor1]
+ val event = ConsumerActorRegistered.forConsumer(c)
+ assert(event === Some(ConsumerActorRegistered(c, consumerOf(c))))
+ }
+
+ @Test def shouldCreateSomeBlockingPublishRequestFromConsumer = {
+ val c = Actor.actorOf[ConsumerActor2]
+ val event = ConsumerActorRegistered.forConsumer(c)
+ assert(event === Some(ConsumerActorRegistered(c, consumerOf(c))))
+ }
+
+ @Test def shouldCreateNoneFromConsumer = {
+ val event = ConsumerActorRegistered.forConsumer(Actor.actorOf[PlainActor])
+ assert(event === None)
+ }
+
+ @Test def shouldCreateSomeNonBlockingPublishRequestFromUntypedConsumer = {
+ val uc = UntypedActor.actorOf(classOf[SampleUntypedConsumer])
+ val event = ConsumerActorRegistered.forConsumer(uc)
+ assert(event === Some(ConsumerActorRegistered(uc, consumerOf(uc))))
+ }
+
+ @Test def shouldCreateSomeBlockingPublishRequestFromUntypedConsumer = {
+ val uc = UntypedActor.actorOf(classOf[SampleUntypedConsumerBlocking])
+ val event = ConsumerActorRegistered.forConsumer(uc)
+ assert(event === Some(ConsumerActorRegistered(uc, consumerOf(uc))))
+ }
+
+ @Test def shouldCreateNoneFromUntypedConsumer = {
+ val a = UntypedActor.actorOf(classOf[SampleUntypedActor])
+ val event = ConsumerActorRegistered.forConsumer(a)
+ assert(event === None)
+ }
+
+ private def consumerOf(ref: ActorRef) = ref.actor.asInstanceOf[Consumer]
+}
+
+object ConsumerRegisteredTest {
+ class ConsumerActor1 extends Actor with Consumer {
+ def endpointUri = "mock:test1"
+ protected def receive = null
+ }
+
+ class ConsumerActor2 extends Actor with Consumer {
+ def endpointUri = "mock:test2"
+ override def blocking = true
+ protected def receive = null
+ }
+
+ class PlainActor extends Actor {
+ protected def receive = null
+ }
+}
diff --git a/akka-camel/src/test/scala/akka/ConsumerScalaTest.scala b/akka-camel/src/test/scala/akka/ConsumerScalaTest.scala
new file mode 100644
index 0000000000..ddbe757a3f
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/ConsumerScalaTest.scala
@@ -0,0 +1,271 @@
+package akka.camel
+
+import java.util.concurrent.{TimeoutException, CountDownLatch, TimeUnit}
+
+import org.apache.camel.CamelExecutionException
+import org.apache.camel.builder.Builder
+import org.apache.camel.model.RouteDefinition
+import org.scalatest.{BeforeAndAfterAll, WordSpec}
+import org.scalatest.matchers.MustMatchers
+
+import akka.actor.Actor._
+import akka.actor._
+
+/**
+ * @author Martin Krasser
+ */
+class ConsumerScalaTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
+ import CamelContextManager.mandatoryTemplate
+ import ConsumerScalaTest._
+
+ var service: CamelService = _
+
+ override protected def beforeAll = {
+ ActorRegistry.shutdownAll
+ // create new CamelService instance
+ service = CamelServiceFactory.createCamelService
+ // Register publish requestor as listener
+ service.registerPublishRequestor
+ // 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
+ service.awaitEndpointActivation(1) {
+ service.start
+ } 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 {
+ mandatoryTemplate.requestBody("direct:publish-test-1", "msg1") must equal ("received msg1")
+ }
+ }
+ "not started" must {
+ "not have an associated endpoint in the CamelContext" in {
+ CamelContextManager.mandatoryContext.hasEndpoint("direct:publish-test-2") must be (null)
+ }
+ }
+ "started" must {
+ "support an in-out message exchange via its endpoint" in {
+ service.awaitEndpointActivation(1) {
+ consumer.start
+ } must be (true)
+ mandatoryTemplate.requestBody("direct:publish-test-2", "msg2") must equal ("received msg2")
+ }
+ "have an associated endpoint in the CamelContext" in {
+ CamelContextManager.mandatoryContext.hasEndpoint("direct:publish-test-2") must not be (null)
+ }
+ }
+ "stopped" must {
+ "not support an in-out message exchange via its endpoint" in {
+ service.awaitEndpointDeactivation(1) {
+ consumer.stop
+ } must be (true)
+ intercept[CamelExecutionException] {
+ mandatoryTemplate.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 {
+ service.awaitEndpointActivation(3) {
+ actor = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
+ } must be (true)
+ mandatoryTemplate.requestBodyAndHeader("direct:m2", "x", "test", "y") must equal ("m2: x y")
+ mandatoryTemplate.requestBodyAndHeader("direct:m3", "x", "test", "y") must equal ("m3: x y")
+ mandatoryTemplate.requestBodyAndHeader("direct:m4", "x", "test", "y") must equal ("m4: x y")
+ }
+ }
+ "stopped" must {
+ "not support in-out message exchanges via its endpoints" in {
+ service.awaitEndpointDeactivation(3) {
+ TypedActor.stop(actor)
+ } must be (true)
+ intercept[CamelExecutionException] {
+ mandatoryTemplate.requestBodyAndHeader("direct:m2", "x", "test", "y")
+ }
+ intercept[CamelExecutionException] {
+ mandatoryTemplate.requestBodyAndHeader("direct:m3", "x", "test", "y")
+ }
+ intercept[CamelExecutionException] {
+ mandatoryTemplate.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 {
+ service.awaitEndpointActivation(2) {
+ actor = TypedActor.newInstance(classOf[TestTypedConsumer], classOf[TestTypedConsumerImpl])
+ } must be (true)
+ mandatoryTemplate.requestBody("direct:publish-test-3", "x") must equal ("foo: x")
+ mandatoryTemplate.requestBody("direct:publish-test-4", "x") must equal ("bar: x")
+ }
+ }
+ "stopped" must {
+ "not support in-out message exchanges via its endpoints" in {
+ service.awaitEndpointDeactivation(2) {
+ TypedActor.stop(actor)
+ } must be (true)
+ intercept[CamelExecutionException] {
+ mandatoryTemplate.requestBody("direct:publish-test-3", "x")
+ }
+ intercept[CamelExecutionException] {
+ mandatoryTemplate.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 {
+ service.awaitEndpointActivation(1) {
+ consumer.start
+ } must be (true)
+ mandatoryTemplate.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 {
+ service.awaitEndpointDeactivation(1) {
+ consumer.stop
+ } must be (true)
+ intercept[CamelExecutionException] {
+ mandatoryTemplate.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 {
+ service.awaitEndpointActivation(1) {
+ actorOf(new TestBlocker("direct:publish-test-5")).start
+ } must be (true)
+
+ try {
+ mandatoryTemplate.requestBody("direct:publish-test-5", "msg3")
+ fail("expected TimoutException not thrown")
+ } catch {
+ case e => {
+ assert(e.getCause.isInstanceOf[TimeoutException])
+ }
+ }
+ }
+ }
+ }
+
+ "A responding, blocking consumer" when {
+ "activated with a custom error handler" must {
+ "handle thrown exceptions by generating a custom response" in {
+ service.awaitEndpointActivation(1) {
+ actorOf[ErrorHandlingConsumer].start
+ } must be (true)
+ mandatoryTemplate.requestBody("direct:error-handler-test", "hello") must equal ("error: hello")
+
+ }
+ }
+ "activated with a custom redelivery handler" must {
+ "handle thrown exceptions by redelivering the initial message" in {
+ service.awaitEndpointActivation(1) {
+ actorOf[RedeliveringConsumer].start
+ } must be (true)
+ mandatoryTemplate.requestBody("direct:redelivery-test", "hello") must equal ("accepted: hello")
+
+ }
+ }
+ }
+}
+
+object ConsumerScalaTest {
+ trait BlockingConsumer extends Consumer { self: Actor =>
+ override def blocking = true
+ }
+
+ 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 BlockingConsumer {
+ self.timeout = 1000
+ def endpointUri = uri
+ protected def receive = {
+ case msg: Message => { /* do not reply */ }
+ }
+ }
+
+ class ErrorHandlingConsumer extends Actor with BlockingConsumer {
+ def endpointUri = "direct:error-handler-test"
+
+ onRouteDefinition {rd: RouteDefinition =>
+ rd.onException(classOf[Exception]).handled(true).transform(Builder.exceptionMessage).end
+ }
+
+ protected def receive = {
+ case msg: Message => throw new Exception("error: %s" format msg.body)
+ }
+ }
+
+ class RedeliveringConsumer extends Actor with BlockingConsumer {
+ def endpointUri = "direct:redelivery-test"
+
+ onRouteDefinition {rd: RouteDefinition =>
+ rd.onException(classOf[Exception]).maximumRedeliveries(1).end
+ }
+
+ //
+ // first message to this actor is not valid and will be rejected
+ //
+
+ var valid = false
+
+ protected def receive = {
+ case msg: Message => try {
+ respondTo(msg)
+ } finally {
+ valid = true
+ }
+ }
+
+ private def respondTo(msg: Message) =
+ if (valid) self.reply("accepted: %s" format msg.body)
+ else throw new Exception("rejected: %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
+ }
+
+
+
+}
diff --git a/akka-camel/src/test/scala/akka/MessageJavaTest.scala b/akka-camel/src/test/scala/akka/MessageJavaTest.scala
new file mode 100644
index 0000000000..3c95887eb4
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/MessageJavaTest.scala
@@ -0,0 +1,5 @@
+package akka.camel
+
+import org.scalatest.junit.JUnitSuite
+
+class MessageJavaTest extends MessageJavaTestBase with JUnitSuite
diff --git a/akka-camel/src/test/scala/akka/MessageScalaTest.scala b/akka-camel/src/test/scala/akka/MessageScalaTest.scala
new file mode 100644
index 0000000000..5f43db596b
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/MessageScalaTest.scala
@@ -0,0 +1,83 @@
+package akka.camel
+
+import java.io.InputStream
+
+import org.apache.camel.NoTypeConversionAvailableException
+import org.junit.Assert._
+import org.junit.Test
+
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.junit.JUnitSuite
+
+
+class MessageScalaTest extends JUnitSuite with BeforeAndAfterAll {
+ override protected def beforeAll = CamelContextManager.init
+
+ @Test def shouldConvertDoubleBodyToString = {
+ assertEquals("1.4", Message(1.4).bodyAs[String])
+ }
+
+ @Test def shouldThrowExceptionWhenConvertingDoubleBodyToInputStream {
+ intercept[NoTypeConversionAvailableException] {
+ Message(1.4).bodyAs[InputStream]
+ }
+ }
+
+ @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"))
+ }
+
+ @Test def shouldReturnSubsetOfHeaders = {
+ val message = Message("test" , Map("A" -> "1", "B" -> "2"))
+ assertEquals(Map("B" -> "2"), message.headers(Set("B")))
+ }
+
+ @Test def shouldTransformBodyAndPreserveHeaders = {
+ assertEquals(
+ Message("ab", Map("A" -> "1")),
+ Message("a" , Map("A" -> "1")).transformBody((body: String) => body + "b"))
+ }
+
+ @Test def shouldConvertBodyAndPreserveHeaders = {
+ assertEquals(
+ Message("1.4", Map("A" -> "1")),
+ Message(1.4 , Map("A" -> "1")).setBodyAs[String])
+ }
+
+ @Test def shouldSetBodyAndPreserveHeaders = {
+ assertEquals(
+ Message("test2" , Map("A" -> "1")),
+ Message("test1" , Map("A" -> "1")).setBody("test2"))
+ }
+
+ @Test def shouldSetHeadersAndPreserveBody = {
+ assertEquals(
+ Message("test1" , Map("C" -> "3")),
+ Message("test1" , Map("A" -> "1")).setHeaders(Map("C" -> "3")))
+
+ }
+
+ @Test def shouldAddHeaderAndPreserveBodyAndHeaders = {
+ assertEquals(
+ Message("test1" , Map("A" -> "1", "B" -> "2")),
+ Message("test1" , Map("A" -> "1")).addHeader("B" -> "2"))
+ }
+
+ @Test def shouldAddHeadersAndPreserveBodyAndHeaders = {
+ assertEquals(
+ Message("test1" , Map("A" -> "1", "B" -> "2")),
+ Message("test1" , Map("A" -> "1")).addHeaders(Map("B" -> "2")))
+ }
+
+ @Test def shouldRemoveHeadersAndPreserveBodyAndRemainingHeaders = {
+ assertEquals(
+ Message("test1" , Map("A" -> "1")),
+ Message("test1" , Map("A" -> "1", "B" -> "2")).removeHeader("B"))
+ }
+}
diff --git a/akka-camel/src/test/scala/akka/ProducerFeatureTest.scala b/akka-camel/src/test/scala/akka/ProducerFeatureTest.scala
new file mode 100644
index 0000000000..66bcbe9186
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/ProducerFeatureTest.scala
@@ -0,0 +1,301 @@
+package 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 akka.actor.Actor._
+import akka.actor.{ActorRef, Actor, ActorRegistry}
+
+class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen {
+ import ProducerFeatureTest._
+
+ override protected def beforeAll = {
+ ActorRegistry.shutdownAll
+ CamelContextManager.init
+ CamelContextManager.mandatoryContext.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(new TestProducer("direct:producer-test-2", true))
+ producer.start
+
+ when("a test message is sent to the producer with !!")
+ val message = Message("test", Map(Message.MessageExchangeId -> "123"))
+ val result = producer !! message
+
+ then("a normal response should have been returned by the producer")
+ val expected = Message("received TEST", Map(Message.MessageExchangeId -> "123"))
+ assert(result === Some(expected))
+ }
+
+ scenario("produce message and receive failure response") {
+ given("a registered two-way producer")
+ val producer = actorOf(new TestProducer("direct:producer-test-2"))
+ 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 !! message).as[Failure]
+
+ then("a failure response should have been returned by the producer")
+ val expectedFailureText = result.get.cause.getMessage
+ val expectedHeaders = result.get.headers
+ assert(expectedFailureText === "failure")
+ assert(expectedHeaders === Map(Message.MessageExchangeId -> "123"))
+ }
+
+ scenario("produce message oneway") {
+ given("a registered one-way producer")
+ val producer = actorOf(new TestProducer("direct:producer-test-1", true) with Oneway)
+ producer.start
+
+ when("a test message is sent to the producer with !")
+ mockEndpoint.expectedBodiesReceived("TEST")
+ producer ! Message("test")
+
+ then("the test message should have been sent to mock:mock")
+ mockEndpoint.assertIsSatisfied
+ }
+
+ scenario("produce message twoway without sender reference") {
+ given("a registered two-way producer")
+ val producer = actorOf(new TestProducer("direct:producer-test-1"))
+ producer.start
+
+ when("a test message is sent to the producer with !")
+ mockEndpoint.expectedBodiesReceived("test")
+ producer ! Message("test")
+
+ then("there should be only a warning that there's no sender reference")
+ mockEndpoint.assertIsSatisfied
+ }
+ }
+
+ feature("Produce a message to an async Camel route") {
+
+ scenario("produce message and receive normal response") {
+ given("a registered two-way producer")
+ val producer = actorOf(new TestProducer("direct:producer-test-3"))
+ producer.start
+
+ when("a test message is sent to the producer with !!")
+ val message = Message("test", Map(Message.MessageExchangeId -> "123"))
+ val result = producer !! message
+
+ then("a normal response should have been returned by the producer")
+ val expected = Message("received test", Map(Message.MessageExchangeId -> "123"))
+ assert(result === Some(expected))
+ }
+
+ scenario("produce message and receive failure response") {
+ given("a registered two-way producer")
+ val producer = actorOf(new TestProducer("direct:producer-test-3"))
+ 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 !! message).as[Failure]
+
+ then("a failure response should have been returned by the producer")
+ val expectedFailureText = result.get.cause.getMessage
+ val expectedHeaders = result.get.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, 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
+
+ when("a test message is sent to the producer with !!")
+ val message = Message("test", Map(Message.MessageExchangeId -> "123"))
+ val result = producer !! message
+
+ then("a normal response should have been returned by the forward target")
+ val expected = Message("received test", Map(Message.MessageExchangeId -> "123", "test" -> "result"))
+ assert(result === Some(expected))
+ }
+
+ 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
+
+ when("a test message causing an exception is sent to the producer with !!")
+ val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
+ val result = (producer !! message).as[Failure]
+
+ then("a failure response should have been returned by the forward target")
+ val expectedFailureText = result.get.cause.getMessage
+ val expectedHeaders = result.get.headers
+ assert(expectedFailureText === "failure")
+ assert(expectedHeaders === Map(Message.MessageExchangeId -> "123", "test" -> "failure"))
+ }
+
+ 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
+
+ when("a test message is sent to the producer with !")
+ mockEndpoint.expectedBodiesReceived("received test")
+ val result = producer.!(Message("test"))(Some(producer))
+
+ then("a normal response should have been produced by the forward target")
+ mockEndpoint.assertIsSatisfied
+ }
+
+ 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
+
+ when("a test message causing an exception is sent to the producer with !")
+ mockEndpoint.expectedMessageCount(1)
+ mockEndpoint.message(0).body().isInstanceOf(classOf[Failure])
+ val result = producer.!(Message("fail"))(Some(producer))
+
+ then("a failure response should have been produced by the forward target")
+ mockEndpoint.assertIsSatisfied
+ }
+ }
+
+ feature("Produce a message to an async Camel route and then forward the 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
+
+ when("a test message is sent to the producer with !!")
+ val message = Message("test", Map(Message.MessageExchangeId -> "123"))
+ val result = producer !! message
+
+ then("a normal response should have been returned by the forward target")
+ val expected = Message("received test", Map(Message.MessageExchangeId -> "123", "test" -> "result"))
+ assert(result === Some(expected))
+ }
+
+ 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
+
+ when("a test message causing an exception is sent to the producer with !!")
+ val message = Message("fail", Map(Message.MessageExchangeId -> "123"))
+ val result = (producer !! message).as[Failure]
+
+ then("a failure response should have been returned by the forward target")
+ val expectedFailureText = result.get.cause.getMessage
+ val expectedHeaders = result.get.headers
+ assert(expectedFailureText === "failure")
+ assert(expectedHeaders === Map(Message.MessageExchangeId -> "123", "test" -> "failure"))
+ }
+
+ 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
+
+ when("a test message is sent to the producer with !")
+ mockEndpoint.expectedBodiesReceived("received test")
+ val result = producer.!(Message("test"))(Some(producer))
+
+ then("a normal response should have been produced by the forward target")
+ mockEndpoint.assertIsSatisfied
+ }
+
+ 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
+
+ when("a test message causing an exception is sent to the producer with !")
+ mockEndpoint.expectedMessageCount(1)
+ mockEndpoint.message(0).body().isInstanceOf(classOf[Failure])
+ val result = producer.!(Message("fail"))(Some(producer))
+
+ then("a failure response should have been produced by the forward target")
+ mockEndpoint.assertIsSatisfied
+ }
+ }
+
+ private def mockEndpoint = CamelContextManager.mandatoryContext.getEndpoint("mock:mock", classOf[MockEndpoint])
+}
+
+object ProducerFeatureTest {
+ class TestProducer(uri: String, upper: Boolean = false) extends Actor with Producer {
+ def endpointUri = uri
+ override protected def receiveBeforeProduce = {
+ case msg: Message => if (upper) msg.transformBody { body: String => body.toUpperCase } else msg
+ }
+ }
+
+ class TestForwarder(uri: String, target: ActorRef) extends Actor with Producer {
+ def endpointUri = uri
+ override protected def receiveAfterProduce = {
+ case msg => target forward msg
+ }
+ }
+
+ class TestResponder extends Actor {
+ protected def receive = {
+ case msg: Message => msg.body match {
+ case "fail" => self.reply(Failure(new Exception("failure"), msg.headers))
+ case _ => self.reply(msg.transformBody { body: String => "received %s" format body })
+ }
+ }
+ }
+
+ class ReplyingForwardTarget extends Actor {
+ protected def receive = {
+ case msg: Message =>
+ self.reply(msg.addHeader("test" -> "result"))
+ case msg: Failure =>
+ self.reply(Failure(msg.cause, msg.headers + ("test" -> "failure")))
+ }
+ }
+
+ class ProducingForwardTarget extends Actor with Producer with Oneway {
+ def endpointUri = "direct:forward-test-1"
+ }
+
+ class TestRoute extends RouteBuilder {
+ val responder = actorOf[TestResponder].start
+ def configure {
+ from("direct:forward-test-1").to("mock:mock")
+ // for one-way messaging tests
+ from("direct:producer-test-1").to("mock:mock")
+ // for two-way messaging tests (async)
+ from("direct:producer-test-3").to("actor:uuid:%s" format responder.uuid)
+ // for two-way messaging tests (sync)
+ from("direct:producer-test-2").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/akka/PublishRequestorTest.scala b/akka-camel/src/test/scala/akka/PublishRequestorTest.scala
new file mode 100644
index 0000000000..8578abef60
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/PublishRequestorTest.scala
@@ -0,0 +1,103 @@
+package akka.camel
+
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+
+import org.junit.{Before, After, Test}
+import org.scalatest.junit.JUnitSuite
+
+import akka.actor._
+import akka.actor.Actor._
+import akka.camel.support.{SetExpectedMessageCount => SetExpectedTestMessageCount, _}
+
+class PublishRequestorTest extends JUnitSuite {
+ import PublishRequestorTest._
+
+ var publisher: ActorRef = _
+ var requestor: ActorRef = _
+ var consumer: ActorRef = _
+
+ 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)
+ consumer = actorOf(new Actor with Consumer {
+ def endpointUri = "mock:test"
+ protected def receive = null
+ }).start
+ }
+
+ @After def tearDown = {
+ AspectInitRegistry.removeListener(requestor);
+ ActorRegistry.shutdownAll
+ }
+
+ @Test def shouldReceiveOneConsumerMethodRegisteredEvent = {
+ AspectInitRegistry.addListener(requestor)
+ val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
+ val obj = TypedActor.newInstance(classOf[SampleTypedSingleConsumer], classOf[SampleTypedSingleConsumerImpl])
+ assert(latch.await(5000, TimeUnit.MILLISECONDS))
+ val event = (publisher !! GetRetainedMessage).as[ConsumerMethodRegistered].get
+ assert(event.endpointUri === "direct:foo")
+ assert(event.typedActor === obj)
+ assert(event.methodName === "foo")
+ }
+
+ @Test def shouldReceiveOneConsumerMethodUnregisteredEvent = {
+ val obj = TypedActor.newInstance(classOf[SampleTypedSingleConsumer], classOf[SampleTypedSingleConsumerImpl])
+ val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
+ AspectInitRegistry.addListener(requestor)
+ TypedActor.stop(obj)
+ assert(latch.await(5000, TimeUnit.MILLISECONDS))
+ val event = (publisher !! GetRetainedMessage).as[ConsumerMethodUnregistered].get
+ assert(event.endpointUri === "direct:foo")
+ assert(event.typedActor === obj)
+ assert(event.methodName === "foo")
+ }
+
+ @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))
+ assert((publisher !! GetRetainedMessage) ===
+ Some(ConsumerActorRegistered(consumer, consumer.actor.asInstanceOf[Consumer])))
+ }
+
+ @Test def shouldReceiveOneConsumerUnregisteredEvent = {
+ val latch = (publisher !! SetExpectedTestMessageCount(1)).as[CountDownLatch].get
+ requestor ! ActorUnregistered(consumer)
+ assert(latch.await(5000, TimeUnit.MILLISECONDS))
+ assert((publisher !! GetRetainedMessage) ===
+ Some(ConsumerActorUnregistered(consumer, consumer.actor.asInstanceOf[Consumer])))
+ }
+}
+
+object PublishRequestorTest {
+ class PublisherMock extends TestActor with Retain with Countdown {
+ def handler = retain andThen countdown
+ }
+}
+
diff --git a/akka-camel/src/test/scala/akka/RemoteConsumerTest.scala b/akka-camel/src/test/scala/akka/RemoteConsumerTest.scala
new file mode 100644
index 0000000000..957080c2ec
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/RemoteConsumerTest.scala
@@ -0,0 +1,101 @@
+package akka.camel
+
+import java.util.concurrent.{CountDownLatch, TimeUnit}
+
+import org.scalatest.{GivenWhenThen, BeforeAndAfterAll, FeatureSpec}
+
+import akka.actor._
+import akka.actor.Actor._
+import akka.remote.{RemoteClient, RemoteServer}
+
+/**
+ * @author Martin Krasser
+ */
+class RemoteConsumerTest extends FeatureSpec with BeforeAndAfterAll with GivenWhenThen {
+ import CamelServiceManager._
+ import RemoteConsumerTest._
+
+ var server: RemoteServer = _
+
+ override protected def beforeAll = {
+ ActorRegistry.shutdownAll
+
+ startCamelService
+
+ server = new RemoteServer()
+ server.start(host, port)
+
+ Thread.sleep(1000)
+ }
+
+ override protected def afterAll = {
+ server.shutdown
+
+ stopCamelService
+
+ RemoteClient.shutdownAll
+ ActorRegistry.shutdownAll
+
+ Thread.sleep(1000)
+ }
+
+ 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")
+ assert(mandatoryService.awaitEndpointActivation(1) {
+ consumer !! "init"
+ })
+
+ then("the published consumer is accessible via its endpoint URI")
+ val response = CamelContextManager.mandatoryTemplate.requestBody("direct:remote-consumer", "test")
+ assert(response === "remote actor: test")
+ }
+ }
+
+ feature("Publish typed consumer on remote node") {
+ scenario("access published remote consumer method") {
+ given("a client-initiated remote typed consumer")
+ val consumer = TypedActor.newRemoteInstance(classOf[SampleRemoteTypedConsumer], classOf[SampleRemoteTypedConsumerImpl], host, port)
+
+ when("remote typed consumer publication is triggered")
+ assert(mandatoryService.awaitEndpointActivation(1) {
+ consumer.foo("init")
+ })
+ then("the published method is accessible via its endpoint URI")
+ val response = CamelContextManager.mandatoryTemplate.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")
+ assert(mandatoryService.awaitEndpointActivation(1) {
+ consumer.sendRequestReply(Message("init", Map("test" -> "init")))
+ })
+ then("the published untyped consumer is accessible via its endpoint URI")
+ val response = CamelContextManager.mandatoryTemplate.requestBodyAndHeader("direct:remote-untyped-consumer", "a", "test", "b")
+ assert(response === "a b")
+ }
+ }
+}
+
+object RemoteConsumerTest {
+ val host = "localhost"
+ val port = 7774
+
+ class RemoteConsumer extends RemoteActor(host, port) with Consumer {
+ def endpointUri = "direct:remote-consumer"
+
+ protected def receive = {
+ case "init" => self.reply("done")
+ case m: Message => self.reply("remote actor: %s" format m.body)
+ }
+ }
+}
diff --git a/akka-camel/src/test/scala/akka/UntypedProducerFeatureTest.scala b/akka-camel/src/test/scala/akka/UntypedProducerFeatureTest.scala
new file mode 100644
index 0000000000..18930f9ab4
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/UntypedProducerFeatureTest.scala
@@ -0,0 +1,98 @@
+package 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 akka.actor.UntypedActor._
+import akka.actor.ActorRegistry
+
+class UntypedProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach with GivenWhenThen {
+ import UntypedProducerFeatureTest._
+
+ override protected def beforeAll = {
+ ActorRegistry.shutdownAll
+ CamelContextManager.init
+ CamelContextManager.mandatoryContext.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.mandatoryContext.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/akka/component/ActorComponentFeatureTest.scala b/akka-camel/src/test/scala/akka/component/ActorComponentFeatureTest.scala
new file mode 100644
index 0000000000..993fe72096
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/component/ActorComponentFeatureTest.scala
@@ -0,0 +1,130 @@
+package 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 akka.actor.Actor._
+import akka.actor.{ActorRegistry, Actor}
+import akka.camel.{Failure, Message, CamelContextManager}
+import akka.camel.support._
+
+class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach {
+ import ActorComponentFeatureTest._
+
+ override protected def beforeAll = {
+ ActorRegistry.shutdownAll
+ CamelContextManager.init
+ CamelContextManager.mandatoryContext.addRoutes(new TestRoute)
+ CamelContextManager.start
+ }
+
+ override protected def afterAll = CamelContextManager.stop
+
+ override protected def afterEach = {
+ ActorRegistry.shutdownAll
+ mockEndpoint.reset
+ }
+
+ feature("Communicate with an actor via an actor:uuid endpoint") {
+ import CamelContextManager.mandatoryTemplate
+
+ scenario("one-way communication") {
+ val actor = actorOf[Tester1].start
+ val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ mandatoryTemplate.sendBody("actor:uuid:%s" format actor.uuid, "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(mandatoryTemplate.requestBody("actor:uuid:%s" format actor.uuid, "Martin") === "Hello Martin")
+ }
+
+ scenario("two-way communication with timeout") {
+ val actor = actorOf[Tester3].start
+ intercept[RuntimeCamelException] {
+ mandatoryTemplate.requestBody("actor:uuid:%s?blocking=true" format actor.uuid, "Martin")
+ }
+ }
+
+ scenario("two-way communication via a custom route with failure response") {
+ mockEndpoint.expectedBodiesReceived("whatever")
+ mandatoryTemplate.requestBody("direct:failure-test-1", "whatever")
+ mockEndpoint.assertIsSatisfied
+ }
+
+ scenario("two-way communication via a custom route with exception") {
+ mockEndpoint.expectedBodiesReceived("whatever")
+ mandatoryTemplate.requestBody("direct:failure-test-2", "whatever")
+ mockEndpoint.assertIsSatisfied
+ }
+ }
+
+ feature("Communicate with an actor via an actor:id endpoint") {
+ import CamelContextManager.mandatoryTemplate
+
+ scenario("one-way communication") {
+ val actor = actorOf[Tester1].start
+ val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ mandatoryTemplate.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(mandatoryTemplate.requestBody("actor:%s" format actor.id, "Martin") === "Hello Martin")
+ }
+
+ scenario("two-way communication via a custom route") {
+ val actor = actorOf[CustomIdActor].start
+ assert(mandatoryTemplate.requestBody("direct:custom-id-test-1", "Martin") === "Received Martin")
+ assert(mandatoryTemplate.requestBody("direct:custom-id-test-2", "Martin") === "Received Martin")
+ }
+ }
+
+ private def mockEndpoint = CamelContextManager.mandatoryContext.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/akka/component/ActorComponentTest.scala b/akka-camel/src/test/scala/akka/component/ActorComponentTest.scala
new file mode 100644
index 0000000000..0af9f00213
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/component/ActorComponentTest.scala
@@ -0,0 +1,79 @@
+package akka.camel.component
+
+import org.apache.camel.{Endpoint, AsyncProcessor}
+import org.apache.camel.impl.DefaultCamelContext
+import org.junit._
+import org.scalatest.junit.JUnitSuite
+
+import akka.actor.uuidFrom
+
+class ActorComponentTest extends JUnitSuite {
+ val component: ActorComponent = ActorComponentTest.actorComponent
+
+ def testUUID = "93da8c80-c3fd-11df-abed-60334b120057"
+
+ @Test def shouldCreateEndpointWithIdDefined = {
+ val ep1: ActorEndpoint = component.createEndpoint("actor:abc").asInstanceOf[ActorEndpoint]
+ val ep2: ActorEndpoint = component.createEndpoint("actor:id:abc").asInstanceOf[ActorEndpoint]
+ assert(ep1.idValue === Some("abc"))
+ assert(ep2.idValue === Some("abc"))
+ assert(ep1.idType === "id")
+ assert(ep2.idType === "id")
+ assert(!ep1.blocking)
+ assert(!ep2.blocking)
+ }
+
+ @Test def shouldCreateEndpointWithIdTemplate = {
+ val ep: ActorEndpoint = component.createEndpoint("actor:id:").asInstanceOf[ActorEndpoint]
+ assert(ep.idValue === None)
+ assert(ep.idType === "id")
+ assert(!ep.blocking)
+ }
+
+ @Test def shouldCreateEndpointWithIdTemplateAndBlockingSet = {
+ val ep: ActorEndpoint = component.createEndpoint("actor:id:?blocking=true").asInstanceOf[ActorEndpoint]
+ assert(ep.idValue === None)
+ assert(ep.idType === "id")
+ assert(ep.blocking)
+ }
+
+ @Test def shouldCreateEndpointWithUuidDefined = {
+ val ep: ActorEndpoint = component.createEndpoint("actor:uuid:%s" format testUUID).asInstanceOf[ActorEndpoint]
+ assert(ep.idValue === Some(testUUID))
+ assert(ep.idType === "uuid")
+ assert(!ep.blocking)
+ }
+
+ @Test def shouldCreateEndpointWithUuidTemplate = {
+ val ep: ActorEndpoint = component.createEndpoint("actor:uuid:").asInstanceOf[ActorEndpoint]
+ assert(ep.idValue === None)
+ assert(ep.idType === "uuid")
+ assert(!ep.blocking)
+ }
+
+ @Test def shouldCreateEndpointWithUuidTemplateandBlockingSet = {
+ val ep: ActorEndpoint = component.createEndpoint("actor:uuid:?blocking=true").asInstanceOf[ActorEndpoint]
+ assert(ep.idValue === None)
+ assert(ep.idType === "uuid")
+ assert(ep.blocking)
+ }
+
+ @Test def shouldCreateEndpointWithBlockingSet = {
+ val ep: ActorEndpoint = component.createEndpoint("actor:uuid:%s?blocking=true" format testUUID).asInstanceOf[ActorEndpoint]
+ assert(ep.idValue === Some(testUUID))
+ assert(ep.idType === "uuid")
+ assert(ep.blocking)
+ }
+}
+
+object ActorComponentTest {
+ def actorComponent = {
+ val component = new ActorComponent
+ component.setCamelContext(new DefaultCamelContext)
+ component
+ }
+
+ def actorEndpoint(uri:String) = actorComponent.createEndpoint(uri)
+ def actorProducer(endpoint: Endpoint) = endpoint.createProducer
+ def actorAsyncProducer(endpoint: Endpoint) = endpoint.createProducer.asInstanceOf[AsyncProcessor]
+}
diff --git a/akka-camel/src/test/scala/akka/component/ActorProducerTest.scala b/akka-camel/src/test/scala/akka/component/ActorProducerTest.scala
new file mode 100644
index 0000000000..62ccf9cb0e
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/component/ActorProducerTest.scala
@@ -0,0 +1,230 @@
+package akka.camel.component
+
+import ActorComponentTest._
+
+import java.util.concurrent.{CountDownLatch, TimeoutException, TimeUnit}
+
+import org.apache.camel.{AsyncCallback, ExchangePattern}
+
+import org.junit.{After, Test}
+import org.scalatest.junit.JUnitSuite
+import org.scalatest.BeforeAndAfterAll
+
+import akka.actor.Actor._
+import akka.actor.ActorRegistry
+import akka.camel.{Failure, Message}
+import akka.camel.support._
+
+class ActorProducerTest extends JUnitSuite with BeforeAndAfterAll {
+ import ActorProducerTest._
+
+ @After def tearDown = ActorRegistry.shutdownAll
+
+ @Test def shouldSendMessageToActorWithSyncProcessor = {
+ val actor = actorOf[Tester1].start
+ val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
+ val exchange = endpoint.createExchange(ExchangePattern.InOnly)
+ exchange.getIn.setBody("Martin")
+ exchange.getIn.setHeader("k1", "v1")
+ actorProducer(endpoint).process(exchange)
+ assert(latch.await(5000, TimeUnit.MILLISECONDS))
+ val reply = (actor !! GetRetainedMessage).get.asInstanceOf[Message]
+ assert(reply.body === "Martin")
+ assert(reply.headers === Map(Message.MessageExchangeId -> exchange.getExchangeId, "k1" -> "v1"))
+ }
+
+ @Test def shouldSendMessageToActorWithAsyncProcessor = {
+ val actor = actorOf[Tester1].start
+ val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
+ val exchange = endpoint.createExchange(ExchangePattern.InOnly)
+ exchange.getIn.setBody("Martin")
+ exchange.getIn.setHeader("k1", "v1")
+ actorAsyncProducer(endpoint).process(exchange, expectSyncCompletion)
+ assert(latch.await(5000, TimeUnit.MILLISECONDS))
+ val reply = (actor !! GetRetainedMessage).get.asInstanceOf[Message]
+ assert(reply.body === "Martin")
+ assert(reply.headers === Map(Message.MessageExchangeId -> exchange.getExchangeId, "k1" -> "v1"))
+ }
+
+ @Test def shouldSendMessageToActorAndReceiveResponseWithSyncProcessor = {
+ val actor = actorOf(new Tester2 {
+ override def response(msg: Message) = Message(super.response(msg), Map("k2" -> "v2"))
+ }).start
+ val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
+ val exchange = endpoint.createExchange(ExchangePattern.InOut)
+ exchange.getIn.setBody("Martin")
+ exchange.getIn.setHeader("k1", "v1")
+ actorProducer(endpoint).process(exchange)
+ assert(exchange.getOut.getBody === "Hello Martin")
+ assert(exchange.getOut.getHeader("k2") === "v2")
+ }
+
+ @Test def shouldSendMessageToActorAndReceiveResponseWithAsyncProcessor = {
+ val actor = actorOf(new Tester2 {
+ override def response(msg: Message) = Message(super.response(msg), Map("k2" -> "v2"))
+ }).start
+ val completion = expectAsyncCompletion
+ val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
+ val exchange = endpoint.createExchange(ExchangePattern.InOut)
+ exchange.getIn.setBody("Martin")
+ exchange.getIn.setHeader("k1", "v1")
+ actorAsyncProducer(endpoint).process(exchange, completion)
+ assert(completion.latch.await(5000, TimeUnit.MILLISECONDS))
+ assert(exchange.getOut.getBody === "Hello Martin")
+ assert(exchange.getOut.getHeader("k2") === "v2")
+ }
+
+ @Test def shouldSendMessageToActorAndReceiveFailureWithAsyncProcessor = {
+ val actor = actorOf(new Tester2 {
+ override def response(msg: Message) = Failure(new Exception("testmsg"), Map("k3" -> "v3"))
+ }).start
+ val completion = expectAsyncCompletion
+ val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
+ val exchange = endpoint.createExchange(ExchangePattern.InOut)
+ exchange.getIn.setBody("Martin")
+ exchange.getIn.setHeader("k1", "v1")
+ actorAsyncProducer(endpoint).process(exchange, completion)
+ assert(completion.latch.await(5000, TimeUnit.MILLISECONDS))
+ assert(exchange.getException.getMessage === "testmsg")
+ assert(exchange.getOut.getBody === null)
+ assert(exchange.getOut.getHeader("k3") === null) // headers from failure message are currently ignored
+ }
+
+ @Test def shouldDynamicallyRouteMessageToActorWithDefaultId = {
+ val actor1 = actorOf[Tester1]
+ val actor2 = actorOf[Tester1]
+ actor1.id = "x"
+ actor2.id = "y"
+ actor1.start
+ actor2.start
+ val latch1 = (actor1 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val latch2 = (actor2 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val endpoint = actorEndpoint("actor:id:%s" format actor1.id)
+ val exchange1 = endpoint.createExchange(ExchangePattern.InOnly)
+ val exchange2 = endpoint.createExchange(ExchangePattern.InOnly)
+ exchange1.getIn.setBody("Test1")
+ exchange2.getIn.setBody("Test2")
+ exchange2.getIn.setHeader(ActorComponent.ActorIdentifier, actor2.id)
+ actorProducer(endpoint).process(exchange1)
+ actorProducer(endpoint).process(exchange2)
+ assert(latch1.await(5, TimeUnit.SECONDS))
+ assert(latch2.await(5, TimeUnit.SECONDS))
+ val reply1 = (actor1 !! GetRetainedMessage).get.asInstanceOf[Message]
+ val reply2 = (actor2 !! GetRetainedMessage).get.asInstanceOf[Message]
+ assert(reply1.body === "Test1")
+ assert(reply2.body === "Test2")
+ }
+
+ @Test def shouldDynamicallyRouteMessageToActorWithoutDefaultId = {
+ val actor1 = actorOf[Tester1]
+ val actor2 = actorOf[Tester1]
+ actor1.id = "x"
+ actor2.id = "y"
+ actor1.start
+ actor2.start
+ val latch1 = (actor1 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val latch2 = (actor2 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val endpoint = actorEndpoint("actor:id:")
+ val exchange1 = endpoint.createExchange(ExchangePattern.InOnly)
+ val exchange2 = endpoint.createExchange(ExchangePattern.InOnly)
+ exchange1.getIn.setBody("Test1")
+ exchange2.getIn.setBody("Test2")
+ exchange1.getIn.setHeader(ActorComponent.ActorIdentifier, actor1.id)
+ exchange2.getIn.setHeader(ActorComponent.ActorIdentifier, actor2.id)
+ actorProducer(endpoint).process(exchange1)
+ actorProducer(endpoint).process(exchange2)
+ assert(latch1.await(5, TimeUnit.SECONDS))
+ assert(latch2.await(5, TimeUnit.SECONDS))
+ val reply1 = (actor1 !! GetRetainedMessage).get.asInstanceOf[Message]
+ val reply2 = (actor2 !! GetRetainedMessage).get.asInstanceOf[Message]
+ assert(reply1.body === "Test1")
+ assert(reply2.body === "Test2")
+ }
+
+ @Test def shouldDynamicallyRouteMessageToActorWithDefaultUuid = {
+ val actor1 = actorOf[Tester1].start
+ val actor2 = actorOf[Tester1].start
+ val latch1 = (actor1 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val latch2 = (actor2 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val endpoint = actorEndpoint("actor:uuid:%s" format actor1.uuid)
+ val exchange1 = endpoint.createExchange(ExchangePattern.InOnly)
+ val exchange2 = endpoint.createExchange(ExchangePattern.InOnly)
+ exchange1.getIn.setBody("Test1")
+ exchange2.getIn.setBody("Test2")
+ exchange2.getIn.setHeader(ActorComponent.ActorIdentifier, actor2.uuid.toString)
+ actorProducer(endpoint).process(exchange1)
+ actorProducer(endpoint).process(exchange2)
+ assert(latch1.await(5, TimeUnit.SECONDS))
+ assert(latch2.await(5, TimeUnit.SECONDS))
+ val reply1 = (actor1 !! GetRetainedMessage).get.asInstanceOf[Message]
+ val reply2 = (actor2 !! GetRetainedMessage).get.asInstanceOf[Message]
+ assert(reply1.body === "Test1")
+ assert(reply2.body === "Test2")
+ }
+
+ @Test def shouldDynamicallyRouteMessageToActorWithoutDefaultUuid = {
+ val actor1 = actorOf[Tester1].start
+ val actor2 = actorOf[Tester1].start
+ val latch1 = (actor1 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val latch2 = (actor2 !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val endpoint = actorEndpoint("actor:uuid:")
+ val exchange1 = endpoint.createExchange(ExchangePattern.InOnly)
+ val exchange2 = endpoint.createExchange(ExchangePattern.InOnly)
+ exchange1.getIn.setBody("Test1")
+ exchange2.getIn.setBody("Test2")
+ exchange1.getIn.setHeader(ActorComponent.ActorIdentifier, actor1.uuid)
+ exchange2.getIn.setHeader(ActorComponent.ActorIdentifier, actor2.uuid.toString)
+ actorProducer(endpoint).process(exchange1)
+ actorProducer(endpoint).process(exchange2)
+ assert(latch1.await(5, TimeUnit.SECONDS))
+ assert(latch2.await(5, TimeUnit.SECONDS))
+ val reply1 = (actor1 !! GetRetainedMessage).get.asInstanceOf[Message]
+ val reply2 = (actor2 !! GetRetainedMessage).get.asInstanceOf[Message]
+ assert(reply1.body === "Test1")
+ assert(reply2.body === "Test2")
+ }
+
+ @Test def shouldThrowExceptionWhenIdNotSet: Unit = {
+ val actor = actorOf[Tester1].start
+ val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val endpoint = actorEndpoint("actor:id:")
+ intercept[ActorIdentifierNotSetException] {
+ actorProducer(endpoint).process(endpoint.createExchange(ExchangePattern.InOnly))
+ }
+ }
+
+ @Test def shouldThrowExceptionWhenUuidNotSet: Unit = {
+ val actor = actorOf[Tester1].start
+ val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
+ val endpoint = actorEndpoint("actor:uuid:")
+ intercept[ActorIdentifierNotSetException] {
+ actorProducer(endpoint).process(endpoint.createExchange(ExchangePattern.InOnly))
+ }
+ }
+
+ @Test def shouldSendMessageToActorAndTimeout(): Unit = {
+ val actor = actorOf[Tester3].start
+ val endpoint = actorEndpoint("actor:uuid:%s" format actor.uuid)
+ val exchange = endpoint.createExchange(ExchangePattern.InOut)
+ exchange.getIn.setBody("Martin")
+ intercept[TimeoutException] {
+ endpoint.createProducer.process(exchange)
+ }
+ }
+}
+
+object ActorProducerTest {
+ def expectSyncCompletion = new AsyncCallback {
+ def done(doneSync: Boolean) = assert(doneSync)
+ }
+
+ def expectAsyncCompletion = new AsyncCallback {
+ val latch = new CountDownLatch(1);
+ def done(doneSync: Boolean) = {
+ assert(!doneSync)
+ latch.countDown
+ }
+ }
+}
diff --git a/akka-camel/src/test/scala/akka/component/TypedActorComponentFeatureTest.scala b/akka-camel/src/test/scala/akka/component/TypedActorComponentFeatureTest.scala
new file mode 100644
index 0000000000..e6e1293a54
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/component/TypedActorComponentFeatureTest.scala
@@ -0,0 +1,108 @@
+package akka.camel.component
+
+import org.apache.camel._
+import org.apache.camel.builder.RouteBuilder
+import org.apache.camel.impl.{DefaultCamelContext, SimpleRegistry}
+import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FeatureSpec}
+
+import akka.actor.{ActorRegistry, TypedActor}
+import akka.camel._
+
+/**
+ * @author Martin Krasser
+ */
+class TypedActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach {
+ import TypedActorComponentFeatureTest._
+ import CamelContextManager.mandatoryTemplate
+
+ 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.mandatoryContext.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 = mandatoryTemplate.requestBodyAndHeader("%s:tc?method=m2" format InternalSchema, "x", "test", "y")
+ val result2 = mandatoryTemplate.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 = mandatoryTemplate.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 = mandatoryTemplate.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 = mandatoryTemplate.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] {
+ mandatoryTemplate.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 = mandatoryTemplate.requestBody("typed-actor:ta?method=foo", "test")
+ assert(result === "foo: test")
+ }
+
+ scenario("two-way communication with method returning String via custom route") {
+ val result = mandatoryTemplate.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/akka/support/TestSupport.scala b/akka-camel/src/test/scala/akka/support/TestSupport.scala
new file mode 100644
index 0000000000..4744d774f5
--- /dev/null
+++ b/akka-camel/src/test/scala/akka/support/TestSupport.scala
@@ -0,0 +1,81 @@
+package akka.camel.support
+
+import java.util.concurrent.{TimeUnit, CountDownLatch}
+
+import collection.mutable.Buffer
+
+import akka.camel.Message
+import akka.actor.Actor
+
+import TestSupport._
+
+object TestSupport {
+ type Handler = PartialFunction[Any, Any]
+}
+
+trait TestActor extends Actor {
+ def receive = {
+ case msg => {
+ handler(msg)
+ }
+ }
+
+ def handler: Handler
+}
+
+class Tester1 extends TestActor with Retain with Countdown {
+ def handler = retain andThen countdown
+}
+
+class Tester2 extends TestActor with Respond {
+ def handler = respond
+}
+
+class Tester3 extends TestActor with Noop {
+ self.timeout = 1
+ def handler = noop
+}
+
+trait Countdown { this: Actor =>
+ var latch: CountDownLatch = new CountDownLatch(0)
+ def countdown: Handler = {
+ case SetExpectedMessageCount(num) => {
+ latch = new CountDownLatch(num)
+ self.reply(latch)
+ }
+ case msg => latch.countDown
+ }
+}
+
+trait Respond { this: Actor =>
+ def respond: Handler = {
+ case msg: Message => self.reply(response(msg))
+ }
+
+ def response(msg: Message): Any = "Hello %s" format msg.body
+}
+
+trait Retain { this: Actor =>
+ val messages = Buffer[Any]()
+
+ def retain: Handler = {
+ case GetRetainedMessage => self.reply(messages.last)
+ case GetRetainedMessages(p) => self.reply(messages.toList.filter(p))
+ case msg => {
+ messages += msg
+ msg
+ }
+ }
+}
+
+trait Noop { this: Actor =>
+ def noop: Handler = {
+ case msg => msg
+ }
+}
+
+case class SetExpectedMessageCount(num: Int)
+case class GetRetainedMessage()
+case class GetRetainedMessages(p: Any => Boolean) {
+ def this() = this(_ => true)
+}
diff --git a/akka-http/src/main/scala/akka/AkkaBroadcaster.scala b/akka-http/src/main/scala/akka/AkkaBroadcaster.scala
new file mode 100644
index 0000000000..fd0f76631a
--- /dev/null
+++ b/akka-http/src/main/scala/akka/AkkaBroadcaster.scala
@@ -0,0 +1,40 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.comet
+
+import org.atmosphere.cpr.{AtmosphereResourceEvent, AtmosphereResource}
+
+import akka.actor.Actor._
+import akka.actor.Actor
+import akka.dispatch.Dispatchers
+import org.atmosphere.jersey.util.JerseyBroadcasterUtil
+
+object AkkaBroadcaster {
+ val broadcasterDispatcher = Dispatchers.fromConfig("akka.rest.comet-dispatcher")
+
+ type Event = AtmosphereResourceEvent[_,_]
+ type Resource = AtmosphereResource[_,_]
+}
+
+class AkkaBroadcaster extends org.atmosphere.jersey.util.JerseySimpleBroadcaster {
+ import AkkaBroadcaster._
+
+ //FIXME should be supervised
+ lazy val caster = actorOf(new Actor {
+ self.dispatcher = broadcasterDispatcher
+ def receive = {
+ case (r: Resource,e: Event) => JerseyBroadcasterUtil.broadcast(r,e)
+ }
+ }).start
+
+ override def destroy {
+ super.destroy
+ caster.stop
+ }
+
+ protected override def broadcast(r: Resource, e : Event) {
+ caster ! ((r,e))
+ }
+}
diff --git a/akka-http/src/main/scala/akka/AkkaCometServlet.scala b/akka-http/src/main/scala/akka/AkkaCometServlet.scala
new file mode 100644
index 0000000000..5b15096c92
--- /dev/null
+++ b/akka-http/src/main/scala/akka/AkkaCometServlet.scala
@@ -0,0 +1,101 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.comet
+
+import akka.util.Logging
+
+import java.util.{List => JList}
+import javax.servlet.{ServletConfig,ServletContext}
+import javax.servlet.http.{HttpServletRequest, HttpServletResponse}
+import com.sun.jersey.spi.container.servlet.ServletContainer
+
+import org.atmosphere.container.GrizzlyCometSupport
+import org.atmosphere.cpr.{AtmosphereServlet, AtmosphereServletProcessor, AtmosphereResource, AtmosphereResourceEvent,CometSupport,CometSupportResolver,DefaultCometSupportResolver}
+import org.atmosphere.handler.{ReflectorServletProcessor, AbstractReflectorAtmosphereHandler}
+
+class AtmosphereRestServlet extends ServletContainer with AtmosphereServletProcessor {
+ //Delegate to implement the behavior for AtmosphereHandler
+ private val handler = new AbstractReflectorAtmosphereHandler {
+ override def onRequest(event: AtmosphereResource[HttpServletRequest, HttpServletResponse]) {
+ if (event ne null) {
+ event.getRequest.setAttribute(AtmosphereServlet.ATMOSPHERE_RESOURCE, event)
+ event.getRequest.setAttribute(AtmosphereServlet.ATMOSPHERE_HANDLER, this)
+ service(event.getRequest, event.getResponse)
+ }
+ }
+ }
+
+ override def onStateChange(event: AtmosphereResourceEvent[HttpServletRequest, HttpServletResponse]) {
+ if (event ne null) handler onStateChange event
+ }
+
+ override def onRequest(resource: AtmosphereResource[HttpServletRequest, HttpServletResponse]) {
+ handler onRequest resource
+ }
+ }
+
+/**
+ * Akka's Comet servlet to be used when deploying actors exposed as Comet (and REST) services in a
+ * standard servlet container, e.g. not using the Akka Kernel.
+ *
+ * Used by the Akka Kernel to bootstrap REST and Comet.
+ */
+class AkkaServlet extends AtmosphereServlet {
+ import akka.config.Config.{config => c}
+
+ /*
+ * Configure Atmosphere and Jersey (default, fall-back values)
+ */
+ addInitParameter(AtmosphereServlet.DISABLE_ONSTATE_EVENT,"true")
+ addInitParameter(AtmosphereServlet.BROADCASTER_CLASS,classOf[AkkaBroadcaster].getName)
+ addInitParameter(AtmosphereServlet.PROPERTY_USE_STREAM,"true")
+ addInitParameter("com.sun.jersey.config.property.packages",c.getList("akka.rest.resource_packages").mkString(";"))
+ addInitParameter("com.sun.jersey.spi.container.ResourceFilters",c.getList("akka.rest.filters").mkString(","))
+
+ c.getInt("akka.rest.maxInactiveActivity") foreach { value => addInitParameter(CometSupport.MAX_INACTIVE,value.toString) }
+ c.getString("akka.rest.cometSupport") foreach { value => addInitParameter("cometSupport",value) }
+
+ /*
+ * Provide a fallback for default values
+ */
+ override def getInitParameter(key : String) =
+ Option(super.getInitParameter(key)).getOrElse(initParams get key)
+
+ /*
+ * Provide a fallback for default values
+ */
+ override def getInitParameterNames() = {
+ import scala.collection.JavaConversions._
+ initParams.keySet.iterator ++ super.getInitParameterNames
+ }
+
+ /**
+ * We override this to avoid Atmosphere looking for it's atmosphere.xml file
+ * Instead we specify what semantics we want in code.
+ */
+ override def loadConfiguration(sc: ServletConfig) {
+ config.setSupportSession(false)
+ isBroadcasterSpecified = true
+
+ //The bridge between Atmosphere and Jersey
+ val servlet = new AtmosphereRestServlet {
+ //These are needed to make sure that Jersey is reading the config from the outer servlet
+ override def getInitParameter(key : String) = AkkaServlet.this.getInitParameter(key)
+ override def getInitParameterNames() = AkkaServlet.this.getInitParameterNames()
+ }
+
+ addAtmosphereHandler("/*", servlet, new AkkaBroadcaster)
+ }
+
+ override lazy val createCometSupportResolver: CometSupportResolver = new DefaultCometSupportResolver(config) {
+ import scala.collection.JavaConversions._
+
+ lazy val desiredCometSupport =
+ Option(AkkaServlet.this.getInitParameter("cometSupport")) filter testClassExists map newCometSupport
+
+ override def resolve(useNativeIfPossible : Boolean, useBlockingAsDefault : Boolean) : CometSupport[_ <: AtmosphereResource[_,_]] =
+ desiredCometSupport.getOrElse(super.resolve(useNativeIfPossible, useBlockingAsDefault))
+ }
+}
diff --git a/akka-http/src/main/scala/akka/AkkaLoader.scala b/akka-http/src/main/scala/akka/AkkaLoader.scala
new file mode 100644
index 0000000000..22ff40016c
--- /dev/null
+++ b/akka-http/src/main/scala/akka/AkkaLoader.scala
@@ -0,0 +1,77 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.servlet
+
+import akka.config.Config
+import akka.util.{Logging, Bootable}
+import akka.actor.Actor
+
+/*
+ * This class is responsible for booting up a stack of bundles and then shutting them down
+ */
+class AkkaLoader extends Logging {
+ @volatile private var hasBooted = false
+
+ @volatile private var _bundles: Option[Bootable] = None
+
+ def bundles = _bundles;
+
+ /*
+ * Boot initializes the specified bundles
+ */
+ def boot(withBanner: Boolean, b : Bootable): Unit = synchronized {
+ if (!hasBooted) {
+ if (withBanner) printBanner
+ log.info("Starting Akka...")
+ b.onLoad
+ Thread.currentThread.setContextClassLoader(getClass.getClassLoader)
+ log.info("Akka started successfully")
+ hasBooted = true
+ _bundles = Some(b)
+ }
+ }
+
+ /*
+ * Shutdown, well, shuts down the bundles used in boot
+ */
+ def shutdown = synchronized {
+ if (hasBooted) {
+ log.info("Shutting down Akka...")
+ _bundles.foreach(_.onUnload)
+ _bundles = None
+ Actor.shutdownHook.run
+ log.info("Akka succesfully shut down")
+ }
+ }
+
+ private def printBanner = {
+ log.info("==================================================")
+ log.info(" t")
+ log.info(" t t t")
+ log.info(" t t tt t")
+ log.info(" tt t t tt t")
+ log.info(" t ttttttt t ttt t")
+ log.info(" t tt ttt t ttt t")
+ log.info(" t t ttt t ttt t t")
+ log.info(" tt t ttt ttt ttt t")
+ log.info(" t t ttt ttt t tt t")
+ log.info(" t ttt ttt t t")
+ log.info(" tt ttt ttt t")
+ log.info(" ttt ttt")
+ log.info(" tttttttt ttt ttt ttt ttt tttttttt")
+ log.info(" ttt tt ttt ttt ttt ttt ttt ttt")
+ log.info(" ttt ttt ttt ttt ttt ttt ttt ttt")
+ log.info(" ttt ttt ttt ttt ttt tt ttt ttt")
+ log.info(" tttt ttttttttt tttttttt tttt")
+ log.info(" ttttttttt ttt ttt ttt ttt ttttttttt")
+ log.info(" ttt ttt ttt ttt ttt ttt ttt ttt")
+ log.info(" ttt ttt ttt ttt ttt ttt ttt ttt")
+ log.info(" ttt tt ttt ttt ttt ttt ttt ttt")
+ log.info(" tttttttt ttt ttt ttt ttt tttttttt")
+ log.info("==================================================")
+ log.info(" Running version %s", Config.VERSION)
+ log.info("==================================================")
+ }
+}
diff --git a/akka-http/src/main/scala/akka/DefaultAkkaLoader.scala b/akka-http/src/main/scala/akka/DefaultAkkaLoader.scala
new file mode 100644
index 0000000000..b0bc3ad020
--- /dev/null
+++ b/akka-http/src/main/scala/akka/DefaultAkkaLoader.scala
@@ -0,0 +1,29 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.http
+
+import akka.config.Config
+import akka.util.{Logging, Bootable}
+import akka.camel.CamelService
+import akka.remote.BootableRemoteActorService
+import akka.actor.BootableActorLoaderService
+import akka.servlet.AkkaLoader
+
+class DefaultAkkaLoader extends AkkaLoader {
+ def boot(): Unit = boot(true,
+ new EmbeddedAppServer with BootableActorLoaderService
+ with BootableRemoteActorService
+ with CamelService)
+}
+
+
+/**
+ * Can be used to boot Akka
+ *
+ * java -cp ... akka.http.Main
+ */
+object Main extends DefaultAkkaLoader {
+ def main(args: Array[String]) = boot
+}
diff --git a/akka-http/src/main/scala/akka/EmbeddedAppServer.scala b/akka-http/src/main/scala/akka/EmbeddedAppServer.scala
new file mode 100644
index 0000000000..ea68bf1106
--- /dev/null
+++ b/akka-http/src/main/scala/akka/EmbeddedAppServer.scala
@@ -0,0 +1,72 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.http
+
+import javax.ws.rs.core.UriBuilder
+import javax.servlet.ServletConfig
+import java.io.File
+
+import akka.actor.BootableActorLoaderService
+import akka.util.{Bootable, Logging}
+import akka.comet.AkkaServlet
+
+import org.eclipse.jetty.xml.XmlConfiguration
+import org.eclipse.jetty.server.{Handler, Server}
+import org.eclipse.jetty.server.handler.{HandlerList, HandlerCollection, ContextHandler}
+
+/**
+ * Handles the Akka Comet Support (load/unload)
+ */
+trait EmbeddedAppServer extends Bootable with Logging {
+ self : BootableActorLoaderService =>
+
+ import akka.config.Config._
+
+ val REST_HOSTNAME = config.getString("akka.rest.hostname", "localhost")
+ val REST_PORT = config.getInt("akka.rest.port", 9998)
+
+ protected var server: Option[Server] = None
+
+ abstract override def onLoad = {
+ super.onLoad
+ if (config.getBool("akka.rest.service", true)) {
+ log.info("Attempting to start Akka REST service (Jersey)")
+
+ System.setProperty("jetty.port",REST_PORT.toString)
+ System.setProperty("jetty.host",REST_HOSTNAME)
+ System.setProperty("jetty.home",HOME.getOrElse(throwNoAkkaHomeException) + "/deploy/root")
+
+ val configuration = new XmlConfiguration(
+ new File(HOME.getOrElse(throwNoAkkaHomeException) + "/config/microkernel-server.xml").toURI.toURL)
+
+ server = Option(configuration.configure.asInstanceOf[Server]) map { s => //Set the correct classloader to our contexts
+ applicationLoader foreach { loader =>
+ //We need to provide the correct classloader to the servlets
+ def setClassLoader(handlers: Seq[Handler]): Unit = {
+ handlers foreach {
+ case c: ContextHandler => c.setClassLoader(loader)
+ case c: HandlerCollection => setClassLoader(c.getHandlers)
+ case _ =>
+ }
+ }
+ setClassLoader(s.getHandlers)
+ }
+ //Start the server
+ s.start()
+ s
+ }
+ log.info("Akka REST service started (Jersey)")
+ }
+ }
+
+ abstract override def onUnload = {
+ super.onUnload
+ server foreach { t => {
+ log.info("Shutting down REST service (Jersey)")
+ t.stop()
+ }
+ }
+ }
+}
diff --git a/akka-http/src/main/scala/akka/Initializer.scala b/akka-http/src/main/scala/akka/Initializer.scala
new file mode 100644
index 0000000000..a470949821
--- /dev/null
+++ b/akka-http/src/main/scala/akka/Initializer.scala
@@ -0,0 +1,34 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.servlet
+
+import akka.remote.BootableRemoteActorService
+import akka.actor.BootableActorLoaderService
+import akka.camel.CamelService
+import akka.config.Config
+import akka.util.{Logging, Bootable}
+
+import javax.servlet.{ServletContextListener, ServletContextEvent}
+
+ /**
+ * This class can be added to web.xml mappings as a listener to start and postStop Akka.
+ *
+ *
+ * ...
+ *
+ * akka.servlet.Initializer
+ *
+ * ...
+ *
+ */
+class Initializer extends ServletContextListener {
+ lazy val loader = new AkkaLoader
+
+ def contextDestroyed(e: ServletContextEvent): Unit =
+ loader.shutdown
+
+ def contextInitialized(e: ServletContextEvent): Unit =
+ loader.boot(true, new BootableActorLoaderService with BootableRemoteActorService with CamelService)
+ }
diff --git a/akka-http/src/main/scala/akka/ListWriter.scala b/akka-http/src/main/scala/akka/ListWriter.scala
new file mode 100644
index 0000000000..3a2c69d02a
--- /dev/null
+++ b/akka-http/src/main/scala/akka/ListWriter.scala
@@ -0,0 +1,41 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.rest
+
+import java.io.OutputStream
+import akka.serialization.Serializer
+import javax.ws.rs.core.{MultivaluedMap, MediaType}
+import javax.ws.rs.ext.{MessageBodyWriter, Provider}
+import javax.ws.rs.Produces
+
+/**
+ * Writes Lists of JSON serializable objects.
+ */
+@Provider
+@Produces(Array("application/json"))
+class ListWriter extends MessageBodyWriter[List[_]] {
+
+ def isWriteable(aClass: Class[_],
+ aType: java.lang.reflect.Type,
+ annotations: Array[java.lang.annotation.Annotation],
+ mediaType: MediaType) =
+ classOf[List[_]].isAssignableFrom(aClass) || aClass == ::.getClass
+
+ def getSize(list: List[_],
+ aClass: Class[_],
+ aType: java.lang.reflect.Type,
+ annotations: Array[java.lang.annotation.Annotation],
+ mediaType: MediaType) =
+ -1L
+
+ def writeTo(list: List[_],
+ aClass: Class[_],
+ aType: java.lang.reflect.Type,
+ annotations: Array[java.lang.annotation.Annotation],
+ mediaType: MediaType,
+ stringObjectMultivaluedMap: MultivaluedMap[String, Object],
+ outputStream: OutputStream): Unit =
+ if (list.isEmpty) outputStream.write(" ".getBytes)
+ else outputStream.write(Serializer.ScalaJSON.toBinary(list))
+}
diff --git a/akka-http/src/main/scala/akka/Security.scala b/akka-http/src/main/scala/akka/Security.scala
new file mode 100644
index 0000000000..b6aec9a1f0
--- /dev/null
+++ b/akka-http/src/main/scala/akka/Security.scala
@@ -0,0 +1,565 @@
+/*
+ * Copyright 2007-2008 WorldWide Conferencing, LLC
+ *
+ * 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.
+ */
+
+/*
+ * AKKA AAS (Authentication and Authorization Service)
+ * Rework of lift's (www.liftweb.com) HTTP Authentication module
+ * All cred to the Lift team (www.liftweb.com), especially David Pollak and Tim Perrett
+ */
+
+package akka.security
+
+import akka.actor.{Scheduler, Actor, ActorRef, ActorRegistry, IllegalActorStateException}
+import akka.actor.Actor._
+import akka.config.Config
+import akka.util.Logging
+
+import com.sun.jersey.api.model.AbstractMethod
+import com.sun.jersey.spi.container.{ResourceFilterFactory, ContainerRequest, ContainerRequestFilter, ContainerResponse, ContainerResponseFilter, ResourceFilter}
+import com.sun.jersey.core.util.Base64
+
+import javax.ws.rs.core.{SecurityContext, Context, Response}
+import javax.ws.rs.WebApplicationException
+import javax.annotation.security.{DenyAll, PermitAll, RolesAllowed}
+import java.security.Principal
+import java.util.concurrent.TimeUnit
+
+case object OK
+
+/**
+ * Authenticate represents a message to authenticate a request
+ */
+case class Authenticate(val req: ContainerRequest, val rolesAllowed: List[String])
+
+/**
+ * User info represents a sign-on with associated credentials/roles
+ */
+case class UserInfo(val username: String, val password: String, val roles: List[String])
+
+trait Credentials
+
+case class BasicCredentials(username: String, password: String) extends Credentials
+
+case class DigestCredentials(method: String,
+ userName: String,
+ realm: String,
+ nonce: String,
+ uri: String,
+ qop: String,
+ nc: String,
+ cnonce: String,
+ response: String,
+ opaque: String) extends Credentials
+
+case class SpnegoCredentials(token: Array[Byte]) extends Credentials
+
+/**
+ * Jersey Filter for invocation intercept and authorization/authentication
+ */
+class AkkaSecurityFilterFactory extends ResourceFilterFactory with Logging {
+ class Filter(actor: ActorRef, rolesAllowed: Option[List[String]])
+ extends ResourceFilter with ContainerRequestFilter with Logging {
+
+ override def getRequestFilter: ContainerRequestFilter = this
+
+ override def getResponseFilter: ContainerResponseFilter = null
+
+ /**
+ * Here's where the magic happens. The request is authenticated by
+ * sending a request for authentication to the configured authenticator actor
+ */
+ override def filter(request: ContainerRequest): ContainerRequest =
+ rolesAllowed match {
+ case Some(roles) => {
+ val result = (authenticator !! Authenticate(request, roles)).as[AnyRef]
+ result match {
+ case Some(OK) => request
+ case Some(r) if r.isInstanceOf[Response] =>
+ throw new WebApplicationException(r.asInstanceOf[Response])
+ case None => throw new WebApplicationException(408)
+ case unknown => {
+ log.warning("Authenticator replied with unexpected result [%s]", unknown);
+ throw new WebApplicationException(Response.Status.INTERNAL_SERVER_ERROR)
+ }
+ }
+ }
+ case None => throw new WebApplicationException(Response.Status.FORBIDDEN)
+ }
+ }
+
+ lazy val authenticatorFQN = {
+ val auth = Config.config.getString("akka.rest.authenticator", "N/A")
+ if (auth == "N/A") throw new IllegalActorStateException("The config option 'akka.rest.authenticator' is not defined in 'akka.conf'")
+ auth
+ }
+
+ /**
+ * Currently we always take the first, since there usually should be at most one authentication actor, but a round-robin
+ * strategy could be implemented in the future
+ */
+ def authenticator: ActorRef = ActorRegistry.actorsFor(authenticatorFQN).head
+
+ def mkFilter(roles: Option[List[String]]): java.util.List[ResourceFilter] =
+ java.util.Collections.singletonList(new Filter(authenticator, roles))
+
+ /**
+ * The create method is invoked for each resource, and we look for javax.annotation.security annotations
+ * and create the appropriate Filter configurations for each.
+ */
+ override def create(am: AbstractMethod): java.util.List[ResourceFilter] = {
+
+ //DenyAll takes precedence
+ if (am.isAnnotationPresent(classOf[DenyAll]))
+ return mkFilter(None)
+
+ //Method-level RolesAllowed takes precedence
+ val ra = am.getAnnotation(classOf[RolesAllowed])
+
+ if (ra ne null)
+ return mkFilter(Some(ra.value.toList))
+
+ //PermitAll takes precedence over resource-level RolesAllowed annotation
+ if (am.isAnnotationPresent(classOf[PermitAll]))
+ return null;
+
+ //Last but not least, the resource-level RolesAllowed
+ val cra = am.getResource.getAnnotation(classOf[RolesAllowed])
+ if (cra ne null)
+ return mkFilter(Some(cra.value.toList))
+
+ return null;
+ }
+}
+
+/**
+ * AuthenticationActor is the super-trait for actors doing Http authentication
+ * It defines the common ground and the flow of execution
+ */
+trait AuthenticationActor[C <: Credentials] extends Actor {
+ type Req = ContainerRequest
+
+ //What realm does the authentication use?
+ def realm: String
+
+ //Creates a response to signal unauthorized
+ def unauthorized: Response
+
+ //Used to extract information from the request, returns None if no credentials found
+ def extractCredentials(r: Req): Option[C]
+
+ //returns None is unverified
+ def verify(c: Option[C]): Option[UserInfo]
+
+ //Contruct a new SecurityContext from the supplied parameters
+ def mkSecurityContext(r: Req, user: UserInfo): SecurityContext
+
+ //This is the default security context factory
+ def mkDefaultSecurityContext(r: Req, u: UserInfo, scheme: String): SecurityContext = {
+ val n = u.username
+ val p = new Principal {def getName = n}
+
+ new SecurityContext {
+ def getAuthenticationScheme = scheme
+ def getUserPrincipal = p
+ def isSecure = r.isSecure
+ def isUserInRole(role: String) = u.roles.exists(_ == role)
+ }
+ }
+
+ /**
+ * Responsible for the execution flow of authentication
+ *
+ * Credentials are extracted and verified from the request,
+ * and a se3curity context is created for the ContainerRequest
+ * this should ensure good integration with current Jersey security
+ */
+ protected val authenticate: Receive = {
+ case Authenticate(req, roles) => {
+ verify(extractCredentials(req)) match {
+ case Some(u: UserInfo) => {
+ req.setSecurityContext(mkSecurityContext(req, u))
+ if (roles.exists(req.isUserInRole(_))) self.reply(OK)
+ else self.reply(Response.status(Response.Status.FORBIDDEN).build)
+ }
+ case _ => self.reply(unauthorized)
+ }
+ }
+ }
+
+ def receive = authenticate
+
+ //returns the string value of the "Authorization"-header of the request
+ def auth(r: Req) = r.getHeaderValue("Authorization")
+
+ //Turns the aforementioned header value into an option
+ def authOption(r: Req): Option[String] = {
+ val a = auth(r)
+ if ((a ne null) && a.length > 0) Some(a) else None
+ }
+}
+
+/**
+ * This trait implements the logic for Http Basic authentication
+ * mix this trait into a class to create an authenticator
+ * Don't forget to set the authenticator FQN in the rest-part of the akka config
+ */
+trait BasicAuthenticationActor extends AuthenticationActor[BasicCredentials] {
+ override def unauthorized =
+ Response.status(401).header("WWW-Authenticate", "Basic realm=\"" + realm + "\"").build
+
+ override def extractCredentials(r: Req): Option[BasicCredentials] = {
+ val Authorization = """(.*):(.*)""".r
+
+ authOption(r) match {
+ case Some(token) => {
+ val authResponse = new String(Base64.decode(token.substring(6).getBytes))
+ authResponse match {
+ case Authorization(username, password) => Some(BasicCredentials(username, password))
+ case _ => None
+ }
+ }
+ case _ => None
+ }
+ }
+
+ override def mkSecurityContext(r: Req, u: UserInfo): SecurityContext =
+ mkDefaultSecurityContext(r, u, SecurityContext.BASIC_AUTH)
+}
+
+/**
+ * This trait implements the logic for Http Digest authentication mix this trait into a
+ * class to create an authenticator. Don't forget to set the authenticator FQN in the
+ * rest-part of the akka config
+ */
+trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] with Logging {
+ import LiftUtils._
+
+ private object InvalidateNonces
+
+ //Holds the generated nonces for the specified validity period
+ val nonceMap = mkNonceMap
+
+ //Discards old nonces
+ protected val invalidateNonces: Receive = {
+ case InvalidateNonces =>
+ val ts = System.currentTimeMillis
+ nonceMap.filter(tuple => (ts - tuple._2) < nonceValidityPeriod)
+ case unknown =>
+ log.error("Don't know what to do with: ", unknown)
+ }
+
+ //Schedule the invalidation of nonces
+ Scheduler.schedule(self, InvalidateNonces, noncePurgeInterval, noncePurgeInterval, TimeUnit.MILLISECONDS)
+
+ //authenticate or invalidate nonces
+ override def receive = authenticate orElse invalidateNonces
+
+ override def unauthorized: Response = {
+ val nonce = randomString(64)
+ nonceMap.put(nonce, System.currentTimeMillis)
+ unauthorized(nonce, "auth", randomString(64))
+ }
+
+ def unauthorized(nonce: String, qop: String, opaque: String): Response = {
+ Response.status(401).header(
+ "WWW-Authenticate",
+ "Digest realm=\"" + realm + "\", " +
+ "qop=\"" + qop + "\", " +
+ "nonce=\"" + nonce + "\", " +
+ "opaque=\"" + opaque + "\"").build
+ }
+
+ //Tests wether the specified credentials are valid
+ def validate(auth: DigestCredentials, user: UserInfo): Boolean = {
+ def h(s: String) = hexEncode(md5(s.getBytes("UTF-8")))
+
+ val ha1 = h(auth.userName + ":" + auth.realm + ":" + user.password)
+ val ha2 = h(auth.method + ":" + auth.uri)
+
+ val response = h(
+ ha1 + ":" + auth.nonce + ":" +
+ auth.nc + ":" + auth.cnonce + ":" +
+ auth.qop + ":" + ha2)
+
+ (response == auth.response) && (nonceMap.getOrElse(auth.nonce, -1) != -1)
+ }
+
+ override def verify(odc: Option[DigestCredentials]): Option[UserInfo] = odc match {
+ case Some(dc) => {
+ userInfo(dc.userName) match {
+ case Some(u) if validate(dc, u) =>
+ nonceMap.get(dc.nonce).map(t => (System.currentTimeMillis - t) < nonceValidityPeriod).map(_ => u)
+ case _ => None
+ }
+ }
+ case _ => None
+ }
+
+ override def extractCredentials(r: Req): Option[DigestCredentials] = {
+ authOption(r).map(s => {
+ val ? = splitNameValuePairs(s.substring(7, s.length))
+ DigestCredentials(r.getMethod.toUpperCase,
+ ?("username"), ?("realm"), ?("nonce"),
+ ?("uri"), ?("qop"), ?("nc"),
+ ?("cnonce"), ?("response"), ?("opaque"))
+ })
+ }
+
+ override def mkSecurityContext(r: Req, u: UserInfo): SecurityContext =
+ mkDefaultSecurityContext(r, u, SecurityContext.DIGEST_AUTH)
+
+ //Mandatory overrides
+ def userInfo(username: String): Option[UserInfo]
+
+ def mkNonceMap: scala.collection.mutable.Map[String, Long]
+
+ //Optional overrides
+ def nonceValidityPeriod = 60 * 1000 //ms
+ def noncePurgeInterval = 2 * 60 * 1000 //ms
+}
+
+import java.security.Principal
+import java.security.PrivilegedActionException
+import java.security.PrivilegedExceptionAction
+
+import javax.security.auth.login.AppConfigurationEntry
+import javax.security.auth.login.Configuration
+import javax.security.auth.login.LoginContext
+import javax.security.auth.Subject
+import javax.security.auth.kerberos.KerberosPrincipal
+
+import org.ietf.jgss.GSSContext
+import org.ietf.jgss.GSSCredential
+import org.ietf.jgss.GSSManager
+
+trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] with Logging {
+ override def unauthorized =
+ Response.status(401).header("WWW-Authenticate", "Negotiate").build
+
+ // for some reason the jersey Base64 class does not work with kerberos
+ // but the commons Base64 does
+ import org.apache.commons.codec.binary.Base64
+ override def extractCredentials(r: Req): Option[SpnegoCredentials] = {
+ val AuthHeader = """Negotiate\s(.*)""".r
+
+ authOption(r) match {
+ case Some(AuthHeader(token)) =>
+ Some(SpnegoCredentials(Base64.decodeBase64(token.trim.getBytes)))
+ case _ => None
+ }
+ }
+
+
+ override def verify(odc: Option[SpnegoCredentials]): Option[UserInfo] = odc match {
+ case Some(dc) => {
+ try {
+ val principal = Subject.doAs(this.serviceSubject, new KerberosValidateAction(dc.token));
+ val user = stripRealmFrom(principal)
+ Some(UserInfo(user, null, rolesFor(user)))
+ } catch {
+ case e: PrivilegedActionException => {
+ log.error(e, "Action not allowed")
+ return None
+ }
+ }
+ }
+ case _ => None
+ }
+
+ override def mkSecurityContext(r: Req, u: UserInfo): SecurityContext =
+ mkDefaultSecurityContext(r, u, SecurityContext.CLIENT_CERT_AUTH) // the security context does not know about spnego/kerberos
+ // not sure whether to use a constant from the security context or something like "SPNEGO/Kerberos"
+
+ /**
+ * returns the roles for the given user
+ */
+ def rolesFor(user: String): List[String]
+
+ // Kerberos
+
+ /**
+ * strips the realm from a kerberos principal name, returning only the user part
+ */
+ private def stripRealmFrom(principal: String): String = principal.split("@")(0)
+
+ /**
+ * principal name for the HTTP kerberos service, i.e HTTP/ { server } @ { realm }
+ */
+ lazy val servicePrincipal = {
+ val p = Config.config.getString("akka.rest.kerberos.servicePrincipal", "N/A")
+ if (p == "N/A") throw new IllegalActorStateException("The config option 'akka.rest.kerberos.servicePrincipal' is not defined in 'akka.conf'")
+ p
+ }
+
+ /**
+ * keytab location with credentials for the service principal
+ */
+ lazy val keyTabLocation = {
+ val p = Config.config.getString("akka.rest.kerberos.keyTabLocation", "N/A")
+ if (p == "N/A") throw new IllegalActorStateException("The config option 'akka.rest.kerberos.keyTabLocation' is not defined in 'akka.conf'")
+ p
+ }
+
+ lazy val kerberosDebug = {
+ val p = Config.config.getString("akka.rest.kerberos.kerberosDebug", "N/A")
+ if (p == "N/A") throw new IllegalActorStateException("The config option 'akka.rest.kerberos.kerberosDebug' is not defined in 'akka.conf'")
+ p
+ }
+
+ /**
+ * is not used by this authenticator, so accept an empty value
+ */
+ lazy val realm = Config.config.getString("akka.rest.kerberos.realm", "")
+
+ /**
+ * verify the kerberos token from a client with the server
+ */
+ class KerberosValidateAction(kerberosTicket: Array[Byte]) extends PrivilegedExceptionAction[String] {
+ def run = {
+ val context = GSSManager.getInstance().createContext(null.asInstanceOf[GSSCredential])
+ context.acceptSecContext(kerberosTicket, 0, kerberosTicket.length)
+ val user = context.getSrcName().toString()
+ context.dispose()
+ user
+ }
+ }
+
+ // service principal login to kerberos on startup
+
+ val serviceSubject = servicePrincipalLogin
+
+ /**
+ * acquire an initial ticket from the kerberos server for the HTTP service
+ */
+ def servicePrincipalLogin = {
+ val loginConfig = new LoginConfig(
+ new java.net.URL(this.keyTabLocation).toExternalForm(),
+ this.servicePrincipal,
+ this.kerberosDebug)
+ val princ = new java.util.HashSet[Principal](1)
+ princ.add(new KerberosPrincipal(this.servicePrincipal))
+ val sub = new Subject(false, princ, new java.util.HashSet[Object], new java.util.HashSet[Object])
+ val lc = new LoginContext("", sub, null, loginConfig)
+ lc.login()
+ lc.getSubject()
+ }
+
+ /**
+ * this class simulates a login-config.xml
+ */
+ class LoginConfig(keyTabLocation: String, servicePrincipal: String, debug: String) extends Configuration {
+ override def getAppConfigurationEntry(name: String): Array[AppConfigurationEntry] = {
+ val options = new java.util.HashMap[String, String]
+ options.put("useKeyTab", "true")
+ options.put("keyTab", this.keyTabLocation)
+ options.put("principal", this.servicePrincipal)
+ options.put("storeKey", "true")
+ options.put("doNotPrompt", "true")
+ options.put("isInitiator", "true")
+ options.put("debug", debug)
+
+ Array(new AppConfigurationEntry(
+ "com.sun.security.auth.module.Krb5LoginModule",
+ AppConfigurationEntry.LoginModuleControlFlag.REQUIRED,
+ options))
+ }
+ }
+
+}
+
+/*
+* Copyright 2006-2010 WorldWide Conferencing, LLC
+*
+* 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.
+*/
+object LiftUtils {
+ import java.security.{MessageDigest,SecureRandom}
+ val random = new SecureRandom()
+
+ def md5(in: Array[Byte]): Array[Byte] = (MessageDigest.getInstance("MD5")).digest(in)
+
+ /**
+ * Create a random string of a given size
+ * @param size size of the string to create. Must be a positive or nul integer
+ * @return the generated string
+ */
+ def randomString(size: Int): String = {
+ def addChar(pos: Int, lastRand: Int, sb: StringBuilder): StringBuilder = {
+ if (pos >= size) sb
+ else {
+ val randNum = if ((pos % 6) == 0) random.nextInt else lastRand
+ sb.append((randNum & 0x1f) match {
+ case n if n < 26 => ('A' + n).toChar
+ case n => ('0' + (n - 26)).toChar
+ })
+ addChar(pos + 1, randNum >> 5, sb)
+ }
+ }
+ addChar(0, 0, new StringBuilder(size)).toString
+ }
+
+/** encode a Byte array as hexadecimal characters */
+ def hexEncode(in: Array[Byte]): String = {
+ val sb = new StringBuilder
+ val len = in.length
+ def addDigit(in: Array[Byte], pos: Int, len: Int, sb: StringBuilder) {
+ if (pos < len) {
+ val b: Int = in(pos)
+ val msb = (b & 0xf0) >> 4
+ val lsb = (b & 0x0f)
+ sb.append((if (msb < 10) ('0' + msb).asInstanceOf[Char] else ('a' + (msb - 10)).asInstanceOf[Char]))
+ sb.append((if (lsb < 10) ('0' + lsb).asInstanceOf[Char] else ('a' + (lsb - 10)).asInstanceOf[Char]))
+ addDigit(in, pos + 1, len, sb)
+ }
+ }
+ addDigit(in, 0, len, sb)
+ sb.toString
+ }
+
+
+ /**
+ * Splits a string of the form <name1=value1, name2=value2, ... > and unquotes the quoted values.
+ * The result is a Map[String, String]
+ */
+ def splitNameValuePairs(props: String): Map[String, String] = {
+ /**
+ * If str is surrounded by quotes it return the content between the quotes
+ */
+ def unquote(str: String) = {
+ if ((str ne null) && str.length >= 2 && str.charAt(0) == '\"' && str.charAt(str.length - 1) == '\"')
+ str.substring(1, str.length - 1)
+ else
+ str
+ }
+
+ val list = props.split(",").toList.map(in => {
+ val pair = in match { case null => Nil case s => s.split("=").toList.map(_.trim).filter(_.length > 0) }
+ (pair(0), unquote(pair(1)))
+ })
+ val map: Map[String, String] = Map.empty
+ (map /: list)((m, next) => m + (next))
+ }
+}
diff --git a/akka-jta/src/main/scala/akka/AtomikosTransactionService.scala b/akka-jta/src/main/scala/akka/AtomikosTransactionService.scala
new file mode 100644
index 0000000000..f2af9c01bf
--- /dev/null
+++ b/akka-jta/src/main/scala/akka/AtomikosTransactionService.scala
@@ -0,0 +1,41 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.jta
+
+import javax.transaction.{TransactionManager, SystemException}
+
+import com.atomikos.icatch.jta.{J2eeTransactionManager, J2eeUserTransaction}
+import com.atomikos.icatch.config.{TSInitInfo, UserTransactionService, UserTransactionServiceImp}
+
+import akka.config.Config._
+import akka.util.Duration
+
+object AtomikosTransactionService extends AtomikosTransactionService
+
+/**
+ * Atomikos implementation of the transaction service trait.
+ *
+ * @author Jonas Bonér
+ */
+class AtomikosTransactionService extends TransactionService with TransactionProtocol {
+ val JTA_TRANSACTION_TIMEOUT = Duration(config.getInt("akka.jta.timeout", 60), TIME_UNIT)
+
+ private val txService: UserTransactionService = new UserTransactionServiceImp
+ private val info: TSInitInfo = txService.createTSInitInfo
+
+ val transactionContainer: TransactionContainer = TransactionContainer(Right(Some(
+ try {
+ txService.init(info)
+ val tm: TransactionManager = new J2eeTransactionManager
+ tm.setTransactionTimeout(JTA_TRANSACTION_TIMEOUT.toSeconds.toInt)
+ tm
+ } catch {
+ case e => throw new SystemException(
+ "Could not create a new Atomikos J2EE Transaction Manager, due to: " + e.toString)
+ }
+ )))
+ // TODO: gracefully postStop of the TM
+ //txService.postStop(false)
+}
diff --git a/akka-jta/src/main/scala/akka/JTA.scala b/akka-jta/src/main/scala/akka/JTA.scala
new file mode 100644
index 0000000000..1f637fc17b
--- /dev/null
+++ b/akka-jta/src/main/scala/akka/JTA.scala
@@ -0,0 +1,223 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.jta
+
+import javax.transaction.{TransactionManager, UserTransaction,
+ Transaction => JtaTransaction, SystemException,
+ Status, Synchronization, TransactionSynchronizationRegistry}
+import javax.naming.{InitialContext, Context, NamingException}
+
+import akka.config.Config._
+import akka.util.Logging
+import akka.stm.Transaction
+import akka.AkkaException
+
+class JtaConfigurationException(message: String) extends AkkaException(message)
+
+/**
+ * Detects if there is a UserTransaction or TransactionManager available in the JNDI.
+ *
+ * @author Jonas Bonér
+ */
+object TransactionContainer extends Logging {
+ val AKKA_JTA_TRANSACTION_SERVICE_CLASS = "akka.jta.AtomikosTransactionService"
+ val DEFAULT_USER_TRANSACTION_NAME = "java:comp/UserTransaction"
+ val FALLBACK_TRANSACTION_MANAGER_NAMES = "java:comp/TransactionManager" ::
+ "java:appserver/TransactionManager" ::
+ "java:pm/TransactionManager" ::
+ "java:/TransactionManager" :: Nil
+ val DEFAULT_TRANSACTION_SYNCHRONIZATION_REGISTRY_NAME = "java:comp/TransactionSynchronizationRegistry"
+
+ val JTA_PROVIDER = config.getString("akka.jta.provider", "from-jndi")
+
+ private var synchronizationRegistry: Option[TransactionSynchronizationRegistry] = None
+
+ def apply(tm: Either[Option[UserTransaction], Option[TransactionManager]]) = new TransactionContainer(tm)
+
+ def apply(): TransactionContainer =
+ JTA_PROVIDER match {
+ case "from-jndi" =>
+ new TransactionContainer(findUserTransaction match {
+ case None => Right(findTransactionManager)
+ case tm => Left(tm)
+ })
+ case "atomikos" =>
+ try {
+ Class.forName(AKKA_JTA_TRANSACTION_SERVICE_CLASS)
+ .newInstance.asInstanceOf[TransactionService]
+ .transactionContainer
+ } catch {
+ case e: ClassNotFoundException =>
+ throw new JtaConfigurationException(
+ "JTA provider defined as 'atomikos', but the AtomikosTransactionService classes can not be found." +
+ "\n\tPlease make sure you have 'akka-jta' JAR and its dependencies on your classpath.")
+ }
+ case _ =>
+ throw new JtaConfigurationException(
+ "No UserTransaction on TransactionManager could be found in scope." +
+ "\n\tEither add 'akka-jta' to the classpath or make sure there is a" +
+ "\n\tTransactionManager or UserTransaction defined in the JNDI.")
+
+ }
+
+ def findUserTransaction: Option[UserTransaction] = {
+ val located = createInitialContext.lookup(DEFAULT_USER_TRANSACTION_NAME)
+ if (located eq null) None
+ else {
+ log.info("JTA UserTransaction detected [%s]", located)
+ Some(located.asInstanceOf[UserTransaction])
+ }
+ }
+
+ def findSynchronizationRegistry: Option[TransactionSynchronizationRegistry] = synchronized {
+ if (synchronizationRegistry.isDefined) synchronizationRegistry
+ else {
+ val located = createInitialContext.lookup(DEFAULT_TRANSACTION_SYNCHRONIZATION_REGISTRY_NAME)
+ if (located eq null) None
+ else {
+ log.info("JTA TransactionSynchronizationRegistry detected [%s]", located)
+ synchronizationRegistry = Some(located.asInstanceOf[TransactionSynchronizationRegistry])
+ synchronizationRegistry
+ }
+ }
+ }
+
+ def findTransactionManager: Option[TransactionManager] = {
+ val context = createInitialContext
+ val tms = for {
+ name <- FALLBACK_TRANSACTION_MANAGER_NAMES
+ tm = context.lookup(name)
+ if tm ne null
+ } yield tm
+ tms match {
+ case Nil => None
+ case tm :: _ =>
+ log.info("JTA TransactionManager detected [%s]", tm)
+ Some(tm.asInstanceOf[TransactionManager])
+ }
+ }
+
+ private def createInitialContext = new InitialContext(new java.util.Hashtable)
+}
+
+/**
+ * JTA transaction container holding either a UserTransaction or a TransactionManager.
+ *
+ * The TransactionContainer is created using the factory val container = TransactionContainer()
+ *
+ * @author Jonas Bonér
+ */
+class TransactionContainer private (
+ val tm: Either[Option[UserTransaction], Option[TransactionManager]]) extends Logging {
+
+ def registerSynchronization(sync: Synchronization) = {
+ TransactionContainer.findSynchronizationRegistry match { // try to use SynchronizationRegistry in JNDI
+ case Some(registry) =>
+ registry.asInstanceOf[TransactionSynchronizationRegistry].registerInterposedSynchronization(sync)
+ case None =>
+ tm match {
+ case Right(Some(txMan)) => // try to use TransactionManager
+ txMan.getTransaction.registerSynchronization(sync)
+ case _ =>
+ log.warning("Cannot find TransactionSynchronizationRegistry in JNDI, can't register STM synchronization")
+ }
+ }
+ }
+
+ def beginWithStmSynchronization(transaction: Transaction) = {
+ begin
+ registerSynchronization(new StmSynchronization(this, transaction))
+ }
+
+ def begin = {
+ TransactionContainer.log.trace("Starting JTA transaction")
+ tm match {
+ case Left(Some(userTx)) => userTx.begin
+ case Right(Some(txMan)) => txMan.begin
+ case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
+ }
+ }
+
+ def commit = {
+ TransactionContainer.log.trace("Committing JTA transaction")
+ tm match {
+ case Left(Some(userTx)) => userTx.commit
+ case Right(Some(txMan)) => txMan.commit
+ case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
+ }
+ }
+
+ def rollback = {
+ TransactionContainer.log.trace("Aborting JTA transaction")
+ tm match {
+ case Left(Some(userTx)) => userTx.rollback
+ case Right(Some(txMan)) => txMan.rollback
+ case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
+ }
+ }
+
+ def getStatus = tm match {
+ case Left(Some(userTx)) => userTx.getStatus
+ case Right(Some(txMan)) => txMan.getStatus
+ case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
+ }
+
+ def isInExistingTransaction = tm match {
+ case Left(Some(userTx)) => userTx.getStatus == Status.STATUS_ACTIVE
+ case Right(Some(txMan)) => txMan.getStatus == Status.STATUS_ACTIVE
+ case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
+ }
+
+ def isRollbackOnly = tm match {
+ case Left(Some(userTx)) => userTx.getStatus == Status.STATUS_MARKED_ROLLBACK
+ case Right(Some(txMan)) => txMan.getStatus == Status.STATUS_MARKED_ROLLBACK
+ case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
+ }
+
+ def setRollbackOnly = tm match {
+ case Left(Some(userTx)) => userTx.setRollbackOnly
+ case Right(Some(txMan)) => txMan.setRollbackOnly
+ case _ => throw new JtaConfigurationException("Does not have a UserTransaction or TransactionManager in scope")
+ }
+
+ def suspend = tm match {
+ case Right(Some(txMan)) => txMan.suspend
+ case _ => throw new JtaConfigurationException("Does not have a TransactionManager in scope")
+ }
+
+ def resume(tx: JtaTransaction) = tm match {
+ case Right(Some(txMan)) => txMan.resume(tx)
+ case _ => throw new JtaConfigurationException("Does not have a TransactionManager in scope")
+ }
+}
+
+/**
+ * STM Synchronization class for synchronizing with the JTA TransactionManager.
+ *
+ * @author Jonas Bonér
+ */
+class StmSynchronization(tc: TransactionContainer, tx: Transaction) extends Synchronization with Logging {
+ def beforeCompletion = {
+ val status = tc.getStatus
+ if (status != Status.STATUS_ROLLEDBACK &&
+ status != Status.STATUS_ROLLING_BACK &&
+ status != Status.STATUS_MARKED_ROLLBACK) {
+ log.debug("JTA transaction has failed, abort STM transaction")
+ tx.transaction.foreach(_.abort) // abort multiverse tx
+ }
+ }
+
+ def afterCompletion(status: Int) = {}
+}
+
+/**
+ * JTA Transaction service.
+ *
+ * @author Jonas Bonér
+ */
+trait TransactionService {
+ def transactionContainer: TransactionContainer
+}
+
diff --git a/akka-jta/src/main/scala/akka/TransactionContext.scala b/akka-jta/src/main/scala/akka/TransactionContext.scala
new file mode 100644
index 0000000000..ca92e5aa75
--- /dev/null
+++ b/akka-jta/src/main/scala/akka/TransactionContext.scala
@@ -0,0 +1,238 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.jta
+
+import javax.transaction.{Transaction, Status, TransactionManager, Synchronization}
+
+import akka.util.Logging
+import akka.config.Config._
+
+/**
+ * The TransactionContext object manages the transactions.
+ * Can be used as higher-order functional 'atomic blocks' or monadic.
+ *
+ * Manages a thread-local stack of TransactionContexts.
+ *
+ * Example usage 1:
+ *
+ *
+ * @author Jonas Bonér
+ */
+object TransactionContext extends TransactionProtocol with Logging {
+ implicit val tc = TransactionContainer()
+
+ private[TransactionContext] val stack = new scala.util.DynamicVariable(new TransactionContext(tc))
+
+ /**
+ * This method can be used to register a Synchronization instance for participating with the JTA transaction.
+ * Here is an example of how to add a JPA EntityManager integration.
+ *
+ * TransactionContext.registerSynchronization(new javax.transaction.Synchronization() {
+ * def beforeCompletion = {
+ * try {
+ * val status = tm.getStatus
+ * if (status != Status.STATUS_ROLLEDBACK &&
+ * status != Status.STATUS_ROLLING_BACK &&
+ * status != Status.STATUS_MARKED_ROLLBACK) {
+ * log.debug("Flushing EntityManager...")
+ * em.flush // flush EntityManager on success
+ * }
+ * } catch {
+ * case e: javax.transaction.SystemException => throw new RuntimeException(e)
+ * }
+ * }
+ *
+ * def afterCompletion(status: Int) = {
+ * val status = tm.getStatus
+ * if (closeAtTxCompletion) em.close
+ * if (status == Status.STATUS_ROLLEDBACK ||
+ * status == Status.STATUS_ROLLING_BACK ||
+ * status == Status.STATUS_MARKED_ROLLBACK) {
+ * em.close
+ * }
+ * }
+ * })
+ *
+ * You should also override the 'joinTransaction' and 'handleException' methods.
+ * See ScalaDoc for these methods in the 'TransactionProtocol' for details.
+ */
+ def registerSynchronization(sync: Synchronization) = synchronization.add(sync)
+
+ /**
+ * Registeres a join transaction function.
+ *
+ * Here is an example on how to integrate with JPA EntityManager.
+ *
+ *
+ * TransactionContext.registerJoinTransactionFun(() => {
+ * val em: EntityManager = ... // get the EntityManager
+ * em.joinTransaction // join JTA transaction
+ * })
+ *
+ */
+ def registerJoinTransactionFun(fn: () => Unit) = joinTransactionFuns.add(fn)
+
+ /**
+ * Handle exception. Can be overriden by concrete transaction service implementation.
+ *
+ * Here is an example on how to handle JPA exceptions.
+ *
+ *
+ */
+ def registerExceptionNotToRollbackOn(e: Class[_ <: Exception]) = exceptionsNotToRollbackOn.add(e)
+
+ object Required extends TransactionMonad {
+ def map[T](f: TransactionMonad => T): T = withTxRequired { f(this) }
+ def flatMap[T](f: TransactionMonad => T): T = withTxRequired { f(this) }
+ def foreach(f: TransactionMonad => Unit): Unit = withTxRequired { f(this) }
+ }
+
+ object RequiresNew extends TransactionMonad {
+ def map[T](f: TransactionMonad => T): T = withTxRequiresNew { f(this) }
+ def flatMap[T](f: TransactionMonad => T): T = withTxRequiresNew { f(this) }
+ def foreach(f: TransactionMonad => Unit): Unit = withTxRequiresNew { f(this) }
+ }
+
+ object Supports extends TransactionMonad {
+ def map[T](f: TransactionMonad => T): T = withTxSupports { f(this) }
+ def flatMap[T](f: TransactionMonad => T): T = withTxSupports { f(this) }
+ def foreach(f: TransactionMonad => Unit): Unit = withTxSupports { f(this) }
+ }
+
+ object Mandatory extends TransactionMonad {
+ def map[T](f: TransactionMonad => T): T = withTxMandatory { f(this) }
+ def flatMap[T](f: TransactionMonad => T): T = withTxMandatory { f(this) }
+ def foreach(f: TransactionMonad => Unit): Unit = withTxMandatory { f(this) }
+ }
+
+ object Never extends TransactionMonad {
+ def map[T](f: TransactionMonad => T): T = withTxNever { f(this) }
+ def flatMap[T](f: TransactionMonad => T): T = withTxNever { f(this) }
+ def foreach(f: TransactionMonad => Unit): Unit = withTxNever { f(this) }
+ }
+
+ object NoOpTransactionMonad extends TransactionMonad {
+ def map[T](f: TransactionMonad => T): T = f(this)
+ def flatMap[T](f: TransactionMonad => T): T = f(this)
+ def foreach(f: TransactionMonad => Unit): Unit = f(this)
+ override def filter(f: TransactionMonad => Boolean): TransactionMonad = this
+ }
+
+ private[jta] def setRollbackOnly = current.setRollbackOnly
+
+ private[jta] def isRollbackOnly = current.isRollbackOnly
+
+ private[jta] def getTransactionContainer: TransactionContainer = current.getTransactionContainer
+
+ private[this] def current = stack.value
+
+ /**
+ * Continues with the invocation defined in 'body' with the brand new context define in 'newCtx', the old
+ * one is put on the stack and will automatically come back in scope when the method exits.
+ *
+ * Suspends and resumes the current JTA transaction.
+ */
+ private[jta] def withNewContext[T](body: => T): T = {
+ val suspendedTx: Option[Transaction] =
+ if (getTransactionContainer.isInExistingTransaction) {
+ log.debug("Suspending TX")
+ Some(getTransactionContainer.suspend)
+ } else None
+ val result = stack.withValue(new TransactionContext(tc)) { body }
+ if (suspendedTx.isDefined) {
+ log.debug("Resuming TX")
+ getTransactionContainer.resume(suspendedTx.get)
+ }
+ result
+ }
+}
+
+/**
+ * Base monad for the transaction monad implementations.
+ *
+ * @author Jonas Bonér
+ */
+trait TransactionMonad {
+
+ // -----------------------------
+ // Monadic definitions
+ // -----------------------------
+
+ def map[T](f: TransactionMonad => T): T
+ def flatMap[T](f: TransactionMonad => T): T
+ def foreach(f: TransactionMonad => Unit): Unit
+ def filter(f: TransactionMonad => Boolean): TransactionMonad =
+ if (f(this)) this else TransactionContext.NoOpTransactionMonad
+
+ // -----------------------------
+ // JTA Transaction definitions
+ // -----------------------------
+
+ /**
+ * Marks the current transaction as doomed.
+ */
+ def setRollbackOnly = TransactionContext.setRollbackOnly
+
+ /**
+ * Marks the current transaction as doomed.
+ */
+ def doom = TransactionContext.setRollbackOnly
+
+ /**
+ * Checks if the current transaction is doomed.
+ */
+ def isRollbackOnly = TransactionContext.isRollbackOnly
+
+ /**
+ * Checks that the current transaction is NOT doomed.
+ */
+ def isNotDoomed = !TransactionContext.isRollbackOnly
+}
+
+/**
+ * Transaction context, holds the EntityManager and the TransactionManager.
+ *
+ * @author Jonas Bonér
+ */
+class TransactionContext(val tc: TransactionContainer) {
+ def registerSynchronization(sync: Synchronization) = TransactionContext.registerSynchronization(sync)
+ def setRollbackOnly = tc.setRollbackOnly
+ def isRollbackOnly: Boolean = tc.getStatus == Status.STATUS_MARKED_ROLLBACK
+ def getTransactionContainer: TransactionContainer = tc
+}
diff --git a/akka-jta/src/main/scala/akka/TransactionProtocol.scala b/akka-jta/src/main/scala/akka/TransactionProtocol.scala
new file mode 100644
index 0000000000..11965df9e6
--- /dev/null
+++ b/akka-jta/src/main/scala/akka/TransactionProtocol.scala
@@ -0,0 +1,227 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.jta
+
+import akka.util.Logging
+
+import java.util.{List => JList}
+import java.util.concurrent.CopyOnWriteArrayList
+
+import javax.naming.{NamingException, Context, InitialContext}
+import javax.transaction.{
+ Transaction,
+ UserTransaction,
+ TransactionManager,
+ Status,
+ RollbackException,
+ SystemException,
+ Synchronization,
+ TransactionRequiredException
+}
+
+/**
+ *
+ * Trait that implements a JTA transaction service that obeys the transaction semantics defined
+ * in the transaction attribute types for the transacted methods according to the EJB 3 draft specification.
+ * The aspect handles UserTransaction, TransactionManager instance variable injection thru @javax.ejb.Inject
+ * (name subject to change as per EJB 3 spec) and method transaction levels thru @javax.ejb.TransactionAttribute.
+ *
+ *
+ *
+ * This trait should be inherited to implement the getTransactionManager() method that should return a concrete
+ * javax.transaction.TransactionManager implementation (from JNDI lookup etc).
+ *
+ * 'Required' is probably the best choice (at least initially) for an EJB method that will need to be transactional. In this case, if the method's caller is already part of a transaction, then the EJB method does not create a new transaction, but continues in the same transaction as its caller. If the caller is not in a transaction, then a new transaction is created for the EJB method. If something happens in the EJB that means that a rollback is required, then the extent of the rollback will include everything done in the EJB method, whatever the condition of the caller. If the caller was in a transaction, then everything done by the caller will be rolled back as well. Thus the 'required' attribute ensures that any work done by the EJB will be rolled back if necessary, and if the caller requires a rollback that too will be rolled back.
+ *
+ *
+ *
RequiresNew
+ * 'RequiresNew' will be appropriate if you want to ensure that the EJB method is rolled back if necessary, but you don't want the rollback to propogate back to the caller. This attribute results in the creation of a new transaction for the method, regardless of the transactional state of the caller. If the caller was operating in a transaction, then its transaction is suspended until the EJB method completes. Because a new transaction is always created, there may be a slight performance penalty if this attribute is over-used.
+ *
+ *
+ *
Mandatory
+ * With the 'mandatory' attribute, the EJB method will not even start unless its caller is in a transaction. It will throw a TransactionRequiredException instead. If the method does start, then it will become part of the transaction of the caller. So if the EJB method signals a failure, the caller will be rolled back as well as the EJB.
+ *
+ *
+ *
Supports
+ * With this attribute, the EJB method does not care about the transactional context of its caller. If the caller is part of a transaction, then the EJB method will be part of the same transaction. If the EJB method fails, the transaction will roll back. If the caller is not part of a transaction, then the EJB method will still operate, but a failure will not cause anything to roll back. 'Supports' is probably the attribute that leads to the fastest method call (as there is no transactional overhead), but it can lead to unpredicatable results. If you want a method to be isolated from transactions, that is, to have no effect on the transaction of its caller, then use 'NotSupported' instead.
+ *
+ *
+ *
NotSupported
+ * With the 'NotSupported' attribute, the EJB method will never take part in a transaction. If the caller is part of a transaction, then the caller's transaction is suspended. If the EJB method fails, there will be no effect on the caller's transaction, and no rollback will occur. Use this method if you want to ensure that the EJB method will not cause a rollback in its caller. This is appropriate if, for example, the method does something non-essential, such as logging a message. It would not be helpful if the failure of this operation caused a transaction rollback.
+ *
+ *
+ *
Never
+ * The 'NotSupported'' attribute will ensure that the EJB method is never called by a transactional caller. Any attempt to do so will result in a RemoteException being thrown. This attribute is probably less useful than `NotSupported', in that NotSupported will assure that the caller's transaction is never affected by the EJB method (just as `Never' does), but will allow a call from a transactional caller if necessary.
+ *
+ *
+ * @author Jonas Bonér
+ */
+trait TransactionProtocol extends Logging {
+
+ protected val synchronization: JList[Synchronization] = new CopyOnWriteArrayList[Synchronization]
+ protected val joinTransactionFuns: JList[() => Unit] = new CopyOnWriteArrayList[() => Unit]
+ protected val exceptionsNotToRollbackOn: JList[Class[_ <: Exception]] = new CopyOnWriteArrayList[Class[_ <: Exception]]
+
+ def joinTransaction(): Unit = {
+ val it = joinTransactionFuns.iterator
+ while (it.hasNext) {
+ val fn = it.next
+ fn()
+ }
+ }
+
+ def handleException(tm: TransactionContainer, e: Exception) = {
+ var rollback = true
+ val it = joinTransactionFuns.iterator
+ while (it.hasNext) {
+ val exception = it.next
+ if (e.getClass.isAssignableFrom(exception.getClass))
+ rollback = false
+ }
+ if (rollback) tm.setRollbackOnly
+ throw e
+ }
+
+ /**
+ * Wraps body in a transaction with REQUIRED semantics.
+ *
+ * Creates a new transaction if no transaction is active in scope, else joins the outer transaction.
+ */
+ def withTxRequired[T](body: => T): T = {
+ val tm = TransactionContext.getTransactionContainer
+ if (!isInExistingTransaction(tm)) {
+ tm.begin
+ registerSynchronization
+ try {
+ joinTransaction
+ body
+ } catch {
+ case e: Exception => handleException(tm, e)
+ } finally {
+ commitOrRollBack(tm)
+ }
+ } else body
+ }
+
+ /**
+ * Wraps body in a transaction with REQUIRES_NEW semantics.
+ *
+ * Suspends existing transaction, starts a new transaction, invokes body,
+ * commits or rollbacks new transaction, finally resumes previous transaction.
+ */
+ def withTxRequiresNew[T](body: => T): T = TransactionContext.withNewContext {
+ val tm = TransactionContext.getTransactionContainer
+ tm.begin
+ registerSynchronization
+ try {
+ joinTransaction
+ body
+ } catch {
+ case e: Exception => handleException(tm, e)
+ } finally {
+ commitOrRollBack(tm)
+ }
+ }
+
+ /**
+ * Wraps body in a transaction with NOT_SUPPORTED semantics.
+ *
+ * Suspends existing transaction, invokes body, resumes transaction.
+ */
+ def withTxNotSupported[T](body: => T): T = TransactionContext.withNewContext {
+ body
+ }
+
+ /**
+ * Wraps body in a transaction with SUPPORTS semantics.
+ *
+ * Basicalla a No-op.
+ */
+ def withTxSupports[T](body: => T): T = {
+ // attach to current if exists else skip -> do nothing
+ body
+ }
+
+ /**
+ * Wraps body in a transaction with MANDATORY semantics.
+ *
+ * Throws a TransactionRequiredException if there is no transaction active in scope.
+ */
+ def withTxMandatory[T](body: => T): T = {
+ if (!isInExistingTransaction(TransactionContext.getTransactionContainer))
+ throw new TransactionRequiredException("No active TX at method with TX type set to MANDATORY")
+ body
+ }
+
+ /**
+ * Wraps body in a transaction with NEVER semantics.
+ *
+ * Throws a SystemException in case of an existing transaction in scope.
+ */
+ def withTxNever[T](body: => T): T = {
+ if (isInExistingTransaction(TransactionContext.getTransactionContainer))
+ throw new SystemException("Detected active TX at method with TX type set to NEVER")
+ body
+ }
+
+ protected def commitOrRollBack(tm: TransactionContainer) = {
+ if (isInExistingTransaction(tm)) {
+ if (isRollbackOnly(tm)) {
+ log.debug("Rolling back TX marked as ROLLBACK_ONLY")
+ tm.rollback
+ } else {
+ log.debug("Committing TX")
+ tm.commit
+ }
+ }
+ }
+
+ // ---------------------------
+ // Helper methods
+ // ---------------------------
+
+ protected def registerSynchronization = {
+ val it = synchronization.iterator
+ while (it.hasNext) TransactionContext.getTransactionContainer.registerSynchronization(it.next)
+ }
+ /**
+ * Checks if a transaction is an existing transaction.
+ *
+ * @param tm the transaction manager
+ * @return boolean
+ */
+ protected def isInExistingTransaction(tm: TransactionContainer): Boolean =
+ tm.getStatus != Status.STATUS_NO_TRANSACTION
+
+ /**
+ * Checks if current transaction is set to rollback only.
+ *
+ * @param tm the transaction manager
+ * @return boolean
+ */
+ protected def isRollbackOnly(tm: TransactionContainer): Boolean =
+ tm.getStatus == Status.STATUS_MARKED_ROLLBACK
+
+ /**
+ * A ThreadLocal variable where to store suspended TX and enable pay as you go
+ * before advice - after advice data sharing in a specific case of requiresNew TX
+ */
+ private val suspendedTx = new ThreadLocal[Transaction] {
+ override def initialValue = null
+ }
+
+ private def storeInThreadLocal(tx: Transaction) = suspendedTx.set(tx)
+
+ private def fetchFromThreadLocal: Option[Transaction] = {
+ if ((suspendedTx ne null) && (suspendedTx.get() ne null)) Some(suspendedTx.get.asInstanceOf[Transaction])
+ else None
+ }
+}
diff --git a/akka-kernel/src/main/scala/akka/Kernel.scala b/akka-kernel/src/main/scala/akka/Kernel.scala
new file mode 100644
index 0000000000..342078913b
--- /dev/null
+++ b/akka-kernel/src/main/scala/akka/Kernel.scala
@@ -0,0 +1,25 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.kernel
+
+import akka.http.{ EmbeddedAppServer, DefaultAkkaLoader }
+import akka.remote.BootableRemoteActorService
+
+object Main {
+ def main(args: Array[String]) = Kernel.boot
+}
+
+/**
+ * The Akka Kernel, is used to start And postStop Akka in standalone/kernel mode.
+ *
+ * @author Jonas Bonér
+ */
+object Kernel extends DefaultAkkaLoader {
+ //For testing purposes only
+ def startRemoteService(): Unit = bundles.foreach( _ match {
+ case x: BootableRemoteActorService => x.startRemoteService
+ case _ =>
+ })
+}
diff --git a/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraSession.scala b/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraSession.scala
new file mode 100644
index 0000000000..b8474812ab
--- /dev/null
+++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraSession.scala
@@ -0,0 +1,199 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.persistence.cassandra
+
+import java.io.{Flushable, Closeable}
+
+import akka.persistence.common._
+import akka.util.Logging
+import akka.util.Helpers._
+import akka.serialization.Serializer
+import akka.config.Config.config
+
+import scala.collection.mutable.Map
+
+import org.apache.cassandra.db.ColumnFamily
+import org.apache.cassandra.thrift._
+
+import org.apache.thrift.transport._
+import org.apache.thrift.protocol._
+
+/**
+ * @author Jonas Bonér
+ */
+trait CassandraSession extends Closeable with Flushable {
+ import scala.collection.JavaConversions._
+ import java.util.{Map => JMap, List => JList}
+
+ val client: Cassandra.Client
+ val keyspace: String
+
+ val obtainedAt: Long
+ val consistencyLevel: ConsistencyLevel
+ val schema: JMap[String, JMap[String, String]]
+
+ /**
+ * Count is always the max number of results to return.
+
+ So it means, starting with `start`, or the first one if start is
+ empty, go until you hit `finish` or `count`, whichever comes first.
+ Empty is not a legal column name so if finish is empty it is ignored
+ and only count is used.
+
+ We don't offer a numeric offset since that can't be supported
+ efficiently with a log-structured merge disk format.
+ */
+
+ // ====================================
+ // ====== Scala-style API names
+ // ====================================
+
+ def /(key: String, columnParent: ColumnParent, start: Array[Byte], end: Array[Byte], ascending: Boolean, count: Int): List[ColumnOrSuperColumn] =
+ /(key, columnParent, start, end, ascending, count, consistencyLevel)
+
+ def /(key: String, columnParent: ColumnParent, start: Array[Byte], end: Array[Byte], ascending: Boolean, count: Int, consistencyLevel: ConsistencyLevel): List[ColumnOrSuperColumn] = {
+ val slicePredicate = new SlicePredicate
+ slicePredicate.setSlice_range(new SliceRange(start, end, ascending, count))
+ client.get_slice(keyspace, key, columnParent, slicePredicate, consistencyLevel).toList
+ }
+
+ def /(key: String, columnParent: ColumnParent, slicePredicate: SlicePredicate): List[ColumnOrSuperColumn] =
+ client.get_slice(keyspace, key, columnParent, slicePredicate, consistencyLevel).toList
+
+ def /(key: String, columnParent: ColumnParent, slicePredicate: SlicePredicate, consistencyLevel: ConsistencyLevel): List[ColumnOrSuperColumn] =
+ client.get_slice(keyspace, key, columnParent, slicePredicate, consistencyLevel).toList
+
+ def |(key: String, colPath: ColumnPath): Option[ColumnOrSuperColumn] =
+ |(key, colPath, consistencyLevel)
+
+ def |(key: String, colPath: ColumnPath, consistencyLevel: ConsistencyLevel): Option[ColumnOrSuperColumn] =
+ client.get(keyspace, key, colPath, consistencyLevel)
+
+ def |#(key: String, columnParent: ColumnParent): Int =
+ |#(key, columnParent, consistencyLevel)
+
+ def |#(key: String, columnParent: ColumnParent, consistencyLevel: ConsistencyLevel): Int =
+ client.get_count(keyspace, key, columnParent, consistencyLevel)
+
+ def ++|(key: String, colPath: ColumnPath, value: Array[Byte]): Unit =
+ ++|(key, colPath, value, obtainedAt, consistencyLevel)
+
+ def ++|(key: String, colPath: ColumnPath, value: Array[Byte], consistencyLevel: ConsistencyLevel): Unit =
+ ++|(key, colPath, value, obtainedAt, consistencyLevel)
+
+ def ++|(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long): Unit =
+ ++|(key, colPath, value, timestamp, consistencyLevel)
+
+ def ++|(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long, consistencyLevel: ConsistencyLevel) =
+ client.insert(keyspace, key, colPath, value, timestamp, consistencyLevel)
+
+ def ++|(key: String, batch: Map[String, List[ColumnOrSuperColumn]]): Unit =
+ ++|(key, batch, consistencyLevel)
+
+ def ++|(key: String, batch: Map[String, List[ColumnOrSuperColumn]], consistencyLevel: ConsistencyLevel): Unit = {
+ val jmap = new java.util.HashMap[String, JList[ColumnOrSuperColumn]]
+ for (entry <- batch; (key, value) = entry) jmap.put(key, new java.util.ArrayList(value))
+ client.batch_insert(keyspace, key, jmap, consistencyLevel)
+ }
+
+ def --(key: String, columnPath: ColumnPath, timestamp: Long): Unit =
+ --(key, columnPath, timestamp, consistencyLevel)
+
+ def --(key: String, columnPath: ColumnPath, timestamp: Long, consistencyLevel: ConsistencyLevel): Unit =
+ client.remove(keyspace, key, columnPath, timestamp, consistencyLevel)
+
+ // ====================================
+ // ====== Java-style API names
+ // ====================================
+
+ def getSlice(key: String, columnParent: ColumnParent, start: Array[Byte], end: Array[Byte], ascending: Boolean, count: Int) = / (key, columnParent, start, end, ascending, count, consistencyLevel)
+
+ def getSlice(key: String, columnParent: ColumnParent, start: Array[Byte], end: Array[Byte], ascending: Boolean, count: Int, consistencyLevel: ConsistencyLevel) = / (key, columnParent, start, end, ascending, count, consistencyLevel)
+
+ def getSlice(key: String, columnParent: ColumnParent, slicePredicate: SlicePredicate) = / (key, columnParent, slicePredicate)
+
+ def getSlice(key: String, columnParent: ColumnParent, slicePredicate: SlicePredicate, consistencyLevel: ConsistencyLevel) = / (key, columnParent, slicePredicate, consistencyLevel)
+
+
+ def get(key: String, colPath: ColumnPath) = |(key, colPath)
+
+ def get(key: String, colPath: ColumnPath, consistencyLevel: ConsistencyLevel) = |(key, colPath, consistencyLevel)
+
+ def getCount(key: String, columnParent: ColumnParent)= |#(key, columnParent)
+
+ def getCount(key: String, columnParent: ColumnParent, consistencyLevel: ConsistencyLevel) = |#(key, columnParent, consistencyLevel)
+
+
+ def insert(key: String, colPath: ColumnPath, value: Array[Byte]): Unit = ++|(key, colPath, value)
+
+ def insert(key: String, colPath: ColumnPath, value: Array[Byte], consistencyLevel: ConsistencyLevel): Unit = ++|(key, colPath, value, consistencyLevel)
+
+ def insert(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long): Unit = ++|(key, colPath, value, timestamp)
+
+ def insert(key: String, colPath: ColumnPath, value: Array[Byte], timestamp: Long, consistencyLevel: ConsistencyLevel) = ++|(key, colPath, value, timestamp, consistencyLevel)
+
+ def insert(key: String, batch: Map[String, List[ColumnOrSuperColumn]]): Unit = ++|(key, batch)
+
+ def insert(key: String, batch: Map[String, List[ColumnOrSuperColumn]], consistencyLevel: ConsistencyLevel): Unit = ++|(key, batch, consistencyLevel)
+
+ def remove(key: String, columnPath: ColumnPath, timestamp: Long): Unit = --(key, columnPath, timestamp)
+
+ def remove(key: String, columnPath: ColumnPath, timestamp: Long, consistencyLevel: ConsistencyLevel): Unit = --(key, columnPath, timestamp, consistencyLevel)
+
+}
+
+class CassandraSessionPool[T <: TTransport](
+ space: String,
+ transportPool: Pool[T],
+ inputProtocol: Protocol,
+ outputProtocol: Protocol,
+ consistency: ConsistencyLevel) extends Closeable with Logging {
+
+ def this(space: String, transportPool: Pool[T], ioProtocol: Protocol, consistency: ConsistencyLevel) =
+ this (space, transportPool, ioProtocol, ioProtocol, consistency)
+
+ def newSession: CassandraSession = newSession(consistency)
+
+ def newSession(consistencyLevel: ConsistencyLevel): CassandraSession = {
+ val socket = transportPool.borrowObject
+ val cassandraClient = new Cassandra.Client(inputProtocol(socket), outputProtocol(socket))
+ val cassandraSchema = cassandraClient.describe_keyspace(space)
+ new CassandraSession {
+ val keyspace = space
+ val client = cassandraClient
+ val obtainedAt = System.currentTimeMillis
+ val consistencyLevel = consistency
+ val schema = cassandraSchema
+ log.debug("Creating %s", toString)
+
+ def flush = socket.flush
+ def close = transportPool.returnObject(socket)
+ override def toString = "[CassandraSession]\n\tkeyspace = " + keyspace + "\n\tschema = " + schema
+ }
+ }
+
+ def withSession[T](body: CassandraSession => T) = {
+ val session = newSession(consistency)
+ try {
+ val result = body(session)
+ session.flush
+ result
+ } finally {
+ session.close
+ }
+ }
+
+ def close = transportPool.close
+}
+
+sealed abstract class Protocol(val factory: TProtocolFactory) {
+ def apply(transport: TTransport) = factory.getProtocol(transport)
+}
+
+object Protocol {
+ object Binary extends Protocol(new TBinaryProtocol.Factory)
+ object SimpleJSON extends Protocol(new TSimpleJSONProtocol.Factory)
+ object JSON extends Protocol(new TJSONProtocol.Factory)
+}
diff --git a/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorage.scala b/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorage.scala
new file mode 100644
index 0000000000..166ccbe676
--- /dev/null
+++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorage.scala
@@ -0,0 +1,59 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.persistence.cassandra
+
+import akka.stm._
+import akka.persistence.common._
+import akka.actor.{newUuid}
+
+object CassandraStorage extends Storage {
+ type ElementType = Array[Byte]
+
+ def newMap: PersistentMap[ElementType, ElementType] = newMap(newUuid.toString)
+ def newVector: PersistentVector[ElementType] = newVector(newUuid.toString)
+ def newRef: PersistentRef[ElementType] = newRef(newUuid.toString)
+ override def newQueue: PersistentQueue[ElementType] = newQueue(newUuid.toString)
+
+ def getMap(id: String): PersistentMap[ElementType, ElementType] = newMap(id)
+ def getVector(id: String): PersistentVector[ElementType] = newVector(id)
+ def getRef(id: String): PersistentRef[ElementType] = newRef(id)
+ override def getQueue(id: String): PersistentQueue[ElementType] = newQueue(id)
+
+ def newMap(id: String): PersistentMap[ElementType, ElementType] = new CassandraPersistentMap(id)
+ def newVector(id: String): PersistentVector[ElementType] = new CassandraPersistentVector(id)
+ def newRef(id: String): PersistentRef[ElementType] = new CassandraPersistentRef(id)
+ override def newQueue(id: String): PersistentQueue[ElementType] = new CassandraPersistentQueue(id)
+}
+
+/**
+ * Implements a persistent transactional map based on the Cassandra distributed P2P key-value storage.
+ *
+ * @author Jonas Bonér
+ */
+class CassandraPersistentMap(id: String) extends PersistentMapBinary {
+ val uuid = id
+ val storage = CassandraStorageBackend
+}
+
+/**
+ * Implements a persistent transactional vector based on the Cassandra
+ * distributed P2P key-value storage.
+ *
+ * @author Jonas Bonér
+ */
+class CassandraPersistentVector(id: String) extends PersistentVector[Array[Byte]] {
+ val uuid = id
+ val storage = CassandraStorageBackend
+}
+
+class CassandraPersistentRef(id: String) extends PersistentRef[Array[Byte]] {
+ val uuid = id
+ val storage = CassandraStorageBackend
+}
+
+class CassandraPersistentQueue(id: String) extends PersistentQueue[Array[Byte]] {
+ val uuid = id
+ val storage = CassandraStorageBackend
+}
diff --git a/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorageBackend.scala b/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorageBackend.scala
new file mode 100644
index 0000000000..a4eb481215
--- /dev/null
+++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/akka/CassandraStorageBackend.scala
@@ -0,0 +1,161 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.persistence.cassandra
+
+import akka.stm._
+import akka.persistence.common._
+import akka.util.Logging
+import akka.util.Helpers._
+import akka.config.Config.config
+
+import org.apache.cassandra.thrift._
+import java.lang.String
+import collection.JavaConversions
+import collection.immutable.{TreeMap, Iterable}
+import java.util.{Map => JMap, HashMap => JHMap, List => JList, ArrayList => JAList}
+
+/**
+ * @author Jonas Bonér
+ */
+
+private[akka] object CassandraStorageBackend extends CommonStorageBackend {
+
+ import CommonStorageBackend._
+
+ type ElementType = Array[Byte]
+
+ val KEYSPACE = "akka"
+ val MAP_COLUMN_PARENT = new ColumnParent("map")
+ val VECTOR_COLUMN_PARENT = new ColumnParent("vector")
+ val REF_COLUMN_PARENT = new ColumnParent("ref")
+ val QUEUE_COLUMN_PARENT = new ColumnParent("queue")
+ val REF_KEY = "item".getBytes("UTF-8")
+ val EMPTY_BYTE_ARRAY = new Array[Byte](0)
+
+ val CASSANDRA_SERVER_HOSTNAME = config.getString("akka.storage.cassandra.hostname", "127.0.0.1")
+ val CASSANDRA_SERVER_PORT = config.getInt("akka.storage.cassandra.port", 9160)
+ val CONSISTENCY_LEVEL = {
+ config.getString("akka.storage.cassandra.consistency-level", "QUORUM") match {
+ case "ZERO" => ConsistencyLevel.ZERO
+ case "ONE" => ConsistencyLevel.ONE
+ case "QUORUM" => ConsistencyLevel.QUORUM
+ case "DCQUORUM" => ConsistencyLevel.DCQUORUM
+ case "DCQUORUMSYNC" => ConsistencyLevel.DCQUORUMSYNC
+ case "ALL" => ConsistencyLevel.ALL
+ case "ANY" => ConsistencyLevel.ANY
+ case unknown => throw new IllegalArgumentException(
+ "Cassandra consistency level [" + unknown + "] is not supported." +
+ "\n\tExpected one of [ZERO, ONE, QUORUM, DCQUORUM, DCQUORUMSYNC, ALL, ANY] in the akka.conf configuration file.")
+ }
+ }
+ val IS_ASCENDING = true
+
+ @volatile private[this] var isRunning = false
+ private[this] val protocol: Protocol = Protocol.Binary
+
+ private[this] val sessions = new CassandraSessionPool(
+ KEYSPACE,
+ StackPool(SocketProvider(CASSANDRA_SERVER_HOSTNAME, CASSANDRA_SERVER_PORT)),
+ protocol,
+ CONSISTENCY_LEVEL)
+
+
+ class CassandraAccess(parent: ColumnParent) extends CommonStorageBackendAccess {
+
+ def path(key: Array[Byte]): ColumnPath = {
+ new ColumnPath(parent.getColumn_family).setColumn(key)
+ }
+
+ def delete(owner: String, key: Array[Byte]) = {
+ sessions.withSession{
+ session => {
+ session -- (owner, path(key), System.currentTimeMillis, CONSISTENCY_LEVEL)
+ }
+ }
+ }
+
+ override def getAll(owner: String, keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] = {
+ sessions.withSession{
+ session => {
+ var predicate = new SlicePredicate().setColumn_names(JavaConversions.asList(keys.toList))
+ val cols = session / (owner, parent, predicate, CONSISTENCY_LEVEL)
+ var map = new TreeMap[Array[Byte], Array[Byte]]()(ordering)
+ cols.foreach{
+ cosc => map += cosc.getColumn.getName -> cosc.getColumn.getValue
+ }
+ map
+ }
+ }
+ }
+
+
+ def get(owner: String, key: Array[Byte], default: Array[Byte]) = {
+ sessions.withSession{
+ session => {
+ try
+ {
+ session | (owner, path(key), CONSISTENCY_LEVEL) match {
+ case Some(cosc) => cosc.getColumn.getValue
+ case None => default
+ }
+ } catch {
+ case e: NotFoundException => default
+ }
+ }
+ }
+ }
+
+ def put(owner: String, key: Array[Byte], value: Array[Byte]) = {
+ sessions.withSession{
+ session => {
+ session ++| (owner, path(key), value, System.currentTimeMillis, CONSISTENCY_LEVEL)
+ }
+ }
+ }
+
+
+ def drop() = {
+ sessions.withSession{
+ session => {
+ val slices = session.client.get_range_slices(session.keyspace, parent,
+ new SlicePredicate().setSlice_range(new SliceRange().setStart(Array.empty[Byte]).setFinish(Array.empty[Byte])),
+ new KeyRange().setStart_key("").setEnd_key(""), CONSISTENCY_LEVEL)
+
+ val mutations = new JHMap[String, JMap[String, JList[Mutation]]]
+ JavaConversions.asIterable(slices).foreach{
+ keySlice: KeySlice => {
+ val key = keySlice.getKey
+ val keyMutations = JavaConversions.asMap(mutations).getOrElse(key, {
+ val km = new JHMap[String, JList[Mutation]]
+ mutations.put(key, km)
+ km
+ })
+ val amutation = new JAList[Mutation]
+ val cols = new JAList[Array[Byte]]
+ keyMutations.put(parent.getColumn_family, amutation)
+ JavaConversions.asIterable(keySlice.getColumns) foreach {
+ cosc: ColumnOrSuperColumn => {
+ cols.add(cosc.getColumn.getName)
+ }
+ }
+ amutation.add(new Mutation().setDeletion(new Deletion(System.currentTimeMillis).setPredicate(new SlicePredicate().setColumn_names(cols))))
+
+ }
+ }
+ session.client.batch_mutate(session.keyspace, mutations, CONSISTENCY_LEVEL)
+ }
+ }
+ }
+
+ }
+
+ def queueAccess = new CassandraAccess(QUEUE_COLUMN_PARENT)
+
+ def mapAccess = new CassandraAccess(MAP_COLUMN_PARENT)
+
+ def vectorAccess = new CassandraAccess(VECTOR_COLUMN_PARENT)
+
+ def refAccess = new CassandraAccess(REF_COLUMN_PARENT)
+}
diff --git a/akka-persistence/akka-persistence-common/src/main/scala/akka/CommonStorageBackend.scala b/akka-persistence/akka-persistence-common/src/main/scala/akka/CommonStorageBackend.scala
new file mode 100644
index 0000000000..18020ff180
--- /dev/null
+++ b/akka-persistence/akka-persistence-common/src/main/scala/akka/CommonStorageBackend.scala
@@ -0,0 +1,741 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.persistence.common
+
+import akka.util.Logging
+import java.lang.String
+import java.nio.ByteBuffer
+import collection.Map
+import java.util.{Map => JMap}
+import akka.persistence.common.PersistentMapBinary.COrdering._
+import collection.immutable._
+import collection.mutable.ArrayBuffer
+
+
+private[akka] trait CommonStorageBackendAccess {
+
+ import CommonStorageBackend._
+
+ /*abstract*/
+
+ def get(owner: String, key: Array[Byte], default: Array[Byte]): Array[Byte]
+
+ def getAll(owner: String, keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] = {
+ keys.foldLeft(new HashMap[Array[Byte], Array[Byte]]) {
+ (map, key) => {
+ Option(get(owner, key)) match {
+ case Some(value) => map + (key -> value)
+ case None => map
+ }
+ }
+ }
+ }
+
+ def put(owner: String, key: Array[Byte], value: Array[Byte]): Unit
+
+ def putAll(owner: String, keyValues: Iterable[(Array[Byte], Array[Byte])]): Unit = {
+ keyValues.foreach{
+ kv => kv match {
+ case (key, value) => put(owner, key, value)
+ }
+ }
+ }
+
+ def delete(owner: String, key: Array[Byte]): Unit
+
+ def deleteAll(owner: String, keys: Iterable[Array[Byte]]): Unit = {
+ keys.foreach(delete(owner, _))
+ }
+
+ def drop(): Unit
+
+ /*concrete*/
+
+ def decodeMapKey(owner: String, key: Array[Byte]): Array[Byte] = key
+
+ def encodeMapKey(owner: String, key: Array[Byte]): Array[Byte] = key
+
+ def decodeIndexedKey(owner: String, key: Array[Byte]): Int = IntSerializer.fromBytes(key)
+
+ def encodeIndexedKey(owner: String, keyint: Int): Array[Byte] = IntSerializer.toBytes(keyint)
+
+ def deleteIndexed(owner: String, index: Int): Unit = delete(owner, encodeIndexedKey(owner, index))
+
+ def getIndexed(owner: String, index: Int): Array[Byte] = get(owner, encodeIndexedKey(owner, index))
+
+ def get(owner: String, key: Array[Byte]): Array[Byte] = get(owner, key, null)
+
+ def putIndexed(owner: String, index: Int, value: Array[Byte]): Unit = put(owner, encodeIndexedKey(owner, index), value)
+
+ def putAllIndexed(owner: String, values: Iterable[(Int, Array[Byte])]): Unit = {
+ putAll(owner, values.map{
+ iv => {
+ iv match {
+ case (i, value) => (encodeIndexedKey(owner, i) -> value)
+ }
+ }
+ })
+ }
+
+ def getAllIndexed(owner: String, keys: Iterable[Int]): Map[Int, Array[Byte]] = {
+ val byteKeys = keys.map(encodeIndexedKey(owner, _))
+ getAll(owner, byteKeys).map{
+ kv => kv match {
+ case (key, value) => (decodeIndexedKey(owner, key) -> value)
+ }
+ }
+ }
+
+ def deleteAllIndexed(owner: String, keys: Iterable[Int]): Unit = {
+ val byteKeys = keys.map(encodeIndexedKey(owner, _))
+ deleteAll(owner, byteKeys)
+ }
+}
+
+private[akka] trait KVStorageBackendAccess extends CommonStorageBackendAccess with Logging {
+
+ import CommonStorageBackend._
+ import KVStorageBackend._
+
+ def put(key: Array[Byte], value: Array[Byte]): Unit
+
+ def get(key: Array[Byte]): Array[Byte]
+
+ def get(key: Array[Byte], default: Array[Byte]): Array[Byte]
+
+ def getAll(keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]]
+
+ def delete(key: Array[Byte]): Unit
+
+ override def decodeMapKey(owner: String, key: Array[Byte]): Array[Byte] = {
+ val mapKeyLength = key.length - IntSerializer.bytesPerInt - owner.getBytes("UTF-8").length
+ val mapkey = new Array[Byte](mapKeyLength)
+ System.arraycopy(key, key.length - mapKeyLength, mapkey, 0, mapKeyLength)
+ mapkey
+ }
+
+
+ override def decodeIndexedKey(owner: String, key: Array[Byte]): Int = {
+ IntSerializer.fromBytes(decodeMapKey(owner,key))
+ }
+
+ override def put(owner: String, key: Array[Byte], value: Array[Byte]): Unit = {
+ put(getKey(owner, key), value)
+ }
+
+ override def putIndexed(owner: String, index: Int, value: Array[Byte]): Unit = {
+ put(getIndexedKey(owner, index), value)
+ }
+
+
+ override def get(owner: String, key: Array[Byte]): Array[Byte] = {
+ get(getKey(owner, key))
+ }
+
+ override def getIndexed(owner: String, index: Int): Array[Byte] = {
+ get(getIndexedKey(owner, index))
+ }
+
+
+ override def get(owner: String, key: Array[Byte], default: Array[Byte]): Array[Byte] = {
+ get(getKey(owner, key), default)
+ }
+
+
+ override def getAll(owner: String, keys: Iterable[Array[Byte]]): Map[Array[Byte], Array[Byte]] = {
+ getAll(keys.map{
+ getKey(owner, _)
+ })
+ }
+
+ override def deleteIndexed(owner: String, index: Int): Unit = {
+ delete(getIndexedKey(owner, index))
+ }
+
+ override def delete(owner: String, key: Array[Byte]): Unit = {
+ delete(getKey(owner, key))
+ }
+}
+
+private[akka] object CommonStorageBackendAccess {
+ implicit def stringToByteArray(st: String): Array[Byte] = {
+ st.getBytes("UTF-8")
+ }
+}
+
+private[akka] object CommonStorageBackend {
+ val nullMapValueHeader = 0x00.byteValue
+ val nullMapValue: Array[Byte] = Array(nullMapValueHeader)
+ val notNullMapValueHeader: Byte = 0xff.byteValue
+ val mapKeySetKeyHeader = 0x00.byteValue
+ val mapKeyHeader = 0xff.byteValue
+ val mapKeysIndex: Array[Byte] = new Array[Byte](1).padTo(1, mapKeySetKeyHeader)
+ val mapKeysWrapperPad: Array[Byte] = new Array[Byte](1).padTo(1, mapKeyHeader)
+
+ /**
+ * Wrap map key prepends mapKeysWrapperPad (1-byte) to map keys so that we can
+ * use a seperate 1 byte key to store the map keyset.
+ *
+ * This basically creates the map key used in underlying storage
+ */
+
+ def wrapMapKey(key: Array[Byte]): Array[Byte] = {
+ val wrapped = new Array[Byte](key.length + mapKeysWrapperPad.length)
+ System.arraycopy(mapKeysWrapperPad, 0, wrapped, 0, mapKeysWrapperPad.length)
+ System.arraycopy(key, 0, wrapped, mapKeysWrapperPad.length, key.length)
+ wrapped
+ }
+
+ /**
+ * unwrapMapKey removes the mapKeysWrapperPad, this translates the map key used
+ * in underlying storage back to a key that is understandable by the frontend
+ */
+
+ def unwrapMapKey(key: Array[Byte]): Array[Byte] = {
+ val unwrapped = new Array[Byte](key.length - mapKeysWrapperPad.length)
+ System.arraycopy(key, mapKeysWrapperPad.length, unwrapped, 0, unwrapped.length)
+ unwrapped
+ }
+
+ def getStoredMapValue(value: Array[Byte]): Array[Byte] = {
+ value match {
+ case null => nullMapValue
+ case value => {
+ val stored = new Array[Byte](value.length + 1)
+ stored(0) = notNullMapValueHeader
+ System.arraycopy(value, 0, stored, 1, value.length)
+ stored
+ }
+ }
+ }
+
+ def getMapValueFromStored(value: Array[Byte]): Array[Byte] = {
+
+ if (value(0) == nullMapValueHeader) {
+ null
+ } else if (value(0) == notNullMapValueHeader) {
+ val returned = new Array[Byte](value.length - 1)
+ System.arraycopy(value, 1, returned, 0, value.length - 1)
+ returned
+ } else {
+ throw new StorageException("unknown header byte on map value:" + value(0))
+ }
+ }
+
+ object IntSerializer {
+ val bytesPerInt = java.lang.Integer.SIZE / java.lang.Byte.SIZE
+
+ def toBytes(i: Int) = ByteBuffer.wrap(new Array[Byte](bytesPerInt)).putInt(i).array()
+
+ def fromBytes(bytes: Array[Byte]) = ByteBuffer.wrap(bytes).getInt()
+
+ def toString(obj: Int) = obj.toString
+
+ def fromString(str: String) = str.toInt
+ }
+
+ object SortedSetSerializer {
+ def toBytes(set: SortedSet[Array[Byte]]): Array[Byte] = {
+ val length = set.foldLeft(0) {
+ (total, bytes) => {
+ total + bytes.length + IntSerializer.bytesPerInt
+ }
+ }
+ val allBytes = new Array[Byte](length)
+ val written = set.foldLeft(0) {
+ (total, bytes) => {
+ val sizeBytes = IntSerializer.toBytes(bytes.length)
+ System.arraycopy(sizeBytes, 0, allBytes, total, sizeBytes.length)
+ System.arraycopy(bytes, 0, allBytes, total + sizeBytes.length, bytes.length)
+ total + sizeBytes.length + bytes.length
+ }
+ }
+ require(length == written, "Bytes Written Did not equal Calculated Length, written %d, length %d".format(written, length))
+ allBytes
+ }
+
+ def fromBytes(bytes: Array[Byte]): SortedSet[Array[Byte]] = {
+ var set = new TreeSet[Array[Byte]]
+ if (bytes.length > IntSerializer.bytesPerInt) {
+ var pos = 0
+ while (pos < bytes.length) {
+ val lengthBytes = new Array[Byte](IntSerializer.bytesPerInt)
+ System.arraycopy(bytes, pos, lengthBytes, 0, IntSerializer.bytesPerInt)
+ pos += IntSerializer.bytesPerInt
+ val length = IntSerializer.fromBytes(lengthBytes)
+ val item = new Array[Byte](length)
+ System.arraycopy(bytes, pos, item, 0, length)
+ set = set + item
+ pos += length
+ }
+ }
+ set
+ }
+
+ }
+
+}
+
+private[akka] object KVStorageBackend {
+
+ import CommonStorageBackend._
+
+ /**
+ * Concat the ownerlenght+owner+key+ of owner so owned data will be colocated
+ * Store the length of owner as first byte to work around the rare case
+ * where ownerbytes1 + keybytes1 == ownerbytes2 + keybytes2 but ownerbytes1 != ownerbytes2
+ */
+
+ def getKey(owner: String, key: Array[Byte]): Array[Byte] = {
+ val ownerBytes: Array[Byte] = owner.getBytes("UTF-8")
+ val ownerLenghtBytes: Array[Byte] = IntSerializer.toBytes(owner.length)
+ val theKey = new Array[Byte](ownerLenghtBytes.length + ownerBytes.length + key.length)
+ System.arraycopy(ownerLenghtBytes, 0, theKey, 0, ownerLenghtBytes.length)
+ System.arraycopy(ownerBytes, 0, theKey, ownerLenghtBytes.length, ownerBytes.length)
+ System.arraycopy(key, 0, theKey, ownerLenghtBytes.length + ownerBytes.length, key.length)
+ theKey
+ }
+
+ def getIndexedKey(owner: String, index: Int): Array[Byte] = {
+ getKey(owner, IntSerializer.toBytes(index))
+ }
+
+}
+
+private[akka] trait CommonStorageBackend extends MapStorageBackend[Array[Byte], Array[Byte]] with VectorStorageBackend[Array[Byte]] with RefStorageBackend[Array[Byte]] with QueueStorageBackend[Array[Byte]] with Logging {
+
+ import CommonStorageBackend._
+
+ val vectorHeadIndex = -1
+ val vectorTailIndex = -2
+ val queueHeadIndex = -1
+ val queueTailIndex = -2
+ val zero = IntSerializer.toBytes(0)
+ val refItem = "refItem".getBytes("UTF-8")
+
+ implicit val ordering = ArrayOrdering
+
+
+ def refAccess: CommonStorageBackendAccess
+
+ def vectorAccess: CommonStorageBackendAccess
+
+ def mapAccess: CommonStorageBackendAccess
+
+ def queueAccess: CommonStorageBackendAccess
+
+
+ def getRefStorageFor(name: String): Option[Array[Byte]] = {
+ val result: Array[Byte] = refAccess.get(name, refItem)
+ Option(result)
+ }
+
+ def insertRefStorageFor(name: String, element: Array[Byte]) = {
+ element match {
+ case null => refAccess.delete(name, refItem)
+ case _ => refAccess.put(name, refItem, element)
+ }
+ }
+
+
+ def getMapStorageRangeFor(name: String, start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = {
+ val allkeys: SortedSet[Array[Byte]] = getMapKeys(name)
+ val range = allkeys.rangeImpl(start, finish).take(count)
+ getKeyValues(name, range)
+ }
+
+ def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = {
+ val keys = getMapKeys(name)
+ getKeyValues(name, keys)
+ }
+
+ private def getKeyValues(name: String, keys: SortedSet[Array[Byte]]): List[(Array[Byte], Array[Byte])] = {
+ val all: Map[Array[Byte], Array[Byte]] =
+ mapAccess.getAll(name, keys)
+
+ var returned = new TreeMap[Array[Byte], Array[Byte]]()(ordering)
+ all.foreach{
+ (entry) => {
+ entry match {
+ case (namePlusKey: Array[Byte], value: Array[Byte]) => {
+ //need to fix here
+ returned += mapAccess.decodeMapKey(name, unwrapMapKey(namePlusKey)) -> getMapValueFromStored(value)
+ }
+ }
+ }
+ }
+ returned.toList
+ }
+
+ def getMapStorageSizeFor(name: String): Int = {
+ val keys = getMapKeys(name)
+ keys.size
+ }
+
+ def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = {
+ val result: Array[Byte] = mapAccess.get(name, wrapMapKey(key))
+ result match {
+ case null => None
+ case _ => Some(getMapValueFromStored(result))
+ }
+ }
+
+ def removeMapStorageFor(name: String, key: Array[Byte]) = {
+ val wrapped = wrapMapKey(key)
+ var keys = getMapKeys(name)
+ keys -= wrapped
+ putMapKeys(name, keys)
+ mapAccess.delete(name, wrapped)
+ }
+
+ def removeMapStorageFor(name: String) = {
+ val keys = getMapKeys(name)
+ keys.foreach{
+ key =>
+ mapAccess.delete(name, key)
+ }
+ mapAccess.delete(name, mapKeysIndex)
+ }
+
+ def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]) = {
+ val wrapped = wrapMapKey(key)
+ mapAccess.put(name, wrapped, getStoredMapValue(value))
+ var keys = getMapKeys(name)
+ keys += wrapped
+ putMapKeys(name, keys)
+ }
+
+ def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) = {
+ val toInsert = entries.map{
+ kv => kv match {
+ case (key, value) => (wrapMapKey(key) -> getStoredMapValue(value))
+ }
+ }
+ mapAccess.putAll(name, toInsert)
+ val newKeys = toInsert.map{
+ case (key, value) => {
+ key
+ }
+ }
+ var keys = getMapKeys(name)
+ keys ++= newKeys
+ putMapKeys(name, keys)
+ }
+
+ def putMapKeys(name: String, keys: SortedSet[Array[Byte]]) = {
+ mapAccess.put(name, mapKeysIndex, SortedSetSerializer.toBytes(keys))
+ }
+
+ def getMapKeys(name: String): SortedSet[Array[Byte]] = {
+ SortedSetSerializer.fromBytes(mapAccess.get(name, mapKeysIndex, Array.empty[Byte]))
+ }
+
+ def getVectorStorageSizeFor(name: String): Int = {
+ getVectorMetadata(name).size
+ }
+
+ def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = {
+ val mdata = getVectorMetadata(name)
+
+ val st = start.getOrElse(0)
+ var cnt =
+ if (finish.isDefined) {
+ val f = finish.get
+ if (f >= st) (f - st) else count
+ } else {
+ count
+ }
+ if (cnt > (mdata.size - st)) {
+ cnt = mdata.size - st
+ }
+
+ val indexes = mdata.getRangeIndexes(st, count)
+ val result = vectorAccess.getAllIndexed(name, indexes)
+ indexes.map(result.get(_).get).toList
+
+ }
+
+ def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = {
+ val mdata = getVectorMetadata(name)
+ if (mdata.size > 0 && index < mdata.size) {
+ vectorAccess.getIndexed(name, mdata.getRangeIndexes(index, 1)(0))
+ } else {
+ throw new StorageException("In Vector:" + name + " No such Index:" + index)
+ }
+ }
+
+ def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) = {
+ val mdata = getVectorMetadata(name)
+ if (mdata.size > 0 && index < mdata.size) {
+ elem match {
+ case null => vectorAccess.deleteIndexed(name, mdata.getRangeIndexes(index, 1)(0))
+ case _ => vectorAccess.putIndexed(name, mdata.getRangeIndexes(index, 1)(0), elem)
+ }
+ } else {
+ throw new StorageException("In Vector:" + name + " No such Index:" + index)
+ }
+ }
+
+ def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) = {
+ var mdata = getVectorMetadata(name)
+ var deletes: List[Int] = Nil
+ var puts: List[(Int, Array[Byte])] = Nil
+ elements.foreach{
+ element => {
+ if (mdata.canInsert) {
+ element match {
+ case null => deletes = mdata.head :: deletes
+ case _ => puts = (mdata.head -> element) :: puts
+ }
+ mdata = mdata.copy(head = mdata.nextInsert)
+ } else {
+ throw new IllegalStateException("The vector dosent have enough capacity to insert these entries")
+ }
+ }
+ }
+
+ vectorAccess.deleteAllIndexed(name, deletes)
+ vectorAccess.putAllIndexed(name, puts)
+ vectorAccess.putIndexed(name, vectorHeadIndex, IntSerializer.toBytes(mdata.head))
+
+ }
+
+ def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = {
+ val mdata = getVectorMetadata(name)
+ if (mdata.canInsert) {
+ element match {
+ case null => vectorAccess.deleteIndexed(name, mdata.head)
+ case _ => vectorAccess.putIndexed(name, mdata.head, element)
+ }
+ vectorAccess.putIndexed(name, vectorHeadIndex, IntSerializer.toBytes(mdata.nextInsert))
+ } else {
+ throw new IllegalStateException("The vector %s is full".format(name))
+ }
+
+ }
+
+
+ override def removeVectorStorageEntryFor(name: String) = {
+ val mdata = getVectorMetadata(name)
+ if (mdata.canRemove) {
+ vectorAccess.putIndexed(name, vectorTailIndex, IntSerializer.toBytes(mdata.nextRemove))
+ try
+ {
+ vectorAccess.deleteIndexed(name, mdata.tail)
+ } catch {
+ case e: Exception => log.warn("Exception while trying to clean up a popped element from the vector, this is acceptable")
+ }
+
+ } else {
+ //blow up or not?
+ }
+ }
+
+ def getVectorMetadata(name: String): VectorMetadata = {
+ val result = vectorAccess.getAllIndexed(name, List(vectorHeadIndex, vectorTailIndex))
+ val head = result.getOrElse(vectorHeadIndex, zero)
+ val tail = result.getOrElse(vectorTailIndex, zero)
+ val mdata = VectorMetadata(IntSerializer.fromBytes(head), IntSerializer.fromBytes(tail))
+ mdata
+ }
+
+ def getOrDefaultToZero(map: Map[Array[Byte], Array[Byte]], key: Array[Byte]): Int = {
+ map.get(key) match {
+ case Some(value) => IntSerializer.fromBytes(value)
+ case None => 0
+ }
+ }
+
+
+ def remove(name: String): Boolean = {
+ val mdata = getQueueMetadata(name)
+ mdata.getActiveIndexes foreach {
+ index =>
+ queueAccess.deleteIndexed(name, index)
+ }
+ queueAccess.deleteIndexed(name, queueHeadIndex)
+ queueAccess.deleteIndexed(name, queueTailIndex)
+ true
+ }
+
+ def peek(name: String, start: Int, count: Int): List[Array[Byte]] = {
+ val mdata = getQueueMetadata(name)
+ val indexes = mdata.getPeekIndexes(start, count)
+ val result = queueAccess.getAllIndexed(name, indexes)
+ indexes.map(result.get(_).get).toList
+ }
+
+ def size(name: String): Int = {
+ getQueueMetadata(name).size
+ }
+
+ def dequeue(name: String): Option[Array[Byte]] = {
+ val mdata = getQueueMetadata(name)
+ if (mdata.canDequeue) {
+ try
+ {
+ val dequeued = queueAccess.getIndexed(name, mdata.head)
+ queueAccess.putIndexed(name, queueHeadIndex, IntSerializer.toBytes(mdata.nextDequeue))
+ Some(dequeued)
+ } finally {
+ try
+ {
+ queueAccess.deleteIndexed(name, mdata.head)
+ } catch {
+ //a failure to delete is ok, just leaves a K-V in Voldemort that will be overwritten if the queue ever wraps around
+ case e: Exception => log.warn(e, "caught an exception while deleting a dequeued element, however this will not cause any inconsistency in the queue")
+ }
+ }
+ } else {
+ None
+ }
+ }
+
+ def enqueue(name: String, item: Array[Byte]): Option[Int] = {
+ val mdata = getQueueMetadata(name)
+ if (mdata.canEnqueue) {
+ item match {
+ case null => queueAccess.deleteIndexed(name, mdata.tail)
+ case _ => queueAccess.putIndexed(name, mdata.tail, item)
+ }
+ queueAccess.putIndexed(name, queueTailIndex, IntSerializer.toBytes(mdata.nextEnqueue))
+ Some(mdata.size + 1)
+ } else {
+ None
+ }
+ }
+
+ def getQueueMetadata(name: String): QueueMetadata = {
+ val result = queueAccess.getAllIndexed(name, List(vectorHeadIndex, vectorTailIndex))
+ val head = result.get(vectorHeadIndex).getOrElse(zero)
+ val tail = result.get(vectorTailIndex).getOrElse(zero)
+ QueueMetadata(IntSerializer.fromBytes(head), IntSerializer.fromBytes(tail))
+ }
+
+
+ //wrapper for null
+
+
+ case class QueueMetadata(head: Int, tail: Int) {
+ //queue is an sequence with indexes from 0 to Int.MAX_VALUE
+ //wraps around when one pointer gets to max value
+ //head has an element in it.
+ //tail is the next slot to write to.
+
+ def size = {
+ if (tail >= head) {
+ tail - head
+ } else {
+ //queue has wrapped
+ (Integer.MAX_VALUE - head) + (tail + 1)
+ }
+ }
+
+ def canEnqueue = {
+ //the -1 stops the tail from catching the head on a wrap around
+ size < Integer.MAX_VALUE - 1
+ }
+
+ def canDequeue = {
+ size > 0
+ }
+
+ def getActiveIndexes(): IndexedSeq[Int] = {
+ if (tail >= head) {
+ Range(head, tail)
+ } else {
+ //queue has wrapped
+ val headRange = Range.inclusive(head, Integer.MAX_VALUE)
+ (if (tail > 0) {
+ headRange ++ Range(0, tail)
+ } else {
+ headRange
+ })
+ }
+ }
+
+ def getPeekIndexes(start: Int, count: Int): IndexedSeq[Int] = {
+ val indexes = getActiveIndexes
+ if (indexes.size < start) {
+ IndexedSeq.empty[Int]
+ } else {
+ indexes.drop(start).take(count)
+ }
+ }
+
+ def nextEnqueue = {
+ tail match {
+ case Integer.MAX_VALUE => 0
+ case _ => tail + 1
+ }
+ }
+
+ def nextDequeue = {
+ head match {
+ case Integer.MAX_VALUE => 0
+ case _ => head + 1
+ }
+ }
+ }
+
+ case class VectorMetadata(head: Int, tail: Int) {
+ def size = {
+ if (head >= tail) {
+ head - tail
+ } else {
+ //queue has wrapped
+ (Integer.MAX_VALUE - tail) + (head + 1)
+ }
+ }
+
+ def canInsert = {
+ //the -1 stops the tail from catching the head on a wrap around
+ size < Integer.MAX_VALUE - 1
+ }
+
+ def canRemove = {
+ size > 0
+ }
+
+ def getActiveIndexes(): IndexedSeq[Int] = {
+ if (head >= tail) {
+ Range(tail, head)
+ } else {
+ //queue has wrapped
+ val headRange = Range.inclusive(tail, Integer.MAX_VALUE)
+ (if (head > 0) {
+ headRange ++ Range(0, head)
+ } else {
+ headRange
+ })
+ }
+ }
+
+ def getRangeIndexes(start: Int, count: Int): IndexedSeq[Int] = {
+ val indexes = getActiveIndexes.reverse
+ if (indexes.size < start) {
+ IndexedSeq.empty[Int]
+ } else {
+ indexes.drop(start).take(count)
+ }
+ }
+
+ def nextInsert = {
+ head match {
+ case Integer.MAX_VALUE => 0
+ case _ => head + 1
+ }
+ }
+
+ def nextRemove = {
+ tail match {
+ case Integer.MAX_VALUE => 0
+ case _ => tail + 1
+ }
+ }
+ }
+
+
+}
diff --git a/akka-persistence/akka-persistence-common/src/main/scala/akka/Pool.scala b/akka-persistence/akka-persistence-common/src/main/scala/akka/Pool.scala
new file mode 100644
index 0000000000..3e205dcfe9
--- /dev/null
+++ b/akka-persistence/akka-persistence-common/src/main/scala/akka/Pool.scala
@@ -0,0 +1,91 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.persistence.common
+
+import org.apache.commons.pool._
+import org.apache.commons.pool.impl._
+
+import org.apache.thrift.transport._
+
+trait Pool[T] extends java.io.Closeable {
+ def borrowObject: T
+ def returnObject(t: T): Unit
+ def invalidateObject(t: T): Unit
+ def addObject(): Unit
+ def getNumIdle: Int
+ def getNumActive: Int
+ def clear(): Unit
+ def setFactory(factory: PoolItemFactory[T]): Unit
+}
+
+trait PoolFactory[T] {
+ def createPool: Pool[T]
+}
+
+trait PoolItemFactory[T] {
+ def makeObject: T
+ def destroyObject(t: T): Unit
+ def validateObject(t: T): Boolean
+ def activateObject(t: T): Unit
+ def passivateObject(t: T): Unit
+}
+
+trait PoolBridge[T, OP <: ObjectPool] extends Pool[T] {
+ val impl: OP
+ override def borrowObject: T = impl.borrowObject.asInstanceOf[T]
+ override def returnObject(t: T) = impl.returnObject(t)
+ override def invalidateObject(t: T) = impl.invalidateObject(t)
+ override def addObject = impl.addObject
+ override def getNumIdle: Int = impl.getNumIdle
+ override def getNumActive: Int = impl.getNumActive
+ override def clear(): Unit = impl.clear()
+ override def close(): Unit = impl.close()
+ override def setFactory(factory: PoolItemFactory[T]) = impl.setFactory(toPoolableObjectFactory(factory))
+
+ def toPoolableObjectFactory[T](pif: PoolItemFactory[T]) = new PoolableObjectFactory {
+ def makeObject: Object = pif.makeObject.asInstanceOf[Object]
+ def destroyObject(o: Object): Unit = pif.destroyObject(o.asInstanceOf[T])
+ def validateObject(o: Object): Boolean = pif.validateObject(o.asInstanceOf[T])
+ def activateObject(o: Object): Unit = pif.activateObject(o.asInstanceOf[T])
+ def passivateObject(o: Object): Unit = pif.passivateObject(o.asInstanceOf[T])
+ }
+}
+
+object StackPool {
+ def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,StackObjectPool] {
+ val impl = new StackObjectPool(toPoolableObjectFactory(factory))
+ }
+
+ def apply[T](factory: PoolItemFactory[T], maxIdle: Int) = new PoolBridge[T,StackObjectPool] {
+ val impl = new StackObjectPool(toPoolableObjectFactory(factory),maxIdle)
+ }
+
+ def apply[T](factory: PoolItemFactory[T], maxIdle: Int, initIdleCapacity: Int) = new PoolBridge[T,StackObjectPool] {
+ val impl = new StackObjectPool(toPoolableObjectFactory(factory),maxIdle,initIdleCapacity)
+ }
+}
+
+object SoftRefPool {
+ def apply[T](factory: PoolItemFactory[T]) = new PoolBridge[T,SoftReferenceObjectPool] {
+ val impl = new SoftReferenceObjectPool(toPoolableObjectFactory(factory))
+ }
+}
+
+trait TransportFactory[T <: TTransport] extends PoolItemFactory[T] {
+ def createTransport: T
+ def makeObject: T = createTransport
+ def destroyObject(transport: T): Unit = transport.close
+ def validateObject(transport: T) = transport.isOpen
+ def activateObject(transport: T): Unit = if( !transport.isOpen ) transport.open else ()
+ def passivateObject(transport: T): Unit = transport.flush
+}
+
+case class SocketProvider(val host: String, val port: Int) extends TransportFactory[TSocket] {
+ def createTransport = {
+ val t = new TSocket(host, port)
+ t.open
+ t
+ }
+}
diff --git a/akka-persistence/akka-persistence-common/src/main/scala/akka/Storage.scala b/akka-persistence/akka-persistence-common/src/main/scala/akka/Storage.scala
new file mode 100644
index 0000000000..de5106d610
--- /dev/null
+++ b/akka-persistence/akka-persistence-common/src/main/scala/akka/Storage.scala
@@ -0,0 +1,876 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.persistence.common
+
+import akka.stm._
+import akka.stm.TransactionManagement.transaction
+import akka.util.Logging
+import akka.japi.{Option => JOption}
+import collection.mutable.ArraySeq
+
+// FIXME move to 'stm' package + add message with more info
+class NoTransactionInScopeException extends RuntimeException
+
+class StorageException(message: String) extends RuntimeException(message)
+
+/**
+ * Example Scala usage.
+ *
+ * New map with generated id.
+ *
+ */
+@serializable trait StatelessActorFormat[T <: Actor] extends Format[T] {
+ def fromBinary(bytes: Array[Byte], act: T) = act
+
+ def toBinary(ac: T) = Array.empty[Byte]
+}
+
+/**
+ * A default implementation of the type class for a Format that specifies a serializer
+ *
+ * Create a Format object with the client actor as the implementation of the type class and
+ * a serializer object
+ *
+ *
+ */
+@serializable trait SerializerBasedActorFormat[T <: Actor] extends Format[T] {
+ val serializer: Serializer
+
+ def fromBinary(bytes: Array[Byte], act: T) = serializer.fromBinary(bytes, Some(act.self.actorClass)).asInstanceOf[T]
+
+ def toBinary(ac: T) = serializer.toBinary(ac)
+}
+
+/**
+ * Module for local actor serialization.
+ */
+object ActorSerialization {
+ def fromBinary[T <: Actor](bytes: Array[Byte])(implicit format: Format[T]): ActorRef =
+ fromBinaryToLocalActorRef(bytes, format)
+
+ def toBinary[T <: Actor](a: ActorRef, serializeMailBox: Boolean = true)(implicit format: Format[T]): Array[Byte] =
+ toSerializedActorRefProtocol(a, format, serializeMailBox).toByteArray
+
+ // wrapper for implicits to be used by Java
+ def fromBinaryJ[T <: Actor](bytes: Array[Byte], format: Format[T]): ActorRef =
+ fromBinary(bytes)(format)
+
+ // wrapper for implicits to be used by Java
+ def toBinaryJ[T <: Actor](a: ActorRef, format: Format[T], srlMailBox: Boolean = true): Array[Byte] =
+ toBinary(a, srlMailBox)(format)
+
+ private[akka] def toSerializedActorRefProtocol[T <: Actor](
+ actorRef: ActorRef, format: Format[T], serializeMailBox: Boolean = true): SerializedActorRefProtocol = {
+ val lifeCycleProtocol: Option[LifeCycleProtocol] = {
+ actorRef.lifeCycle match {
+ case Permanent => Some(LifeCycleProtocol.newBuilder.setLifeCycle(LifeCycleType.PERMANENT).build)
+ case Temporary => Some(LifeCycleProtocol.newBuilder.setLifeCycle(LifeCycleType.TEMPORARY).build)
+ case UndefinedLifeCycle => None//No need to send the undefined lifecycle over the wire //builder.setLifeCycle(LifeCycleType.UNDEFINED)
+ }
+ }
+
+ val originalAddress = AddressProtocol.newBuilder
+ .setHostname(actorRef.homeAddress.getHostName)
+ .setPort(actorRef.homeAddress.getPort)
+ .build
+
+ val builder = SerializedActorRefProtocol.newBuilder
+ .setUuid(UuidProtocol.newBuilder.setHigh(actorRef.uuid.getTime).setLow(actorRef.uuid.getClockSeqAndNode).build)
+ .setId(actorRef.id)
+ .setActorClassname(actorRef.actorClass.getName)
+ .setOriginalAddress(originalAddress)
+ .setTimeout(actorRef.timeout)
+
+
+ if (serializeMailBox == true) {
+ val messages =
+ actorRef.mailbox match {
+ case q: java.util.Queue[MessageInvocation] =>
+ val l = new scala.collection.mutable.ListBuffer[MessageInvocation]
+ val it = q.iterator
+ while (it.hasNext == true) l += it.next
+ l
+ }
+
+ val requestProtocols =
+ messages.map(m =>
+ RemoteActorSerialization.createRemoteMessageProtocolBuilder(
+ Some(actorRef),
+ Left(actorRef.uuid),
+ actorRef.id,
+ actorRef.actorClassName,
+ actorRef.timeout,
+ Left(m.message),
+ false,
+ actorRef.getSender,
+ None,
+ ActorType.ScalaActor,
+ RemoteClient.SECURE_COOKIE).build)
+
+ requestProtocols.foreach(rp => builder.addMessages(rp))
+ }
+
+ actorRef.receiveTimeout.foreach(builder.setReceiveTimeout(_))
+ builder.setActorInstance(ByteString.copyFrom(format.toBinary(actorRef.actor.asInstanceOf[T])))
+ lifeCycleProtocol.foreach(builder.setLifeCycle(_))
+ actorRef.supervisor.foreach(s => builder.setSupervisor(RemoteActorSerialization.toRemoteActorRefProtocol(s)))
+ if (!actorRef.hotswap.isEmpty) builder.setHotswapStack(ByteString.copyFrom(Serializer.Java.toBinary(actorRef.hotswap)))
+ builder.build
+ }
+
+ private def fromBinaryToLocalActorRef[T <: Actor](bytes: Array[Byte], format: Format[T]): ActorRef =
+ fromProtobufToLocalActorRef(SerializedActorRefProtocol.newBuilder.mergeFrom(bytes).build, format, None)
+
+ private[akka] def fromProtobufToLocalActorRef[T <: Actor](
+ protocol: SerializedActorRefProtocol, format: Format[T], loader: Option[ClassLoader]): ActorRef = {
+ Actor.log.debug("Deserializing SerializedActorRefProtocol to LocalActorRef:\n" + protocol)
+
+ val serializer =
+ if (format.isInstanceOf[SerializerBasedActorFormat[_]])
+ Some(format.asInstanceOf[SerializerBasedActorFormat[_]].serializer)
+ else None
+
+ val lifeCycle =
+ if (protocol.hasLifeCycle) {
+ protocol.getLifeCycle.getLifeCycle match {
+ case LifeCycleType.PERMANENT => Permanent
+ case LifeCycleType.TEMPORARY => Temporary
+ case unknown => throw new IllegalActorStateException("LifeCycle type is not valid: " + unknown)
+ }
+ } else UndefinedLifeCycle
+
+ val supervisor =
+ if (protocol.hasSupervisor) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader))
+ else None
+
+ val hotswap =
+ if (serializer.isDefined && protocol.hasHotswapStack) serializer.get
+ .fromBinary(protocol.getHotswapStack.toByteArray, Some(classOf[Stack[PartialFunction[Any, Unit]]]))
+ .asInstanceOf[Stack[PartialFunction[Any, Unit]]]
+ else Stack[PartialFunction[Any, Unit]]()
+
+ val classLoader = loader.getOrElse(getClass.getClassLoader)
+
+ val factory = () => {
+ val actorClass = classLoader.loadClass(protocol.getActorClassname)
+ if (format.isInstanceOf[SerializerBasedActorFormat[_]])
+ format.asInstanceOf[SerializerBasedActorFormat[_]].serializer.fromBinary(
+ protocol.getActorInstance.toByteArray, Some(actorClass)).asInstanceOf[Actor]
+ else actorClass.newInstance.asInstanceOf[Actor]
+ }
+
+ val ar = new LocalActorRef(
+ uuidFrom(protocol.getUuid.getHigh, protocol.getUuid.getLow),
+ protocol.getId,
+ protocol.getOriginalAddress.getHostname,
+ protocol.getOriginalAddress.getPort,
+ if (protocol.hasTimeout) protocol.getTimeout else Actor.TIMEOUT,
+ if (protocol.hasReceiveTimeout) Some(protocol.getReceiveTimeout) else None,
+ lifeCycle,
+ supervisor,
+ hotswap,
+ factory)
+
+ val messages = protocol.getMessagesList.toArray.toList.asInstanceOf[List[RemoteMessageProtocol]]
+ messages.foreach(message => ar ! MessageSerializer.deserialize(message.getMessage))
+
+ if (format.isInstanceOf[SerializerBasedActorFormat[_]] == false)
+ format.fromBinary(protocol.getActorInstance.toByteArray, ar.actor.asInstanceOf[T])
+ ar
+ }
+}
+
+object RemoteActorSerialization {
+ /**
+ * Deserializes a byte array (Array[Byte]) into an RemoteActorRef instance.
+ */
+ def fromBinaryToRemoteActorRef(bytes: Array[Byte]): ActorRef =
+ fromProtobufToRemoteActorRef(RemoteActorRefProtocol.newBuilder.mergeFrom(bytes).build, None)
+
+ /**
+ * Deserializes a byte array (Array[Byte]) into an RemoteActorRef instance.
+ */
+ def fromBinaryToRemoteActorRef(bytes: Array[Byte], loader: ClassLoader): ActorRef =
+ fromProtobufToRemoteActorRef(RemoteActorRefProtocol.newBuilder.mergeFrom(bytes).build, Some(loader))
+
+ /**
+ * Deserializes a RemoteActorRefProtocol Protocol Buffers (protobuf) Message into an RemoteActorRef instance.
+ */
+ private[akka] def fromProtobufToRemoteActorRef(protocol: RemoteActorRefProtocol, loader: Option[ClassLoader]): ActorRef = {
+ Actor.log.debug("Deserializing RemoteActorRefProtocol to RemoteActorRef:\n %s", protocol)
+ RemoteActorRef(
+ protocol.getClassOrServiceName,
+ protocol.getActorClassname,
+ protocol.getHomeAddress.getHostname,
+ protocol.getHomeAddress.getPort,
+ protocol.getTimeout,
+ loader)
+ }
+
+ /**
+ * Serializes the ActorRef instance into a Protocol Buffers (protobuf) Message.
+ */
+ def toRemoteActorRefProtocol(ar: ActorRef): RemoteActorRefProtocol = {
+ import ar._
+ val host = homeAddress.getHostName
+ val port = homeAddress.getPort
+
+ Actor.log.debug("Register serialized Actor [%s] as remote @ [%s:%s]", actorClassName, host, port)
+ RemoteServer.getOrCreateServer(homeAddress)
+ ActorRegistry.registerActorByUuid(homeAddress, uuid.toString, ar)
+
+ RemoteActorRefProtocol.newBuilder
+ .setClassOrServiceName(uuid.toString)
+ .setActorClassname(actorClassName)
+ .setHomeAddress(AddressProtocol.newBuilder.setHostname(host).setPort(port).build)
+ .setTimeout(timeout)
+ .build
+ }
+
+ def createRemoteMessageProtocolBuilder(
+ actorRef: Option[ActorRef],
+ uuid: Either[Uuid, UuidProtocol],
+ actorId: String,
+ actorClassName: String,
+ timeout: Long,
+ message: Either[Any, Throwable],
+ isOneWay: Boolean,
+ senderOption: Option[ActorRef],
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: ActorType,
+ secureCookie: Option[String]): RemoteMessageProtocol.Builder = {
+
+ val uuidProtocol = uuid match {
+ case Left(uid) => UuidProtocol.newBuilder.setHigh(uid.getTime).setLow(uid.getClockSeqAndNode).build
+ case Right(protocol) => protocol
+ }
+
+ val actorInfoBuilder = ActorInfoProtocol.newBuilder
+ .setUuid(uuidProtocol)
+ .setId(actorId)
+ .setTarget(actorClassName)
+ .setTimeout(timeout)
+
+ typedActorInfo.foreach { typedActor =>
+ actorInfoBuilder.setTypedActorInfo(
+ TypedActorInfoProtocol.newBuilder
+ .setInterface(typedActor._1)
+ .setMethod(typedActor._2)
+ .build)
+ }
+
+ actorType match {
+ case ActorType.ScalaActor => actorInfoBuilder.setActorType(SCALA_ACTOR)
+ case ActorType.TypedActor => actorInfoBuilder.setActorType(TYPED_ACTOR)
+ }
+ val actorInfo = actorInfoBuilder.build
+ val messageBuilder = RemoteMessageProtocol.newBuilder
+ .setUuid(uuidProtocol)
+ .setActorInfo(actorInfo)
+ .setOneWay(isOneWay)
+
+ message match {
+ case Left(message) =>
+ messageBuilder.setMessage(MessageSerializer.serialize(message))
+ case Right(exception) =>
+ messageBuilder.setException(ExceptionProtocol.newBuilder
+ .setClassname(exception.getClass.getName)
+ .setMessage(exception.getMessage)
+ .build)
+ }
+
+ secureCookie.foreach(messageBuilder.setCookie(_))
+
+ actorRef.foreach { ref =>
+ ref.registerSupervisorAsRemoteActor.foreach { id =>
+ messageBuilder.setSupervisorUuid(
+ UuidProtocol.newBuilder
+ .setHigh(id.getTime)
+ .setLow(id.getClockSeqAndNode)
+ .build)
+ }
+ }
+
+ senderOption.foreach { sender =>
+ RemoteServer.getOrCreateServer(sender.homeAddress).register(sender.uuid.toString, sender)
+ messageBuilder.setSender(toRemoteActorRefProtocol(sender))
+
+ }
+ messageBuilder
+ }
+}
+
+
+/**
+ * Module for local typed actor serialization.
+ */
+object TypedActorSerialization {
+
+ def fromBinary[T <: Actor, U <: AnyRef](bytes: Array[Byte])(implicit format: Format[T]): U =
+ fromBinaryToLocalTypedActorRef(bytes, format)
+
+ def toBinary[T <: Actor](proxy: AnyRef)(implicit format: Format[T]): Array[Byte] = {
+ toSerializedTypedActorRefProtocol(proxy, format).toByteArray
+ }
+
+ // wrapper for implicits to be used by Java
+ def fromBinaryJ[T <: Actor, U <: AnyRef](bytes: Array[Byte], format: Format[T]): U =
+ fromBinary(bytes)(format)
+
+ // wrapper for implicits to be used by Java
+ def toBinaryJ[T <: Actor](a: AnyRef, format: Format[T]): Array[Byte] =
+ toBinary(a)(format)
+
+ private def toSerializedTypedActorRefProtocol[T <: Actor](
+ proxy: AnyRef, format: Format[T]): SerializedTypedActorRefProtocol = {
+
+ val init = AspectInitRegistry.initFor(proxy)
+ if (init eq null) throw new IllegalArgumentException("Proxy for typed actor could not be found in AspectInitRegistry.")
+
+ SerializedTypedActorRefProtocol.newBuilder
+ .setActorRef(ActorSerialization.toSerializedActorRefProtocol(init.actorRef, format))
+ .setInterfaceName(init.interfaceClass.getName)
+ .build
+ }
+
+ private def fromBinaryToLocalTypedActorRef[T <: Actor, U <: AnyRef](bytes: Array[Byte], format: Format[T]): U =
+ fromProtobufToLocalTypedActorRef(SerializedTypedActorRefProtocol.newBuilder.mergeFrom(bytes).build, format, None)
+
+ private def fromProtobufToLocalTypedActorRef[T <: Actor, U <: AnyRef](
+ protocol: SerializedTypedActorRefProtocol, format: Format[T], loader: Option[ClassLoader]): U = {
+ Actor.log.debug("Deserializing SerializedTypedActorRefProtocol to LocalActorRef:\n" + protocol)
+ val actorRef = ActorSerialization.fromProtobufToLocalActorRef(protocol.getActorRef, format, loader)
+ val intfClass = toClass(loader, protocol.getInterfaceName)
+ TypedActor.newInstance(intfClass, actorRef).asInstanceOf[U]
+ }
+
+ private[akka] def toClass[U <: AnyRef](loader: Option[ClassLoader], name: String): Class[U] = {
+ val classLoader = loader.getOrElse(getClass.getClassLoader)
+ val clazz = classLoader.loadClass(name)
+ clazz.asInstanceOf[Class[U]]
+ }
+}
+
+/**
+ * Module for remote typed actor serialization.
+ */
+object RemoteTypedActorSerialization {
+ /**
+ * Deserializes a byte array (Array[Byte]) into an RemoteActorRef instance.
+ */
+ def fromBinaryToRemoteTypedActorRef[T <: AnyRef](bytes: Array[Byte]): T =
+ fromProtobufToRemoteTypedActorRef(RemoteTypedActorRefProtocol.newBuilder.mergeFrom(bytes).build, None)
+
+ /**
+ * Deserializes a byte array (Array[Byte]) into a AW RemoteActorRef proxy.
+ */
+ def fromBinaryToRemoteTypedActorRef[T <: AnyRef](bytes: Array[Byte], loader: ClassLoader): T =
+ fromProtobufToRemoteTypedActorRef(RemoteTypedActorRefProtocol.newBuilder.mergeFrom(bytes).build, Some(loader))
+
+ /**
+ * Serialize as AW RemoteActorRef proxy.
+ */
+ def toBinary[T <: Actor](proxy: AnyRef): Array[Byte] = {
+ toRemoteTypedActorRefProtocol(proxy).toByteArray
+ }
+
+ /**
+ * Deserializes a RemoteTypedActorRefProtocol Protocol Buffers (protobuf) Message into AW RemoteActorRef proxy.
+ */
+ private[akka] def fromProtobufToRemoteTypedActorRef[T](protocol: RemoteTypedActorRefProtocol, loader: Option[ClassLoader]): T = {
+ Actor.log.debug("Deserializing RemoteTypedActorRefProtocol to AW RemoteActorRef proxy:\n" + protocol)
+ val actorRef = RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getActorRef, loader)
+ val intfClass = TypedActorSerialization.toClass(loader, protocol.getInterfaceName)
+ TypedActor.createProxyForRemoteActorRef(intfClass, actorRef).asInstanceOf[T]
+ }
+
+ /**
+ * Serializes the AW TypedActor proxy into a Protocol Buffers (protobuf) Message.
+ */
+ def toRemoteTypedActorRefProtocol(proxy: AnyRef): RemoteTypedActorRefProtocol = {
+ val init = AspectInitRegistry.initFor(proxy)
+ RemoteTypedActorRefProtocol.newBuilder
+ .setActorRef(RemoteActorSerialization.toRemoteActorRefProtocol(init.actorRef))
+ .setInterfaceName(init.interfaceClass.getName)
+ .build
+ }
+}
diff --git a/akka-remote/src/main/scala/akka/serialization/Serializer.scala b/akka-remote/src/main/scala/akka/serialization/Serializer.scala
new file mode 100644
index 0000000000..e30e615322
--- /dev/null
+++ b/akka-remote/src/main/scala/akka/serialization/Serializer.scala
@@ -0,0 +1,179 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.serialization
+
+import java.io.{ObjectOutputStream, ByteArrayOutputStream, ObjectInputStream, ByteArrayInputStream}
+
+import org.apache.commons.io.input.ClassLoaderObjectInputStream
+
+import com.google.protobuf.Message
+
+import org.codehaus.jackson.map.ObjectMapper
+
+import sjson.json.{Serializer => SJSONSerializer}
+
+/**
+ * @author Jonas Bonér
+ */
+@serializable trait Serializer {
+ var classLoader: Option[ClassLoader] = None
+ def deepClone(obj: AnyRef): AnyRef = fromBinary(toBinary(obj), Some(obj.getClass))
+
+ def toBinary(obj: AnyRef): Array[Byte]
+ def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef
+}
+
+// For Java API
+class SerializerFactory {
+ import Serializer._
+ def getJava: Java.type = Java
+ def getJavaJSON: JavaJSON.type = JavaJSON
+ def getScalaJSON: ScalaJSON.type = ScalaJSON
+ def getSBinary: SBinary.type = SBinary
+ def getProtobuf: Protobuf.type = Protobuf
+}
+
+/**
+ * @author Jonas Bonér
+ */
+object Serializer {
+ val ARRAY_OF_BYTE_ARRAY = Array[Class[_]](classOf[Array[Byte]])
+
+ object NOOP extends NOOP
+ class NOOP extends Serializer {
+ def toBinary(obj: AnyRef): Array[Byte] = Array[Byte]()
+ def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = null.asInstanceOf[AnyRef]
+ }
+
+ /**
+ * @author Jonas Bonér
+ */
+ object Java extends Java
+ trait Java extends Serializer {
+ def toBinary(obj: AnyRef): Array[Byte] = {
+ val bos = new ByteArrayOutputStream
+ val out = new ObjectOutputStream(bos)
+ out.writeObject(obj)
+ out.close
+ bos.toByteArray
+ }
+
+ def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
+ val in =
+ if (classLoader.isDefined) new ClassLoaderObjectInputStream(classLoader.get, new ByteArrayInputStream(bytes))
+ else new ObjectInputStream(new ByteArrayInputStream(bytes))
+ val obj = in.readObject
+ in.close
+ obj
+ }
+ }
+
+ /**
+ * @author Jonas Bonér
+ */
+ object Protobuf extends Protobuf
+ trait Protobuf extends Serializer {
+ def toBinary(obj: AnyRef): Array[Byte] = {
+ if (!obj.isInstanceOf[Message]) throw new IllegalArgumentException(
+ "Can't serialize a non-protobuf message using protobuf [" + obj + "]")
+ obj.asInstanceOf[Message].toByteArray
+ }
+
+ def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
+ if (!clazz.isDefined) throw new IllegalArgumentException(
+ "Need a protobuf message class to be able to serialize bytes using protobuf")
+ clazz.get.getDeclaredMethod("parseFrom", ARRAY_OF_BYTE_ARRAY: _*).invoke(null, bytes).asInstanceOf[Message]
+ }
+
+ def fromBinary(bytes: Array[Byte], clazz: Class[_]): AnyRef = {
+ if (clazz eq null) throw new IllegalArgumentException("Protobuf message can't be null")
+ fromBinary(bytes, Some(clazz))
+ }
+ }
+
+ /**
+ * @author Jonas Bonér
+ */
+ object JavaJSON extends JavaJSON
+ trait JavaJSON extends Serializer {
+ private val mapper = new ObjectMapper
+
+ def toBinary(obj: AnyRef): Array[Byte] = {
+ val bos = new ByteArrayOutputStream
+ val out = new ObjectOutputStream(bos)
+ mapper.writeValue(out, obj)
+ out.close
+ bos.toByteArray
+ }
+
+ def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = {
+ if (!clazz.isDefined) throw new IllegalArgumentException(
+ "Can't deserialize JSON to instance if no class is provided")
+ val in =
+ if (classLoader.isDefined) new ClassLoaderObjectInputStream(classLoader.get, new ByteArrayInputStream(bytes))
+ else new ObjectInputStream(new ByteArrayInputStream(bytes))
+ val obj = mapper.readValue(in, clazz.get).asInstanceOf[AnyRef]
+ in.close
+ obj
+ }
+
+ def fromJSON(json: String, clazz: Class[_]): AnyRef = {
+ if (clazz eq null) throw new IllegalArgumentException("Can't deserialize JSON to instance if no class is provided")
+ mapper.readValue(json, clazz).asInstanceOf[AnyRef]
+ }
+ }
+
+ /**
+ * @author Jonas Bonér
+ */
+ trait ScalaJSON {
+ import sjson.json._
+
+ var classLoader: Option[ClassLoader] = None
+
+ def tojson[T](o: T)(implicit tjs: Writes[T]): JsValue = JsonSerialization.tojson(o)(tjs)
+
+ def fromjson[T](json: JsValue)(implicit fjs: Reads[T]): T = JsonSerialization.fromjson(json)(fjs)
+
+ def tobinary[T](o: T)(implicit tjs: Writes[T]): Array[Byte] = JsonSerialization.tobinary(o)(tjs)
+
+ def frombinary[T](bytes: Array[Byte])(implicit fjs: Reads[T]): T = JsonSerialization.frombinary(bytes)(fjs)
+
+ // backward compatibility
+ // implemented using refelction based json serialization
+ def toBinary(obj: AnyRef): Array[Byte] = SJSONSerializer.SJSON.out(obj)
+
+ def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef = SJSONSerializer.SJSON.in(bytes)
+
+ import scala.reflect.Manifest
+ def fromJSON[T](json: String)(implicit m: Manifest[T]): AnyRef = {
+ SJSONSerializer.SJSON.in(json)(m)
+ }
+
+ def fromBinary[T](bytes: Array[Byte])(implicit m: Manifest[T]): AnyRef = {
+ SJSONSerializer.SJSON.in(bytes)(m)
+ }
+ }
+ object ScalaJSON extends ScalaJSON
+
+ /**
+ * @author Jonas Bonér
+ */
+ object SBinary extends SBinary
+ class SBinary {
+ import sbinary._
+ import sbinary.Operations._
+ import sbinary.DefaultProtocol._
+
+ var classLoader: Option[ClassLoader] = None
+
+ def deepClone[T <: AnyRef](obj: T)(implicit w : Writes[T], r : Reads[T]): T = fromBinary[T](toBinary[T](obj), None)
+
+ def toBinary[T](t : T)(implicit bin : Writes[T]): Array[Byte] = toByteArray[T](t)
+
+ def fromBinary[T](array : Array[Byte], clazz: Option[Class[T]])(implicit bin : Reads[T]): T = fromByteArray[T](array)
+ }
+}
+
diff --git a/akka-remote/src/main/scala/akka/serialization/package.scala b/akka-remote/src/main/scala/akka/serialization/package.scala
new file mode 100644
index 0000000000..f75560403a
--- /dev/null
+++ b/akka-remote/src/main/scala/akka/serialization/package.scala
@@ -0,0 +1,9 @@
+package akka
+
+package object serialization {
+ type JsValue = _root_.dispatch.json.JsValue
+ val JsValue = _root_.dispatch.json.JsValue
+ val Js = _root_.dispatch.json.Js
+ val JsonSerialization = sjson.json.JsonSerialization
+ val DefaultProtocol = sjson.json.DefaultProtocol
+}
diff --git a/akka-remote/src/test/scala/ticket/Ticket519Spec.scala b/akka-remote/src/test/scala/ticket/Ticket519Spec.scala
new file mode 100644
index 0000000000..8457f10f45
--- /dev/null
+++ b/akka-remote/src/test/scala/ticket/Ticket519Spec.scala
@@ -0,0 +1,30 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.actor.ticket
+
+import org.scalatest.Spec
+import org.scalatest.matchers.ShouldMatchers
+import akka.remote.{RemoteClient, RemoteServer}
+import akka.actor._
+
+
+class Ticket519Spec extends Spec with ShouldMatchers {
+
+ val HOSTNAME = "localhost"
+ val PORT = 6666
+
+ describe("A remote TypedActor") {
+ it("should handle remote future replies") {
+ import akka.remote._
+
+ val server = { val s = new RemoteServer; s.start(HOSTNAME,PORT); s}
+ val actor = TypedActor.newRemoteInstance(classOf[SamplePojo], classOf[SamplePojoImpl],7000,HOSTNAME,PORT)
+ val r = actor.someFutureString
+
+ r.await.result.get should equal ("foo")
+ TypedActor.stop(actor)
+ server.shutdown
+ }
+ }
+}
diff --git a/akka-samples/akka-sample-rest-scala/src/main/scala/SimpleService.scala b/akka-samples/akka-sample-rest-scala/src/main/scala/SimpleService.scala
new file mode 100644
index 0000000000..5ef2181e3b
--- /dev/null
+++ b/akka-samples/akka-sample-rest-scala/src/main/scala/SimpleService.scala
@@ -0,0 +1,208 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package sample.rest.scala
+
+import akka.actor.{SupervisorFactory, Actor}
+import akka.actor.Actor._
+import akka.stm._
+import akka.stm.TransactionalMap
+import akka.persistence.cassandra.CassandraStorage
+import akka.config.Supervision._
+import akka.util.Logging
+import scala.xml.NodeSeq
+import java.lang.Integer
+import java.nio.ByteBuffer
+import javax.ws.rs.core.MultivaluedMap
+import javax.ws.rs.{GET, POST, Path, Produces, WebApplicationException, Consumes,PathParam}
+import akka.actor.ActorRegistry.actorFor
+import org.atmosphere.annotation.{Broadcast, Suspend,Cluster}
+import org.atmosphere.util.XSSHtmlFilter
+import org.atmosphere.cpr.{Broadcaster, BroadcastFilter}
+import org.atmosphere.jersey.Broadcastable
+
+class Boot {
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ OneForOneStrategy(List(classOf[Exception]), 3, 100),
+ Supervise(
+ actorOf[SimpleServiceActor],
+ Permanent) ::
+ Supervise(
+ actorOf[ChatActor],
+ Permanent) ::
+ Supervise(
+ actorOf[PersistentSimpleServiceActor],
+ Permanent)
+ :: Nil))
+ factory.newInstance.start
+}
+
+/**
+ * Try service out by invoking (multiple times):
+ *
+ * curl http://localhost:9998/scalacount
+ *
+ * Or browse to the URL from a web browser.
+ */
+@Path("/scalacount")
+class SimpleService {
+ @GET
+ @Produces(Array("text/html"))
+ def count = {
+ //Fetch the first actor of type SimpleServiceActor
+ //Send it the "Tick" message and expect a NodeSeq back
+ val result = for{a <- actorFor[SimpleServiceActor]
+ r <- (a !! "Tick").as[NodeSeq]} yield r
+ //Return either the resulting NodeSeq or a default one
+ result getOrElse Error in counter
+ }
+}
+
+class SimpleServiceActor extends Actor {
+ private val KEY = "COUNTER"
+ private var hasStartedTicking = false
+ private val storage = TransactionalMap[String, Integer]()
+
+ def receive = {
+ case "Tick" => if (hasStartedTicking) {
+ val count = atomic {
+ val current = storage.get(KEY).get.asInstanceOf[Integer].intValue
+ val updated = current + 1
+ storage.put(KEY, new Integer(updated))
+ updated
+ }
+ self.reply(Tick:{count})
+ } else {
+ atomic {
+ storage.put(KEY, new Integer(0))
+ }
+ hasStartedTicking = true
+ self.reply(Tick: 0)
+ }
+ }
+}
+
+@Path("/pubsub/")
+class PubSub {
+ @GET
+ @Suspend
+ @Produces(Array("text/plain;charset=ISO-8859-1"))
+ @Path("/topic/{topic}/")
+ def subscribe(@PathParam("topic") topic: Broadcaster): Broadcastable = new Broadcastable("", topic)
+
+ @GET
+ @Broadcast
+ @Path("/topic/{topic}/{message}/")
+ @Produces(Array("text/plain;charset=ISO-8859-1"))
+ def say(@PathParam("topic") topic: Broadcaster, @PathParam("message") message: String): Broadcastable = new Broadcastable(message, topic)
+}
+
+/**
+ * Try service out by invoking (multiple times):
+ *
+ * Or browse to the URL from a web browser.
+ */
+@Path("/persistentscalacount")
+class PersistentSimpleService {
+ @GET
+ @Produces(Array("text/html"))
+ def count = {
+ //Fetch the first actor of type PersistentSimpleServiceActor
+ //Send it the "Tick" message and expect a NodeSeq back
+ val result = for{a <- actorFor[PersistentSimpleServiceActor]
+ r <- (a !! "Tick").as[NodeSeq]} yield r
+ //Return either the resulting NodeSeq or a default one
+ result getOrElse Error in counter
+ }
+}
+
+class PersistentSimpleServiceActor extends Actor {
+ private val KEY = "COUNTER"
+ private var hasStartedTicking = false
+ private lazy val storage = CassandraStorage.newMap
+
+ def receive = {
+ case "Tick" => if (hasStartedTicking) {
+ val count = atomic {
+ val bytes = storage.get(KEY.getBytes).get
+ val current = Integer.parseInt(new String(bytes, "UTF8"))
+ val updated = current + 1
+ storage.put(KEY.getBytes, (updated).toString.getBytes)
+ updated
+ }
+// val bytes = storage.get(KEY.getBytes).get
+// val counter = ByteBuffer.wrap(bytes).getInt
+// storage.put(KEY.getBytes, ByteBuffer.allocate(4).putInt(counter + 1).array)
+ self.reply(Tick:{count})
+ } else {
+ atomic {
+ storage.put(KEY.getBytes, "0".getBytes)
+ }
+// storage.put(KEY.getBytes, Array(0.toByte))
+ hasStartedTicking = true
+ self.reply(Tick: 0)
+ }
+ }
+}
+
+@Path("/chat")
+class Chat {
+ import ChatActor.ChatMsg
+ @Suspend
+ @GET
+ @Produces(Array("text/html"))
+ def suspend = ()
+
+ @POST
+ @Broadcast(Array(classOf[XSSHtmlFilter], classOf[JsonpFilter]))
+ @Consumes(Array("application/x-www-form-urlencoded"))
+ @Produces(Array("text/html"))
+ def publishMessage(form: MultivaluedMap[String, String]) = {
+ val msg = ChatMsg(form.getFirst("name"),form.getFirst("action"),form.getFirst("message"))
+ //Fetch the first actor of type ChatActor
+ //Send it the "Tick" message and expect a NodeSeq back
+ val result = for{a <- actorFor[ChatActor]
+ r <- (a !! msg).as[String]} yield r
+ //Return either the resulting String or a default one
+ result getOrElse "System__error"
+ }
+}
+
+object ChatActor {
+ case class ChatMsg(val who: String, val what: String, val msg: String)
+}
+
+class ChatActor extends Actor with Logging {
+ import ChatActor.ChatMsg
+ def receive = {
+ case ChatMsg(who, what, msg) => {
+ what match {
+ case "login" => self.reply("System Message__" + who + " has joined.")
+ case "post" => self.reply("" + who + "__" + msg)
+ case _ => throw new WebApplicationException(422)
+ }
+ }
+ case x => log.info("recieve unknown: " + x)
+ }
+}
+
+
+class JsonpFilter extends BroadcastFilter with Logging {
+ def filter(an: AnyRef) = {
+ val m = an.toString
+ var name = m
+ var message = ""
+
+ if (m.indexOf("__") > 0) {
+ name = m.substring(0, m.indexOf("__"))
+ message = m.substring(m.indexOf("__") + 2)
+ }
+
+ new BroadcastFilter.BroadcastAction("\n")
+ }
+}
diff --git a/akka-spring/src/main/scala/akka/ActorBeanDefinitionParser.scala b/akka-spring/src/main/scala/akka/ActorBeanDefinitionParser.scala
new file mode 100644
index 0000000000..2abb1024d8
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/ActorBeanDefinitionParser.scala
@@ -0,0 +1,93 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.springframework.beans.factory.support.BeanDefinitionBuilder
+import org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser
+import org.springframework.beans.factory.xml.ParserContext
+import AkkaSpringConfigurationTags._
+import org.w3c.dom.Element
+
+
+/**
+ * Parser for custom namespace configuration.
+ * @author michaelkober
+ */
+class TypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser {
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ val typedActorConf = parseActor(element)
+ typedActorConf.typed = TYPED_ACTOR_TAG
+ typedActorConf.setAsProperties(builder)
+ }
+
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean]
+}
+
+
+/**
+ * Parser for custom namespace configuration.
+ * @author michaelkober
+ */
+class UntypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser {
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ val untypedActorConf = parseActor(element)
+ untypedActorConf.typed = UNTYPED_ACTOR_TAG
+ untypedActorConf.setAsProperties(builder)
+ }
+
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean]
+}
+
+
+/**
+ * Parser for custom namespace configuration.
+ * @author michaelkober
+ */
+class ActorForBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorForParser {
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ val actorForConf = parseActorFor(element)
+ actorForConf.setAsProperties(builder)
+ }
+
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[ActorForFactoryBean]
+}
+
+/**
+ * Parser for custom namespace configuration.
+ * @author michaelkober
+ */
+class ConfigBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser {
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ val location = element.getAttribute(LOCATION)
+ builder.addPropertyValue(LOCATION, location)
+ }
+
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[ConfiggyPropertyPlaceholderConfigurer]
+
+ override def shouldGenerateId() = true
+}
diff --git a/akka-spring/src/main/scala/akka/ActorFactoryBean.scala b/akka-spring/src/main/scala/akka/ActorFactoryBean.scala
new file mode 100644
index 0000000000..0d3d407475
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/ActorFactoryBean.scala
@@ -0,0 +1,265 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.spring
+
+import org.springframework.beans.{BeanUtils,BeansException,BeanWrapper,BeanWrapperImpl}
+import akka.remote.{RemoteClient, RemoteServer}
+import org.springframework.beans.factory.config.AbstractFactoryBean
+import org.springframework.context.{ApplicationContext,ApplicationContextAware}
+import org.springframework.util.StringUtils
+
+import akka.actor.{ActorRef, AspectInitRegistry, TypedActorConfiguration, TypedActor,Actor}
+import akka.dispatch.MessageDispatcher
+import akka.util.{Logging, Duration}
+import scala.reflect.BeanProperty
+import java.net.InetSocketAddress
+
+/**
+ * Exception to use when something goes wrong during bean creation.
+ *
+ * @author Johan Rask
+ */
+class AkkaBeansException(message: String, cause:Throwable) extends BeansException(message, cause) {
+ def this(message: String) = this(message, null)
+}
+
+/**
+ * Factory bean for typed and untyped actors.
+ *
+ * @author michaelkober
+ * @author Johan Rask
+ * @author Martin Krasser
+ * @author Jonas Bonér
+ */
+class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with ApplicationContextAware {
+ import StringReflect._
+ import AkkaSpringConfigurationTags._
+
+ @BeanProperty var typed: String = ""
+ @BeanProperty var interface: String = ""
+ @BeanProperty var implementation: String = ""
+ @BeanProperty var beanRef: String = null
+ @BeanProperty var timeoutStr: String = ""
+ @BeanProperty var host: String = ""
+ @BeanProperty var port: String = ""
+ @BeanProperty var serverManaged: Boolean = false
+ @BeanProperty var serviceName: String = ""
+ @BeanProperty var lifecycle: String = ""
+ @BeanProperty var dispatcher: DispatcherProperties = _
+ @BeanProperty var scope: String = VAL_SCOPE_SINGLETON
+ @BeanProperty var property: PropertyEntries = _
+ @BeanProperty var applicationContext: ApplicationContext = _
+
+ lazy val timeout = parseTimeout
+
+ private def parseTimeout() : Long = {
+ var result = -1L
+ try {
+ result = if (!timeoutStr.isEmpty) timeoutStr.toLong else -1L
+ } catch {
+ case nfe: NumberFormatException =>
+ log.error(nfe, "could not parse timeout %s", timeoutStr)
+ throw nfe
+ }
+ result
+ }
+
+ // Holds info about if deps have been set or not. Depends on
+ // if interface is specified or not. We must set deps on
+ // target instance if interface is specified
+ var hasSetDependecies = false
+
+ override def isSingleton = scope.equals(VAL_SCOPE_SINGLETON)
+
+ /*
+ * @see org.springframework.beans.factory.FactoryBean#getObjectType()
+ */
+ def getObjectType: Class[AnyRef] = try {
+ implementation.toClass
+ } catch {
+ // required by contract to return null
+ case e: IllegalArgumentException => null
+ }
+
+ /*
+ * @see org.springframework.beans.factory.config.AbstractFactoryBean#createInstance()
+ */
+ def createInstance: AnyRef = {
+ val ref = typed match {
+ case TYPED_ACTOR_TAG => val typedActor = createTypedInstance()
+ setProperties(AspectInitRegistry.initFor(typedActor).targetInstance)
+ typedActor
+ case UNTYPED_ACTOR_TAG => val untypedActor = createUntypedInstance()
+ setProperties(untypedActor.actor)
+ untypedActor
+ case _ => throw new IllegalArgumentException("Unknown actor type")
+ }
+ ref
+ }
+
+ private[akka] def createTypedInstance() : AnyRef = {
+ if ((interface eq null) || interface == "") throw new AkkaBeansException(
+ "The 'interface' part of the 'akka:actor' element in the Spring config file can't be null or empty string")
+ if (((implementation eq null) || implementation == "") && (beanRef eq null)) throw new AkkaBeansException(
+ "Either 'implementation' or 'ref' must be specified as attribute of the 'akka:typed-actor' element in the Spring config file ")
+
+ val typedActor: AnyRef = if (beanRef eq null ) {
+ TypedActor.newInstance(interface.toClass, implementation.toClass, createConfig)
+ }
+ else
+ {
+ TypedActor.newInstance(interface.toClass, getBeanFactory().getBean(beanRef), createConfig)
+ }
+
+
+ if (isRemote && serverManaged) {
+ val server = RemoteServer.getOrCreateServer(new InetSocketAddress(host, port.toInt))
+ if (serviceName.isEmpty) {
+ server.registerTypedActor(interface, typedActor)
+ } else {
+ server.registerTypedActor(serviceName, typedActor)
+ }
+ }
+ typedActor
+ }
+
+ /**
+ * Create an UntypedActor.
+ */
+ private[akka] def createUntypedInstance() : ActorRef = {
+ if (((implementation eq null) || implementation == "") && (beanRef eq null)) throw new AkkaBeansException(
+ "Either 'implementation' or 'ref' must be specified as attribute of the 'akka:untyped-actor' element in the Spring config file ")
+ val actorRef = if (beanRef eq null )
+ Actor.actorOf(implementation.toClass)
+ else
+ Actor.actorOf(getBeanFactory().getBean(beanRef).asInstanceOf[Actor])
+
+ if (timeout > 0) {
+ actorRef.setTimeout(timeout)
+ }
+ if (isRemote) {
+ if (serverManaged) {
+ val server = RemoteServer.getOrCreateServer(new InetSocketAddress(host, port.toInt))
+ if (serviceName.isEmpty) {
+ server.register(actorRef)
+ } else {
+ server.register(serviceName, actorRef)
+ }
+ } else {
+ actorRef.makeRemote(host, port.toInt)
+ }
+ }
+ if (hasDispatcher) {
+ if (dispatcher.dispatcherType != THREAD_BASED){
+ actorRef.setDispatcher(dispatcherInstance())
+ } else {
+ actorRef.setDispatcher(dispatcherInstance(Some(actorRef)))
+ }
+ }
+ actorRef
+ }
+
+ /**
+ * Stop the typed actor if it is a singleton.
+ */
+ override def destroyInstance(instance: AnyRef) {
+ typed match {
+ case TYPED_ACTOR_TAG => TypedActor.stop(instance)
+ case UNTYPED_ACTOR_TAG => instance.asInstanceOf[ActorRef].stop
+ }
+ }
+
+ private def setProperties(ref: AnyRef): AnyRef = {
+ if (hasSetDependecies) return ref
+ log.debug("Processing properties and dependencies for implementation class\n\t[%s]", implementation)
+ val beanWrapper = new BeanWrapperImpl(ref);
+ if (ref.isInstanceOf[ApplicationContextAware]) {
+ log.debug("Setting application context")
+ beanWrapper.setPropertyValue("applicationContext", applicationContext)
+ }
+ for (entry <- property.entryList) {
+ val propertyDescriptor = BeanUtils.getPropertyDescriptor(ref.getClass, entry.name)
+ val method = propertyDescriptor.getWriteMethod
+ if (StringUtils.hasText(entry.ref)) {
+ log.debug("Setting property %s with bean ref %s using method %s", entry.name, entry.ref, method.getName)
+ method.invoke(ref,getBeanFactory().getBean(entry.ref))
+ } else if(StringUtils.hasText(entry.value)) {
+ log.debug("Setting property %s with value %s using method %s", entry.name, entry.value, method.getName)
+ beanWrapper.setPropertyValue(entry.name,entry.value)
+ } else throw new AkkaBeansException("Either property@ref or property@value must be set on property element")
+ }
+ ref
+ }
+
+
+ private[akka] def createConfig: TypedActorConfiguration = {
+ val config = new TypedActorConfiguration().timeout(Duration(timeout, "millis"))
+ if (isRemote && !serverManaged) config.makeRemote(host, port.toInt)
+ if (hasDispatcher) {
+ if (dispatcher.dispatcherType != THREAD_BASED) {
+ config.dispatcher(dispatcherInstance())
+ } else {
+ config.threadBasedDispatcher()
+ }
+ }
+ config
+ }
+
+ private[akka] def isRemote = (host ne null) && (!host.isEmpty)
+
+ private[akka] def hasDispatcher =
+ (dispatcher ne null) &&
+ (dispatcher.dispatcherType ne null) &&
+ (!dispatcher.dispatcherType.isEmpty)
+
+ /**
+ * Create dispatcher instance with dispatcher properties.
+ * @param actorRef actorRef for thread based dispatcher
+ * @return new dispatcher instance
+ */
+ private[akka] def dispatcherInstance(actorRef: Option[ActorRef] = None) : MessageDispatcher = {
+ import DispatcherFactoryBean._
+ if (dispatcher.dispatcherType != THREAD_BASED) {
+ createNewInstance(dispatcher)
+ } else {
+ createNewInstance(dispatcher, actorRef)
+ }
+ }
+}
+
+/**
+ * Factory bean for remote client actor-for.
+ *
+ * @author michaelkober
+ */
+class ActorForFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with ApplicationContextAware {
+ import StringReflect._
+ import AkkaSpringConfigurationTags._
+
+ @BeanProperty var interface: String = ""
+ @BeanProperty var host: String = ""
+ @BeanProperty var port: String = ""
+ @BeanProperty var serviceName: String = ""
+ @BeanProperty var applicationContext: ApplicationContext = _
+
+ override def isSingleton = false
+
+ /*
+ * @see org.springframework.beans.factory.FactoryBean#getObjectType()
+ */
+ def getObjectType: Class[AnyRef] = classOf[AnyRef]
+
+ /*
+ * @see org.springframework.beans.factory.config.AbstractFactoryBean#createInstance()
+ */
+ def createInstance: AnyRef = {
+ if (interface.isEmpty) {
+ RemoteClient.actorFor(serviceName, host, port.toInt)
+ } else {
+ RemoteClient.typedActorFor(interface.toClass, serviceName, host, port.toInt)
+ }
+ }
+}
+
diff --git a/akka-spring/src/main/scala/akka/ActorParser.scala b/akka-spring/src/main/scala/akka/ActorParser.scala
new file mode 100644
index 0000000000..da059d2f8e
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/ActorParser.scala
@@ -0,0 +1,231 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.springframework.util.xml.DomUtils
+import org.w3c.dom.Element
+import scala.collection.JavaConversions._
+import akka.util.Logging
+
+/**
+ * Parser trait for custom namespace configuration for typed-actor.
+ * @author michaelkober
+ * @author Johan Rask
+ * @author Martin Krasser
+ */
+trait ActorParser extends BeanParser with DispatcherParser {
+ import AkkaSpringConfigurationTags._
+
+ /**
+ * Parses the given element and returns a TypedActorProperties.
+ * @param element dom element to parse
+ * @return configuration for the typed actor
+ */
+ def parseActor(element: Element): ActorProperties = {
+ val objectProperties = new ActorProperties()
+ val remoteElement = DomUtils.getChildElementByTagName(element, REMOTE_TAG);
+ val dispatcherElement = DomUtils.getChildElementByTagName(element, DISPATCHER_TAG)
+ val propertyEntries = DomUtils.getChildElementsByTagName(element, PROPERTYENTRY_TAG)
+
+ if (remoteElement ne null) {
+ objectProperties.host = mandatory(remoteElement, HOST)
+ objectProperties.port = mandatory(remoteElement, PORT)
+ objectProperties.serverManaged = (remoteElement.getAttribute(MANAGED_BY) ne null) && (remoteElement.getAttribute(MANAGED_BY).equals(SERVER_MANAGED))
+ val serviceName = remoteElement.getAttribute(SERVICE_NAME)
+ if ((serviceName ne null) && (!serviceName.isEmpty)) {
+ objectProperties.serviceName = serviceName
+ objectProperties.serverManaged = true
+ }
+ }
+
+ if (dispatcherElement ne null) {
+ val dispatcherProperties = parseDispatcher(dispatcherElement)
+ objectProperties.dispatcher = dispatcherProperties
+ }
+
+ for (element <- propertyEntries) {
+ val entry = new PropertyEntry
+ entry.name = element.getAttribute("name");
+ entry.value = element.getAttribute("value")
+ entry.ref = element.getAttribute("ref")
+ objectProperties.propertyEntries.add(entry)
+ }
+
+ objectProperties.timeoutStr = element.getAttribute(TIMEOUT)
+ objectProperties.target = if (element.getAttribute(IMPLEMENTATION).isEmpty) null else element.getAttribute(IMPLEMENTATION)
+ objectProperties.beanRef = if (element.getAttribute(BEANREF).isEmpty) null else element.getAttribute(BEANREF)
+
+ if (objectProperties.target == null && objectProperties.beanRef == null) {
+ throw new IllegalArgumentException("Mandatory attribute missing, you need to provide either implementation or ref ")
+ }
+
+ if (element.hasAttribute(INTERFACE)) {
+ objectProperties.interface = element.getAttribute(INTERFACE)
+ }
+ if (element.hasAttribute(LIFECYCLE)) {
+ objectProperties.lifecycle = element.getAttribute(LIFECYCLE)
+ }
+ if (element.hasAttribute(SCOPE)) {
+ objectProperties.scope = element.getAttribute(SCOPE)
+ }
+
+ objectProperties
+ }
+
+}
+
+/**
+ * Parser trait for custom namespace configuration for RemoteClient actor-for.
+ * @author michaelkober
+ */
+trait ActorForParser extends BeanParser {
+ import AkkaSpringConfigurationTags._
+
+ /**
+ * Parses the given element and returns a ActorForProperties.
+ * @param element dom element to parse
+ * @return configuration for the typed actor
+ */
+ def parseActorFor(element: Element): ActorForProperties = {
+ val objectProperties = new ActorForProperties()
+
+ objectProperties.host = mandatory(element, HOST)
+ objectProperties.port = mandatory(element, PORT)
+ objectProperties.serviceName = mandatory(element, SERVICE_NAME)
+ if (element.hasAttribute(INTERFACE)) {
+ objectProperties.interface = element.getAttribute(INTERFACE)
+ }
+ objectProperties
+ }
+
+}
+
+/**
+ * Base trait with utility methods for bean parsing.
+ */
+trait BeanParser extends Logging {
+
+ /**
+ * Get a mandatory element attribute.
+ * @param element the element with the mandatory attribute
+ * @param attribute name of the mandatory attribute
+ */
+ def mandatory(element: Element, attribute: String): String = {
+ if ((element.getAttribute(attribute) eq null) || (element.getAttribute(attribute).isEmpty)) {
+ throw new IllegalArgumentException("Mandatory attribute missing: " + attribute)
+ } else {
+ element.getAttribute(attribute)
+ }
+ }
+
+ /**
+ * Get a mandatory child element.
+ * @param element the parent element
+ * @param childName name of the mandatory child element
+ */
+ def mandatoryElement(element: Element, childName: String): Element = {
+ val childElement = DomUtils.getChildElementByTagName(element, childName);
+ if (childElement eq null) {
+ throw new IllegalArgumentException("Mandatory element missing: ''")
+ } else {
+ childElement
+ }
+ }
+
+}
+
+
+/**
+ * Parser trait for custom namespace for Akka dispatcher configuration.
+ * @author michaelkober
+ */
+trait DispatcherParser extends BeanParser {
+ import AkkaSpringConfigurationTags._
+
+ /**
+ * Parses the given element and returns a DispatcherProperties.
+ * @param element dom element to parse
+ * @return configuration for the dispatcher
+ */
+ def parseDispatcher(element: Element): DispatcherProperties = {
+ val properties = new DispatcherProperties()
+ var dispatcherElement = element
+ if (hasRef(element)) {
+ val ref = element.getAttribute(REF)
+ dispatcherElement = element.getOwnerDocument.getElementById(ref)
+ if (dispatcherElement eq null) {
+ throw new IllegalArgumentException("Referenced dispatcher not found: '" + ref + "'")
+ }
+ }
+
+ properties.dispatcherType = mandatory(dispatcherElement, TYPE)
+ if (properties.dispatcherType == THREAD_BASED) {
+ val allowedParentNodes = "akka:typed-actor" :: "akka:untyped-actor" :: "typed-actor" :: "untyped-actor" :: Nil
+ if (!allowedParentNodes.contains(dispatcherElement.getParentNode.getNodeName)) {
+ throw new IllegalArgumentException("Thread based dispatcher must be nested in 'typed-actor' or 'untyped-actor' element!")
+ }
+ }
+
+ if (properties.dispatcherType == HAWT) { // no name for HawtDispatcher
+ properties.name = dispatcherElement.getAttribute(NAME)
+ if (dispatcherElement.hasAttribute(AGGREGATE)) {
+ properties.aggregate = dispatcherElement.getAttribute(AGGREGATE).toBoolean
+ }
+ } else {
+ properties.name = mandatory(dispatcherElement, NAME)
+ }
+
+ val threadPoolElement = DomUtils.getChildElementByTagName(dispatcherElement, THREAD_POOL_TAG);
+ if (threadPoolElement ne null) {
+ if (properties.dispatcherType == THREAD_BASED) {
+ throw new IllegalArgumentException("Element 'thread-pool' not allowed for this dispatcher type.")
+ }
+ val threadPoolProperties = parseThreadPool(threadPoolElement)
+ properties.threadPool = threadPoolProperties
+ }
+ properties
+ }
+
+ /**
+ * Parses the given element and returns a ThreadPoolProperties.
+ * @param element dom element to parse
+ * @return configuration for the thread pool
+ */
+ def parseThreadPool(element: Element): ThreadPoolProperties = {
+ val properties = new ThreadPoolProperties()
+ properties.queue = element.getAttribute(QUEUE)
+ if (element.hasAttribute(CAPACITY)) {
+ properties.capacity = element.getAttribute(CAPACITY).toInt
+ }
+ if (element.hasAttribute(BOUND)) {
+ properties.bound = element.getAttribute(BOUND).toInt
+ }
+ if (element.hasAttribute(FAIRNESS)) {
+ properties.fairness = element.getAttribute(FAIRNESS).toBoolean
+ }
+ if (element.hasAttribute(CORE_POOL_SIZE)) {
+ properties.corePoolSize = element.getAttribute(CORE_POOL_SIZE).toInt
+ }
+ if (element.hasAttribute(MAX_POOL_SIZE)) {
+ properties.maxPoolSize = element.getAttribute(MAX_POOL_SIZE).toInt
+ }
+ if (element.hasAttribute(KEEP_ALIVE)) {
+ properties.keepAlive = element.getAttribute(KEEP_ALIVE).toLong
+ }
+ if (element.hasAttribute(REJECTION_POLICY)) {
+ properties.rejectionPolicy = element.getAttribute(REJECTION_POLICY)
+ }
+ if (element.hasAttribute(MAILBOX_CAPACITY)) {
+ properties.mailboxCapacity = element.getAttribute(MAILBOX_CAPACITY).toInt
+ }
+ properties
+ }
+
+ def hasRef(element: Element): Boolean = {
+ val ref = element.getAttribute(REF)
+ (ref ne null) && !ref.isEmpty
+ }
+
+}
+
diff --git a/akka-spring/src/main/scala/akka/ActorProperties.scala b/akka-spring/src/main/scala/akka/ActorProperties.scala
new file mode 100644
index 0000000000..d0e7c49392
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/ActorProperties.scala
@@ -0,0 +1,78 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.spring
+
+import org.springframework.beans.factory.support.BeanDefinitionBuilder
+import AkkaSpringConfigurationTags._
+
+/**
+ * Data container for actor configuration data.
+ * @author michaelkober
+ * @author Martin Krasser
+ */
+class ActorProperties {
+ var typed: String = ""
+ var target: String = ""
+ var beanRef: String = ""
+ var timeoutStr: String = ""
+ var interface: String = ""
+ var host: String = ""
+ var port: String = ""
+ var serverManaged: Boolean = false
+ var serviceName: String = ""
+ var lifecycle: String = ""
+ var scope:String = VAL_SCOPE_SINGLETON
+ var dispatcher: DispatcherProperties = _
+ var propertyEntries = new PropertyEntries()
+
+
+ /**
+ * Sets the properties to the given builder.
+ * @param builder bean definition builder
+ */
+ def setAsProperties(builder: BeanDefinitionBuilder) {
+ builder.addPropertyValue("typed", typed)
+ builder.addPropertyValue(HOST, host)
+ builder.addPropertyValue(PORT, port)
+ builder.addPropertyValue("serverManaged", serverManaged)
+ builder.addPropertyValue("serviceName", serviceName)
+ builder.addPropertyValue("timeoutStr", timeoutStr)
+ builder.addPropertyValue(IMPLEMENTATION, target)
+ builder.addPropertyValue("beanRef", beanRef)
+ builder.addPropertyValue(INTERFACE, interface)
+ builder.addPropertyValue(LIFECYCLE, lifecycle)
+ builder.addPropertyValue(SCOPE, scope)
+ builder.addPropertyValue(DISPATCHER_TAG, dispatcher)
+ builder.addPropertyValue(PROPERTYENTRY_TAG,propertyEntries)
+ }
+
+ def timeout() : Long = {
+ if (!timeoutStr.isEmpty) timeoutStr.toLong else -1L
+ }
+
+}
+
+/**
+ * Data container for actor configuration data.
+ * @author michaelkober
+ */
+class ActorForProperties {
+ var interface: String = ""
+ var host: String = ""
+ var port: String = ""
+ var serviceName: String = ""
+
+ /**
+ * Sets the properties to the given builder.
+ * @param builder bean definition builder
+ */
+ def setAsProperties(builder: BeanDefinitionBuilder) {
+ builder.addPropertyValue(HOST, host)
+ builder.addPropertyValue(PORT, port)
+ builder.addPropertyValue("serviceName", serviceName)
+ builder.addPropertyValue(INTERFACE, interface)
+ }
+
+}
diff --git a/akka-spring/src/main/scala/akka/AkkaNamespaceHandler.scala b/akka-spring/src/main/scala/akka/AkkaNamespaceHandler.scala
new file mode 100644
index 0000000000..38041a3ea4
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/AkkaNamespaceHandler.scala
@@ -0,0 +1,23 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.springframework.beans.factory.xml.NamespaceHandlerSupport
+import AkkaSpringConfigurationTags._
+
+/**
+ * Custom spring namespace handler for Akka.
+ * @author michaelkober
+ */
+class AkkaNamespaceHandler extends NamespaceHandlerSupport {
+ def init = {
+ registerBeanDefinitionParser(CONFIG_TAG, new ConfigBeanDefinitionParser());
+ registerBeanDefinitionParser(TYPED_ACTOR_TAG, new TypedActorBeanDefinitionParser())
+ registerBeanDefinitionParser(UNTYPED_ACTOR_TAG, new UntypedActorBeanDefinitionParser())
+ registerBeanDefinitionParser(SUPERVISION_TAG, new SupervisionBeanDefinitionParser())
+ registerBeanDefinitionParser(DISPATCHER_TAG, new DispatcherBeanDefinitionParser())
+ registerBeanDefinitionParser(CAMEL_SERVICE_TAG, new CamelServiceBeanDefinitionParser)
+ registerBeanDefinitionParser(ACTOR_FOR_TAG, new ActorForBeanDefinitionParser());
+ }
+}
diff --git a/akka-spring/src/main/scala/akka/AkkaSpringConfigurationTags.scala b/akka-spring/src/main/scala/akka/AkkaSpringConfigurationTags.scala
new file mode 100644
index 0000000000..6548495070
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/AkkaSpringConfigurationTags.scala
@@ -0,0 +1,115 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+/**
+ * XML configuration tags.
+ * @author michaelkober
+ * @author Martin Krasser
+ */
+object AkkaSpringConfigurationTags {
+
+ // --- TAGS
+ //
+ // top level tags
+ val CONFIG_TAG = "property-placeholder"
+ val TYPED_ACTOR_TAG = "typed-actor"
+ val UNTYPED_ACTOR_TAG = "untyped-actor"
+ val SUPERVISION_TAG = "supervision"
+ val DISPATCHER_TAG = "dispatcher"
+ val PROPERTYENTRY_TAG = "property"
+ val CAMEL_SERVICE_TAG = "camel-service"
+ val ACTOR_FOR_TAG = "actor-for"
+
+ // actor sub tags
+ val REMOTE_TAG = "remote"
+
+ // superivision sub tags
+ val TYPED_ACTORS_TAG = "typed-actors"
+ val UNTYPED_ACTORS_TAG = "untyped-actors"
+ val STRATEGY_TAG = "restart-strategy"
+ val TRAP_EXISTS_TAG = "trap-exits"
+ val TRAP_EXIT_TAG = "trap-exit"
+
+ // dispatcher sub tags
+ val THREAD_POOL_TAG = "thread-pool"
+
+ // camel-service sub tags
+ val CAMEL_CONTEXT_TAG = "camel-context"
+
+ // --- ATTRIBUTES
+ //
+ // actor attributes
+ val TIMEOUT = "timeout"
+ val IMPLEMENTATION = "implementation"
+ val BEANREF = "ref"
+ val INTERFACE = "interface"
+ val HOST = "host"
+ val PORT = "port"
+ val MANAGED_BY = "managed-by"
+ val SERVICE_NAME = "service-name"
+ val LIFECYCLE = "lifecycle"
+ val SCOPE = "scope"
+
+ // supervision attributes
+ val FAILOVER = "failover"
+ val RETRIES = "retries"
+ val TIME_RANGE = "timerange"
+
+ // dispatcher attributes
+ val NAME = "name"
+ val REF = "ref"
+ val TYPE = "type"
+ val AGGREGATE = "aggregate" // HawtDispatcher
+
+ // thread pool attributes
+ val QUEUE = "queue"
+ val CAPACITY = "capacity"
+ val FAIRNESS = "fairness"
+ val CORE_POOL_SIZE = "core-pool-size"
+ val MAX_POOL_SIZE = "max-pool-size"
+ val KEEP_ALIVE = "keep-alive"
+ val BOUND ="bound"
+ val REJECTION_POLICY ="rejection-policy"
+ val MAILBOX_CAPACITY ="mailbox-capacity"
+
+ // config attribute
+ val LOCATION = "location"
+
+ // --- VALUES
+ //
+ // Lifecycle
+ val VAL_LIFECYCYLE_TEMPORARY = "temporary"
+ val VAL_LIFECYCYLE_PERMANENT = "permanent"
+
+ val VAL_SCOPE_SINGLETON = "singleton"
+ val VAL_SCOPE_PROTOTYPE = "prototype"
+
+ // Failover
+ val VAL_ALL_FOR_ONE = "AllForOne"
+ val VAL_ONE_FOR_ONE = "OneForOne"
+
+ // rejection policies
+ val VAL_ABORT_POLICY = "abort-policy"
+ val VAL_CALLER_RUNS_POLICY = "caller-runs-policy"
+ val VAL_DISCARD_OLDEST_POLICY = "discard-oldest-policy"
+ val VAL_DISCARD_POLICY = "discard-policy"
+
+ // dispatcher queue types
+ val VAL_BOUNDED_LINKED_BLOCKING_QUEUE = "bounded-linked-blocking-queue"
+ val VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE = "unbounded-linked-blocking-queue"
+ val VAL_SYNCHRONOUS_QUEUE = "synchronous-queue"
+ val VAL_BOUNDED_ARRAY_BLOCKING_QUEUE = "bounded-array-blocking-queue"
+
+ // dispatcher types
+ val EXECUTOR_BASED_EVENT_DRIVEN = "executor-based-event-driven"
+ val EXECUTOR_BASED_EVENT_DRIVEN_WORK_STEALING = "executor-based-event-driven-work-stealing"
+ val THREAD_BASED = "thread-based"
+ val HAWT = "hawt"
+
+ // managed by types
+ val SERVER_MANAGED = "server"
+ val CLIENT_MANAGED = "client"
+
+}
diff --git a/akka-spring/src/main/scala/akka/CamelServiceBeanDefinitionParser.scala b/akka-spring/src/main/scala/akka/CamelServiceBeanDefinitionParser.scala
new file mode 100644
index 0000000000..4025a831a8
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/CamelServiceBeanDefinitionParser.scala
@@ -0,0 +1,41 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.springframework.beans.factory.support.BeanDefinitionBuilder
+import org.springframework.beans.factory.xml.{ParserContext, AbstractSingleBeanDefinitionParser}
+import org.springframework.util.xml.DomUtils
+import org.w3c.dom.Element
+
+import akka.spring.AkkaSpringConfigurationTags._
+
+
+/**
+ * Parser for <camel-service> elements.
+ *
+ * @author Martin Krasser
+ */
+class CamelServiceBeanDefinitionParser extends AbstractSingleBeanDefinitionParser {
+ /**
+ * Parses the <camel-service> element. If a nested <camel-context> element
+ * is defined then the referenced context is set on the {@link CamelServiceFactoryBean}.
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ val camelContextElement = DomUtils.getChildElementByTagName(element, CAMEL_CONTEXT_TAG);
+ if (camelContextElement ne null) {
+ val camelContextReference = camelContextElement.getAttribute("ref")
+ builder.addPropertyReference("camelContext", camelContextReference)
+ }
+ }
+
+ /**
+ * Returns the class of {@link CamelServiceFactoryBean}
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[CamelServiceFactoryBean]
+
+ /**
+ * Returns true.
+ */
+ override def shouldGenerateIdAsFallback = true
+}
diff --git a/akka-spring/src/main/scala/akka/CamelServiceFactoryBean.scala b/akka-spring/src/main/scala/akka/CamelServiceFactoryBean.scala
new file mode 100644
index 0000000000..337413f0eb
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/CamelServiceFactoryBean.scala
@@ -0,0 +1,45 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.apache.camel.CamelContext
+import org.springframework.beans.factory.{DisposableBean, InitializingBean, FactoryBean}
+
+import akka.camel.{CamelContextManager, CamelService, CamelServiceFactory}
+
+/**
+ * Factory bean for a {@link CamelService}.
+ *
+ * @author Martin Krasser
+ */
+class CamelServiceFactoryBean extends FactoryBean[CamelService] with InitializingBean with DisposableBean {
+ @scala.reflect.BeanProperty var camelContext: CamelContext = _
+
+ var instance: CamelService = _
+
+ def isSingleton = true
+
+ def getObjectType = classOf[CamelService]
+
+ def getObject = instance
+
+ /**
+ * Initializes the {@link CamelContextManager} with camelService if defined, then
+ * creates and starts the {@link CamelService} singleton.
+ */
+ def afterPropertiesSet = {
+ if (camelContext ne null) {
+ CamelContextManager.init(camelContext)
+ }
+ instance = CamelServiceFactory.createCamelService
+ instance.start
+ }
+
+ /**
+ * Stops the {@link CamelService} singleton.
+ */
+ def destroy = {
+ instance.stop
+ }
+}
diff --git a/akka-spring/src/main/scala/akka/ConfiggyPropertyPlaceholderConfigurer.scala b/akka-spring/src/main/scala/akka/ConfiggyPropertyPlaceholderConfigurer.scala
new file mode 100644
index 0000000000..e32c331688
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/ConfiggyPropertyPlaceholderConfigurer.scala
@@ -0,0 +1,37 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.springframework.beans.factory.config.PropertyPlaceholderConfigurer
+import org.springframework.core.io.Resource
+import net.lag.configgy.Configgy
+import java.util.Properties
+
+/**
+ * ConfiggyPropertyPlaceholderConfigurer. Property resource configurer for configgy files.
+ */
+class ConfiggyPropertyPlaceholderConfigurer extends PropertyPlaceholderConfigurer {
+
+ /**
+ * Sets the akka properties as local properties, leaves the location empty.
+ * @param configgyResource akka.conf
+ */
+ override def setLocation(configgyResource: Resource) {
+ if (configgyResource eq null) throw new IllegalArgumentException("Property 'config' must be set")
+ val properties = loadAkkaConfig(configgyResource)
+ setProperties(properties)
+ }
+
+ /**
+ * Load the akka.conf and transform to properties.
+ */
+ private def loadAkkaConfig(configgyResource: Resource) : Properties = {
+ Configgy.configure(configgyResource.getFile.getPath)
+ val config = Configgy.config
+ val properties = new Properties()
+ config.asMap.foreach {case (k, v) => properties.put(k, v); println("(k,v)=" + k + ", " + v)}
+ properties
+ }
+
+}
diff --git a/akka-spring/src/main/scala/akka/DispatcherBeanDefinitionParser.scala b/akka-spring/src/main/scala/akka/DispatcherBeanDefinitionParser.scala
new file mode 100644
index 0000000000..4f2a40469f
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/DispatcherBeanDefinitionParser.scala
@@ -0,0 +1,28 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.w3c.dom.Element
+import org.springframework.beans.factory.support.BeanDefinitionBuilder
+import org.springframework.beans.factory.xml.{ParserContext, AbstractSingleBeanDefinitionParser}
+
+
+/**
+ * Parser for custom namespace configuration.
+ * @author michaelkober
+ */
+class DispatcherBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser with DispatcherParser {
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ val dispatcherProperties = parseDispatcher(element)
+ dispatcherProperties.setAsProperties(builder)
+ }
+
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[DispatcherFactoryBean]
+}
diff --git a/akka-spring/src/main/scala/akka/DispatcherFactoryBean.scala b/akka-spring/src/main/scala/akka/DispatcherFactoryBean.scala
new file mode 100644
index 0000000000..bdcfca5d33
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/DispatcherFactoryBean.scala
@@ -0,0 +1,110 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.springframework.beans.factory.config.AbstractFactoryBean
+import akka.config.Supervision._
+import AkkaSpringConfigurationTags._
+import reflect.BeanProperty
+import akka.actor.ActorRef
+import java.util.concurrent.RejectedExecutionHandler
+import java.util.concurrent.ThreadPoolExecutor.{DiscardPolicy, DiscardOldestPolicy, CallerRunsPolicy, AbortPolicy}
+import akka.dispatch._
+import akka.util.Duration
+
+/**
+ * Reusable factory method for dispatchers.
+ */
+object DispatcherFactoryBean {
+
+ /**
+ * factory method for dispatchers
+ * @param properties dispatcher properties
+ * @param actorRef actorRef needed for thread based dispatcher
+ */
+ def createNewInstance(properties: DispatcherProperties, actorRef: Option[ActorRef] = None): MessageDispatcher = {
+
+ //Creates a ThreadPoolConfigDispatcherBuilder and applies the configuration to it
+ def configureThreadPool(createDispatcher: => (ThreadPoolConfig) => MessageDispatcher): ThreadPoolConfigDispatcherBuilder = {
+ if ((properties.threadPool ne null) && (properties.threadPool.queue ne null)) {
+ import ThreadPoolConfigDispatcherBuilder.conf_?
+ import properties._
+ val queueDef = Some(threadPool.queue)
+ val corePoolSize = if (threadPool.corePoolSize > -1) Some(threadPool.corePoolSize) else None
+ val maxPoolSize = if (threadPool.maxPoolSize > -1) Some(threadPool.maxPoolSize) else None
+ val keepAlive = if (threadPool.keepAlive > -1) Some(threadPool.keepAlive) else None
+ val executorBounds = if (threadPool.bound > -1) Some(threadPool.bound) else None
+ val flowHandler = threadPool.rejectionPolicy match {
+ case null | "" => None
+ case "abort-policy" => Some(new AbortPolicy())
+ case "caller-runs-policy" => Some(new CallerRunsPolicy())
+ case "discard-oldest-policy" => Some(new DiscardOldestPolicy())
+ case "discard-policy" => Some(new DiscardPolicy())
+ case x => throw new IllegalArgumentException("Unknown rejection-policy '" + x + "'")
+ }
+
+ //Apply the following options to the config if they are present in the cfg
+ ThreadPoolConfigDispatcherBuilder(createDispatcher,ThreadPoolConfig()).configure(
+ conf_?(queueDef )(definition => definition match {
+ case VAL_BOUNDED_ARRAY_BLOCKING_QUEUE =>
+ _.withNewThreadPoolWithArrayBlockingQueueWithCapacityAndFairness(threadPool.capacity,threadPool.fairness)
+ case VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE if threadPool.capacity > -1 =>
+ _.withNewThreadPoolWithLinkedBlockingQueueWithCapacity(threadPool.capacity)
+ case VAL_UNBOUNDED_LINKED_BLOCKING_QUEUE if threadPool.capacity <= 0 =>
+ _.withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
+ case VAL_BOUNDED_LINKED_BLOCKING_QUEUE =>
+ _.withNewBoundedThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity(threadPool.bound)
+ case VAL_SYNCHRONOUS_QUEUE =>
+ _.withNewThreadPoolWithSynchronousQueueWithFairness(threadPool.fairness)
+ case unknown =>
+ throw new IllegalArgumentException("Unknown queue type " + unknown)
+ }),
+ conf_?(keepAlive )(time => _.setKeepAliveTimeInMillis(time)),
+ conf_?(corePoolSize )(count => _.setCorePoolSize(count)),
+ conf_?(maxPoolSize )(count => _.setMaxPoolSize(count)),
+ conf_?(executorBounds)(bounds => _.setExecutorBounds(bounds)),
+ conf_?(flowHandler )(policy => _.setRejectionPolicy(policy)))
+ }
+ else
+ ThreadPoolConfigDispatcherBuilder(createDispatcher,ThreadPoolConfig())
+ }
+
+ //Create the dispatcher
+ properties.dispatcherType match {
+ case EXECUTOR_BASED_EVENT_DRIVEN =>
+ configureThreadPool(poolConfig => new ExecutorBasedEventDrivenDispatcher(properties.name, poolConfig)).build
+ case EXECUTOR_BASED_EVENT_DRIVEN_WORK_STEALING =>
+ configureThreadPool(poolConfig => new ExecutorBasedEventDrivenWorkStealingDispatcher(properties.name,Dispatchers.MAILBOX_TYPE,poolConfig)).build
+ case THREAD_BASED if actorRef.isEmpty =>
+ throw new IllegalArgumentException("Need an ActorRef to create a thread based dispatcher.")
+ case THREAD_BASED if actorRef.isDefined =>
+ Dispatchers.newThreadBasedDispatcher(actorRef.get)
+ case HAWT =>
+ Dispatchers.newHawtDispatcher(properties.aggregate)
+ case unknown =>
+ throw new IllegalArgumentException("Unknown dispatcher type " + unknown)
+ }
+ }
+}
+
+/**
+ * Factory bean for supervisor configuration.
+ * @author michaelkober
+ */
+class DispatcherFactoryBean extends AbstractFactoryBean[MessageDispatcher] {
+ @BeanProperty var properties: DispatcherProperties = _
+
+ /*
+ * @see org.springframework.beans.factory.FactoryBean#getObjectType()
+ */
+ def getObjectType: Class[MessageDispatcher] = classOf[MessageDispatcher]
+
+ /*
+ * @see org.springframework.beans.factory.config.AbstractFactoryBean#createInstance()
+ */
+ def createInstance: MessageDispatcher = {
+ import DispatcherFactoryBean._
+ createNewInstance(properties)
+ }
+}
diff --git a/akka-spring/src/main/scala/akka/DispatcherProperties.scala b/akka-spring/src/main/scala/akka/DispatcherProperties.scala
new file mode 100644
index 0000000000..8dd33602df
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/DispatcherProperties.scala
@@ -0,0 +1,61 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.springframework.beans.factory.support.BeanDefinitionBuilder
+
+/**
+ * Data container for dispatcher configuration data.
+ * @author michaelkober
+ */
+class DispatcherProperties {
+ var ref: String = ""
+ var dispatcherType: String = ""
+ var name: String = ""
+ var threadPool: ThreadPoolProperties = _
+ var aggregate = true
+
+ /**
+ * Sets the properties to the given builder.
+ * @param builder bean definition builder
+ */
+ def setAsProperties(builder: BeanDefinitionBuilder) {
+ builder.addPropertyValue("properties", this)
+ }
+
+ override def toString : String = {
+ "DispatcherProperties[ref=" + ref +
+ ", dispatcher-type=" + dispatcherType +
+ ", name=" + name +
+ ", threadPool=" + threadPool + "]"
+ }
+}
+
+/**
+ * Data container for thread pool configuration data.
+ * @author michaelkober
+ */
+class ThreadPoolProperties {
+ var queue = ""
+ var bound = -1
+ var capacity = -1
+ var fairness = false
+ var corePoolSize = -1
+ var maxPoolSize = -1
+ var keepAlive = -1L
+ var rejectionPolicy = ""
+ var mailboxCapacity = -1
+
+ override def toString : String = {
+ "ThreadPoolProperties[queue=" + queue +
+ ", bound=" + bound +
+ ", capacity=" + capacity +
+ ", fairness=" + fairness +
+ ", corePoolSize=" + corePoolSize +
+ ", maxPoolSize=" + maxPoolSize +
+ ", keepAlive=" + keepAlive +
+ ", policy=" + rejectionPolicy +
+ ", mailboxCapacity=" + mailboxCapacity + "]"
+ }
+}
diff --git a/akka-spring/src/main/scala/akka/PropertyEntries.scala b/akka-spring/src/main/scala/akka/PropertyEntries.scala
new file mode 100644
index 0000000000..9f6493bbb3
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/PropertyEntries.scala
@@ -0,0 +1,36 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.springframework.beans.factory.support.BeanDefinitionBuilder
+
+import scala.collection.mutable._
+
+/**
+ * Simple container for Properties
+ * @author Johan Rask
+ */
+class PropertyEntries {
+ var entryList: ListBuffer[PropertyEntry] = ListBuffer[PropertyEntry]()
+
+ def add(entry: PropertyEntry) = {
+ entryList.append(entry)
+ }
+}
+
+/**
+ * Represents a property element
+ * @author Johan Rask
+ */
+class PropertyEntry {
+ var name: String = _
+ var value: String = null
+ var ref: String = null
+
+
+ override def toString(): String = {
+ format("name = %s,value = %s, ref = %s", name, value, ref)
+ }
+}
+
diff --git a/akka-spring/src/main/scala/akka/StringReflect.scala b/akka-spring/src/main/scala/akka/StringReflect.scala
new file mode 100644
index 0000000000..2b77f8caa6
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/StringReflect.scala
@@ -0,0 +1,25 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.spring
+
+object StringReflect {
+
+ /**
+ * Implicit conversion from String to StringReflect.
+ */
+ implicit def string2StringReflect(x: String) = new StringReflect(x)
+}
+
+/**
+ * Reflection helper class.
+ * @author michaelkober
+ */
+class StringReflect(val self: String) {
+ if ((self eq null) || self == "") throw new IllegalArgumentException("Class name can't be null or empty string [" + self + "]")
+ def toClass[T <: AnyRef]: Class[T] = {
+ val clazz = Class.forName(self)
+ clazz.asInstanceOf[Class[T]]
+ }
+}
diff --git a/akka-spring/src/main/scala/akka/SupervisionBeanDefinitionParser.scala b/akka-spring/src/main/scala/akka/SupervisionBeanDefinitionParser.scala
new file mode 100644
index 0000000000..c4753d1d5b
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/SupervisionBeanDefinitionParser.scala
@@ -0,0 +1,86 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import akka.util.Logging
+import org.springframework.beans.factory.support.BeanDefinitionBuilder
+import org.springframework.beans.factory.xml.{ParserContext, AbstractSingleBeanDefinitionParser}
+import akka.config.Supervision._
+import AkkaSpringConfigurationTags._
+
+
+import org.w3c.dom.Element
+import org.springframework.util.xml.DomUtils
+
+
+/**
+ * Parser for custom namespace for Akka declarative supervisor configuration.
+ * @author michaelkober
+ */
+class SupervisionBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser {
+ /* (non-Javadoc)
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ parseSupervisor(element, builder)
+ }
+
+ /**
+ * made accessible for testing
+ */
+ private[akka] def parseSupervisor(element: Element, builder: BeanDefinitionBuilder) {
+ val strategyElement = mandatoryElement(element, STRATEGY_TAG)
+ val typedActorsElement = DomUtils.getChildElementByTagName(element, TYPED_ACTORS_TAG)
+ val untypedActorsElement = DomUtils.getChildElementByTagName(element, UNTYPED_ACTORS_TAG)
+ if ((typedActorsElement eq null) && (untypedActorsElement eq null)) {
+ throw new IllegalArgumentException("One of 'akka:typed-actors' or 'akka:untyped-actors' needed.")
+ }
+ parseRestartStrategy(strategyElement, builder)
+ if (typedActorsElement ne null) {
+ builder.addPropertyValue("typed", AkkaSpringConfigurationTags.TYPED_ACTOR_TAG)
+ parseTypedActorList(typedActorsElement, builder)
+ } else {
+ builder.addPropertyValue("typed", AkkaSpringConfigurationTags.UNTYPED_ACTOR_TAG)
+ parseUntypedActorList(untypedActorsElement, builder)
+ }
+ }
+
+ private[akka] def parseRestartStrategy(element: Element, builder: BeanDefinitionBuilder) {
+ val failover = mandatory(element, FAILOVER)
+ val timeRange = mandatory(element, TIME_RANGE).toInt
+ val retries = mandatory(element, RETRIES).toInt
+ val trapExitsElement = mandatoryElement(element, TRAP_EXISTS_TAG)
+ val trapExceptions = parseTrapExits(trapExitsElement)
+
+ val restartStrategy = failover match {
+ case "AllForOne" => new AllForOneStrategy(trapExceptions, retries, timeRange)
+ case "OneForOne" => new OneForOneStrategy(trapExceptions, retries, timeRange)
+ case _ => new OneForOneStrategy(trapExceptions, retries, timeRange) //Default to OneForOne
+ }
+ builder.addPropertyValue("restartStrategy", restartStrategy)
+ }
+
+ private[akka] def parseTypedActorList(element: Element, builder: BeanDefinitionBuilder) {
+ val typedActors = DomUtils.getChildElementsByTagName(element, TYPED_ACTOR_TAG).toArray.toList.asInstanceOf[List[Element]]
+ val actorProperties = typedActors.map(parseActor(_))
+ builder.addPropertyValue("supervised", actorProperties)
+ }
+
+ private[akka] def parseUntypedActorList(element: Element, builder: BeanDefinitionBuilder) {
+ val untypedActors = DomUtils.getChildElementsByTagName(element, UNTYPED_ACTOR_TAG).toArray.toList.asInstanceOf[List[Element]]
+ val actorProperties = untypedActors.map(parseActor(_))
+ builder.addPropertyValue("supervised", actorProperties)
+ }
+
+ private def parseTrapExits(element: Element): Array[Class[_ <: Throwable]] = {
+ import StringReflect._
+ val trapExits = DomUtils.getChildElementsByTagName(element, TRAP_EXIT_TAG).toArray.toList.asInstanceOf[List[Element]]
+ trapExits.map(DomUtils.getTextValue(_).toClass.asInstanceOf[Class[_ <: Throwable]]).toArray
+ }
+
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[SupervisionFactoryBean]
+}
diff --git a/akka-spring/src/main/scala/akka/SupervisionFactoryBean.scala b/akka-spring/src/main/scala/akka/SupervisionFactoryBean.scala
new file mode 100644
index 0000000000..5e88374f18
--- /dev/null
+++ b/akka-spring/src/main/scala/akka/SupervisionFactoryBean.scala
@@ -0,0 +1,97 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package akka.spring
+
+import org.springframework.beans.factory.config.AbstractFactoryBean
+import akka.config.Supervision._
+import akka.actor.{Supervisor, SupervisorFactory, Actor}
+import AkkaSpringConfigurationTags._
+import reflect.BeanProperty
+import akka.config.{TypedActorConfigurator, RemoteAddress}
+
+/**
+ * Factory bean for supervisor configuration.
+ * @author michaelkober
+ */
+class SupervisionFactoryBean extends AbstractFactoryBean[AnyRef] {
+ @BeanProperty var restartStrategy: FaultHandlingStrategy = _
+ @BeanProperty var supervised: List[ActorProperties] = _
+ @BeanProperty var typed: String = ""
+
+ /*
+ * @see org.springframework.beans.factory.FactoryBean#getObjectType()
+ */
+ def getObjectType: Class[AnyRef] = classOf[AnyRef]
+
+ /*
+ * @see org.springframework.beans.factory.config.AbstractFactoryBean#createInstance()
+ */
+ def createInstance: AnyRef = typed match {
+ case AkkaSpringConfigurationTags.TYPED_ACTOR_TAG => createInstanceForTypedActors
+ case AkkaSpringConfigurationTags.UNTYPED_ACTOR_TAG => createInstanceForUntypedActors
+ }
+
+ private def createInstanceForTypedActors() : TypedActorConfigurator = {
+ val configurator = new TypedActorConfigurator()
+ configurator.configure(
+ restartStrategy,
+ supervised.map(createComponent(_)).toArray
+ ).supervise
+
+ }
+
+ private def createInstanceForUntypedActors() : Supervisor = {
+ val factory = new SupervisorFactory(
+ new SupervisorConfig(
+ restartStrategy,
+ supervised.map(createSupervise(_))))
+ factory.newInstance
+ }
+
+ /**
+ * Create configuration for TypedActor
+ */
+ private[akka] def createComponent(props: ActorProperties): SuperviseTypedActor = {
+ import StringReflect._
+ val lifeCycle = if (!props.lifecycle.isEmpty && props.lifecycle.equalsIgnoreCase(VAL_LIFECYCYLE_TEMPORARY)) Temporary else Permanent
+ val isRemote = (props.host ne null) && (!props.host.isEmpty)
+ val withInterface = (props.interface ne null) && (!props.interface.isEmpty)
+ if (isRemote) {
+ //val remote = new RemoteAddress(props.host, props.port)
+ val remote = new RemoteAddress(props.host, props.port.toInt)
+ if (withInterface) {
+ new SuperviseTypedActor(props.interface.toClass, props.target.toClass, lifeCycle, props.timeout, remote)
+ } else {
+ new SuperviseTypedActor(props.target.toClass, lifeCycle, props.timeout, remote)
+ }
+ } else {
+ if (withInterface) {
+ new SuperviseTypedActor(props.interface.toClass, props.target.toClass, lifeCycle, props.timeout)
+ } else {
+ new SuperviseTypedActor(props.target.toClass, lifeCycle, props.timeout)
+ }
+ }
+ }
+
+ /**
+ * Create configuration for UntypedActor
+ */
+ private[akka] def createSupervise(props: ActorProperties): Server = {
+ import StringReflect._
+ val lifeCycle = if (!props.lifecycle.isEmpty && props.lifecycle.equalsIgnoreCase(VAL_LIFECYCYLE_TEMPORARY)) Temporary else Permanent
+ val isRemote = (props.host ne null) && (!props.host.isEmpty)
+ val actorRef = Actor.actorOf(props.target.toClass)
+ if (props.timeout > 0) {
+ actorRef.setTimeout(props.timeout)
+ }
+
+ val supervise = if (isRemote) {
+ val remote = new RemoteAddress(props.host, props.port.toInt)
+ Supervise(actorRef, lifeCycle, remote)
+ } else {
+ Supervise(actorRef, lifeCycle)
+ }
+ supervise
+ }
+}
diff --git a/akka-stm/src/main/scala/akka/agent/Agent.scala b/akka-stm/src/main/scala/akka/agent/Agent.scala
new file mode 100644
index 0000000000..f250a65f15
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/agent/Agent.scala
@@ -0,0 +1,244 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.agent
+
+import akka.stm._
+import akka.actor.Actor
+import akka.japi.{Function => JFunc, Procedure => JProc}
+import akka.dispatch.Dispatchers
+
+/**
+ * Used internally to send functions.
+ */
+private[akka] case class Update[T](function: T => T)
+
+/**
+ * Factory method for creating an Agent.
+ */
+object Agent {
+ def apply[T](initialValue: T) = new Agent(initialValue)
+}
+
+/**
+ * The Agent class was inspired by agents in Clojure.
+ *
+ * Agents provide asynchronous change of individual locations. Agents
+ * are bound to a single storage location for their lifetime, and only
+ * allow mutation of that location (to a new state) to occur as a result
+ * of an action. Update actions are functions that are asynchronously
+ * applied to the Agent's state and whose return value becomes the
+ * Agent's new state. The state of an Agent should be immutable.
+ *
+ * While updates to Agents are asynchronous, the state of an Agent is
+ * always immediately available for reading by any thread (using ''get''
+ * or ''apply'') without any messages.
+ *
+ * Agents are reactive. The update actions of all Agents get interleaved
+ * amongst threads in a thread pool. At any point in time, at most one
+ * ''send'' action for each Agent is being executed. Actions dispatched to
+ * an agent from another thread will occur in the order they were sent,
+ * potentially interleaved with actions dispatched to the same agent from
+ * other sources.
+ *
+ * If an Agent is used within an enclosing transaction, then it will
+ * participate in that transaction. Agents are integrated with the STM -
+ * any dispatches made in a transaction are held until that transaction
+ * commits, and are discarded if it is retried or aborted.
+ *
+ *
+ * Example of usage:
+ * {{{
+ * val agent = Agent(5)
+ *
+ * agent send (_ * 2)
+ *
+ * ...
+ *
+ * val result = agent()
+ * // use result ...
+ *
+ * agent.close
+ * }}}
+ *
+ *
+ * Agent is also monadic, which means that you can compose operations using
+ * for-comprehensions. In monadic usage the original agents are not touched
+ * but new agents are created. So the old values (agents) are still available
+ * as-is. They are so-called 'persistent'.
+ *
+ *
+ * Example of monadic usage:
+ * {{{
+ * val agent1 = Agent(3)
+ * val agent2 = Agent(5)
+ *
+ * for (value <- agent1) {
+ * result = value + 1
+ * }
+ *
+ * val agent3 = for (value <- agent1) yield value + 1
+ *
+ * val agent4 = for {
+ * value1 <- agent1
+ * value2 <- agent2
+ * } yield value1 + value2
+ *
+ * agent1.close
+ * agent2.close
+ * agent3.close
+ * agent4.close
+ * }}}
+ */
+class Agent[T](initialValue: T) {
+ private[akka] val ref = Ref(initialValue)
+ private[akka] val updater = Actor.actorOf(new AgentUpdater(this)).start
+
+ /**
+ * Read the internal state of the agent.
+ */
+ def get() = ref.get
+
+ /**
+ * Read the internal state of the agent.
+ */
+ def apply() = get
+
+ /**
+ * Dispatch a function to update the internal state.
+ */
+ def send(f: T => T): Unit = {
+ def dispatch = updater ! Update(f)
+ if (Stm.activeTransaction) { get; deferred(dispatch) }
+ else dispatch
+ }
+
+ /**
+ * Dispatch a new value for the internal state. Behaves the same
+ * as sending a fuction (x => newValue).
+ */
+ def send(newValue: T): Unit = send(x => newValue)
+
+ /**
+ * Dispatch a new value for the internal state. Behaves the same
+ * as sending a fuction (x => newValue).
+ */
+ def update(newValue: T) = send(newValue)
+
+ /**
+ * Dispatch a function to update the internal state but on its own thread.
+ * This does not use the reactive thread pool and can be used for long-running
+ * or blocking operations. Dispatches using either `sendOff` or `send` will
+ * still be executed in order.
+ */
+ def sendOff(f: T => T): Unit = send((value: T) => {
+ suspend
+ val threadBased = Actor.actorOf(new ThreadBasedAgentUpdater(this)).start
+ threadBased ! Update(f)
+ value
+ })
+
+ /**
+ * Map this agent to a new agent, applying the function to the internal state.
+ * Does not change the value of this agent.
+ */
+ def map[B](f: T => B): Agent[B] = Agent(f(get))
+
+ /**
+ * Flatmap this agent to a new agent, applying the function to the internal state.
+ * Does not change the value of this agent.
+ */
+ def flatMap[B](f: T => Agent[B]): Agent[B] = f(get)
+
+ /**
+ * Applies the function to the internal state. Does not change the value of this agent.
+ */
+ def foreach[U](f: T => U): Unit = f(get)
+
+ /**
+ * Suspends processing of `send` actions for the agent.
+ */
+ def suspend() = updater.dispatcher.suspend(updater)
+
+ /**
+ * Resumes processing of `send` actions for the agent.
+ */
+ def resume() = updater.dispatcher.resume(updater)
+
+ /**
+ * Closes the agents and makes it eligable for garbage collection.
+ * A closed agent cannot accept any `send` actions.
+ */
+ def close() = updater.stop
+
+ // ---------------------------------------------
+ // Support for Java API Functions and Procedures
+ // ---------------------------------------------
+
+ /**
+ * Java API:
+ * Dispatch a function to update the internal state.
+ */
+ def send(f: JFunc[T, T]): Unit = send(x => f(x))
+
+ /**
+ * Java API:
+ * Dispatch a function to update the internal state but on its own thread.
+ * This does not use the reactive thread pool and can be used for long-running
+ * or blocking operations. Dispatches using either `sendOff` or `send` will
+ * still be executed in order.
+ */
+ def sendOff(f: JFunc[T, T]): Unit = sendOff(x => f(x))
+
+ /**
+ * Java API:
+ * Map this agent to a new agent, applying the function to the internal state.
+ * Does not change the value of this agent.
+ */
+ def map[B](f: JFunc[T, B]): Agent[B] = Agent(f(get))
+
+ /**
+ * Java API:
+ * Flatmap this agent to a new agent, applying the function to the internal state.
+ * Does not change the value of this agent.
+ */
+ def flatMap[B](f: JFunc[T, Agent[B]]): Agent[B] = f(get)
+
+ /**
+ * Java API:
+ * Applies the function to the internal state. Does not change the value of this agent.
+ */
+ def foreach(f: JProc[T]): Unit = f(get)
+}
+
+/**
+ * Agent updater actor. Used internally for `send` actions.
+ */
+class AgentUpdater[T](agent: Agent[T]) extends Actor {
+ val txFactory = TransactionFactory(familyName = "AgentUpdater", readonly = false)
+
+ def receive = {
+ case update: Update[T] =>
+ atomic(txFactory) { agent.ref alter update.function }
+ case _ => ()
+ }
+}
+
+/**
+ * Thread-based agent updater actor. Used internally for `sendOff` actions.
+ */
+class ThreadBasedAgentUpdater[T](agent: Agent[T]) extends Actor {
+ self.dispatcher = Dispatchers.newThreadBasedDispatcher(self)
+
+ val txFactory = TransactionFactory(familyName = "ThreadBasedAgentUpdater", readonly = false)
+
+ def receive = {
+ case update: Update[T] => {
+ atomic(txFactory) { agent.ref alter update.function }
+ agent.resume
+ self.stop
+ }
+ case _ => self.stop
+ }
+}
diff --git a/akka-stm/src/main/scala/akka/stm/Atomic.scala b/akka-stm/src/main/scala/akka/stm/Atomic.scala
new file mode 100644
index 0000000000..dc5f192573
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/stm/Atomic.scala
@@ -0,0 +1,40 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.stm
+
+/**
+ * Java-friendly atomic blocks.
+ *
+ * Example usage ''(Java)''
+ *
+ * {{{
+ * import akka.stm.*;
+ *
+ * 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](val factory: TransactionFactory) {
+ def this() = this(DefaultTransactionFactory)
+ def atomically: T
+ def execute: T = atomic(factory)(atomically)
+}
diff --git a/akka-stm/src/main/scala/akka/stm/Ref.scala b/akka-stm/src/main/scala/akka/stm/Ref.scala
new file mode 100644
index 0000000000..398ab82eee
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/stm/Ref.scala
@@ -0,0 +1,124 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.stm
+
+import akka.actor.{newUuid, Uuid}
+
+import org.multiverse.transactional.refs.BasicRef
+
+/**
+ * Transactional managed reference. See the companion class for more information.
+ */
+object Ref {
+ def apply[T]() = new Ref[T]()
+
+ def apply[T](initialValue: T) = new Ref[T](initialValue)
+
+ /**
+ * An implicit conversion that converts a Ref to an Iterable value.
+ */
+ implicit def ref2Iterable[T](ref: Ref[T]): Iterable[T] = ref.toList
+}
+
+/**
+ * Refs (transactional references) are mutable references to values and through
+ * the STM allow the safe sharing of mutable data. Refs separate identity from value.
+ * To ensure safety the value stored in a Ref should be immutable (they can also
+ * contain refs themselves). The value referenced by a Ref can only be accessed
+ * or swapped within a transaction. If a transaction is not available, the call will
+ * be executed in its own transaction.
+ *
+ *
+ * Creating a Ref ''(Scala)''
+ *
+ * {{{
+ * import akka.stm._
+ *
+ * // giving an initial value
+ * val ref = Ref(0)
+ *
+ * // specifying a type but no initial value
+ * val ref = Ref[Int]
+ * }}}
+ *
+ *
+ * Creating a Ref ''(Java)''
+ *
+ * {{{
+ * import akka.stm.*;
+ *
+ * // giving an initial value
+ * final Ref ref = new Ref(0);
+ *
+ * // specifying a type but no initial value
+ * final Ref ref = new Ref();
+ * }}}
+ */
+class Ref[T](initialValue: T) extends BasicRef[T](initialValue) with Transactional {
+ self =>
+
+ def this() = this(null.asInstanceOf[T])
+
+ val uuid = newUuid.toString
+
+ def apply() = get
+
+ def update(newValue: T) = set(newValue)
+
+ def swap(newValue: T) = set(newValue)
+
+ def alter(f: T => T): T = {
+ val value = f(get)
+ set(value)
+ value
+ }
+
+ def opt: Option[T] = Option(get)
+
+ def getOrWait: T = getOrAwait
+
+ def getOrElse(default: => T): T =
+ if (isNull) default else get
+
+ def isDefined: Boolean = !isNull
+
+ def isEmpty: Boolean = isNull
+
+ def map[B](f: T => B): Ref[B] =
+ if (isEmpty) Ref[B] else Ref(f(get))
+
+ def flatMap[B](f: T => Ref[B]): Ref[B] =
+ if (isEmpty) Ref[B] else f(get)
+
+ def filter(p: T => Boolean): Ref[T] =
+ if (isDefined && p(get)) Ref(get) else Ref[T]
+
+ /**
+ * Necessary to keep from being implicitly converted to Iterable in for comprehensions.
+ */
+ def withFilter(p: T => Boolean): WithFilter = new WithFilter(p)
+
+ class WithFilter(p: T => Boolean) {
+ def map[B](f: T => B): Ref[B] = self filter p map f
+ def flatMap[B](f: T => Ref[B]): Ref[B] = self filter p flatMap f
+ def foreach[U](f: T => U): Unit = self filter p foreach f
+ def withFilter(q: T => Boolean): WithFilter = new WithFilter(x => p(x) && q(x))
+ }
+
+ def foreach[U](f: T => U): Unit =
+ if (isDefined) f(get)
+
+ def elements: Iterator[T] =
+ if (isEmpty) Iterator.empty else Iterator(get)
+
+ def toList: List[T] =
+ if (isEmpty) List() else List(get)
+
+ def toRight[X](left: => X) =
+ if (isEmpty) Left(left) else Right(get)
+
+ def toLeft[X](right: => X) =
+ if (isEmpty) Right(right) else Left(get)
+}
diff --git a/akka-stm/src/main/scala/akka/stm/Stm.scala b/akka-stm/src/main/scala/akka/stm/Stm.scala
new file mode 100644
index 0000000000..2ca9844442
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/stm/Stm.scala
@@ -0,0 +1,149 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.stm
+
+import org.multiverse.api.{StmUtils => MultiverseStmUtils}
+import org.multiverse.api.{Transaction => MultiverseTransaction}
+import org.multiverse.templates.{TransactionalCallable, OrElseTemplate}
+
+object Stm {
+ /**
+ * Check whether there is an active Multiverse transaction.
+ */
+ def activeTransaction() = {
+ val tx = org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction
+ (tx ne null) && !tx.getStatus.isDead
+ }
+}
+
+/**
+ * Defines the atomic block for local transactions. Automatically imported with:
+ *
+ * {{{
+ * import akka.stm._
+ * }}}
+ *
+ *
+ * If you need to coordinate transactions across actors see [[akka.stm.Coordinated]].
+ *
+ *
+ * Example of using the atomic block ''(Scala)''
+ *
+ * {{{
+ * atomic {
+ * // do something within a transaction
+ * }
+ * }}}
+ *
+ * @see [[akka.stm.Atomic]] for creating atomic blocks in Java.
+ * @see [[akka.stm.StmUtil]] for useful methods to combine with `atomic`
+ */
+trait Stm {
+ val DefaultTransactionFactory = TransactionFactory(DefaultTransactionConfig, "DefaultTransaction")
+
+ def atomic[T](body: => T)(implicit factory: TransactionFactory = DefaultTransactionFactory): 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
+ body
+ }
+ })
+ }
+}
+
+/**
+ * Stm utility methods for scheduling transaction lifecycle tasks and for blocking transactions.
+ * Automatically imported with:
+ *
+ * {{{
+ * import akka.stm._
+ * }}}
+ *
+ *
+ * Schedule a deferred task on the thread local transaction (use within an atomic).
+ * This is executed when the transaction commits.
+ *
+ * {{{
+ * atomic {
+ * deferred {
+ * // executes when transaction successfully commits
+ * }
+ * }
+ * }}}
+ *
+ *
+ * Schedule a compensating task on the thread local transaction (use within an atomic).
+ * This is executed when the transaction aborts.
+ *
+ * {{{
+ * atomic {
+ * compensating {
+ * // executes when transaction aborts
+ * }
+ * }
+ * }}}
+ *
+ *
+ * STM retry for blocking transactions (use within an atomic).
+ * Can be used to wait for a condition.
+ *
+ * {{{
+ * atomic {
+ * if (!someCondition) retry
+ * // ...
+ * }
+ * }}}
+ *
+ *
+ * Use either-orElse to combine two blocking transactions.
+ *
+ * {{{
+ * atomic {
+ * either {
+ * // ...
+ * } orElse {
+ * // ...
+ * }
+ * }
+ * }}}
+ *
+ */
+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 =
+ 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 =
+ MultiverseStmUtils.scheduleCompensatingTask(new Runnable { def run = body })
+
+ /**
+ * STM retry for blocking transactions (use within an atomic).
+ * Can be used to wait for a condition.
+ */
+ def retry() = MultiverseStmUtils.retry
+
+ /**
+ * Use either-orElse to combine two blocking transactions.
+ */
+ def either[T](firstBody: => T) = new {
+ def orElse(secondBody: => T) = new OrElseTemplate[T] {
+ def either(mtx: MultiverseTransaction) = firstBody
+ def orelse(mtx: MultiverseTransaction) = secondBody
+ }.execute()
+ }
+}
+
+
+
diff --git a/akka-stm/src/main/scala/akka/stm/Transaction.scala b/akka-stm/src/main/scala/akka/stm/Transaction.scala
new file mode 100644
index 0000000000..880a0af05a
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/stm/Transaction.scala
@@ -0,0 +1,232 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.stm
+
+import java.util.concurrent.atomic.AtomicLong
+import java.util.concurrent.atomic.AtomicInteger
+
+import scala.collection.mutable.HashMap
+
+import akka.util.{Logging, ReflectiveAccess}
+import akka.config.Config._
+import akka.config.ModuleNotAvailableException
+import akka.AkkaException
+
+import org.multiverse.api.{Transaction => MultiverseTransaction}
+import org.multiverse.api.lifecycle.{TransactionLifecycleListener, TransactionLifecycleEvent}
+import org.multiverse.api.ThreadLocalTransaction._
+import org.multiverse.api.{PropagationLevel => MultiversePropagationLevel}
+import org.multiverse.api.{TraceLevel => MultiverseTraceLevel}
+
+class NoTransactionInScopeException extends AkkaException("No transaction in scope")
+class TransactionRetryException(message: String) extends AkkaException(message)
+class StmConfigurationException(message: String) extends AkkaException(message)
+
+
+/**
+ * Internal helper methods for managing Akka-specific transaction.
+ */
+object TransactionManagement extends TransactionManagement {
+ private[akka] val transaction = new ThreadLocal[Option[Transaction]]() {
+ override protected def initialValue: Option[Transaction] = None
+ }
+
+ private[akka] def getTransaction: Transaction = {
+ val option = transaction.get
+ if ((option eq null) || option.isEmpty) throw new StmConfigurationException("No Transaction in scope")
+ option.get
+ }
+}
+
+/**
+ * Internal helper methods for managing Akka-specific transaction.
+ */
+trait TransactionManagement {
+ private[akka] def setTransaction(tx: Option[Transaction]) =
+ if (tx.isDefined) TransactionManagement.transaction.set(tx)
+
+ private[akka] def clearTransaction = {
+ TransactionManagement.transaction.set(None)
+ setThreadLocalTransaction(null)
+ }
+
+ private[akka] def getTransactionInScope = TransactionManagement.getTransaction
+
+ private[akka] def isTransactionInScope = {
+ val option = TransactionManagement.transaction.get
+ (option ne null) && option.isDefined
+ }
+}
+
+object Transaction {
+ val idFactory = new AtomicLong(-1L)
+
+ /**
+ * Attach an Akka-specific Transaction to the current Multiverse transaction.
+ * Must be called within a Multiverse transaction. Used by TransactionFactory.addHooks
+ */
+ private[akka] def attach = {
+ val mtx = getRequiredThreadLocalTransaction
+ val tx = new Transaction
+ tx.begin
+ tx.transaction = Some(mtx)
+ TransactionManagement.transaction.set(Some(tx))
+ mtx.registerLifecycleListener(new TransactionLifecycleListener() {
+ def notify(mtx: MultiverseTransaction, event: TransactionLifecycleEvent) = event match {
+ case TransactionLifecycleEvent.PostCommit => tx.commit
+ case TransactionLifecycleEvent.PostAbort => tx.abort
+ case _ => {}
+ }
+ })
+ }
+}
+
+/**
+ * The Akka-specific Transaction class.
+ * For integration with persistence modules and JTA support.
+ */
+@serializable class Transaction extends Logging {
+ val JTA_AWARE = config.getBool("akka.stm.jta-aware", false)
+
+ val id = Transaction.idFactory.incrementAndGet
+ @volatile private[this] var status: TransactionStatus = TransactionStatus.New
+ private[akka] var transaction: Option[MultiverseTransaction] = None
+ private[this] val persistentStateMap = new HashMap[String, Committable with Abortable]
+ private[akka] val depth = new AtomicInteger(0)
+
+ val jta: Option[ReflectiveJtaModule.TransactionContainer] =
+ if (JTA_AWARE) Some(ReflectiveJtaModule.createTransactionContainer)
+ else None
+
+ log.trace("Creating transaction " + toString)
+
+ // --- public methods ---------
+
+ def begin = synchronized {
+ log.trace("Starting transaction " + toString)
+ jta.foreach { _.beginWithStmSynchronization(this) }
+ }
+
+ def commit = synchronized {
+ log.trace("Committing transaction " + toString)
+ persistentStateMap.valuesIterator.foreach(_.commit)
+ status = TransactionStatus.Completed
+ jta.foreach(_.commit)
+ }
+
+ def abort = synchronized {
+ log.trace("Aborting transaction " + toString)
+ jta.foreach(_.rollback)
+ persistentStateMap.valuesIterator.foreach(_.abort)
+ persistentStateMap.clear
+ }
+
+ def isNew = synchronized { status == TransactionStatus.New }
+
+ def isActive = synchronized { status == TransactionStatus.Active }
+
+ def isCompleted = synchronized { status == TransactionStatus.Completed }
+
+ def isAborted = synchronized { status == TransactionStatus.Aborted }
+
+ // --- internal methods ---------
+
+ //private def isJtaTxActive(status: Int) = status == Status.STATUS_ACTIVE
+
+ private[akka] def status_? = status
+
+ private[akka] def increment = depth.incrementAndGet
+
+ private[akka] def decrement = depth.decrementAndGet
+
+ private[akka] def isTopLevel = depth.get == 0
+ //when calling this method, make sure to prefix the uuid with the type so you
+ //have no possibility of kicking a diffferent type with the same uuid out of a transction
+ private[akka] def register(uuid: String, storage: Committable with Abortable) = persistentStateMap.put(uuid, storage)
+
+ private def ensureIsActive = if (status != TransactionStatus.Active)
+ throw new StmConfigurationException(
+ "Expected ACTIVE transaction - current status [" + status + "]: " + toString)
+
+ private def ensureIsActiveOrAborted =
+ if (!(status == TransactionStatus.Active || status == TransactionStatus.Aborted))
+ throw new StmConfigurationException(
+ "Expected ACTIVE or ABORTED transaction - current status [" + status + "]: " + toString)
+
+ private def ensureIsActiveOrNew =
+ if (!(status == TransactionStatus.Active || status == TransactionStatus.New))
+ throw new StmConfigurationException(
+ "Expected ACTIVE or NEW transaction - current status [" + status + "]: " + toString)
+
+ override def equals(that: Any): Boolean = synchronized {
+ that.isInstanceOf[Transaction] &&
+ that.asInstanceOf[Transaction].id == this.id
+ }
+
+ override def hashCode: Int = synchronized { id.toInt }
+
+ override def toString = synchronized { "Transaction[" + id + ", " + status + "]" }
+}
+
+@serializable sealed abstract class TransactionStatus
+
+object TransactionStatus {
+ case object New extends TransactionStatus
+ case object Active extends TransactionStatus
+ case object Aborted extends TransactionStatus
+ case object Completed extends TransactionStatus
+}
+
+/**
+ * Common trait for all the transactional objects:
+ * Ref, TransactionalMap, TransactionalVector,
+ * PersistentRef, PersistentMap, PersistentVector, PersistentQueue, PersistentSortedSet
+ */
+@serializable trait Transactional {
+ val uuid: String
+}
+
+/**
+ * Used for integration with the persistence modules.
+ */
+trait Committable {
+ def commit(): Unit
+}
+
+/**
+ * Used for integration with the persistence modules.
+ */
+trait Abortable {
+ def abort(): Unit
+}
+
+/**
+ * Used internally for reflective access to the JTA module.
+ * Allows JTA integration to work when akka-jta.jar is on the classpath.
+ */
+object ReflectiveJtaModule {
+ type TransactionContainerObject = {
+ def apply(): TransactionContainer
+ }
+
+ type TransactionContainer = {
+ def beginWithStmSynchronization(transaction: Transaction): Unit
+ def commit: Unit
+ def rollback: Unit
+ }
+
+ lazy val isJtaEnabled = transactionContainerObjectInstance.isDefined
+
+ def ensureJtaEnabled = if (!isJtaEnabled) throw new ModuleNotAvailableException(
+ "Can't load the JTA module, make sure that akka-jta.jar is on the classpath")
+
+ val transactionContainerObjectInstance: Option[TransactionContainerObject] =
+ ReflectiveAccess.getObjectFor("akka.jta.TransactionContainer$")
+
+ def createTransactionContainer: TransactionContainer = {
+ ensureJtaEnabled
+ transactionContainerObjectInstance.get.apply.asInstanceOf[TransactionContainer]
+ }
+}
diff --git a/akka-stm/src/main/scala/akka/stm/TransactionFactory.scala b/akka-stm/src/main/scala/akka/stm/TransactionFactory.scala
new file mode 100644
index 0000000000..a6c847002b
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/stm/TransactionFactory.scala
@@ -0,0 +1,225 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.stm
+
+import java.lang.{Boolean => JBoolean}
+
+import akka.config.Config._
+import akka.util.Duration
+
+import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance
+import org.multiverse.stms.alpha.AlphaStm
+import org.multiverse.templates.TransactionBoilerplate
+import org.multiverse.api.{PropagationLevel => MPropagation}
+import org.multiverse.api.{TraceLevel => MTraceLevel}
+
+/**
+ * For configuring multiverse transactions.
+ */
+object TransactionConfig {
+ // note: null values are so that we can default to Multiverse inference when not set
+ val FAMILY_NAME = "DefaultTransaction"
+ val READONLY = null.asInstanceOf[JBoolean]
+ val MAX_RETRIES = config.getInt("akka.stm.max-retries", 1000)
+ val TIMEOUT = config.getLong("akka.stm.timeout", 10)
+ val TRACK_READS = null.asInstanceOf[JBoolean]
+ val WRITE_SKEW = config.getBool("akka.stm.write-skew", true)
+ val BLOCKING_ALLOWED = config.getBool("akka.stm.blocking-allowed", false)
+ val INTERRUPTIBLE = config.getBool("akka.stm.interruptible", false)
+ val SPECULATIVE = config.getBool("akka.stm.speculative", true)
+ val QUICK_RELEASE = config.getBool("akka.stm.quick-release", true)
+ val PROPAGATION = propagation(config.getString("akka.stm.propagation", "requires"))
+ val TRACE_LEVEL = traceLevel(config.getString("akka.stm.trace-level", "none"))
+ val HOOKS = config.getBool("akka.stm.hooks", true)
+
+ val DefaultTimeout = Duration(TIMEOUT, TIME_UNIT)
+
+ def propagation(level: String) = level.toLowerCase match {
+ case "requiresnew" => Propagation.RequiresNew
+ case "fine" => Propagation.Mandatory
+ case "supports" => Propagation.Supports
+ case "never" => Propagation.Never
+ case _ => Propagation.Requires
+ }
+
+ def traceLevel(level: String) = level.toLowerCase match {
+ case "coarse" | "course" => TraceLevel.Coarse
+ case "fine" => TraceLevel.Fine
+ case _ => TraceLevel.None
+ }
+
+ /**
+ * For configuring multiverse transactions.
+ *
+ * @param familyName Family name for transactions. Useful for debugging.
+ * @param readonly Sets transaction as readonly. Readonly transactions are cheaper.
+ * @param maxRetries The maximum number of times a transaction will retry.
+ * @param timeout The maximum time a transaction will block for.
+ * @param trackReads Whether all reads should be tracked. Needed for blocking operations.
+ * @param writeSkew Whether writeskew is allowed. Disable with care.
+ * @param blockingAllowed Whether explicit retries are allowed.
+ * @param interruptible Whether a blocking transaction can be interrupted.
+ * @param speculative Whether speculative configuration should be enabled.
+ * @param quickRelease Whether locks should be released as quickly as possible (before whole commit).
+ * @param propagation For controlling how nested transactions behave.
+ * @param traceLevel Transaction trace level.
+ * @param hooks Whether hooks for persistence modules and JTA should be added to the transaction.
+ */
+ def apply(familyName: String = FAMILY_NAME,
+ readonly: JBoolean = READONLY,
+ maxRetries: Int = MAX_RETRIES,
+ timeout: Duration = DefaultTimeout,
+ trackReads: JBoolean = TRACK_READS,
+ writeSkew: Boolean = WRITE_SKEW,
+ blockingAllowed: Boolean = BLOCKING_ALLOWED,
+ interruptible: Boolean = INTERRUPTIBLE,
+ speculative: Boolean = SPECULATIVE,
+ quickRelease: Boolean = QUICK_RELEASE,
+ propagation: MPropagation = PROPAGATION,
+ traceLevel: MTraceLevel = TRACE_LEVEL,
+ hooks: Boolean = HOOKS) = {
+ new TransactionConfig(familyName, readonly, maxRetries, timeout, trackReads, writeSkew, blockingAllowed,
+ interruptible, speculative, quickRelease, propagation, traceLevel, hooks)
+ }
+}
+
+/**
+ * For configuring multiverse transactions.
+ *
+ *
familyName - Family name for transactions. Useful for debugging.
+ *
readonly - Sets transaction as readonly. Readonly transactions are cheaper.
+ *
maxRetries - The maximum number of times a transaction will retry.
+ *
timeout - The maximum time a transaction will block for.
+ *
trackReads - Whether all reads should be tracked. Needed for blocking operations.
+ *
writeSkew - Whether writeskew is allowed. Disable with care.
+ *
blockingAllowed - Whether explicit retries are allowed.
+ *
interruptible - Whether a blocking transaction can be interrupted.
+ *
speculative - Whether speculative configuration should be enabled.
+ *
quickRelease - Whether locks should be released as quickly as possible (before whole commit).
+ *
propagation - For controlling how nested transactions behave.
+ *
traceLevel - Transaction trace level.
+ *
hooks - Whether hooks for persistence modules and JTA should be added to the transaction.
+ */
+class TransactionConfig(val familyName: String = TransactionConfig.FAMILY_NAME,
+ val readonly: JBoolean = TransactionConfig.READONLY,
+ val maxRetries: Int = TransactionConfig.MAX_RETRIES,
+ val timeout: Duration = TransactionConfig.DefaultTimeout,
+ val trackReads: JBoolean = TransactionConfig.TRACK_READS,
+ val writeSkew: Boolean = TransactionConfig.WRITE_SKEW,
+ val blockingAllowed: Boolean = TransactionConfig.BLOCKING_ALLOWED,
+ val interruptible: Boolean = TransactionConfig.INTERRUPTIBLE,
+ val speculative: Boolean = TransactionConfig.SPECULATIVE,
+ val quickRelease: Boolean = TransactionConfig.QUICK_RELEASE,
+ val propagation: MPropagation = TransactionConfig.PROPAGATION,
+ val traceLevel: MTraceLevel = TransactionConfig.TRACE_LEVEL,
+ val hooks: Boolean = TransactionConfig.HOOKS)
+
+object DefaultTransactionConfig extends TransactionConfig
+
+/**
+ * Wrapper for transaction config, factory, and boilerplate. Used by atomic.
+ */
+object TransactionFactory {
+ def apply(config: TransactionConfig) = new TransactionFactory(config)
+
+ def apply(config: TransactionConfig, defaultName: String) = new TransactionFactory(config, defaultName)
+
+ def apply(familyName: String = TransactionConfig.FAMILY_NAME,
+ readonly: JBoolean = TransactionConfig.READONLY,
+ maxRetries: Int = TransactionConfig.MAX_RETRIES,
+ timeout: Duration = TransactionConfig.DefaultTimeout,
+ trackReads: JBoolean = TransactionConfig.TRACK_READS,
+ writeSkew: Boolean = TransactionConfig.WRITE_SKEW,
+ blockingAllowed: Boolean = TransactionConfig.BLOCKING_ALLOWED,
+ interruptible: Boolean = TransactionConfig.INTERRUPTIBLE,
+ speculative: Boolean = TransactionConfig.SPECULATIVE,
+ quickRelease: Boolean = TransactionConfig.QUICK_RELEASE,
+ propagation: MPropagation = TransactionConfig.PROPAGATION,
+ traceLevel: MTraceLevel = TransactionConfig.TRACE_LEVEL,
+ hooks: Boolean = TransactionConfig.HOOKS) = {
+ val config = new TransactionConfig(
+ familyName, readonly, maxRetries, timeout, trackReads, writeSkew, blockingAllowed,
+ interruptible, speculative, quickRelease, propagation, traceLevel, hooks)
+ new TransactionFactory(config)
+ }
+}
+
+/**
+ * Wrapper for transaction config, factory, and boilerplate. Used by atomic.
+ * Can be passed to atomic implicitly or explicitly.
+ *
+ * {{{
+ * implicit val txFactory = TransactionFactory(readonly = true)
+ * ...
+ * atomic {
+ * // do something within a readonly transaction
+ * }
+ * }}}
+ *
+ * Can be created at different levels as needed. For example: as an implicit object
+ * used throughout a package, as a static implicit val within a singleton object and
+ * imported where needed, or as an implicit val within each instance of a class.
+ *
+ * If no explicit transaction factory is passed to atomic and there is no implicit
+ * transaction factory in scope, then a default transaction factory is used.
+ *
+ * @see [[akka.stm.TransactionConfig]] for configuration options.
+ */
+class TransactionFactory(
+ val config: TransactionConfig = DefaultTransactionConfig,
+ defaultName: String = TransactionConfig.FAMILY_NAME) { self =>
+
+ // use the config family name if it's been set, otherwise defaultName - used by actors to set class name as default
+ val familyName = if (config.familyName != TransactionConfig.FAMILY_NAME) config.familyName else defaultName
+
+ val factory = {
+ var builder = (getGlobalStmInstance().asInstanceOf[AlphaStm].getTransactionFactoryBuilder()
+ .setFamilyName(familyName)
+ .setMaxRetries(config.maxRetries)
+ .setTimeoutNs(config.timeout.toNanos)
+ .setWriteSkewAllowed(config.writeSkew)
+ .setExplicitRetryAllowed(config.blockingAllowed)
+ .setInterruptible(config.interruptible)
+ .setSpeculativeConfigurationEnabled(config.speculative)
+ .setQuickReleaseEnabled(config.quickRelease)
+ .setPropagationLevel(config.propagation)
+ .setTraceLevel(config.traceLevel))
+
+ if (config.readonly ne null) {
+ builder = builder.setReadonly(config.readonly.booleanValue)
+ } // otherwise default to Multiverse inference
+
+ if (config.trackReads ne null) {
+ builder = builder.setReadTrackingEnabled(config.trackReads.booleanValue)
+ } // otherwise default to Multiverse inference
+
+ builder.build()
+ }
+
+ val boilerplate = new TransactionBoilerplate(factory)
+
+ def addHooks = if (config.hooks) Transaction.attach
+}
+
+/**
+ * Mapping to Multiverse PropagationLevel.
+ */
+object Propagation {
+ val RequiresNew = MPropagation.RequiresNew
+ val Mandatory = MPropagation.Mandatory
+ val Requires = MPropagation.Requires
+ val Supports = MPropagation.Supports
+ val Never = MPropagation.Never
+}
+
+/**
+ * Mapping to Multiverse TraceLevel.
+ */
+object TraceLevel {
+ val None = MTraceLevel.none
+ val Coarse = MTraceLevel.course // mispelling?
+ val Course = MTraceLevel.course
+ val Fine = MTraceLevel.fine
+}
diff --git a/akka-stm/src/main/scala/akka/stm/TransactionFactoryBuilder.scala b/akka-stm/src/main/scala/akka/stm/TransactionFactoryBuilder.scala
new file mode 100644
index 0000000000..07eb1ed6df
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/stm/TransactionFactoryBuilder.scala
@@ -0,0 +1,89 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.stm
+
+import java.lang.{Boolean => JBoolean}
+
+import akka.util.Duration
+
+import org.multiverse.api.{TraceLevel => MTraceLevel}
+import org.multiverse.api.{PropagationLevel => MPropagation}
+
+/**
+ * For more easily creating TransactionConfig from Java.
+ */
+class TransactionConfigBuilder {
+ var familyName: String = TransactionConfig.FAMILY_NAME
+ var readonly: JBoolean = TransactionConfig.READONLY
+ var maxRetries: Int = TransactionConfig.MAX_RETRIES
+ var timeout: Duration = TransactionConfig.DefaultTimeout
+ var trackReads: JBoolean = TransactionConfig.TRACK_READS
+ var writeSkew: Boolean = TransactionConfig.WRITE_SKEW
+ var blockingAllowed: Boolean = TransactionConfig.BLOCKING_ALLOWED
+ var interruptible: Boolean = TransactionConfig.INTERRUPTIBLE
+ var speculative: Boolean = TransactionConfig.SPECULATIVE
+ var quickRelease: Boolean = TransactionConfig.QUICK_RELEASE
+ var propagation: MPropagation = TransactionConfig.PROPAGATION
+ var traceLevel: MTraceLevel = TransactionConfig.TRACE_LEVEL
+ var hooks: Boolean = TransactionConfig.HOOKS
+
+ def setFamilyName(familyName: String) = { this.familyName = familyName; this }
+ def setReadonly(readonly: JBoolean) = { this.readonly = readonly; this }
+ def setMaxRetries(maxRetries: Int) = { this.maxRetries = maxRetries; this }
+ def setTimeout(timeout: Duration) = { this.timeout = timeout; this }
+ def setTrackReads(trackReads: JBoolean) = { this.trackReads = trackReads; this }
+ def setWriteSkew(writeSkew: Boolean) = { this.writeSkew = writeSkew; this }
+ def setBlockingAllowed(blockingAllowed: Boolean) = { this.blockingAllowed = blockingAllowed; this }
+ def setInterruptible(interruptible: Boolean) = { this.interruptible = interruptible; this }
+ def setSpeculative(speculative: Boolean) = { this.speculative = speculative; this }
+ def setQuickRelease(quickRelease: Boolean) = { this.quickRelease = quickRelease; this }
+ def setPropagation(propagation: MPropagation) = { this.propagation = propagation; this }
+ def setTraceLevel(traceLevel: MTraceLevel) = { this.traceLevel = traceLevel; this }
+ def setHooks(hooks: Boolean) = { this.hooks = hooks; this }
+
+ def build() = new TransactionConfig(
+ familyName, readonly, maxRetries, timeout, trackReads, writeSkew, blockingAllowed,
+ interruptible, speculative, quickRelease, propagation, traceLevel, hooks)
+}
+
+/**
+ * For more easily creating TransactionFactory from Java.
+ */
+class TransactionFactoryBuilder {
+ var familyName: String = TransactionConfig.FAMILY_NAME
+ var readonly: JBoolean = TransactionConfig.READONLY
+ var maxRetries: Int = TransactionConfig.MAX_RETRIES
+ var timeout: Duration = TransactionConfig.DefaultTimeout
+ var trackReads: JBoolean = TransactionConfig.TRACK_READS
+ var writeSkew: Boolean = TransactionConfig.WRITE_SKEW
+ var blockingAllowed: Boolean = TransactionConfig.BLOCKING_ALLOWED
+ var interruptible: Boolean = TransactionConfig.INTERRUPTIBLE
+ var speculative: Boolean = TransactionConfig.SPECULATIVE
+ var quickRelease: Boolean = TransactionConfig.QUICK_RELEASE
+ var propagation: MPropagation = TransactionConfig.PROPAGATION
+ var traceLevel: MTraceLevel = TransactionConfig.TRACE_LEVEL
+ var hooks: Boolean = TransactionConfig.HOOKS
+
+ def setFamilyName(familyName: String) = { this.familyName = familyName; this }
+ def setReadonly(readonly: JBoolean) = { this.readonly = readonly; this }
+ def setMaxRetries(maxRetries: Int) = { this.maxRetries = maxRetries; this }
+ def setTimeout(timeout: Duration) = { this.timeout = timeout; this }
+ def setTrackReads(trackReads: JBoolean) = { this.trackReads = trackReads; this }
+ def setWriteSkew(writeSkew: Boolean) = { this.writeSkew = writeSkew; this }
+ def setBlockingAllowed(blockingAllowed: Boolean) = { this.blockingAllowed = blockingAllowed; this }
+ def setInterruptible(interruptible: Boolean) = { this.interruptible = interruptible; this }
+ def setSpeculative(speculative: Boolean) = { this.speculative = speculative; this }
+ def setQuickRelease(quickRelease: Boolean) = { this.quickRelease = quickRelease; this }
+ def setPropagation(propagation: MPropagation) = { this.propagation = propagation; this }
+ def setTraceLevel(traceLevel: MTraceLevel) = { this.traceLevel = traceLevel; this }
+ def setHooks(hooks: Boolean) = { this.hooks = hooks; this }
+
+ def build() = {
+ val config = new TransactionConfig(
+ familyName, readonly, maxRetries, timeout, trackReads, writeSkew, blockingAllowed,
+ interruptible, speculative, quickRelease, propagation, traceLevel, hooks)
+ new TransactionFactory(config)
+ }
+}
diff --git a/akka-stm/src/main/scala/akka/stm/TransactionalMap.scala b/akka-stm/src/main/scala/akka/stm/TransactionalMap.scala
new file mode 100644
index 0000000000..a149171763
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/stm/TransactionalMap.scala
@@ -0,0 +1,89 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.stm
+
+import scala.collection.immutable.HashMap
+
+import akka.actor.{newUuid}
+
+/**
+ * Transactional map that implements the mutable Map interface with an underlying Ref and HashMap.
+ */
+object TransactionalMap {
+ def apply[K, V]() = new TransactionalMap[K, V]()
+
+ def apply[K, V](pairs: (K, V)*) = new TransactionalMap(HashMap(pairs: _*))
+}
+
+/**
+ * Transactional map that implements the mutable Map interface with an underlying Ref and HashMap.
+ *
+ * TransactionalMap and TransactionalVector look like regular mutable datastructures, they even
+ * implement the standard Scala 'Map' and 'IndexedSeq' interfaces, but they are implemented using
+ * persistent datastructures and managed references under the hood. Therefore they are safe to use
+ * in a concurrent environment through the STM. Underlying TransactionalMap is HashMap, an immutable
+ * Map but with near constant time access and modification operations.
+ *
+ * From Scala you can use TMap as a shorter alias for TransactionalMap.
+ */
+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 = newUuid.toString
+
+ private[this] val ref = Ref(initialValue)
+
+ def -=(key: K) = {
+ remove(key)
+ this
+ }
+
+ def +=(key: K, value: V) = put(key, value)
+
+ def +=(kv: (K, V)) = {
+ put(kv._1,kv._2)
+ this
+ }
+
+ override def remove(key: K) = {
+ val map = ref.get
+ val oldValue = map.get(key)
+ ref.swap(ref.get - key)
+ oldValue
+ }
+
+ def get(key: K): Option[V] = ref.get.get(key)
+
+ override def put(key: K, value: V): Option[V] = {
+ 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
+ val oldValue = map.get(key)
+ ref.swap(map.updated(key, value))
+ }
+
+ def iterator = ref.get.iterator
+
+ override def elements: Iterator[(K, V)] = ref.get.iterator
+
+ override def contains(key: K): Boolean = ref.get.contains(key)
+
+ override def clear = ref.swap(HashMap[K, V]())
+
+ override def size: Int = ref.get.size
+
+ override def hashCode: Int = System.identityHashCode(this);
+
+ override def equals(other: Any): Boolean =
+ other.isInstanceOf[TransactionalMap[_, _]] &&
+ other.hashCode == hashCode
+
+ override def toString = if (Stm.activeTransaction) super.toString else ""
+}
diff --git a/akka-stm/src/main/scala/akka/stm/TransactionalVector.scala b/akka-stm/src/main/scala/akka/stm/TransactionalVector.scala
new file mode 100644
index 0000000000..b3ef8fea34
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/stm/TransactionalVector.scala
@@ -0,0 +1,66 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.stm
+
+import scala.collection.immutable.Vector
+
+import akka.actor.newUuid
+
+
+/**
+ * Transactional vector that implements the IndexedSeq interface with an underlying Ref and Vector.
+ */
+object TransactionalVector {
+ def apply[T]() = new TransactionalVector[T]()
+
+ def apply[T](elems: T*) = new TransactionalVector(Vector(elems: _*))
+}
+
+/**
+ * Transactional vector that implements the IndexedSeq interface with an underlying Ref and Vector.
+ *
+ * TransactionalMap and TransactionalVector look like regular mutable datastructures, they even
+ * implement the standard Scala 'Map' and 'IndexedSeq' interfaces, but they are implemented using
+ * persistent datastructures and managed references under the hood. Therefore they are safe to use
+ * in a concurrent environment through the STM. Underlying TransactionalVector is Vector, an immutable
+ * sequence but with near constant time access and modification operations.
+ *
+ * From Scala you can use TVector as a shorter alias for TransactionalVector.
+ */
+class TransactionalVector[T](initialValue: Vector[T]) extends Transactional with IndexedSeq[T] {
+ def this() = this(Vector[T]())
+
+ val uuid = newUuid.toString
+
+ 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 :+ elem)
+
+ def get(index: Int): T = ref.get.apply(index)
+
+ /**
+ * Removes the tail element of this vector.
+ */
+ def pop = ref.swap(ref.get.dropRight(1))
+
+ def update(index: Int, elem: T) = ref.swap(ref.get.updated(index, elem))
+
+ def length: Int = ref.get.length
+
+ def apply(index: Int): T = ref.get.apply(index)
+
+ override def hashCode: Int = System.identityHashCode(this);
+
+ override def equals(other: Any): Boolean =
+ other.isInstanceOf[TransactionalVector[_]] &&
+ other.hashCode == hashCode
+
+ override def toString = if (Stm.activeTransaction) super.toString else ""
+}
+
diff --git a/akka-stm/src/main/scala/akka/stm/package.scala b/akka-stm/src/main/scala/akka/stm/package.scala
new file mode 100644
index 0000000000..50f9e709b9
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/stm/package.scala
@@ -0,0 +1,41 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka
+
+/**
+ * For easily importing everthing needed for STM.
+ */
+package object stm extends akka.stm.Stm with akka.stm.StmUtil {
+
+ // Shorter aliases for transactional map and vector
+
+ type TMap[K, V] = akka.stm.TransactionalMap[K, V]
+ val TMap = akka.stm.TransactionalMap
+
+ type TVector[T] = akka.stm.TransactionalVector[T]
+ val TVector = akka.stm.TransactionalVector
+
+ // Multiverse primitive refs
+
+ type BooleanRef = org.multiverse.transactional.refs.BooleanRef
+ type ByteRef = org.multiverse.transactional.refs.ByteRef
+ type CharRef = org.multiverse.transactional.refs.CharRef
+ type DoubleRef = org.multiverse.transactional.refs.DoubleRef
+ type FloatRef = org.multiverse.transactional.refs.FloatRef
+ type IntRef = org.multiverse.transactional.refs.IntRef
+ type LongRef = org.multiverse.transactional.refs.LongRef
+ type ShortRef = org.multiverse.transactional.refs.ShortRef
+
+ // Multiverse transactional datastructures
+
+ type TransactionalReferenceArray[T] = org.multiverse.transactional.arrays.TransactionalReferenceArray[T]
+ type TransactionalThreadPoolExecutor = org.multiverse.transactional.executors.TransactionalThreadPoolExecutor
+
+ // These won't compile:
+ // Transaction arg is added after varargs with byte code rewriting but Scala compiler doesn't allow this
+
+ // type TransactionalArrayList[T] = org.multiverse.transactional.collections.TransactionalArrayList[T]
+ // type TransactionalLinkedList[T] = org.multiverse.transactional.collections.TransactionalLinkedList[T]
+}
diff --git a/akka-stm/src/main/scala/akka/transactor/Atomically.scala b/akka-stm/src/main/scala/akka/transactor/Atomically.scala
new file mode 100644
index 0000000000..e20c85d4b4
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/transactor/Atomically.scala
@@ -0,0 +1,21 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.transactor
+
+import akka.stm.TransactionFactory
+
+/**
+ * For Java-friendly coordinated atomic blocks.
+ *
+ * Similar to [[akka.stm.Atomic]] but used to pass a block to Coordinated.atomic
+ * or to Coordination.coordinate.
+ *
+ * @see [[akka.transactor.Coordinated]]
+ * @see [[akka.transactor.Coordination]]
+ */
+abstract class Atomically(val factory: TransactionFactory) {
+ def this() = this(Coordinated.DefaultFactory)
+ def atomically: Unit
+}
diff --git a/akka-stm/src/main/scala/akka/transactor/Coordinated.scala b/akka-stm/src/main/scala/akka/transactor/Coordinated.scala
new file mode 100644
index 0000000000..81040fce44
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/transactor/Coordinated.scala
@@ -0,0 +1,165 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.transactor
+
+import akka.config.Config
+import akka.stm.{Atomic, DefaultTransactionConfig, TransactionFactory}
+
+import org.multiverse.api.{Transaction => MultiverseTransaction}
+import org.multiverse.commitbarriers.CountDownCommitBarrier
+import org.multiverse.templates.TransactionalCallable
+
+/**
+ * Coordinated transactions across actors.
+ */
+object Coordinated {
+ val DefaultFactory = TransactionFactory(DefaultTransactionConfig, "DefaultCoordinatedTransaction")
+ val Fair = Config.config.getBool("akka.stm.fair", true)
+
+ def apply(message: Any = null) = new Coordinated(message, createBarrier)
+
+ def unapply(c: Coordinated): Option[Any] = Some(c.message)
+
+ def createBarrier = new CountDownCommitBarrier(1, Fair)
+}
+
+/**
+ * `Coordinated` is a message wrapper that adds a `CountDownCommitBarrier` for explicitly
+ * coordinating transactions across actors or threads.
+ *
+ * Creating a `Coordinated` will create a count down barrier with initially one member.
+ * For each member in the coordination set a transaction is expected to be created using
+ * the coordinated atomic method. The number of included parties must match the number of
+ * transactions, otherwise a successful transaction cannot be coordinated.
+ *
+ *
+ * To start a new coordinated transaction set that you will also participate in just create
+ * a `Coordinated` object:
+ *
+ * {{{
+ * val coordinated = Coordinated()
+ * }}}
+ *
+ *
+ * To start a coordinated transaction that you won't participate in yourself you can create a
+ * `Coordinated` object with a message and send it directly to an actor. The recipient of the message
+ * will be the first member of the coordination set:
+ *
+ * {{{
+ * actor ! Coordinated(Message)
+ * }}}
+ *
+ *
+ * To receive a coordinated message in an actor simply match it in a case statement:
+ *
+ * {{{
+ * def receive = {
+ * case coordinated @ Coordinated(Message) => ...
+ * }
+ * }}}
+ *
+ *
+ * To include another actor in the same coordinated transaction set that you've created or
+ * received, use the apply method on that object. This will increment the number of parties
+ * involved by one and create a new `Coordinated` object to be sent.
+ *
+ * {{{
+ * actor ! coordinated(Message)
+ * }}}
+ *
+ *
+ * To enter the coordinated transaction use the atomic method of the coordinated object:
+ *
+ * {{{
+ * coordinated atomic {
+ * // do something in transaction ...
+ * }
+ * }}}
+ *
+ * The coordinated transaction will wait for the other transactions before committing.
+ * If any of the coordinated transactions fail then they all fail.
+ *
+ * @see [[akka.actor.Transactor]] for an actor that implements coordinated transactions
+ */
+class Coordinated(val message: Any, barrier: CountDownCommitBarrier) {
+
+ // Java API constructors
+ def this(message: Any) = this(message, Coordinated.createBarrier)
+ def this() = this(null, Coordinated.createBarrier)
+
+ /**
+ * Create a new Coordinated object and increment the number of parties by one.
+ * Use this method to ''pass on'' the coordination.
+ */
+ def apply(msg: Any) = {
+ barrier.incParties(1)
+ new Coordinated(msg, barrier)
+ }
+
+ /**
+ * Create a new Coordinated object but *do not* increment the number of parties by one.
+ * Only use this method if you know this is what you need.
+ */
+ def noIncrement(msg: Any) = new Coordinated(msg, barrier)
+
+ /**
+ * Java API: get the message for this Coordinated.
+ */
+ def getMessage() = message
+
+ /**
+ * Java API: create a new Coordinated object and increment the number of parties by one.
+ * Use this method to ''pass on'' the coordination.
+ */
+ def coordinate(msg: Any) = apply(msg)
+
+ /**
+ * Delimits the coordinated transaction. The transaction will wait for all other transactions
+ * in this coordination before committing. The timeout is specified by the transaction factory.
+ */
+ def atomic[T](body: => T)(implicit factory: TransactionFactory = Coordinated.DefaultFactory): T =
+ atomic(factory)(body)
+
+ /**
+ * Delimits the coordinated transaction. The transaction will wait for all other transactions
+ * in this coordination before committing. The timeout is specified by the transaction factory.
+ */
+ def atomic[T](factory: TransactionFactory)(body: => T): T = {
+ factory.boilerplate.execute(new TransactionalCallable[T]() {
+ def call(mtx: MultiverseTransaction): T = {
+ factory.addHooks
+ val result = body
+ val timeout = factory.config.timeout
+ try {
+ barrier.tryJoinCommit(mtx, timeout.length, timeout.unit)
+ } catch {
+ // Need to catch IllegalStateException until we have fix in Multiverse, since it throws it by mistake
+ case e: IllegalStateException => ()
+ }
+ result
+ }
+ })
+ }
+
+ /**
+ * Java API: coordinated atomic method that accepts an [[akka.stm.Atomic]].
+ * Delimits the coordinated transaction. The transaction will wait for all other transactions
+ * in this coordination before committing. The timeout is specified by the transaction factory.
+ */
+ def atomic[T](jatomic: Atomic[T]): T = atomic(jatomic.factory)(jatomic.atomically)
+
+ /**
+ * Java API: coordinated atomic method that accepts an [[akka.transactor.Atomically]].
+ * Delimits the coordinated transaction. The transaction will wait for all other transactions
+ * in this coordination before committing. The timeout is specified by the transaction factory.
+ */
+ def atomic(atomically: Atomically): Unit = atomic(atomically.factory)(atomically.atomically)
+
+ /**
+ * An empty coordinated atomic block. Can be used to complete the number of parties involved
+ * and wait for all transactions to complete. The default timeout is used.
+ */
+ def await() = atomic(Coordinated.DefaultFactory) {}
+}
diff --git a/akka-stm/src/main/scala/akka/transactor/Transactor.scala b/akka-stm/src/main/scala/akka/transactor/Transactor.scala
new file mode 100644
index 0000000000..72ba139d85
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/transactor/Transactor.scala
@@ -0,0 +1,179 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.transactor
+
+import akka.actor.{Actor, ActorRef}
+import akka.stm.{DefaultTransactionConfig, TransactionFactory}
+
+
+/**
+ * Used for specifying actor refs and messages to send to during coordination.
+ */
+case class SendTo(actor: ActorRef, message: Option[Any] = None)
+
+/**
+ * An actor with built-in support for coordinated transactions.
+ *
+ * Transactors implement the general pattern for using [[akka.stm.Coordinated]] where
+ * first any coordination messages are sent to other transactors, then the coordinated
+ * transaction is entered.
+ * Transactors can also accept explicitly sent `Coordinated` messages.
+ *
+ *
+ * Simple transactors will just implement the `atomically` method which is similar to
+ * the actor `receive` method but runs within a coordinated transaction.
+ *
+ * Example of a simple transactor that will join a coordinated transaction:
+ *
+ * {{{
+ * class Counter extends Transactor {
+ * val count = Ref(0)
+ *
+ * def atomically = {
+ * case Increment => count alter (_ + 1)
+ * }
+ * }
+ * }}}
+ *
+ *
+ * To coordinate with other transactors override the `coordinate` method.
+ * The `coordinate` method maps a message to a set
+ * of [[akka.actor.Transactor.SendTo]] objects, pairs of `ActorRef` and a message.
+ * You can use the `include` and `sendTo` methods to easily coordinate with other transactors.
+ * The `include` method will send on the same message that was received to other transactors.
+ * The `sendTo` method allows you to specify both the actor to send to, and message to send.
+ *
+ * Example of coordinating an increment:
+ *
+ * {{{
+ * class FriendlyCounter(friend: ActorRef) extends Transactor {
+ * val count = Ref(0)
+ *
+ * override def coordinate = {
+ * case Increment => include(friend)
+ * }
+ *
+ * def atomically = {
+ * case Increment => count alter (_ + 1)
+ * }
+ * }
+ * }}}
+ *
+ *
+ * Using `include` to include more than one transactor:
+ *
+ * {{{
+ * override def coordinate = {
+ * case Message => include(actor1, actor2, actor3)
+ * }
+ * }}}
+ *
+ *
+ * Using `sendTo` to coordinate transactions but send on a different message
+ * than the one that was received:
+ *
+ * {{{
+ * override def coordinate = {
+ * case Message => sendTo(someActor -> SomeOtherMessage)
+ * case SomeMessage => sendTo(actor1 -> Message1, actor2 -> Message2)
+ * }
+ * }}}
+ *
+ *
+ * To exeucte directly before or after the coordinated transaction, override
+ * the `before` and `after` methods. These methods also expect partial functions
+ * like the receive method. They do not execute within the transaction.
+ *
+ * To completely bypass coordinated transactions override the `normally` method.
+ * Any message matched by `normally` will not be matched by the other methods,
+ * and will not be involved in coordinated transactions. In this method you
+ * can implement normal actor behavior, or use the normal STM atomic for
+ * local transactions.
+ *
+ * @see [[akka.stm.Coordinated]] for more information about the underlying mechanism
+ */
+trait Transactor extends Actor {
+ private lazy val txFactory = transactionFactory
+
+ /**
+ * Create default transaction factory. Override to provide custom configuration.
+ */
+ def transactionFactory = TransactionFactory(DefaultTransactionConfig)
+
+ /**
+ * Implement a general pattern for using coordinated transactions.
+ */
+ final def receive = {
+ case coordinated @ Coordinated(message) => {
+ val others = (coordinate orElse alone)(message)
+ for (sendTo <- others) {
+ sendTo.actor ! coordinated(sendTo.message.getOrElse(message))
+ }
+ (before orElse doNothing)(message)
+ coordinated.atomic(txFactory) { (atomically orElse doNothing)(message) }
+ (after orElse doNothing)(message)
+ }
+ case message => {
+ if (normally.isDefinedAt(message)) normally(message)
+ else receive(Coordinated(message))
+ }
+ }
+
+ /**
+ * Override this method to coordinate with other transactors.
+ * The other transactors are added to the coordinated transaction barrier
+ * and sent a Coordinated message. The message to send can be specified
+ * or otherwise the same message as received is sent. Use the 'include' and
+ * 'sendTo' methods to easily create the set of transactors to be involved.
+ */
+ def coordinate: PartialFunction[Any, Set[SendTo]] = alone
+
+ /**
+ * Default coordination - no other transactors.
+ */
+ def alone: PartialFunction[Any, Set[SendTo]] = { case _ => nobody }
+
+ /**
+ * Empty set of transactors to send to.
+ */
+ def nobody: Set[SendTo] = Set.empty
+
+ /**
+ * Include other actors in this coordinated transaction and send
+ * them the same message as received. Use as the result in 'coordinated'.
+ */
+ def include(actors: ActorRef*): Set[SendTo] = actors map (SendTo(_)) toSet
+
+ /**
+ * Include other actors in this coordinated transaction and specify the message
+ * to send by providing ActorRef -> Message pairs. Use as the result in 'coordinated'.
+ */
+ def sendTo(pairs: (ActorRef, Any)*): Set[SendTo] = pairs map (p => SendTo(p._1, Some(p._2))) toSet
+
+ /**
+ * A Receive block that runs before the coordinated transaction is entered.
+ */
+ def before: Receive = doNothing
+
+ /**
+ * The Receive block to run inside the coordinated transaction.
+ */
+ def atomically: Receive
+
+ /**
+ * A Receive block that runs after the coordinated transaction.
+ */
+ def after: Receive = doNothing
+
+ /**
+ * Bypass transactionality and behave like a normal actor.
+ */
+ def normally: Receive = doNothing
+
+ /**
+ * Default catch-all for the different Receive methods.
+ */
+ def doNothing: Receive = { case _ => }
+}
diff --git a/akka-stm/src/main/scala/akka/transactor/UntypedTransactor.scala b/akka-stm/src/main/scala/akka/transactor/UntypedTransactor.scala
new file mode 100644
index 0000000000..9383d67aaf
--- /dev/null
+++ b/akka-stm/src/main/scala/akka/transactor/UntypedTransactor.scala
@@ -0,0 +1,110 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.transactor
+
+import akka.actor.{UntypedActor, ActorRef}
+import akka.stm.{DefaultTransactionConfig, TransactionFactory}
+
+import java.util.{Set => JSet}
+
+import scala.collection.JavaConversions._
+
+
+/**
+ * An UntypedActor version of transactor for using from Java.
+ */
+abstract class UntypedTransactor extends UntypedActor {
+ private lazy val txFactory = transactionFactory
+
+ /**
+ * Create default transaction factory. Override to provide custom configuration.
+ */
+ def transactionFactory = TransactionFactory(DefaultTransactionConfig)
+
+ /**
+ * Implement a general pattern for using coordinated transactions.
+ */
+ @throws(classOf[Exception])
+ final def onReceive(message: Any): Unit = {
+ message match {
+ case coordinated @ Coordinated(message) => {
+ val others = coordinate(message)
+ for (sendTo <- others) {
+ sendTo.actor.sendOneWay(coordinated(sendTo.message.getOrElse(message)))
+ }
+ before(message)
+ coordinated.atomic(txFactory) { atomically(message) }
+ after(message)
+ }
+ case message => {
+ val normal = normally(message)
+ if (!normal) onReceive(Coordinated(message))
+ }
+ }
+ }
+
+ /**
+ * Override this method to coordinate with other transactors.
+ * The other transactors are added to the coordinated transaction barrier
+ * and sent a Coordinated message. The message to send can be specified
+ * or otherwise the same message as received is sent. Use the 'include' and
+ * 'sendTo' methods to easily create the set of transactors to be involved.
+ */
+ @throws(classOf[Exception])
+ def coordinate(message: Any): JSet[SendTo] = nobody
+
+ /**
+ * Empty set of transactors to send to.
+ */
+ def nobody: JSet[SendTo] = Set[SendTo]()
+
+ /**
+ * For including one other actor in this coordinated transaction and sending
+ * them the same message as received. Use as the result in `coordinated`.
+ */
+ def include(actor: ActorRef): JSet[SendTo] = Set(SendTo(actor))
+
+ /**
+ * For including one other actor in this coordinated transaction and specifying the
+ * message to send. Use as the result in `coordinated`.
+ */
+ def include(actor: ActorRef, message: Any): JSet[SendTo] = Set(SendTo(actor, Some(message)))
+
+ /**
+ * For including another actor in this coordinated transaction and sending
+ * them the same message as received. Use to create the result in `coordinated`.
+ */
+ def sendTo(actor: ActorRef): SendTo = SendTo(actor)
+
+ /**
+ * For including another actor in this coordinated transaction and specifying the
+ * message to send. Use to create the result in `coordinated`.
+ */
+ def sendTo(actor: ActorRef, message: Any): SendTo = SendTo(actor, Some(message))
+
+ /**
+ * A Receive block that runs before the coordinated transaction is entered.
+ */
+ @throws(classOf[Exception])
+ def before(message: Any): Unit = {}
+
+ /**
+ * The Receive block to run inside the coordinated transaction.
+ */
+ @throws(classOf[Exception])
+ def atomically(message: Any): Unit = {}
+
+ /**
+ * A Receive block that runs after the coordinated transaction.
+ */
+ @throws(classOf[Exception])
+ def after(message: Any): Unit = {}
+
+ /**
+ * Bypass transactionality and behave like a normal actor.
+ */
+ @throws(classOf[Exception])
+ def normally(message: Any): Boolean = false
+}
diff --git a/akka-stm/src/test/java/akka/stm/example/Address.java b/akka-stm/src/test/java/akka/stm/example/Address.java
new file mode 100644
index 0000000000..03201dd627
--- /dev/null
+++ b/akka-stm/src/test/java/akka/stm/example/Address.java
@@ -0,0 +1,13 @@
+package akka.stm.example;
+
+public class Address {
+ private String location;
+
+ public Address(String location) {
+ this.location = location;
+ }
+
+ @Override public String toString() {
+ return "Address(" + location + ")";
+ }
+}
diff --git a/akka-stm/src/test/java/akka/stm/example/CounterExample.java b/akka-stm/src/test/java/akka/stm/example/CounterExample.java
new file mode 100644
index 0000000000..0624cadf43
--- /dev/null
+++ b/akka-stm/src/test/java/akka/stm/example/CounterExample.java
@@ -0,0 +1,25 @@
+package akka.stm.example;
+
+import akka.stm.*;
+
+public class CounterExample {
+ final static Ref ref = new Ref(0);
+
+ public static int counter() {
+ return new Atomic() {
+ public Integer atomically() {
+ int inc = ref.get() + 1;
+ ref.set(inc);
+ return inc;
+ }
+ }.execute();
+ }
+
+ public static void main(String[] args) {
+ System.out.println();
+ System.out.println("Counter example");
+ System.out.println();
+ System.out.println("counter 1: " + counter());
+ System.out.println("counter 2: " + counter());
+ }
+}
diff --git a/akka-stm/src/test/java/akka/stm/example/RefExample.java b/akka-stm/src/test/java/akka/stm/example/RefExample.java
new file mode 100644
index 0000000000..ddc1d744d1
--- /dev/null
+++ b/akka-stm/src/test/java/akka/stm/example/RefExample.java
@@ -0,0 +1,35 @@
+package akka.stm.example;
+
+import akka.stm.*;
+
+public class RefExample {
+ public static void main(String[] args) {
+ System.out.println();
+ System.out.println("Ref example");
+ System.out.println();
+
+ final Ref ref = new Ref(0);
+
+ Integer value1 = new Atomic() {
+ public Integer atomically() {
+ return ref.get();
+ }
+ }.execute();
+
+ System.out.println("value 1: " + value1);
+
+ new Atomic() {
+ public Object atomically() {
+ return ref.set(5);
+ }
+ }.execute();
+
+ Integer value2 = new Atomic() {
+ public Integer atomically() {
+ return ref.get();
+ }
+ }.execute();
+
+ System.out.println("value 2: " + value2);
+ }
+}
diff --git a/akka-stm/src/test/java/akka/stm/example/StmExamples.java b/akka-stm/src/test/java/akka/stm/example/StmExamples.java
new file mode 100644
index 0000000000..8e104b181a
--- /dev/null
+++ b/akka-stm/src/test/java/akka/stm/example/StmExamples.java
@@ -0,0 +1,15 @@
+package akka.stm.example;
+
+public class StmExamples {
+ public static void main(String[] args) {
+ System.out.println();
+ System.out.println("STM examples");
+ System.out.println();
+
+ CounterExample.main(args);
+ RefExample.main(args);
+ TransactionFactoryExample.main(args);
+ TransactionalMapExample.main(args);
+ TransactionalVectorExample.main(args);
+ }
+}
diff --git a/akka-stm/src/test/java/akka/stm/example/TransactionFactoryExample.java b/akka-stm/src/test/java/akka/stm/example/TransactionFactoryExample.java
new file mode 100644
index 0000000000..10945df4d6
--- /dev/null
+++ b/akka-stm/src/test/java/akka/stm/example/TransactionFactoryExample.java
@@ -0,0 +1,29 @@
+package akka.stm.example;
+
+import akka.stm.*;
+
+import org.multiverse.api.ThreadLocalTransaction;
+import org.multiverse.api.TransactionConfiguration;
+
+public class TransactionFactoryExample {
+ public static void main(String[] args) {
+ System.out.println();
+ System.out.println("TransactionFactory example");
+ System.out.println();
+
+ TransactionFactory txFactory = new TransactionFactoryBuilder()
+ .setFamilyName("example")
+ .setReadonly(true)
+ .build();
+
+ new Atomic(txFactory) {
+ public Object atomically() {
+ // check config has been passed to multiverse
+ TransactionConfiguration config = ThreadLocalTransaction.getThreadLocalTransaction().getConfiguration();
+ System.out.println("family name: " + config.getFamilyName());
+ System.out.println("readonly: " + config.isReadonly());
+ return null;
+ }
+ }.execute();
+ }
+}
diff --git a/akka-stm/src/test/java/akka/stm/example/TransactionalMapExample.java b/akka-stm/src/test/java/akka/stm/example/TransactionalMapExample.java
new file mode 100644
index 0000000000..8276dc480c
--- /dev/null
+++ b/akka-stm/src/test/java/akka/stm/example/TransactionalMapExample.java
@@ -0,0 +1,34 @@
+package akka.stm.example;
+
+import akka.stm.*;
+
+public class TransactionalMapExample {
+ public static void main(String[] args) {
+ System.out.println();
+ System.out.println("TransactionalMap example");
+ System.out.println();
+
+ final TransactionalMap users = new TransactionalMap();
+
+ // fill users map (in a transaction)
+ new Atomic() {
+ public Object atomically() {
+ users.put("bill", new User("bill"));
+ users.put("mary", new User("mary"));
+ users.put("john", new User("john"));
+ return null;
+ }
+ }.execute();
+
+ System.out.println("users: " + users);
+
+ // access users map (in a transaction)
+ User user = new Atomic() {
+ public User atomically() {
+ return users.get("bill").get();
+ }
+ }.execute();
+
+ System.out.println("user: " + user);
+ }
+}
diff --git a/akka-stm/src/test/java/akka/stm/example/TransactionalVectorExample.java b/akka-stm/src/test/java/akka/stm/example/TransactionalVectorExample.java
new file mode 100644
index 0000000000..4340dab619
--- /dev/null
+++ b/akka-stm/src/test/java/akka/stm/example/TransactionalVectorExample.java
@@ -0,0 +1,33 @@
+package akka.stm.example;
+
+import akka.stm.*;
+
+public class TransactionalVectorExample {
+ public static void main(String[] args) {
+ System.out.println();
+ System.out.println("TransactionalVector example");
+ System.out.println();
+
+ final TransactionalVector addresses = new TransactionalVector();
+
+ // fill addresses vector (in a transaction)
+ new Atomic() {
+ public Object atomically() {
+ addresses.add(new Address("somewhere"));
+ addresses.add(new Address("somewhere else"));
+ return null;
+ }
+ }.execute();
+
+ System.out.println("addresses: " + addresses);
+
+ // access addresses vector (in a transaction)
+ Address address = new Atomic() {
+ public Address atomically() {
+ return addresses.get(0);
+ }
+ }.execute();
+
+ System.out.println("address: " + address);
+ }
+}
diff --git a/akka-stm/src/test/java/akka/stm/example/User.java b/akka-stm/src/test/java/akka/stm/example/User.java
new file mode 100644
index 0000000000..400f538d03
--- /dev/null
+++ b/akka-stm/src/test/java/akka/stm/example/User.java
@@ -0,0 +1,13 @@
+package akka.stm.example;
+
+public class User {
+ private String name;
+
+ public User(String name) {
+ this.name = name;
+ }
+
+ @Override public String toString() {
+ return "User(" + name + ")";
+ }
+}
diff --git a/akka-stm/src/test/java/akka/stm/test/JavaStmTests.java b/akka-stm/src/test/java/akka/stm/test/JavaStmTests.java
new file mode 100644
index 0000000000..79547c1bb6
--- /dev/null
+++ b/akka-stm/src/test/java/akka/stm/test/JavaStmTests.java
@@ -0,0 +1,90 @@
+package akka.stm.test;
+
+import static org.junit.Assert.*;
+import org.junit.Test;
+import org.junit.Before;
+
+import akka.stm.*;
+
+import org.multiverse.api.ThreadLocalTransaction;
+import org.multiverse.api.TransactionConfiguration;
+import org.multiverse.api.exceptions.ReadonlyException;
+
+public class JavaStmTests {
+
+ private Ref ref;
+
+ private int getRefValue() {
+ return new Atomic() {
+ public Integer atomically() {
+ return ref.get();
+ }
+ }.execute();
+ }
+
+ public int increment() {
+ return new Atomic() {
+ public Integer atomically() {
+ int inc = ref.get() + 1;
+ ref.set(inc);
+ return inc;
+ }
+ }.execute();
+ }
+
+ @Before public void initialise() {
+ ref = new Ref(0);
+ }
+
+ @Test public void incrementRef() {
+ assertEquals(0, getRefValue());
+ increment();
+ increment();
+ increment();
+ assertEquals(3, getRefValue());
+ }
+
+ @Test public void failSetRef() {
+ assertEquals(0, getRefValue());
+ try {
+ new Atomic() {
+ public Object atomically() {
+ ref.set(3);
+ throw new RuntimeException();
+ }
+ }.execute();
+ } catch(RuntimeException e) {}
+ assertEquals(0, getRefValue());
+ }
+
+ @Test public void configureTransaction() {
+ TransactionFactory txFactory = new TransactionFactoryBuilder()
+ .setFamilyName("example")
+ .setReadonly(true)
+ .build();
+
+ // get transaction config from multiverse
+ TransactionConfiguration config = new Atomic(txFactory) {
+ public TransactionConfiguration atomically() {
+ ref.get();
+ return ThreadLocalTransaction.getThreadLocalTransaction().getConfiguration();
+ }
+ }.execute();
+
+ assertEquals("example", config.getFamilyName());
+ assertEquals(true, config.isReadonly());
+ }
+
+ @Test(expected=ReadonlyException.class) public void failReadonlyTransaction() {
+ TransactionFactory txFactory = new TransactionFactoryBuilder()
+ .setFamilyName("example")
+ .setReadonly(true)
+ .build();
+
+ new Atomic(txFactory) {
+ public Object atomically() {
+ return ref.set(3);
+ }
+ }.execute();
+ }
+}
diff --git a/akka-stm/src/test/java/akka/transactor/example/Increment.java b/akka-stm/src/test/java/akka/transactor/example/Increment.java
new file mode 100644
index 0000000000..bcb0988d41
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/example/Increment.java
@@ -0,0 +1,21 @@
+package akka.transactor.example;
+
+import akka.actor.ActorRef;
+
+public class Increment {
+ private ActorRef friend = null;
+
+ public Increment() {}
+
+ public Increment(ActorRef friend) {
+ this.friend = friend;
+ }
+
+ public boolean hasFriend() {
+ return friend != null;
+ }
+
+ public ActorRef getFriend() {
+ return friend;
+ }
+}
diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java
new file mode 100644
index 0000000000..649ef42d3b
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedCounter.java
@@ -0,0 +1,39 @@
+package akka.transactor.example;
+
+import akka.transactor.Coordinated;
+import akka.transactor.Atomically;
+import akka.actor.ActorRef;
+import akka.actor.UntypedActor;
+import akka.stm.Ref;
+
+public class UntypedCoordinatedCounter extends UntypedActor {
+ private Ref count = new Ref(0);
+
+ private void increment() {
+ System.out.println("incrementing");
+ count.set(count.get() + 1);
+ }
+
+ public void onReceive(Object incoming) throws Exception {
+ if (incoming instanceof Coordinated) {
+ Coordinated coordinated = (Coordinated) incoming;
+ Object message = coordinated.getMessage();
+ if (message instanceof Increment) {
+ Increment increment = (Increment) message;
+ if (increment.hasFriend()) {
+ increment.getFriend().sendOneWay(coordinated.coordinate(new Increment()));
+ }
+ coordinated.atomic(new Atomically() {
+ public void atomically() {
+ increment();
+ }
+ });
+ }
+ } else if (incoming instanceof String) {
+ String message = (String) incoming;
+ if (message.equals("GetCount")) {
+ getContext().replyUnsafe(count.get());
+ }
+ }
+ }
+}
diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java
new file mode 100644
index 0000000000..d3a2a14107
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCoordinatedExample.java
@@ -0,0 +1,44 @@
+package akka.transactor.example;
+
+import akka.transactor.Coordinated;
+import akka.actor.ActorRef;
+import akka.actor.UntypedActor;
+import akka.dispatch.Future;
+import akka.dispatch.Futures;
+
+public class UntypedCoordinatedExample {
+ public static void main(String[] args) throws InterruptedException {
+ System.out.println();
+ System.out.println("Untyped transactor example");
+ System.out.println();
+
+ ActorRef counter1 = UntypedActor.actorOf(UntypedCoordinatedCounter.class).start();
+ ActorRef counter2 = UntypedActor.actorOf(UntypedCoordinatedCounter.class).start();
+
+ counter1.sendOneWay(new Coordinated(new Increment(counter2)));
+
+ Thread.sleep(3000);
+
+ Future future1 = counter1.sendRequestReplyFuture("GetCount");
+ Future future2 = counter2.sendRequestReplyFuture("GetCount");
+
+ future1.await();
+ if (future1.isCompleted()) {
+ if (future1.result().isDefined()) {
+ int result = (Integer) future1.result().get();
+ System.out.println("counter 1: " + result);
+ }
+ }
+
+ future2.await();
+ if (future2.isCompleted()) {
+ if (future2.result().isDefined()) {
+ int result = (Integer) future2.result().get();
+ System.out.println("counter 2: " + result);
+ }
+ }
+
+ counter1.stop();
+ counter2.stop();
+ }
+}
diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedCounter.java b/akka-stm/src/test/java/akka/transactor/example/UntypedCounter.java
new file mode 100644
index 0000000000..56cc12f6c2
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/example/UntypedCounter.java
@@ -0,0 +1,33 @@
+package akka.transactor.example;
+
+import akka.transactor.UntypedTransactor;
+import akka.transactor.SendTo;
+import akka.stm.Ref;
+
+import java.util.Set;
+
+public class UntypedCounter extends UntypedTransactor {
+ Ref count = new Ref(0);
+
+ @Override public Set coordinate(Object message) {
+ if (message instanceof Increment) {
+ Increment increment = (Increment) message;
+ if (increment.hasFriend())
+ return include(increment.getFriend(), new Increment());
+ }
+ return nobody();
+ }
+
+ public void atomically(Object message) {
+ if (message instanceof Increment) {
+ count.set(count.get() + 1);
+ }
+ }
+
+ @Override public boolean normally(Object message) {
+ if ("GetCount".equals(message)) {
+ getContext().replyUnsafe(count.get());
+ return true;
+ } else return false;
+ }
+}
diff --git a/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java b/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java
new file mode 100644
index 0000000000..fdef74ca60
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/example/UntypedTransactorExample.java
@@ -0,0 +1,43 @@
+package akka.transactor.example;
+
+import akka.actor.ActorRef;
+import akka.actor.UntypedActor;
+import akka.dispatch.Future;
+import akka.dispatch.Futures;
+
+public class UntypedTransactorExample {
+ public static void main(String[] args) throws InterruptedException {
+ System.out.println();
+ System.out.println("Untyped transactor example");
+ System.out.println();
+
+ ActorRef counter1 = UntypedActor.actorOf(UntypedCounter.class).start();
+ ActorRef counter2 = UntypedActor.actorOf(UntypedCounter.class).start();
+
+ counter1.sendOneWay(new Increment(counter2));
+
+ Thread.sleep(3000);
+
+ Future future1 = counter1.sendRequestReplyFuture("GetCount");
+ Future future2 = counter2.sendRequestReplyFuture("GetCount");
+
+ future1.await();
+ if (future1.isCompleted()) {
+ if (future1.result().isDefined()) {
+ int result = (Integer) future1.result().get();
+ System.out.println("counter 1: " + result);
+ }
+ }
+
+ future2.await();
+ if (future2.isCompleted()) {
+ if (future2.result().isDefined()) {
+ int result = (Integer) future2.result().get();
+ System.out.println("counter 2: " + result);
+ }
+ }
+
+ counter1.stop();
+ counter2.stop();
+ }
+}
diff --git a/akka-stm/src/test/java/akka/transactor/test/Increment.java b/akka-stm/src/test/java/akka/transactor/test/Increment.java
new file mode 100644
index 0000000000..1d1e3399fc
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/test/Increment.java
@@ -0,0 +1,23 @@
+package akka.transactor.test;
+
+import akka.actor.ActorRef;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+public class Increment {
+ private List friends;
+ private CountDownLatch latch;
+
+ public Increment(List friends, CountDownLatch latch) {
+ this.friends = friends;
+ this.latch = latch;
+ }
+
+ public List getFriends() {
+ return friends;
+ }
+
+ public CountDownLatch getLatch() {
+ return latch;
+ }
+}
\ No newline at end of file
diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java
new file mode 100644
index 0000000000..b1030106de
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedCounter.java
@@ -0,0 +1,63 @@
+package akka.transactor.test;
+
+import akka.transactor.Coordinated;
+import akka.transactor.Atomically;
+import akka.actor.ActorRef;
+import akka.actor.UntypedActor;
+import akka.stm.*;
+import akka.util.Duration;
+
+import org.multiverse.api.StmUtils;
+
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+public class UntypedCoordinatedCounter extends UntypedActor {
+ private String name;
+ private Ref count = new Ref(0);
+ private TransactionFactory txFactory = new TransactionFactoryBuilder()
+ .setTimeout(new Duration(3, TimeUnit.SECONDS))
+ .build();
+
+ public UntypedCoordinatedCounter(String name) {
+ this.name = name;
+ }
+
+ private void increment() {
+ System.out.println(name + ": incrementing");
+ count.set(count.get() + 1);
+ }
+
+ public void onReceive(Object incoming) throws Exception {
+ if (incoming instanceof Coordinated) {
+ Coordinated coordinated = (Coordinated) incoming;
+ Object message = coordinated.getMessage();
+ if (message instanceof Increment) {
+ Increment increment = (Increment) message;
+ List friends = increment.getFriends();
+ final CountDownLatch latch = increment.getLatch();
+ if (!friends.isEmpty()) {
+ Increment coordMessage = new Increment(friends.subList(1, friends.size()), latch);
+ friends.get(0).sendOneWay(coordinated.coordinate(coordMessage));
+ }
+ coordinated.atomic(new Atomically(txFactory) {
+ public void atomically() {
+ increment();
+ StmUtils.scheduleDeferredTask(new Runnable() {
+ public void run() { latch.countDown(); }
+ });
+ StmUtils.scheduleCompensatingTask(new Runnable() {
+ public void run() { latch.countDown(); }
+ });
+ }
+ });
+ }
+ } else if (incoming instanceof String) {
+ String message = (String) incoming;
+ if (message.equals("GetCount")) {
+ getContext().replyUnsafe(count.get());
+ }
+ }
+ }
+}
diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java
new file mode 100644
index 0000000000..35635ceb4f
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCoordinatedIncrementTest.java
@@ -0,0 +1,88 @@
+package akka.transactor.test;
+
+import static org.junit.Assert.*;
+import org.junit.Test;
+import org.junit.Before;
+
+import akka.transactor.Coordinated;
+import akka.actor.ActorRef;
+import akka.actor.UntypedActor;
+import akka.actor.UntypedActorFactory;
+import akka.dispatch.Future;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import scala.Option;
+
+public class UntypedCoordinatedIncrementTest {
+ List counters;
+ ActorRef failer;
+
+ int numCounters = 5;
+ int timeout = 5;
+
+ @Before public void initialise() {
+ counters = new ArrayList();
+ for (int i = 1; i <= numCounters; i++) {
+ final String name = "counter" + i;
+ ActorRef counter = UntypedActor.actorOf(new UntypedActorFactory() {
+ public UntypedActor create() {
+ return new UntypedCoordinatedCounter(name);
+ }
+ });
+ counter.start();
+ counters.add(counter);
+ }
+ failer = UntypedActor.actorOf(UntypedFailer.class);
+ failer.start();
+ }
+
+ @Test public void incrementAllCountersWithSuccessfulTransaction() {
+ CountDownLatch incrementLatch = new CountDownLatch(numCounters);
+ Increment message = new Increment(counters.subList(1, counters.size()), incrementLatch);
+ counters.get(0).sendOneWay(new Coordinated(message));
+ try {
+ incrementLatch.await(timeout, TimeUnit.SECONDS);
+ } catch (InterruptedException exception) {}
+ for (ActorRef counter : counters) {
+ Future future = counter.sendRequestReplyFuture("GetCount");
+ future.await();
+ if (future.isCompleted()) {
+ Option resultOption = future.result();
+ if (resultOption.isDefined()) {
+ Object result = resultOption.get();
+ int count = (Integer) result;
+ assertEquals(1, count);
+ }
+ }
+ }
+ }
+
+ @Test public void incrementNoCountersWithFailingTransaction() {
+ CountDownLatch incrementLatch = new CountDownLatch(numCounters);
+ List actors = new ArrayList(counters);
+ actors.add(failer);
+ Increment message = new Increment(actors.subList(1, actors.size()), incrementLatch);
+ actors.get(0).sendOneWay(new Coordinated(message));
+ try {
+ incrementLatch.await(timeout, TimeUnit.SECONDS);
+ } catch (InterruptedException exception) {}
+ for (ActorRef counter : counters) {
+ Future future = counter.sendRequestReplyFuture("GetCount");
+ future.await();
+ if (future.isCompleted()) {
+ Option resultOption = future.result();
+ if (resultOption.isDefined()) {
+ Object result = resultOption.get();
+ int count = (Integer) result;
+ assertEquals(0, count);
+ }
+ }
+ }
+ }
+}
+
+
diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java b/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java
new file mode 100644
index 0000000000..d343ceea31
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/test/UntypedCounter.java
@@ -0,0 +1,77 @@
+package akka.transactor.test;
+
+import akka.transactor.UntypedTransactor;
+import akka.transactor.SendTo;
+import akka.actor.ActorRef;
+import akka.stm.*;
+import akka.util.Duration;
+
+import org.multiverse.api.StmUtils;
+
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+public class UntypedCounter extends UntypedTransactor {
+ private String name;
+ private Ref count = new Ref(0);
+
+ public UntypedCounter(String name) {
+ this.name = name;
+ }
+
+ @Override public TransactionFactory transactionFactory() {
+ return new TransactionFactoryBuilder()
+ .setTimeout(new Duration(3, TimeUnit.SECONDS))
+ .build();
+ }
+
+ private void increment() {
+ System.out.println(name + ": incrementing");
+ count.set(count.get() + 1);
+ }
+
+ @Override public Set coordinate(Object message) {
+ if (message instanceof Increment) {
+ Increment increment = (Increment) message;
+ List friends = increment.getFriends();
+ if (!friends.isEmpty()) {
+ Increment coordMessage = new Increment(friends.subList(1, friends.size()), increment.getLatch());
+ return include(friends.get(0), coordMessage);
+ } else {
+ return nobody();
+ }
+ } else {
+ return nobody();
+ }
+ }
+
+ @Override public void before(Object message) {
+ System.out.println(name + ": before transaction");
+ }
+
+ public void atomically(Object message) {
+ if (message instanceof Increment) {
+ increment();
+ final Increment increment = (Increment) message;
+ StmUtils.scheduleDeferredTask(new Runnable() {
+ public void run() { increment.getLatch().countDown(); }
+ });
+ StmUtils.scheduleCompensatingTask(new Runnable() {
+ public void run() { increment.getLatch().countDown(); }
+ });
+ }
+ }
+
+ @Override public void after(Object message) {
+ System.out.println(name + ": after transaction");
+ }
+
+ @Override public boolean normally(Object message) {
+ if ("GetCount".equals(message)) {
+ getContext().replyUnsafe(count.get());
+ return true;
+ } else return false;
+ }
+}
\ No newline at end of file
diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedFailer.java b/akka-stm/src/test/java/akka/transactor/test/UntypedFailer.java
new file mode 100644
index 0000000000..6d2db1b803
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/test/UntypedFailer.java
@@ -0,0 +1,9 @@
+package akka.transactor.test;
+
+import akka.actor.UntypedActor;
+
+public class UntypedFailer extends UntypedActor {
+ public void onReceive(Object incoming) throws Exception {
+ throw new RuntimeException("Expected failure");
+ }
+}
diff --git a/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java
new file mode 100644
index 0000000000..e378b1c598
--- /dev/null
+++ b/akka-stm/src/test/java/akka/transactor/test/UntypedTransactorTest.java
@@ -0,0 +1,87 @@
+package akka.transactor.test;
+
+import static org.junit.Assert.*;
+import org.junit.Test;
+import org.junit.Before;
+
+import akka.actor.ActorRef;
+import akka.actor.UntypedActor;
+import akka.actor.UntypedActorFactory;
+import akka.dispatch.Future;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import scala.Option;
+
+public class UntypedTransactorTest {
+ List counters;
+ ActorRef failer;
+
+ int numCounters = 5;
+ int timeout = 5;
+
+ @Before public void initialise() {
+ counters = new ArrayList();
+ for (int i = 1; i <= numCounters; i++) {
+ final String name = "counter" + i;
+ ActorRef counter = UntypedActor.actorOf(new UntypedActorFactory() {
+ public UntypedActor create() {
+ return new UntypedCounter(name);
+ }
+ });
+ counter.start();
+ counters.add(counter);
+ }
+ failer = UntypedActor.actorOf(UntypedFailer.class);
+ failer.start();
+ }
+
+ @Test public void incrementAllCountersWithSuccessfulTransaction() {
+ CountDownLatch incrementLatch = new CountDownLatch(numCounters);
+ Increment message = new Increment(counters.subList(1, counters.size()), incrementLatch);
+ counters.get(0).sendOneWay(message);
+ try {
+ incrementLatch.await(timeout, TimeUnit.SECONDS);
+ } catch (InterruptedException exception) {}
+ for (ActorRef counter : counters) {
+ Future future = counter.sendRequestReplyFuture("GetCount");
+ future.await();
+ if (future.isCompleted()) {
+ Option resultOption = future.result();
+ if (resultOption.isDefined()) {
+ Object result = resultOption.get();
+ int count = (Integer) result;
+ assertEquals(1, count);
+ }
+ }
+ }
+ }
+
+ @Test public void incrementNoCountersWithFailingTransaction() {
+ CountDownLatch incrementLatch = new CountDownLatch(numCounters);
+ List actors = new ArrayList(counters);
+ actors.add(failer);
+ Increment message = new Increment(actors.subList(1, actors.size()), incrementLatch);
+ actors.get(0).sendOneWay(message);
+ try {
+ incrementLatch.await(timeout, TimeUnit.SECONDS);
+ } catch (InterruptedException exception) {}
+ for (ActorRef counter : counters) {
+ Future future = counter.sendRequestReplyFuture("GetCount");
+ future.await();
+ if (future.isCompleted()) {
+ Option resultOption = future.result();
+ if (resultOption.isDefined()) {
+ Object result = resultOption.get();
+ int count = (Integer) result;
+ assertEquals(0, count);
+ }
+ }
+ }
+ }
+}
+
+
diff --git a/akka-stm/src/test/scala/agent/AgentSpec.scala b/akka-stm/src/test/scala/agent/AgentSpec.scala
new file mode 100644
index 0000000000..7f99f24664
--- /dev/null
+++ b/akka-stm/src/test/scala/agent/AgentSpec.scala
@@ -0,0 +1,170 @@
+package akka.agent.test
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+
+import akka.agent.Agent
+import akka.stm._
+import akka.util.Duration
+import akka.util.duration._
+
+import java.util.concurrent.CountDownLatch
+
+class CountDownFunction[A](num: Int = 1) extends Function1[A, A] {
+ val latch = new CountDownLatch(num)
+ def apply(a: A) = { latch.countDown; a }
+ def await(timeout: Duration) = latch.await(timeout.length, timeout.unit)
+}
+
+class AgentSpec extends WordSpec with MustMatchers {
+ "Agent" should {
+ "update with send dispatches in order sent" in {
+ val countDown = new CountDownFunction[String]
+
+ val agent = Agent("a")
+ agent send (_ + "b")
+ agent send (_ + "c")
+ agent send (_ + "d")
+ agent send countDown
+
+ countDown.await(5 seconds)
+ agent() must be ("abcd")
+
+ agent.close
+ }
+
+ "maintain order between send and sendOff" in {
+ val countDown = new CountDownFunction[String]
+
+ val agent = Agent("a")
+ agent send (_ + "b")
+ val longRunning = (s: String) => { Thread.sleep(2000); s + "c" }
+ agent sendOff longRunning
+ agent send (_ + "d")
+ agent send countDown
+
+ countDown.await(5 seconds)
+ agent() must be ("abcd")
+
+ agent.close
+ }
+
+ "be immediately readable" in {
+ val countDown = new CountDownFunction[Int]
+ val readLatch = new CountDownLatch(1)
+ val readTimeout = 5 seconds
+
+ val agent = Agent(5)
+ val f1 = (i: Int) => {
+ readLatch.await(readTimeout.length, readTimeout.unit)
+ i + 5
+ }
+ agent send f1
+ val read = agent()
+ readLatch.countDown
+ agent send countDown
+
+ countDown.await(5 seconds)
+ read must be (5)
+ agent() must be (10)
+
+ agent.close
+ }
+
+ "be readable within a transaction" in {
+ val agent = Agent(5)
+ val value = atomic { agent() }
+ value must be (5)
+ agent.close
+ }
+
+ "dispatch sends in successful transactions" in {
+ val countDown = new CountDownFunction[Int]
+
+ val agent = Agent(5)
+ atomic {
+ agent send (_ * 2)
+ }
+ agent send countDown
+
+ countDown.await(5 seconds)
+ agent() must be (10)
+
+ agent.close
+ }
+
+ "not dispatch sends in aborted transactions" in {
+ val countDown = new CountDownFunction[Int]
+
+ val agent = Agent(5)
+
+ try {
+ atomic(DefaultTransactionFactory) {
+ agent send (_ * 2)
+ throw new RuntimeException("Expected failure")
+ }
+ } catch { case _ => }
+
+ agent send countDown
+
+ countDown.await(5 seconds)
+ agent() must be (5)
+
+ agent.close
+ }
+
+ "be able to be mapped" in {
+ val agent1 = Agent(5)
+ val agent2 = agent1 map (_ * 2)
+
+ agent1() must be (5)
+ agent2() must be (10)
+
+ agent1.close
+ agent2.close
+ }
+
+ "be able to be used in a 'foreach' for comprehension" in {
+ val agent = Agent(3)
+ var result = 0
+
+ for (value <- agent) {
+ result += value
+ }
+
+ result must be (3)
+
+ agent.close
+ }
+
+ "be able to be used in a 'map' for comprehension" in {
+ val agent1 = Agent(5)
+ val agent2 = for (value <- agent1) yield value * 2
+
+ agent1() must be (5)
+ agent2() must be (10)
+
+ agent1.close
+ agent2.close
+ }
+
+ "be able to be used in a 'flatMap' for comprehension" in {
+ val agent1 = Agent(1)
+ val agent2 = Agent(2)
+
+ val agent3 = for {
+ value1 <- agent1
+ value2 <- agent2
+ } yield value1 + value2
+
+ agent1() must be (1)
+ agent2() must be (2)
+ agent3() must be (3)
+
+ agent1.close
+ agent2.close
+ agent3.close
+ }
+ }
+}
+
diff --git a/akka-stm/src/test/scala/stm/JavaStmSpec.scala b/akka-stm/src/test/scala/stm/JavaStmSpec.scala
new file mode 100644
index 0000000000..a5847d2e87
--- /dev/null
+++ b/akka-stm/src/test/scala/stm/JavaStmSpec.scala
@@ -0,0 +1,5 @@
+package akka.stm.test
+
+import org.scalatest.junit.JUnitWrapperSuite
+
+class JavaStmSpec extends JUnitWrapperSuite("akka.stm.test.JavaStmTests", Thread.currentThread.getContextClassLoader)
diff --git a/akka-stm/src/test/scala/stm/RefSpec.scala b/akka-stm/src/test/scala/stm/RefSpec.scala
new file mode 100644
index 0000000000..8270706146
--- /dev/null
+++ b/akka-stm/src/test/scala/stm/RefSpec.scala
@@ -0,0 +1,152 @@
+package akka.stm.test
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+
+class RefSpec extends WordSpec with MustMatchers {
+
+ import akka.stm._
+
+ "A Ref" should {
+
+ "optionally accept an initial value" in {
+ val emptyRef = Ref[Int]
+ val empty = atomic { emptyRef.opt }
+
+ empty must be(None)
+
+ val ref = Ref(3)
+ val value = atomic { ref.get }
+
+ value must be (3)
+ }
+
+ "keep the initial value, even if the first transaction is rolled back" in {
+ val ref = Ref(3)
+
+ try {
+ atomic(DefaultTransactionFactory) {
+ ref.swap(5)
+ throw new Exception
+ }
+ } catch {
+ case e => {}
+ }
+
+ val value = atomic { ref.get }
+
+ value must be (3)
+ }
+
+ "be settable using set" in {
+ val ref = Ref[Int]
+
+ atomic { ref.set(3) }
+
+ val value = atomic { ref.get }
+
+ value must be (3)
+ }
+
+ "be settable using swap" in {
+ val ref = Ref[Int]
+
+ atomic { ref.swap(3) }
+
+ val value = atomic { ref.get }
+
+ value must be (3)
+ }
+
+ "be changeable using alter" in {
+ val ref = Ref(0)
+
+ def increment = atomic {
+ ref alter (_ + 1)
+ }
+
+ increment
+ increment
+ increment
+
+ val value = atomic { ref.get }
+
+ value must be (3)
+ }
+
+ "be able to be mapped" in {
+ val ref1 = Ref(1)
+
+ val ref2 = atomic {
+ ref1 map (_ + 1)
+ }
+
+ val value1 = atomic { ref1.get }
+ val value2 = atomic { ref2.get }
+
+ value1 must be (1)
+ value2 must be (2)
+ }
+
+ "be able to be used in a 'foreach' for comprehension" in {
+ val ref = Ref(3)
+
+ var result = 0
+
+ atomic {
+ for (value <- ref) {
+ result += value
+ }
+ }
+
+ result must be (3)
+ }
+
+ "be able to be used in a 'map' for comprehension" in {
+ val ref1 = Ref(1)
+
+ val ref2 = atomic {
+ for (value <- ref1) yield value + 2
+ }
+
+ val value2 = atomic { ref2.get }
+
+ value2 must be (3)
+ }
+
+ "be able to be used in a 'flatMap' for comprehension" in {
+ val ref1 = Ref(1)
+ val ref2 = Ref(2)
+
+ val ref3 = atomic {
+ for {
+ value1 <- ref1
+ value2 <- ref2
+ } yield value1 + value2
+ }
+
+ val value3 = atomic { ref3.get }
+
+ value3 must be (3)
+ }
+
+ "be able to be used in a 'filter' for comprehension" in {
+ val ref1 = Ref(1)
+
+ val refLess2 = atomic {
+ for (value <- ref1 if value < 2) yield value
+ }
+
+ val optLess2 = atomic { refLess2.opt }
+
+ val refGreater2 = atomic {
+ for (value <- ref1 if value > 2) yield value
+ }
+
+ val optGreater2 = atomic { refGreater2.opt }
+
+ optLess2 must be (Some(1))
+ optGreater2 must be (None)
+ }
+ }
+}
diff --git a/akka-stm/src/test/scala/stm/StmSpec.scala b/akka-stm/src/test/scala/stm/StmSpec.scala
new file mode 100644
index 0000000000..f20545e03b
--- /dev/null
+++ b/akka-stm/src/test/scala/stm/StmSpec.scala
@@ -0,0 +1,129 @@
+package akka.stm.test
+
+import akka.actor.Actor
+import Actor._
+
+import org.multiverse.api.exceptions.ReadonlyException
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+
+class StmSpec extends WordSpec with MustMatchers {
+
+ import akka.stm._
+
+ "Local STM" should {
+
+ "be able to do multiple consecutive atomic {..} statements" in {
+ val ref = Ref(0)
+
+ def increment = atomic {
+ ref alter (_ + 1)
+ }
+
+ def total: Int = atomic {
+ ref.getOrElse(0)
+ }
+
+ increment
+ increment
+ increment
+
+ total must be (3)
+ }
+
+ "be able to do nested atomic {..} statements" in {
+ val ref = Ref(0)
+
+ def increment = atomic {
+ ref alter (_ + 1)
+ }
+
+ def total: Int = atomic {
+ ref.getOrElse(0)
+ }
+
+ atomic {
+ increment
+ increment
+ }
+
+ atomic {
+ increment
+ total must be (3)
+ }
+ }
+
+ "roll back failing nested atomic {..} statements" in {
+ val ref = Ref(0)
+
+ def increment = atomic {
+ ref alter (_ + 1)
+ }
+
+ def total: Int = atomic {
+ ref.getOrElse(0)
+ }
+
+ try {
+ atomic(DefaultTransactionFactory) {
+ increment
+ increment
+ throw new Exception
+ }
+ } catch {
+ case e => {}
+ }
+
+ total must be (0)
+ }
+
+ "use the outer transaction settings by default" in {
+ val readonlyFactory = TransactionFactory(readonly = true)
+ val writableFactory = TransactionFactory(readonly = false)
+
+ val ref = Ref(0)
+
+ def writableOuter =
+ atomic(writableFactory) {
+ atomic(readonlyFactory) {
+ ref alter (_ + 1)
+ }
+ }
+
+ def readonlyOuter =
+ atomic(readonlyFactory) {
+ atomic(writableFactory) {
+ ref alter (_ + 1)
+ }
+ }
+
+ writableOuter must be (1)
+ evaluating { readonlyOuter } must produce [ReadonlyException]
+ }
+
+ "allow propagation settings for nested transactions" in {
+ val readonlyFactory = TransactionFactory(readonly = true)
+ val writableRequiresNewFactory = TransactionFactory(readonly = false, propagation = Propagation.RequiresNew)
+
+ val ref = Ref(0)
+
+ def writableOuter =
+ atomic(writableRequiresNewFactory) {
+ atomic(readonlyFactory) {
+ ref alter (_ + 1)
+ }
+ }
+
+ def readonlyOuter =
+ atomic(readonlyFactory) {
+ atomic(writableRequiresNewFactory) {
+ ref alter (_ + 1)
+ }
+ }
+
+ writableOuter must be (1)
+ readonlyOuter must be (2)
+ }
+ }
+}
diff --git a/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala
new file mode 100644
index 0000000000..e9a2ab20fa
--- /dev/null
+++ b/akka-stm/src/test/scala/transactor/CoordinatedIncrementSpec.scala
@@ -0,0 +1,86 @@
+package akka.transactor.test
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+
+import akka.transactor.Coordinated
+import akka.actor.{Actor, ActorRef}
+import akka.stm.{Ref, TransactionFactory}
+import akka.util.duration._
+
+object CoordinatedIncrement {
+ case class Increment(friends: Seq[ActorRef])
+ case object GetCount
+
+ class Counter(name: String) extends Actor {
+ val count = Ref(0)
+
+ implicit val txFactory = TransactionFactory(timeout = 3 seconds)
+
+ def increment = {
+ log.info(name + ": incrementing")
+ count alter (_ + 1)
+ }
+
+ def receive = {
+ case coordinated @ Coordinated(Increment(friends)) => {
+ if (friends.nonEmpty) {
+ friends.head ! coordinated(Increment(friends.tail))
+ }
+ coordinated atomic {
+ increment
+ }
+ }
+
+ case GetCount => self.reply(count.get)
+ }
+ }
+
+ class Failer extends Actor {
+ def receive = {
+ case Coordinated(Increment(friends)) => {
+ throw new RuntimeException("Expected failure")
+ }
+ }
+ }
+}
+
+class CoordinatedIncrementSpec extends WordSpec with MustMatchers {
+ import CoordinatedIncrement._
+
+ val numCounters = 5
+ val timeout = 5 seconds
+
+ def createActors = {
+ def createCounter(i: Int) = Actor.actorOf(new Counter("counter" + i)).start
+ val counters = (1 to numCounters) map createCounter
+ val failer = Actor.actorOf(new Failer).start
+ (counters, failer)
+ }
+
+ "Coordinated increment" should {
+ "increment all counters by one with successful transactions" in {
+ val (counters, failer) = createActors
+ val coordinated = Coordinated()
+ counters(0) ! coordinated(Increment(counters.tail))
+ coordinated.await
+ for (counter <- counters) {
+ (counter !! GetCount).get must be === 1
+ }
+ counters foreach (_.stop)
+ failer.stop
+ }
+
+ "increment no counters with a failing transaction" in {
+ val (counters, failer) = createActors
+ val coordinated = Coordinated()
+ counters(0) ! Coordinated(Increment(counters.tail :+ failer))
+ coordinated.await
+ for (counter <- counters) {
+ (counter !! GetCount).get must be === 0
+ }
+ counters foreach (_.stop)
+ failer.stop
+ }
+ }
+}
diff --git a/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala
new file mode 100644
index 0000000000..80a6e0f081
--- /dev/null
+++ b/akka-stm/src/test/scala/transactor/FickleFriendsSpec.scala
@@ -0,0 +1,122 @@
+package akka.transactor.test
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+
+import akka.transactor.Coordinated
+import akka.actor.{Actor, ActorRef}
+import akka.stm._
+import akka.util.duration._
+
+import scala.util.Random.{nextInt => random}
+
+import java.util.concurrent.CountDownLatch
+
+object FickleFriends {
+ case class FriendlyIncrement(friends: Seq[ActorRef], latch: CountDownLatch)
+ case class Increment(friends: Seq[ActorRef])
+ case object GetCount
+
+ /**
+ * Coordinator will keep trying to coordinate an increment until successful.
+ */
+ class Coordinator(name: String) extends Actor {
+ val count = Ref(0)
+
+ implicit val txFactory = TransactionFactory(timeout = 3 seconds)
+
+ def increment = {
+ log.info(name + ": incrementing")
+ count alter (_ + 1)
+ }
+
+ def receive = {
+ case FriendlyIncrement(friends, latch) => {
+ var success = false
+ while (!success) {
+ try {
+ val coordinated = Coordinated()
+ if (friends.nonEmpty) {
+ friends.head ! coordinated(Increment(friends.tail))
+ }
+ coordinated atomic {
+ increment
+ deferred {
+ success = true
+ latch.countDown
+ }
+ }
+ } catch {
+ case _ => () // swallow exceptions
+ }
+ }
+ }
+
+ case GetCount => self.reply(count.get)
+ }
+ }
+
+ /**
+ * FickleCounter randomly fails at different points with 50% chance of failing overall.
+ */
+ class FickleCounter(name: String) extends Actor {
+ val count = Ref(0)
+
+ implicit val txFactory = TransactionFactory(timeout = 3 seconds)
+
+ def increment = {
+ log.info(name + ": incrementing")
+ count alter (_ + 1)
+ }
+
+ def failIf(x: Int, y: Int) = {
+ if (x == y) throw new RuntimeException("Random fail at position " + x)
+ }
+
+ def receive = {
+ case coordinated @ Coordinated(Increment(friends)) => {
+ val failAt = random(8)
+ failIf(failAt, 0)
+ if (friends.nonEmpty) {
+ friends.head ! coordinated(Increment(friends.tail))
+ }
+ failIf(failAt, 1)
+ coordinated atomic {
+ failIf(failAt, 2)
+ increment
+ failIf(failAt, 3)
+ }
+ }
+
+ case GetCount => self.reply(count.get)
+ }
+ }
+}
+
+class FickleFriendsSpec extends WordSpec with MustMatchers {
+ import FickleFriends._
+
+ val numCounters = 2
+
+ def createActors = {
+ def createCounter(i: Int) = Actor.actorOf(new FickleCounter("counter" + i)).start
+ val counters = (1 to numCounters) map createCounter
+ val coordinator = Actor.actorOf(new Coordinator("coordinator")).start
+ (counters, coordinator)
+ }
+
+ "Coordinated fickle friends" should {
+ "eventually succeed to increment all counters by one" in {
+ val (counters, coordinator) = createActors
+ val latch = new CountDownLatch(1)
+ coordinator ! FriendlyIncrement(counters, latch)
+ latch.await // this could take a while
+ (coordinator !! GetCount).get must be === 1
+ for (counter <- counters) {
+ (counter !! GetCount).get must be === 1
+ }
+ counters foreach (_.stop)
+ coordinator.stop
+ }
+ }
+}
diff --git a/akka-stm/src/test/scala/transactor/JavaUntypedCoordinatedSpec.scala b/akka-stm/src/test/scala/transactor/JavaUntypedCoordinatedSpec.scala
new file mode 100644
index 0000000000..a7de3fc63b
--- /dev/null
+++ b/akka-stm/src/test/scala/transactor/JavaUntypedCoordinatedSpec.scala
@@ -0,0 +1,8 @@
+package akka.transactor.test
+
+import org.scalatest.junit.JUnitWrapperSuite
+
+class JavaUntypedCoordinatedSpec extends JUnitWrapperSuite(
+ "akka.transactor.test.UntypedCoordinatedIncrementTest",
+ Thread.currentThread.getContextClassLoader
+)
diff --git a/akka-stm/src/test/scala/transactor/JavaUntypedTransactorSpec.scala b/akka-stm/src/test/scala/transactor/JavaUntypedTransactorSpec.scala
new file mode 100644
index 0000000000..6efc2f0f1b
--- /dev/null
+++ b/akka-stm/src/test/scala/transactor/JavaUntypedTransactorSpec.scala
@@ -0,0 +1,8 @@
+package akka.transactor.test
+
+import org.scalatest.junit.JUnitWrapperSuite
+
+class JavaUntypedTransactorSpec extends JUnitWrapperSuite(
+ "akka.transactor.test.UntypedTransactorTest",
+ Thread.currentThread.getContextClassLoader
+)
diff --git a/akka-stm/src/test/scala/transactor/TransactorSpec.scala b/akka-stm/src/test/scala/transactor/TransactorSpec.scala
new file mode 100644
index 0000000000..aeb559c410
--- /dev/null
+++ b/akka-stm/src/test/scala/transactor/TransactorSpec.scala
@@ -0,0 +1,100 @@
+package akka.transactor.test
+
+import org.scalatest.WordSpec
+import org.scalatest.matchers.MustMatchers
+
+import akka.transactor.Transactor
+import akka.actor.{Actor, ActorRef}
+import akka.stm._
+import akka.util.duration._
+
+import java.util.concurrent.CountDownLatch
+
+object TransactorIncrement {
+ case class Increment(friends: Seq[ActorRef], latch: CountDownLatch)
+ case object GetCount
+
+ class Counter(name: String) extends Transactor {
+ val count = Ref(0)
+
+ override def transactionFactory = TransactionFactory(timeout = 3 seconds)
+
+ def increment = {
+ log.info(name + ": incrementing")
+ count alter (_ + 1)
+ }
+
+ override def coordinate = {
+ case Increment(friends, latch) => {
+ if (friends.nonEmpty) sendTo(friends.head -> Increment(friends.tail, latch))
+ else nobody
+ }
+ }
+
+ override def before = {
+ case i: Increment => log.info(name + ": before transaction")
+ }
+
+ def atomically = {
+ case Increment(friends, latch) => {
+ increment
+ deferred { latch.countDown }
+ compensating { latch.countDown }
+ }
+ }
+
+ override def after = {
+ case i: Increment => log.info(name + ": after transaction")
+ }
+
+ override def normally = {
+ case GetCount => self.reply(count.get)
+ }
+ }
+
+ class Failer extends Transactor {
+ def atomically = {
+ case _ => throw new RuntimeException("Expected failure")
+ }
+ }
+}
+
+class TransactorSpec extends WordSpec with MustMatchers {
+ import TransactorIncrement._
+
+ val numCounters = 5
+ val timeout = 5 seconds
+
+ def createTransactors = {
+ def createCounter(i: Int) = Actor.actorOf(new Counter("counter" + i)).start
+ val counters = (1 to numCounters) map createCounter
+ val failer = Actor.actorOf(new Failer).start
+ (counters, failer)
+ }
+
+ "Transactor increment" should {
+ "increment all counters by one with successful transactions" in {
+ val (counters, failer) = createTransactors
+ val incrementLatch = new CountDownLatch(numCounters)
+ counters(0) ! Increment(counters.tail, incrementLatch)
+ incrementLatch.await(timeout.length, timeout.unit)
+ for (counter <- counters) {
+ (counter !! GetCount).get must be === 1
+ }
+ counters foreach (_.stop)
+ failer.stop
+ }
+
+ "increment no counters with a failing transaction" in {
+ val (counters, failer) = createTransactors
+ val failLatch = new CountDownLatch(numCounters + 1)
+ counters(0) ! Increment(counters.tail :+ failer, failLatch)
+ failLatch.await(timeout.length, timeout.unit)
+ for (counter <- counters) {
+ (counter !! GetCount).get must be === 0
+ }
+ counters foreach (_.stop)
+ failer.stop
+ }
+ }
+}
diff --git a/akka-typed-actor/src/main/java/akka/transactor/annotation/Coordinated.java b/akka-typed-actor/src/main/java/akka/transactor/annotation/Coordinated.java
new file mode 100644
index 0000000000..326f45ae71
--- /dev/null
+++ b/akka-typed-actor/src/main/java/akka/transactor/annotation/Coordinated.java
@@ -0,0 +1,7 @@
+package akka.transactor.annotation;
+
+import java.lang.annotation.*;
+
+@Retention(RetentionPolicy.RUNTIME)
+@Target(ElementType.METHOD)
+public @interface Coordinated { }
diff --git a/akka-typed-actor/src/main/scala/akka/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/akka/actor/TypedActor.scala
new file mode 100644
index 0000000000..1a39eab01d
--- /dev/null
+++ b/akka-typed-actor/src/main/scala/akka/actor/TypedActor.scala
@@ -0,0 +1,952 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package akka.actor
+
+import Actor._
+import akka.dispatch.{MessageDispatcher, Future, CompletableFuture, Dispatchers}
+import akka.config.Supervision._
+import akka.util._
+import ReflectiveAccess._
+import akka.transactor.{Coordinated, Coordination}
+import akka.transactor.annotation.{Coordinated => CoordinatedAnnotation}
+
+import org.codehaus.aspectwerkz.joinpoint.{MethodRtti, JoinPoint}
+import org.codehaus.aspectwerkz.proxy.Proxy
+import org.codehaus.aspectwerkz.annotation.{Aspect, Around}
+
+import java.net.InetSocketAddress
+import scala.reflect.BeanProperty
+import java.lang.reflect.{Method, Field, InvocationHandler, Proxy => JProxy}
+
+/**
+ * TypedActor is a type-safe actor made out of a POJO with interface.
+ * Void methods are turned into fire-forget messages.
+ * Non-void methods are turned into request-reply messages with the exception of methods returning
+ * a 'Future' which will be sent using request-reply-with-future semantics and need to return the
+ * result using the 'future(..)' method: 'return future(... future result ...);'.
+ * Methods returning akka.japi.Option will block until a timeout expires,
+ * if the implementation of the method returns "none", some(null) will be returned, "none" will only be
+ * returned when the method didn't respond within the timeout.
+ *
+ * Here is an example of usage (in Java):
+ *
+ * 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 preStart() {
+ * ... // optional initialization on start
+ * }
+ *
+ * @Override
+ * public void postStop() {
+ * ... // 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 preStart = {
+ * ... // optional initialization on start
+ * }
+ *
+ * override def postStop = {
+ * ... // 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 with Proxyable {
+ 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):
+ *
+ */
+ @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 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 coordinated @ Coordinated(joinPoint: JoinPoint) =>
+ SenderContextInfo.senderActorRef.value = self
+ SenderContextInfo.senderProxy.value = proxy
+ if (Actor.SERIALIZE_MESSAGES) serializeArguments(joinPoint)
+ coordinated atomic { 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 swapProxiedActor(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) {
+
+ //FIXME serializeArguments
+ // val copyOfArgs = Serializer.Java.deepClone(args)
+ // joinPoint.getRtti.asInstanceOf[MethodRtti].setParameterValues(copyOfArgs.asInstanceOf[Array[AnyRef]])
+ joinPoint
+ }
+ }
+}
+
+/**
+ * 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):
+ *