Merge branch 'master' of git@github.com:jboner/akka

This commit is contained in:
Jonas Bonér 2010-10-09 11:23:00 +02:00
commit 75ff6f3d37
68 changed files with 1914 additions and 742 deletions

View file

@ -6,8 +6,8 @@ package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.dispatch._
import se.scalablesolutions.akka.config.Config._
import se.scalablesolutions.akka.config.{ AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy }
import se.scalablesolutions.akka.config.ScalaConfig._
import se.scalablesolutions.akka.config.{NoFaultHandlingStrategy, AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
import se.scalablesolutions.akka.stm.global._
import se.scalablesolutions.akka.stm.TransactionManagement._
import se.scalablesolutions.akka.stm.{ TransactionManagement, TransactionSetAbortedException }
@ -127,39 +127,19 @@ trait ActorRef extends ActorRefShared with TransactionManagement with Logging wi
/**
* Akka Java API
* Set 'trapExit' to the list of exception classes that the actor should be able to trap
* from the actor it is supervising. When the supervising actor throws these exceptions
* then they will trigger a restart.
* <p/>
*
* Trap all exceptions:
* <pre>
* getContext().setTrapExit(new Class[]{Throwable.class});
* </pre>
*
* Trap specific exceptions only:
* <pre>
* getContext().setTrapExit(new Class[]{MyApplicationException.class, MyApplicationError.class});
* </pre>
*/
def setTrapExit(exceptions: Array[Class[_ <: Throwable]]) = trapExit = exceptions.toList
def getTrapExit(): Array[Class[_ <: Throwable]] = trapExit.toArray
/**
* Akka Java API
* If 'trapExit' is set for the actor to act as supervisor, then a 'faultHandler' must be defined.
* A faultHandler defines what should be done when a linked actor signals an error.
* <p/>
* Can be one of:
* <pre>
* getContext().setFaultHandler(new AllForOneStrategy(maxNrOfRetries, withinTimeRange));
* getContext().setFaultHandler(new AllForOneStrategy(new Class[]{Throwable.class},maxNrOfRetries, withinTimeRange));
* </pre>
* Or:
* <pre>
* getContext().setFaultHandler(new OneForOneStrategy(maxNrOfRetries, withinTimeRange));
* getContext().setFaultHandler(new OneForOneStrategy(new Class[]{Throwable.class},maxNrOfRetries, withinTimeRange));
* </pre>
*/
def setFaultHandler(handler: FaultHandlingStrategy) = this.faultHandler = Some(handler)
def getFaultHandler(): Option[FaultHandlingStrategy] = faultHandler
def setFaultHandler(handler: FaultHandlingStrategy)
def getFaultHandler(): FaultHandlingStrategy
@volatile
private[akka] var _dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher
@ -521,7 +501,7 @@ trait ActorRef extends ActorRefShared with TransactionManagement with Logging wi
* 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.
* <p/>
* If the 'trapExit' member field has been set to at contain at least one exception class then it will
* 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'.
*/
@ -621,8 +601,6 @@ trait ActorRef extends ActorRefShared with TransactionManagement with Logging wi
protected[akka] def linkedActors: JMap[Uuid, ActorRef]
protected[akka] def linkedActorsAsList: List[ActorRef]
override def hashCode: Int = HashCode.hash(HashCode.SEED, uuid)
override def equals(that: Any): Boolean = {
@ -661,12 +639,10 @@ class LocalActorRef private[akka] (
@volatile
private[akka] var _remoteAddress: Option[InetSocketAddress] = None // only mutable to maintain identity across nodes
@volatile
private[akka] var _linkedActors: Option[ConcurrentHashMap[Uuid, ActorRef]] = None
private[akka] lazy val _linkedActors = new ConcurrentHashMap[Uuid, ActorRef]
@volatile
private[akka] var _supervisor: Option[ActorRef] = None
@volatile
private var isInInitialization = false
@volatile
private var maxNrOfRetriesCount: Int = 0
@volatile
private var restartsWithinTimeRangeTimestamp: Long = 0L
@ -814,7 +790,9 @@ class LocalActorRef private[akka] (
}
_status = ActorRefStatus.RUNNING
if (!isInInitialization) initializeActorInstance
//If we are not currently creating this ActorRef instance
if ((actorInstance ne null) && (actorInstance.get ne null))
initializeActorInstance
checkReceiveTimeout //Schedule the initial Receive timeout
}
@ -826,6 +804,7 @@ class LocalActorRef private[akka] (
*/
def stop() = guard.withGuard {
if (isRunning) {
receiveTimeout = None
cancelReceiveTimeout
dispatcher.unregister(this)
_transactionFactory = None
@ -845,7 +824,7 @@ class LocalActorRef private[akka] (
* 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.
* <p/>
* If the 'trapExit' member field has been set to at contain at least one exception class then it will
* 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'.
* <p/>
@ -964,9 +943,12 @@ class LocalActorRef private[akka] (
/**
* Shuts down and removes all linked actors.
*/
def shutdownLinkedActors(): Unit = {
linkedActorsAsList.foreach(_.stop)
linkedActors.clear
def shutdownLinkedActors() {
val i = linkedActors.values.iterator
while(i.hasNext) {
i.next.stop
i.remove
}
}
/**
@ -1034,36 +1016,39 @@ class LocalActorRef private[akka] (
}
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = {
if (trapExit.exists(_.isAssignableFrom(reason.getClass))) {
if (faultHandler.trapExit.exists(_.isAssignableFrom(reason.getClass))) {
faultHandler match {
case Some(AllForOneStrategy(maxNrOfRetries, withinTimeRange)) =>
case AllForOneStrategy(_,maxNrOfRetries, withinTimeRange) =>
restartLinkedActors(reason, maxNrOfRetries, withinTimeRange)
case Some(OneForOneStrategy(maxNrOfRetries, withinTimeRange)) =>
case OneForOneStrategy(_,maxNrOfRetries, withinTimeRange) =>
dead.restart(reason, maxNrOfRetries, withinTimeRange)
case None => throw new IllegalActorStateException(
"No 'faultHandler' defined for an actor with the 'trapExit' member field defined " +
"\n\tto non-empty list of exception classes - can't proceed " + toString)
case NoFaultHandlingStrategy =>
notifySupervisorWithMessage(Exit(this, reason)) //This shouldn't happen
}
} else {
notifySupervisorWithMessage(Exit(this, reason)) // if 'trapExit' is not defined then pass the Exit on
notifySupervisorWithMessage(Exit(this, reason)) // if 'trapExit' isn't triggered then pass the Exit on
}
}
protected[akka] def canRestart(maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Boolean = {
if (maxNrOfRetries.isEmpty && withinTimeRange.isEmpty) { //Immortal
true
}
else if (withinTimeRange.isEmpty) { // restrict number of restarts
maxNrOfRetriesCount < maxNrOfRetries.get
} else { // cannot restart more than N within M timerange
val maxRetries = if (maxNrOfRetries.isEmpty) 1 else maxNrOfRetries.get //Default to 1, has to match timerange also
!((maxNrOfRetriesCount >= maxRetries) &&
(System.currentTimeMillis - restartsWithinTimeRangeTimestamp < withinTimeRange.get))
}
}
protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = {
if (maxNrOfRetriesCount == 0) restartsWithinTimeRangeTimestamp = System.currentTimeMillis // first time around
if (maxNrOfRetriesCount == 0) restartsWithinTimeRangeTimestamp = System.currentTimeMillis
val tooManyRestarts = if (maxNrOfRetries.isDefined) {
maxNrOfRetriesCount += 1
maxNrOfRetriesCount > maxNrOfRetries.get
} else false
val restartingHasExpired = if (withinTimeRange.isDefined)
(System.currentTimeMillis - restartsWithinTimeRangeTimestamp) > withinTimeRange.get
else false
if (tooManyRestarts || restartingHasExpired) {
if (!canRestart(maxNrOfRetries, withinTimeRange)) {
val notification = MaximumNumberOfRestartsWithinTimeRangeReached(this, maxNrOfRetries, withinTimeRange, reason)
Actor.log.warning(
"Maximum number of restarts [%s] within time range [%s] reached." +
@ -1097,13 +1082,21 @@ class LocalActorRef private[akka] (
else restartActor(failedActor, reason)
_status = ActorRefStatus.RUNNING
// update restart parameters
if (maxNrOfRetries.isDefined && maxNrOfRetriesCount % maxNrOfRetries.get == 0 && maxNrOfRetriesCount != 0)
restartsWithinTimeRangeTimestamp = System.currentTimeMillis
else if (!maxNrOfRetries.isDefined)
restartsWithinTimeRangeTimestamp = System.currentTimeMillis
maxNrOfRetriesCount += 1
}
}
}
}
protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) = {
linkedActorsAsList.foreach { actorRef =>
import scala.collection.JavaConversions._
linkedActors.values foreach { actorRef =>
actorRef.lifeCycle match {
// either permanent or none where default is permanent
case Temporary => shutDownTemporaryActor(actorRef)
@ -1120,16 +1113,7 @@ class LocalActorRef private[akka] (
} else None
}
protected[akka] def linkedActors: JMap[Uuid, ActorRef] = guard.withGuard {
if (_linkedActors.isEmpty) {
val actors = new ConcurrentHashMap[Uuid, ActorRef]
_linkedActors = Some(actors)
actors
} else _linkedActors.get
}
protected[akka] def linkedActorsAsList: List[ActorRef] =
linkedActors.values.toArray.toList.asInstanceOf[List[ActorRef]]
protected[akka] def linkedActors: JMap[Uuid, ActorRef] = _linkedActors
// ========= PRIVATE FUNCTIONS =========
@ -1156,7 +1140,6 @@ class LocalActorRef private[akka] (
private[this] def newActor: Actor = {
Actor.actorRefInCreation.withValue(Some(this)) {
isInInitialization = true
val actor = actorFactory match {
case Left(Some(clazz)) =>
import ReflectiveAccess.{ createInstance, noParams, noArgs }
@ -1174,7 +1157,6 @@ class LocalActorRef private[akka] (
}
if (actor eq null) throw new ActorInitializationException(
"Actor instance passed to ActorRef can not be 'null'")
isInInitialization = false
actor
}
}
@ -1360,7 +1342,7 @@ object RemoteActorSystemMessage {
*/
private[akka] case class RemoteActorRef private[akka] (
classOrServiceName: String,
val className: String,
val actorClassName: String,
val hostname: String,
val port: Int,
_timeout: Long,
@ -1374,7 +1356,6 @@ private[akka] case class RemoteActorRef private[akka] (
timeout = _timeout
start
lazy val remoteClient = RemoteClientModule.clientFor(hostname, port, loader)
def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit =
RemoteClientModule.send[Any](
@ -1391,21 +1372,18 @@ private[akka] case class RemoteActorRef private[akka] (
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
}
def start: ActorRef = {
def start: ActorRef = synchronized {
_status = ActorRefStatus.RUNNING
this
}
def stop: Unit = {
_status = ActorRefStatus.SHUTDOWN
postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
def stop: Unit = synchronized {
if (_status == ActorRefStatus.RUNNING) {
_status = ActorRefStatus.SHUTDOWN
postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
}
}
/**
* Returns the class name for the Actor instance that is managed by the ActorRef.
*/
def actorClassName: String = className
protected[akka] def registerSupervisorAsRemoteActor: Option[Uuid] = None
val remoteAddress: Option[InetSocketAddress] = Some(new InetSocketAddress(hostname, port))
@ -1436,7 +1414,6 @@ private[akka] case class RemoteActorRef private[akka] (
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 linkedActorsAsList: List[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
@ -1496,47 +1473,21 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
/**
* User overridable callback/setting.
*
* <p/>
* Set trapExit to the list of exception classes that the actor should be able to trap
* from the actor it is supervising. When the supervising actor throws these exceptions
* then they will trigger a restart.
* <p/>
*
* Trap no exceptions:
* <pre>
* trapExit = Nil
* </pre>
*
* Trap all exceptions:
* <pre>
* trapExit = List(classOf[Throwable])
* </pre>
*
* Trap specific exceptions only:
* <pre>
* trapExit = List(classOf[MyApplicationException], classOf[MyApplicationError])
* </pre>
*/
@volatile
var trapExit: List[Class[_ <: Throwable]] = Nil
/**
* User overridable callback/setting.
* <p/>
* If 'trapExit' is set for the actor to act as supervisor, then a faultHandler must be defined.
* Don't forget to supply a List of exception types to intercept (trapExit)
* <p/>
* Can be one of:
* <pre>
* faultHandler = Some(AllForOneStrategy(maxNrOfRetries, withinTimeRange))
* faultHandler = AllForOneStrategy(trapExit = List(classOf[Exception]),maxNrOfRetries, withinTimeRange)
* </pre>
* Or:
* <pre>
* faultHandler = Some(OneForOneStrategy(maxNrOfRetries, withinTimeRange))
* faultHandler = OneForOneStrategy(trapExit = List(classOf[Exception]),maxNrOfRetries, withinTimeRange)
* </pre>
*/
@volatile
var faultHandler: Option[FaultHandlingStrategy] = None
@BeanProperty
var faultHandler: FaultHandlingStrategy = NoFaultHandlingStrategy
/**
* The reference sender Actor of the last received message.

View file

@ -46,6 +46,11 @@ object ActorRegistry extends ListenerManagement {
* 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.
@ -127,6 +132,7 @@ object ActorRegistry extends ListenerManagement {
* 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)
@ -141,6 +147,7 @@ object ActorRegistry extends ListenerManagement {
* 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)
@ -178,8 +185,9 @@ object ActorRegistry extends ListenerManagement {
* 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 = actorFor(proxy)
val actorRef = TypedActorModule.typedActorObjectInstance.get.actorFor(proxy)
actorRef.isDefined && manifest.erasure.isAssignableFrom(actorRef.get.actor.getClass)
}
findTypedActor({ case a:AnyRef if predicate(a) => a })
@ -191,7 +199,7 @@ object ActorRegistry extends ListenerManagement {
def typedActorsFor[T <: AnyRef](clazz: Class[T]): Array[AnyRef] = {
TypedActorModule.ensureTypedActorEnabled
def predicate(proxy: AnyRef) : Boolean = {
val actorRef = actorFor(proxy)
val actorRef = TypedActorModule.typedActorObjectInstance.get.actorFor(proxy)
actorRef.isDefined && clazz.isAssignableFrom(actorRef.get.actor.getClass)
}
filterTypedActors(predicate)
@ -225,13 +233,6 @@ object ActorRegistry extends ListenerManagement {
TypedActorModule.typedActorObjectInstance.get.proxyFor(actorRef)
}
/**
* Get the underlying typed actor for a given proxy.
*/
private def actorFor(proxy: AnyRef): Option[ActorRef] = {
TypedActorModule.typedActorObjectInstance.get.actorFor(proxy)
}
/**
* Registers an actor in the ActorRegistry.

View file

@ -6,7 +6,7 @@ package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.stm.Ref
import se.scalablesolutions.akka.AkkaException
import se.scalablesolutions.akka.util.{ Function => JFunc, Procedure => JProc }
import se.scalablesolutions.akka.japi.{ Function => JFunc, Procedure => JProc }
import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.CountDownLatch

View file

@ -79,14 +79,14 @@ object Supervisor {
object SupervisorFactory {
def apply(config: SupervisorConfig) = new SupervisorFactory(config)
private[akka] def retrieveFaultHandlerAndTrapExitsFrom(config: SupervisorConfig):
Tuple2[FaultHandlingStrategy, List[Class[_ <: Throwable]]] = config match {
case SupervisorConfig(RestartStrategy(scheme, maxNrOfRetries, timeRange, trapExceptions), _) =>
scheme match {
case AllForOne => (AllForOneStrategy(maxNrOfRetries, timeRange), trapExceptions)
case OneForOne => (OneForOneStrategy(maxNrOfRetries, timeRange), trapExceptions)
}
}
private[akka] def retrieveFaultHandlerAndTrapExitsFrom(config: SupervisorConfig): FaultHandlingStrategy =
config match {
case SupervisorConfig(RestartStrategy(scheme, maxNrOfRetries, timeRange, trapExceptions), _) =>
scheme match {
case AllForOne => AllForOneStrategy(trapExceptions,maxNrOfRetries, timeRange)
case OneForOne => OneForOneStrategy(trapExceptions,maxNrOfRetries, timeRange)
}
}
}
/**
@ -99,9 +99,8 @@ class SupervisorFactory private[akka] (val config: SupervisorConfig) extends Log
def newInstance: Supervisor = newInstanceFor(config)
def newInstanceFor(config: SupervisorConfig): Supervisor = {
val (handler, trapExits) = SupervisorFactory.retrieveFaultHandlerAndTrapExitsFrom(config)
val supervisor = new Supervisor(handler, trapExits)
def newInstanceFor(config: SupervisorConfig): Supervisor = {
val supervisor = new Supervisor(SupervisorFactory.retrieveFaultHandlerAndTrapExitsFrom(config))
supervisor.configure(config)
supervisor.start
supervisor
@ -121,13 +120,13 @@ class SupervisorFactory private[akka] (val config: SupervisorConfig) extends Log
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
sealed class Supervisor private[akka] (
handler: FaultHandlingStrategy, trapExceptions: List[Class[_ <: Throwable]]) {
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, trapExceptions)).start
private[akka] val supervisor = actorOf(new SupervisorActor(handler)).start
def uuid = supervisor.uuid
@ -179,13 +178,9 @@ sealed class Supervisor private[akka] (
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
final class SupervisorActor private[akka] (
handler: FaultHandlingStrategy,
trapExceptions: List[Class[_ <: Throwable]]) extends Actor {
final class SupervisorActor private[akka] (handler: FaultHandlingStrategy) extends Actor {
import self._
trapExit = trapExceptions
faultHandler = Some(handler)
faultHandler = handler
override def postStop(): Unit = shutdownLinkedActors

View file

@ -7,20 +7,45 @@ package se.scalablesolutions.akka.config
import se.scalablesolutions.akka.actor.{ActorRef}
import se.scalablesolutions.akka.dispatch.MessageDispatcher
sealed abstract class FaultHandlingStrategy
object AllForOneStrategy {
def apply(maxNrOfRetries: Int, withinTimeRange: Int): AllForOneStrategy =
AllForOneStrategy(if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
if (withinTimeRange < 0) None else Some(withinTimeRange))
sealed abstract class FaultHandlingStrategy {
def trapExit: List[Class[_ <: Throwable]]
}
object AllForOneStrategy {
def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
new AllForOneStrategy(trapExit, if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
def apply(trapExit: Array[Class[Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
new AllForOneStrategy(trapExit.toList,maxNrOfRetries,withinTimeRange)
}
case class AllForOneStrategy(trapExit: List[Class[_ <: Throwable]],
maxNrOfRetries: Option[Int] = None,
withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy {
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,maxNrOfRetries,withinTimeRange)
}
case class AllForOneStrategy(maxNrOfRetries: Option[Int] = None, withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy
object OneForOneStrategy {
def apply(maxNrOfRetries: Int, withinTimeRange: Int): OneForOneStrategy =
this(if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
if (withinTimeRange < 0) None else Some(withinTimeRange))
def apply(trapExit: List[Class[_ <: Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
new OneForOneStrategy(trapExit, if (maxNrOfRetries < 0) None else Some(maxNrOfRetries), if (withinTimeRange < 0) None else Some(withinTimeRange))
def apply(trapExit: Array[Class[Throwable]], maxNrOfRetries: Int, withinTimeRange: Int) =
new OneForOneStrategy(trapExit.toList,maxNrOfRetries,withinTimeRange)
}
case class OneForOneStrategy(trapExit: List[Class[_ <: Throwable]],
maxNrOfRetries: Option[Int] = None,
withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy {
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,maxNrOfRetries,withinTimeRange)
}
case object NoFaultHandlingStrategy extends FaultHandlingStrategy {
def trapExit: List[Class[_ <: Throwable]] = Nil
}
case class OneForOneStrategy(maxNrOfRetries: Option[Int] = None, withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy
/**
* Configuration classes - not to be used as messages.

View file

@ -11,7 +11,7 @@ import se.scalablesolutions.akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.dispatch.CompletableFuture
import se.scalablesolutions.akka.AkkaException
import se.scalablesolutions.akka.util.{ Function, SideEffect }
import se.scalablesolutions.akka.japi.{ Function, SideEffect }
/**
* Implements Oz-style dataflow (single assignment) variables.

View file

@ -0,0 +1,78 @@
package se.scalablesolutions.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
}
/**
* This class represents optional values. Instances of <code>Option</code>
* are either instances of case class <code>Some</code> or it is case
* object <code>None</code>.
* <p>
* 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 {
/**
* <code>Option</code> factory that creates <code>Some</code>
*/
def some[A](v: A): Option[A] = Some(v)
/**
* <code>Option</code> factory that creates <code>None</code>
*/
def none[A] = None.asInstanceOf[Option[A]]
/**
* <code>Option</code> factory that creates <code>None</code> if
* <code>v</code> is <code>null</code>, <code>Some(v)</code> otherwise.
*/
def option[A](v: A): Option[A] = if (v == null) none else some(v)
/**
* Class <code>Some[A]</code> represents existing values of type
* <code>A</code>.
*/
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)
}

View file

@ -1,23 +0,0 @@
package se.scalablesolutions.akka.util
/** A Function interface
* Used to create first-class-functions is Java (sort of)
* Java API
*/
trait Function[T,R] {
def apply(param: T): R
}
/** A Procedure is like a Function, but it doesn't produce a return value
* Java API
*/
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
}

View file

@ -111,4 +111,62 @@ class SimpleLock {
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 = {
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 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
}

View file

@ -0,0 +1,42 @@
package se.scalablesolutions.akka.japi;
import org.junit.Test;
import static org.junit.Assert.*;
public class JavaAPITestBase {
@Test public void shouldCreateSomeString() {
Option<String> o = Option.some("abc");
assertFalse(o.isEmpty());
assertTrue(o.isDefined());
assertEquals("abc", o.get());
}
@Test public void shouldCreateNone() {
Option<String> o1 = Option.none();
assertTrue(o1.isEmpty());
assertFalse(o1.isDefined());
Option<Float> o2 = Option.none();
assertTrue(o2.isEmpty());
assertFalse(o2.isDefined());
}
@Test public void shouldEnterForLoop() {
for(String s : Option.some("abc")) {
return;
}
fail("for-loop not entered");
}
@Test public void shouldNotEnterForLoop() {
for(Object o : Option.none()) {
fail("for-loop entered");
}
}
@Test public void shouldBeSingleton() {
assertSame(Option.none(), Option.none());
}
}

View file

@ -4,6 +4,8 @@
package se.scalablesolutions.akka.actor
import java.lang.Thread.sleep
import org.scalatest.junit.JUnitSuite
import org.junit.Test
@ -19,17 +21,17 @@ class RestartStrategySpec extends JUnitSuite {
object Crash
@Test
def slaveShouldStayDeadAfterMaxRestarts = {
def slaveShouldStayDeadAfterMaxRestartsWithinTimeRange = {
val boss = actorOf(new Actor{
self.trapExit = List(classOf[Throwable])
self.faultHandler = Some(OneForOneStrategy(1, 1000))
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(2)
val countDownLatch = new CountDownLatch(3)
val stopLatch = new StandardLatch
val slave = actorOf(new Actor{
@ -39,13 +41,14 @@ class RestartStrategySpec extends JUnitSuite {
case Crash => throw new Exception("Crashing...")
}
override def postRestart(reason: Throwable) = {
restartLatch.open
if (!restartLatch.isOpen)
restartLatch.open
else
secondRestartLatch.open
}
override def postStop = {
if (restartLatch.isOpen) {
secondRestartLatch.open
}
stopLatch.open
}
})
boss.startLink(slave)
@ -56,27 +59,27 @@ class RestartStrategySpec extends JUnitSuite {
// test restart and post restart ping
assert(restartLatch.tryAwait(1, TimeUnit.SECONDS))
assert(countDownLatch.await(1, TimeUnit.SECONDS))
assert(slave.isRunning)
// now crash again... should not restart
slave ! Crash
slave ! Ping
assert(secondRestartLatch.tryAwait(1, TimeUnit.SECONDS))
val exceptionLatch = new StandardLatch
try {
slave ! Ping // this should fail
} catch {
case e => exceptionLatch.open // expected here
}
assert(exceptionLatch.tryAwait(1, TimeUnit.SECONDS))
assert(countDownLatch.await(1, TimeUnit.SECONDS))
slave ! Crash
assert(stopLatch.tryAwait(1, TimeUnit.SECONDS))
assert(!slave.isRunning)
}
@Test
def slaveShouldBeImmortalWithoutMaxRestarts = {
def slaveShouldBeImmortalWithoutMaxRestartsAndTimeRange = {
val boss = actorOf(new Actor{
self.trapExit = List(classOf[Throwable])
self.faultHandler = Some(OneForOneStrategy(None, None))
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, None)
protected def receive = { case _ => () }
}).start
@ -96,6 +99,174 @@ class RestartStrategySpec extends JUnitSuite {
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 slaveShouldNotRestartWithinTimeRange = {
val boss = actorOf(new Actor{
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), None, Some(1000))
protected def receive = { case _ => () }
}).start
val restartLatch = 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) = {
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(!slave.isRunning)
}
}

View file

@ -37,8 +37,7 @@ class SupervisorHierarchySpec extends JUnitSuite {
val workerThree = actorOf(new CountDownActor(countDown))
val boss = actorOf(new Actor{
self.trapExit = List(classOf[Throwable])
self.faultHandler = Some(OneForOneStrategy(5, 1000))
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 5, 1000)
protected def receive = { case _ => () }
}).start
@ -63,8 +62,7 @@ class SupervisorHierarchySpec extends JUnitSuite {
val countDown = new CountDownLatch(2)
val crasher = actorOf(new CountDownActor(countDown))
val boss = actorOf(new Actor{
self.trapExit = List(classOf[Throwable])
self.faultHandler = Some(OneForOneStrategy(1, 5000))
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]), 1, 5000)
protected def receive = {
case MaximumNumberOfRestartsWithinTimeRangeReached(_, _, _, _) =>
countDown.countDown

View file

@ -95,8 +95,7 @@ object SupervisorSpec {
}
class Master extends Actor {
self.trapExit = classOf[Exception] :: Nil
self.faultHandler = Some(OneForOneStrategy(5, 1000))
self.faultHandler = OneForOneStrategy(List(classOf[Exception]), 5, 1000)
val temp = self.spawnLink[TemporaryActor]
override def receive = {
case Die => temp !! (Die, 5000)

View file

@ -130,9 +130,9 @@ class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite {
slowOne ! "hogexecutor"
slowOne ! "ping"
fastOne ! "ping"
assert(ready.await(5,TimeUnit.SECONDS) === true)
Thread.sleep(deadlineMs)
assert(ready.await(2,TimeUnit.SECONDS) === true)
Thread.sleep(deadlineMs+10) // wait just a bit more than the deadline
start.countDown
assert(latch.await(10,TimeUnit.SECONDS) === true)
assert(latch.await(2,TimeUnit.SECONDS) === true)
}
}

View file

@ -0,0 +1,5 @@
package se.scalablesolutions.akka.japi
import org.scalatest.junit.JUnitSuite
class JavaAPITest extends JavaAPITestBase with JUnitSuite

View file

@ -0,0 +1,267 @@
package se.scalablesolutions.akka.amqp;
import org.multiverse.api.latches.StandardLatch;
import scala.Option;
import se.scalablesolutions.akka.actor.ActorRef;
import se.scalablesolutions.akka.actor.ActorRegistry;
import se.scalablesolutions.akka.actor.UntypedActor;
import se.scalablesolutions.akka.actor.UntypedActorFactory;
import se.scalablesolutions.akka.amqp.rpc.RPC;
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol;
import se.scalablesolutions.akka.japi.Function;
import se.scalablesolutions.akka.japi.Procedure;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
@SuppressWarnings({"unchecked"})
public class ExampleSessionJava {
public static void main(String... args) {
new ExampleSessionJava();
}
public ExampleSessionJava() {
printTopic("DIRECT");
direct();
printTopic("CALLBACK");
callback();
printTopic("EASY STRING PRODUCER AND CONSUMER");
easyStringProducerConsumer();
printTopic("EASY PROTOBUF PRODUCER AND CONSUMER");
easyProtobufProducerConsumer();
printTopic("EASY STRING RPC");
easyStringRpc();
printTopic("EASY PROTOBUF RPC");
easyProtobufRpc();
// postStop everything the amqp tree except the main AMQP supervisor
// all connections/consumers/producers will be stopped
AMQP.shutdownAll();
ActorRegistry.shutdownAll();
printTopic("Happy hAkking :-)");
System.exit(0);
}
private void printTopic(String topic) {
System.out.println("");
System.out.println("==== " + topic + " ===");
System.out.println("");
try {
TimeUnit.SECONDS.sleep(2);
} catch (InterruptedException ignore) {
}
}
private void direct() {
// defaults to amqp://guest:guest@localhost:5672/
ActorRef connection = AMQP.newConnection();
AMQP.ExchangeParameters exchangeParameters = new AMQP.ExchangeParameters("my_direct_exchange", Direct.getInstance());
ActorRef deliveryHandler = UntypedActor.actorOf(DirectDeliveryHandlerActor.class);
AMQP.ConsumerParameters consumerParameters = new AMQP.ConsumerParameters("some.routing", deliveryHandler, exchangeParameters);
ActorRef consumer = AMQP.newConsumer(connection, consumerParameters);
ActorRef producer = AMQP.newProducer(connection, new AMQP.ProducerParameters(exchangeParameters));
producer.sendOneWay(new Message("@jonas_boner: You sucked!!".getBytes(), "some.routing"));
}
private void callback() {
final CountDownLatch channelCountdown = new CountDownLatch(2);
ActorRef connectionCallback = UntypedActor.actorOf(ConnectionCallbackActor.class);
connectionCallback.start();
AMQP.ConnectionParameters connectionParameters = new AMQP.ConnectionParameters(connectionCallback);
ActorRef connection = AMQP.newConnection(connectionParameters);
ActorRef channelCallback = UntypedActor.actorOf(new UntypedActorFactory() {
public UntypedActor create() {
return new ChannelCallbackActor(channelCountdown);
}
});
channelCallback.start();
AMQP.ExchangeParameters exchangeParameters = new AMQP.ExchangeParameters("my_callback_exchange", Direct.getInstance());
AMQP.ChannelParameters channelParameters = new AMQP.ChannelParameters(channelCallback);
ActorRef dummyHandler = UntypedActor.actorOf(DummyActor.class);
AMQP.ConsumerParameters consumerParameters = new AMQP.ConsumerParameters("callback.routing", dummyHandler, exchangeParameters, channelParameters);
ActorRef consumer = AMQP.newConsumer(connection, consumerParameters);
ActorRef producer = AMQP.newProducer(connection, new AMQP.ProducerParameters(exchangeParameters, channelParameters));
// Wait until both channels (producer & consumer) are started before stopping the connection
try {
channelCountdown.await(2, TimeUnit.SECONDS);
} catch (InterruptedException ignore) {
}
connection.stop();
}
public void easyStringProducerConsumer() {
ActorRef connection = AMQP.newConnection();
String exchangeName = "easy.string";
// listen by default to:
// exchange = optional exchangeName
// routingKey = provided routingKey or <exchangeName>.request
// queueName = <routingKey>.in
Procedure<String> procedure = new Procedure<String>() {
public void apply(String message) {
System.out.println("### >> Received message: " + message);
}
};
AMQP.newStringConsumer(connection, procedure, exchangeName);
// send by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
AMQP.ProducerClient<String> producer = AMQP.newStringProducer(connection, exchangeName);
producer.send("This shit is easy!");
}
public void easyProtobufProducerConsumer() {
ActorRef connection = AMQP.newConnection();
String exchangeName = "easy.protobuf";
Procedure<RemoteProtocol.AddressProtocol> procedure = new Procedure<RemoteProtocol.AddressProtocol>() {
public void apply(RemoteProtocol.AddressProtocol message) {
System.out.println("### >> Received message: " + message);
}
};
AMQP.newProtobufConsumer(connection, procedure, exchangeName, RemoteProtocol.AddressProtocol.class);
AMQP.ProducerClient<RemoteProtocol.AddressProtocol> producerClient = AMQP.newProtobufProducer(connection, exchangeName);
producerClient.send(RemoteProtocol.AddressProtocol.newBuilder().setHostname("akkarocks.com").setPort(1234).build());
}
public void easyStringRpc() {
ActorRef connection = AMQP.newConnection();
String exchangeName = "easy.stringrpc";
// listen by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
// queueName = <routingKey>.in
RPC.newStringRpcServer(connection, exchangeName, new Function<String, String>() {
public String apply(String request) {
System.out.println("### >> Got request: " + request);
return "Response to: '" + request + "'";
}
});
// send by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
RPC.RpcClient<String, String> stringRpcClient = RPC.newStringRpcClient(connection, exchangeName);
Option<String> response = stringRpcClient.call("AMQP Rocks!");
System.out.println("### >> Got response: " + response);
final StandardLatch standardLatch = new StandardLatch();
stringRpcClient.callAsync("AMQP is dead easy", new Procedure<String>() {
public void apply(String request) {
System.out.println("### >> This is handled async: " + request);
standardLatch.open();
}
});
try {
standardLatch.tryAwait(2, TimeUnit.SECONDS);
} catch (InterruptedException ignore) {
}
}
public void easyProtobufRpc() {
ActorRef connection = AMQP.newConnection();
String exchangeName = "easy.protobuf.rpc";
RPC.newProtobufRpcServer(connection, exchangeName, new Function<RemoteProtocol.AddressProtocol, RemoteProtocol.AddressProtocol>() {
public RemoteProtocol.AddressProtocol apply(RemoteProtocol.AddressProtocol request) {
return RemoteProtocol.AddressProtocol.newBuilder().setHostname(request.getHostname()).setPort(request.getPort()).build();
}
}, RemoteProtocol.AddressProtocol.class);
RPC.RpcClient<RemoteProtocol.AddressProtocol, RemoteProtocol.AddressProtocol> protobufRpcClient =
RPC.newProtobufRpcClient(connection, exchangeName, RemoteProtocol.AddressProtocol.class);
scala.Option<RemoteProtocol.AddressProtocol> response =
protobufRpcClient.call(RemoteProtocol.AddressProtocol.newBuilder().setHostname("localhost").setPort(4321).build());
System.out.println("### >> Got response: " + response);
}
}
class DummyActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
// not used
}
}
class ChannelCallbackActor extends UntypedActor {
private final CountDownLatch channelCountdown;
public ChannelCallbackActor(CountDownLatch channelCountdown) {
this.channelCountdown = channelCountdown;
}
public void onReceive(Object message) throws Exception {
if (Started.getInstance().getClass().isAssignableFrom(message.getClass())) {
System.out.println("### >> Channel callback: Started");
channelCountdown.countDown();
} else if (Restarting.getInstance().getClass().isAssignableFrom(message.getClass())) {
} else if (Stopped.getInstance().getClass().isAssignableFrom(message.getClass())) {
System.out.println("### >> Channel callback: Stopped");
} else throw new IllegalArgumentException("Unknown message: " + message);
}
}
class ConnectionCallbackActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
if (Connected.getInstance().getClass().isAssignableFrom(message.getClass())) {
System.out.println("### >> Connection callback: Connected!");
} else if (Reconnecting.getInstance().getClass().isAssignableFrom(message.getClass())) {
} else if (Disconnected.getInstance().getClass().isAssignableFrom(message.getClass())) {
System.out.println("### >> Connection callback: Disconnected!");
} else throw new IllegalArgumentException("Unknown message: " + message);
}
}
class DirectDeliveryHandlerActor extends UntypedActor {
public void onReceive(Object message) throws Exception {
if (Delivery.class.isAssignableFrom(message.getClass())) {
Delivery delivery = (Delivery) message;
System.out.println("### >> @george_bush received message from: " + new String(delivery.payload()));
} else throw new IllegalArgumentException("Unknown message: " + message);
}
}

View file

@ -8,8 +8,11 @@ import se.scalablesolutions.akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.config.OneForOneStrategy
import com.rabbitmq.client.{ReturnListener, ShutdownListener, ConnectionFactory}
import ConnectionFactory._
import com.rabbitmq.client.AMQP.BasicProperties
import java.lang.{String, IllegalArgumentException}
import reflect.Manifest
import se.scalablesolutions.akka.japi.Procedure
/**
* AMQP Actor API. Implements Connection, Producer and Consumer materialized as Actors.
@ -19,56 +22,214 @@ import java.lang.{String, IllegalArgumentException}
* @author Irmo Manie
*/
object AMQP {
case class ConnectionParameters(
host: String = ConnectionFactory.DEFAULT_HOST,
port: Int = ConnectionFactory.DEFAULT_AMQP_PORT,
username: String = ConnectionFactory.DEFAULT_USER,
password: String = ConnectionFactory.DEFAULT_PASS,
virtualHost: String = ConnectionFactory.DEFAULT_VHOST,
initReconnectDelay: Long = 5000,
connectionCallback: Option[ActorRef] = None)
/**
* Parameters used to make the connection to the amqp broker. Uses the rabbitmq defaults.
*/
case class ConnectionParameters(
host: String = DEFAULT_HOST,
port: Int = DEFAULT_AMQP_PORT,
username: String = DEFAULT_USER,
password: String = DEFAULT_PASS,
virtualHost: String = DEFAULT_VHOST,
initReconnectDelay: Long = 5000,
connectionCallback: Option[ActorRef] = None) {
// Needed for Java API usage
def this() = this (DEFAULT_HOST, DEFAULT_AMQP_PORT, DEFAULT_USER, DEFAULT_PASS, DEFAULT_VHOST, 5000, None)
// Needed for Java API usage
def this(host: String, port: Int, username: String, password: String, virtualHost: String) =
this (host, port, username, password, virtualHost, 5000, None)
// Needed for Java API usage
def this(host: String, port: Int, username: String, password: String, virtualHost: String, initReconnectDelay: Long, connectionCallback: ActorRef) =
this (host, port, username, password, virtualHost, initReconnectDelay, Some(connectionCallback))
// Needed for Java API usage
def this(connectionCallback: ActorRef) =
this (DEFAULT_HOST, DEFAULT_AMQP_PORT, DEFAULT_USER, DEFAULT_PASS, DEFAULT_VHOST, 5000, Some(connectionCallback))
}
/**
* Additional parameters for the channel
*/
case class ChannelParameters(
shutdownListener: Option[ShutdownListener] = None,
channelCallback: Option[ActorRef] = None)
channelCallback: Option[ActorRef] = None) {
// Needed for Java API usage
def this() = this (None, None)
// Needed for Java API usage
def this(channelCallback: ActorRef) = this (None, Some(channelCallback))
// Needed for Java API usage
def this(shutdownListener: ShutdownListener, channelCallback: ActorRef) =
this (Some(shutdownListener), Some(channelCallback))
}
/**
* Declaration type used for either exchange or queue declaration
*/
sealed trait Declaration
case object NoActionDeclaration extends Declaration {
def getInstance() = this // Needed for Java API usage
}
case object PassiveDeclaration extends Declaration {
def getInstance() = this // Needed for Java API usage
}
case class ActiveDeclaration(durable: Boolean = false, autoDelete: Boolean = true, exclusive: Boolean = false) extends Declaration {
// Needed for Java API usage
def this() = this (false, true, false)
// Needed for Java API usage
def this(durable: Boolean, autoDelete: Boolean) = this (durable, autoDelete, false)
}
/**
* Exchange specific parameters
*/
case class ExchangeParameters(
exchangeName: String,
exchangeType: ExchangeType,
exchangeDurable: Boolean = false,
exchangeAutoDelete: Boolean = true,
exchangePassive: Boolean = false,
configurationArguments: Map[String, AnyRef] = Map())
exchangeType: ExchangeType = Topic,
exchangeDeclaration: Declaration = ActiveDeclaration(),
configurationArguments: Map[String, AnyRef] = Map.empty) {
// Needed for Java API usage
def this(exchangeName: String) =
this (exchangeName, Topic, ActiveDeclaration(), Map.empty)
// Needed for Java API usage
def this(exchangeName: String, exchangeType: ExchangeType) =
this (exchangeName, exchangeType, ActiveDeclaration(), Map.empty)
// Needed for Java API usage
def this(exchangeName: String, exchangeType: ExchangeType, exchangeDeclaration: Declaration) =
this (exchangeName, exchangeType, exchangeDeclaration, Map.empty)
}
/**
* Producer specific parameters
*/
case class ProducerParameters(
exchangeParameters: ExchangeParameters,
exchangeParameters: Option[ExchangeParameters] = None,
producerId: Option[String] = None,
returnListener: Option[ReturnListener] = None,
channelParameters: Option[ChannelParameters] = None)
channelParameters: Option[ChannelParameters] = None) {
def this() = this (None, None, None, None)
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters) = this (Some(exchangeParameters), None, None, None)
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters, producerId: String) =
this (Some(exchangeParameters), Some(producerId), None, None)
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters, returnListener: ReturnListener) =
this (Some(exchangeParameters), None, Some(returnListener), None)
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters, channelParameters: ChannelParameters) =
this (Some(exchangeParameters), None, None, Some(channelParameters))
// Needed for Java API usage
def this(exchangeParameters: ExchangeParameters, producerId: String, returnListener: ReturnListener, channelParameters: ChannelParameters) =
this (Some(exchangeParameters), Some(producerId), Some(returnListener), Some(channelParameters))
}
/**
* Consumer specific parameters
*/
case class ConsumerParameters(
exchangeParameters: ExchangeParameters,
routingKey: String,
deliveryHandler: ActorRef,
queueName: Option[String] = None,
queueDurable: Boolean = false,
queueAutoDelete: Boolean = true,
queuePassive: Boolean = false,
queueExclusive: Boolean = false,
exchangeParameters: Option[ExchangeParameters],
queueDeclaration: Declaration = ActiveDeclaration(),
selfAcknowledging: Boolean = true,
channelParameters: Option[ChannelParameters] = None) {
if (queueDurable && queueName.isEmpty) {
throw new IllegalArgumentException("A queue name is required when requesting a durable queue.")
if (queueName.isEmpty) {
queueDeclaration match {
case ActiveDeclaration(true, _, _) =>
throw new IllegalArgumentException("A queue name is required when requesting a durable queue.")
case PassiveDeclaration =>
throw new IllegalArgumentException("A queue name is required when requesting passive declaration.")
case _ => () // ignore
}
}
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef) =
this (routingKey, deliveryHandler, None, None, ActiveDeclaration(), true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, None, None, ActiveDeclaration(), true, Some(channelParameters))
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, selfAcknowledging: Boolean) =
this (routingKey, deliveryHandler, None, None, ActiveDeclaration(), selfAcknowledging, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, selfAcknowledging: Boolean, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, None, None, ActiveDeclaration(), selfAcknowledging, Some(channelParameters))
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String) =
this (routingKey, deliveryHandler, Some(queueName), None, ActiveDeclaration(), true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, queueDeclaration: Declaration, selfAcknowledging: Boolean, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, Some(queueName), None, queueDeclaration, selfAcknowledging, Some(channelParameters))
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, exchangeParameters: ExchangeParameters) =
this (routingKey, deliveryHandler, None, Some(exchangeParameters), ActiveDeclaration(), true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, exchangeParameters: ExchangeParameters, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, None, Some(exchangeParameters), ActiveDeclaration(), true, Some(channelParameters))
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, exchangeParameters: ExchangeParameters, selfAcknowledging: Boolean) =
this (routingKey, deliveryHandler, None, Some(exchangeParameters), ActiveDeclaration(), selfAcknowledging, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, exchangeParameters: ExchangeParameters) =
this (routingKey, deliveryHandler, Some(queueName), Some(exchangeParameters), ActiveDeclaration(), true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, exchangeParameters: ExchangeParameters, queueDeclaration: Declaration) =
this (routingKey, deliveryHandler, Some(queueName), Some(exchangeParameters), queueDeclaration, true, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, exchangeParameters: ExchangeParameters, queueDeclaration: Declaration, selfAcknowledging: Boolean) =
this (routingKey, deliveryHandler, Some(queueName), Some(exchangeParameters), queueDeclaration, selfAcknowledging, None)
// Needed for Java API usage
def this(routingKey: String, deliveryHandler: ActorRef, queueName: String, exchangeParameters: ExchangeParameters, queueDeclaration: Declaration, selfAcknowledging: Boolean, channelParameters: ChannelParameters) =
this (routingKey, deliveryHandler, Some(queueName), Some(exchangeParameters), queueDeclaration, selfAcknowledging, Some(channelParameters))
// How about that for some overloading... huh? :P (yes, I know, there are still possibilities left...sue me!)
// Who said java is easy :(
}
def newConnection(connectionParameters: ConnectionParameters = new ConnectionParameters): ActorRef = {
def newConnection(connectionParameters: ConnectionParameters = new ConnectionParameters()): ActorRef = {
val connection = actorOf(new FaultTolerantConnectionActor(connectionParameters))
supervisor.startLink(connection)
connection ! Connect
connection
}
// Needed for Java API usage
def newConnection(): ActorRef = {
newConnection(new ConnectionParameters())
}
def newProducer(connection: ActorRef, producerParameters: ProducerParameters): ActorRef = {
val producer: ActorRef = Actor.actorOf(new ProducerActor(producerParameters))
connection.startLink(producer)
@ -86,29 +247,58 @@ object AMQP {
}
/**
* Convenience
* Convenience
*/
class ProducerClient[O](client: ActorRef, routingKey: String, toBinary: ToBinary[O]) {
// Needed for Java API usage
def send(request: O): Unit = {
send(request, None)
}
// Needed for Java API usage
def send(request: O, replyTo: String): Unit = {
send(request, Some(replyTo))
}
def send(request: O, replyTo: Option[String] = None) = {
val basicProperties = new BasicProperties
basicProperties.setReplyTo(replyTo.getOrElse(null))
client ! Message(toBinary.toBinary(request), routingKey, false, false, Some(basicProperties))
}
def stop = client.stop
def stop() = client.stop
}
// Needed for Java API usage
def newStringProducer(connection: ActorRef,
exchangeName: String): ProducerClient[String] = {
newStringProducer(connection, Some(exchangeName))
}
// Needed for Java API usage
def newStringProducer(connection: ActorRef,
exchangeName: String,
routingKey: String): ProducerClient[String] = {
newStringProducer(connection, Some(exchangeName), Some(routingKey))
}
// Needed for Java API usage
def newStringProducer(connection: ActorRef,
exchangeName: String,
routingKey: String,
producerId: String): ProducerClient[String] = {
newStringProducer(connection, Some(exchangeName), Some(routingKey), Some(producerId))
}
def newStringProducer(connection: ActorRef,
exchange: String,
routingKey: Option[String] = None,
producerId: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true,
passive: Boolean = true): ProducerClient[String] = {
exchangeName: Option[String],
routingKey: Option[String] = None,
producerId: Option[String] = None): ProducerClient[String] = {
val exchangeParameters = ExchangeParameters(exchange, ExchangeType.Topic,
exchangeDurable = durable, exchangeAutoDelete = autoDelete)
val rKey = routingKey.getOrElse("%s.request".format(exchange))
if (exchangeName.isEmpty && routingKey.isEmpty) {
throw new IllegalArgumentException("Either exchange name or routing key is mandatory")
}
val exchangeParameters = exchangeName.flatMap(name => Some(ExchangeParameters(name)))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName.get))
val producerRef = newProducer(connection, ProducerParameters(exchangeParameters, producerId))
val toBinary = new ToBinary[String] {
@ -117,37 +307,83 @@ object AMQP {
new ProducerClient(producerRef, rKey, toBinary)
}
// Needed for Java API usage
def newStringConsumer(connection: ActorRef,
exchange: String,
handler: String => Unit,
routingKey: Option[String] = None,
queueName: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true): ActorRef = {
handler: Procedure[String],
exchangeName: String): ActorRef = {
newStringConsumer(connection, handler.apply _, Some(exchangeName))
}
// Needed for Java API usage
def newStringConsumer(connection: ActorRef,
handler: Procedure[String],
exchangeName: String,
routingKey: String): ActorRef = {
newStringConsumer(connection, handler.apply _, Some(exchangeName), Some(routingKey))
}
// Needed for Java API usage
def newStringConsumer(connection: ActorRef,
handler: Procedure[String],
exchangeName: String,
routingKey: String,
queueName: String): ActorRef = {
newStringConsumer(connection, handler.apply _, Some(exchangeName), Some(routingKey), Some(queueName))
}
def newStringConsumer(connection: ActorRef,
handler: String => Unit,
exchangeName: Option[String],
routingKey: Option[String] = None,
queueName: Option[String] = None): ActorRef = {
if (exchangeName.isEmpty && routingKey.isEmpty) {
throw new IllegalArgumentException("Either exchange name or routing key is mandatory")
}
val deliveryHandler = actor {
case Delivery(payload, _, _, _, _) => handler.apply(new String(payload))
}
val exchangeParameters = ExchangeParameters(exchange, ExchangeType.Topic,
exchangeDurable = durable, exchangeAutoDelete = autoDelete)
val rKey = routingKey.getOrElse("%s.request".format(exchange))
val exchangeParameters = exchangeName.flatMap(name => Some(ExchangeParameters(name)))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName.get))
val qName = queueName.getOrElse("%s.in".format(rKey))
newConsumer(connection, ConsumerParameters(exchangeParameters, rKey, deliveryHandler, Some(qName), durable, autoDelete))
newConsumer(connection, ConsumerParameters(rKey, deliveryHandler, Some(qName), exchangeParameters))
}
// Needed for Java API usage
def newProtobufProducer[O <: com.google.protobuf.Message](connection: ActorRef,
exchangeName: String): ProducerClient[O] = {
newProtobufProducer(connection, Some(exchangeName))
}
// Needed for Java API usage
def newProtobufProducer[O <: com.google.protobuf.Message](connection: ActorRef,
exchangeName: String,
routingKey: String): ProducerClient[O] = {
newProtobufProducer(connection, Some(exchangeName), Some(routingKey))
}
// Needed for Java API usage
def newProtobufProducer[O <: com.google.protobuf.Message](connection: ActorRef,
exchangeName: String,
routingKey: String,
producerId: String): ProducerClient[O] = {
newProtobufProducer(connection, Some(exchangeName), Some(routingKey), Some(producerId))
}
def newProtobufProducer[O <: com.google.protobuf.Message](connection: ActorRef,
exchange: String,
routingKey: Option[String] = None,
producerId: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true,
passive: Boolean = true): ProducerClient[O] = {
exchangeName: Option[String],
routingKey: Option[String] = None,
producerId: Option[String] = None): ProducerClient[O] = {
val exchangeParameters = ExchangeParameters(exchange, ExchangeType.Topic,
exchangeDurable = durable, exchangeAutoDelete = autoDelete)
val rKey = routingKey.getOrElse("%s.request".format(exchange))
if (exchangeName.isEmpty && routingKey.isEmpty) {
throw new IllegalArgumentException("Either exchange name or routing key is mandatory")
}
val exchangeParameters = exchangeName.flatMap(name => Some(ExchangeParameters(name)))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName.get))
val producerRef = newProducer(connection, ProducerParameters(exchangeParameters, producerId))
new ProducerClient(producerRef, rKey, new ToBinary[O] {
@ -155,13 +391,45 @@ object AMQP {
})
}
// Needed for Java API usage
def newProtobufConsumer[I <: com.google.protobuf.Message](connection: ActorRef,
exchange: String,
handler: I => Unit,
routingKey: Option[String] = None,
queueName: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true)(implicit manifest: Manifest[I]): ActorRef = {
handler: Procedure[I],
exchangeName: String,
clazz: Class[I]): ActorRef = {
implicit val manifest = Manifest.classType[I](clazz)
newProtobufConsumer[I](connection, handler.apply _, Some(exchangeName))
}
// Needed for Java API usage
def newProtobufConsumer[I <: com.google.protobuf.Message](connection: ActorRef,
handler: Procedure[I],
exchangeName: String,
routingKey: String,
clazz: Class[I]): ActorRef = {
implicit val manifest = Manifest.classType[I](clazz)
newProtobufConsumer[I](connection, handler.apply _, Some(exchangeName), Some(routingKey))
}
// Needed for Java API usage
def newProtobufConsumer[I <: com.google.protobuf.Message](connection: ActorRef,
handler: Procedure[I],
exchangeName: String,
routingKey: String,
queueName: String,
clazz: Class[I]): ActorRef = {
implicit val manifest = Manifest.classType[I](clazz)
newProtobufConsumer[I](connection, handler.apply _, Some(exchangeName), Some(routingKey), Some(queueName))
}
def newProtobufConsumer[I <: com.google.protobuf.Message](connection: ActorRef,
handler: I => Unit,
exchangeName: Option[String],
routingKey: Option[String] = None,
queueName: Option[String] = None)(implicit manifest: Manifest[I]): ActorRef = {
if (exchangeName.isEmpty && routingKey.isEmpty) {
throw new IllegalArgumentException("Either exchange name or routing key is mandatory")
}
val deliveryHandler = actor {
case Delivery(payload, _, _, _, _) => {
@ -169,23 +437,21 @@ object AMQP {
}
}
val exchangeParameters = ExchangeParameters(exchange, ExchangeType.Topic,
exchangeDurable = durable, exchangeAutoDelete = autoDelete)
val rKey = routingKey.getOrElse("%s.request".format(exchange))
val exchangeParameters = exchangeName.flatMap(name => Some(ExchangeParameters(name)))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName.get))
val qName = queueName.getOrElse("%s.in".format(rKey))
newConsumer(connection, ConsumerParameters(exchangeParameters, rKey, deliveryHandler, Some(qName), durable, autoDelete))
newConsumer(connection, ConsumerParameters(rKey, deliveryHandler, Some(qName), exchangeParameters))
}
/**
* Main supervisor
*/
class AMQPSupervisorActor extends Actor {
import self._
faultHandler = Some(OneForOneStrategy(5, 5000))
trapExit = List(classOf[Throwable])
faultHandler = OneForOneStrategy(List(classOf[Throwable]))
def receive = {
case _ => {} // ignore all messages
@ -194,7 +460,7 @@ object AMQP {
private val supervisor = actorOf(new AMQPSupervisorActor).start
def shutdownAll = {
def shutdownAll() = {
supervisor.shutdownLinkedActors
}

View file

@ -5,8 +5,6 @@
package se.scalablesolutions.akka.amqp
import se.scalablesolutions.akka.actor.ActorRef
import se.scalablesolutions.akka.AkkaException
import com.rabbitmq.client.AMQP.BasicProperties
import com.rabbitmq.client.ShutdownSignalException
@ -18,7 +16,23 @@ case class Message(
routingKey: String,
mandatory: Boolean = false,
immediate: Boolean = false,
properties: Option[BasicProperties] = None) extends AMQPMessage
properties: Option[BasicProperties] = None) extends AMQPMessage {
// Needed for Java API usage
def this(payload: Array[Byte], routingKey: String) = this(payload, routingKey, false, false, None)
// Needed for Java API usage
def this(payload: Array[Byte], routingKey: String, mandatory: Boolean, immediate: Boolean) =
this(payload, routingKey, mandatory, immediate, None)
// Needed for Java API usage
def this(payload: Array[Byte], routingKey: String, properties: BasicProperties) =
this(payload, routingKey, false, false, Some(properties))
// Needed for Java API usage
def this(payload: Array[Byte], routingKey: String, mandatory: Boolean, immediate: Boolean, properties: BasicProperties) =
this(payload, routingKey, mandatory, immediate, Some(properties))
}
case class Delivery(
payload: Array[Byte],
@ -30,18 +44,30 @@ case class Delivery(
// connection messages
case object Connect extends AMQPMessage
case object Connected extends AMQPMessage
case object Reconnecting extends AMQPMessage
case object Disconnected extends AMQPMessage
case object Connected extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object Reconnecting extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object Disconnected extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object ChannelRequest extends InternalAMQPMessage
// channel messages
case object Start extends AMQPMessage
case object Started extends AMQPMessage
case object Restarting extends AMQPMessage
case object Stopped extends AMQPMessage
case object Started extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object Restarting extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
case object Stopped extends AMQPMessage {
def getInstance() = this // Needed for Java API usage
}
// delivery messages
case class Acknowledge(deliveryTag: Long) extends AMQPMessage
@ -52,8 +78,8 @@ class RejectionException(deliveryTag: Long) extends RuntimeException
// internal messages
private[akka] case class Failure(cause: Throwable) extends InternalAMQPMessage
private[akka] case class ConnectionShutdown(cause: ShutdownSignalException) extends InternalAMQPMessage
private[akka] case class ChannelShutdown(cause: ShutdownSignalException) extends InternalAMQPMessage
case class ConnectionShutdown(cause: ShutdownSignalException) extends InternalAMQPMessage
case class ChannelShutdown(cause: ShutdownSignalException) extends InternalAMQPMessage
private[akka] class MessageNotDeliveredException(
val message: String,

View file

@ -6,20 +6,16 @@ package se.scalablesolutions.akka.amqp
import collection.JavaConversions
import se.scalablesolutions.akka.amqp.AMQP.ConsumerParameters
import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.AkkaException
import com.rabbitmq.client.AMQP.Queue.DeclareOk
import com.rabbitmq.client.AMQP.BasicProperties
import com.rabbitmq.client.{Channel, Envelope, DefaultConsumer}
import se.scalablesolutions.akka.amqp.AMQP.{NoActionDeclaration, ActiveDeclaration, PassiveDeclaration, ConsumerParameters}
private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters)
extends FaultTolerantChannelActor(
consumerParameters.exchangeParameters, consumerParameters.channelParameters) {
extends FaultTolerantChannelActor(
consumerParameters.exchangeParameters, consumerParameters.channelParameters) {
import consumerParameters._
import exchangeParameters._
var listenerTag: Option[String] = None
@ -34,15 +30,21 @@ private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters)
protected def setupChannel(ch: Channel) = {
val queueDeclare: DeclareOk = {
val queueDeclare: com.rabbitmq.client.AMQP.Queue.DeclareOk = {
queueName match {
case Some(name) =>
log.debug("Declaring new queue [%s] for %s", name, toString)
if (queuePassive) ch.queueDeclarePassive(name)
else {
ch.queueDeclare(
name, queueDurable, queueExclusive, queueAutoDelete,
JavaConversions.asMap(configurationArguments))
queueDeclaration match {
case PassiveDeclaration =>
log.debug("Passively declaring new queue [%s] for %s", name, toString)
ch.queueDeclarePassive(name)
case ActiveDeclaration(durable, autoDelete, exclusive) =>
log.debug("Actively declaring new queue [%s] for %s", name, toString)
val configurationArguments = exchangeParameters match {
case Some(params) => params.configurationArguments
case _ => Map.empty
}
ch.queueDeclare(name, durable, exclusive, autoDelete, JavaConversions.asMap(configurationArguments.toMap))
case NoActionDeclaration => new com.rabbitmq.client.impl.AMQImpl.Queue.DeclareOk(name, 0, 0) // do nothing here
}
case None =>
log.debug("Declaring new generated queue for %s", toString)
@ -50,8 +52,9 @@ private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters)
}
}
val exchangeName = exchangeParameters.flatMap(params => Some(params.exchangeName))
log.debug("Binding new queue [%s] for %s", queueDeclare.getQueue, toString)
ch.queueBind(queueDeclare.getQueue, exchangeName, routingKey)
ch.queueBind(queueDeclare.getQueue, exchangeName.getOrElse(""), routingKey)
val tag = ch.basicConsume(queueDeclare.getQueue, false, new DefaultConsumer(ch) with Logging {
override def handleDelivery(tag: String, envelope: Envelope, properties: BasicProperties, payload: Array[Byte]) {
@ -77,11 +80,12 @@ private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters)
private def acknowledgeDeliveryTag(deliveryTag: Long, remoteAcknowledgement: Boolean) = {
log.debug("Acking message with delivery tag [%s]", deliveryTag)
channel.foreach{ch =>
ch.basicAck(deliveryTag, false)
if (remoteAcknowledgement) {
deliveryHandler ! Acknowledged(deliveryTag)
}
channel.foreach {
ch =>
ch.basicAck(deliveryTag, false)
if (remoteAcknowledgement) {
deliveryHandler ! Acknowledged(deliveryTag)
}
}
}
@ -90,10 +94,11 @@ private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters)
// FIXME: when rabbitmq 1.9 arrives, basicReject should be available on the API and implemented instead of this
log.warning("Consumer is rejecting delivery with tag [%s] - " +
"for now this means we have to self terminate and kill the channel - see you in a second.")
channel.foreach{ch =>
if (remoteAcknowledgement) {
deliveryHandler ! Rejected(deliveryTag)
}
channel.foreach {
ch =>
if (remoteAcknowledgement) {
deliveryHandler ! Rejected(deliveryTag)
}
}
throw new RejectionException(deliveryTag)
}
@ -115,10 +120,8 @@ private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters)
}
override def toString =
"AMQP.Consumer[id= "+ self.id +
", exchange=" + exchangeName +
", exchangeType=" + exchangeType +
", durable=" + exchangeDurable +
", autoDelete=" + exchangeAutoDelete + "]"
"AMQP.Consumer[id= " + self.id +
", exchangeParameters=" + exchangeParameters +
", queueDeclaration=" + queueDeclaration + "]"
}

View file

@ -67,13 +67,13 @@ object ExampleSession {
// defaults to amqp://guest:guest@localhost:5672/
val connection = AMQP.newConnection()
val exchangeParameters = ExchangeParameters("my_direct_exchange", ExchangeType.Direct)
val exchangeParameters = ExchangeParameters("my_direct_exchange", Direct)
val consumer = AMQP.newConsumer(connection, ConsumerParameters(exchangeParameters, "some.routing", actor {
val consumer = AMQP.newConsumer(connection, ConsumerParameters("some.routing", actor {
case Delivery(payload, _, _, _, _) => log.info("@george_bush received message from: %s", new String(payload))
}))
}, None, Some(exchangeParameters)))
val producer = AMQP.newProducer(connection, ProducerParameters(exchangeParameters))
val producer = AMQP.newProducer(connection, ProducerParameters(Some(exchangeParameters)))
producer ! Message("@jonas_boner: You sucked!!".getBytes, "some.routing")
}
@ -82,17 +82,17 @@ object ExampleSession {
// defaults to amqp://guest:guest@localhost:5672/
val connection = AMQP.newConnection()
val exchangeParameters = ExchangeParameters("my_fanout_exchange", ExchangeType.Fanout)
val exchangeParameters = ExchangeParameters("my_fanout_exchange", Fanout)
val bushConsumer = AMQP.newConsumer(connection, ConsumerParameters(exchangeParameters, "@george_bush", actor {
val bushConsumer = AMQP.newConsumer(connection, ConsumerParameters("@george_bush", actor {
case Delivery(payload, _, _, _, _) => log.info("@george_bush received message from: %s", new String(payload))
}))
}, None, Some(exchangeParameters)))
val obamaConsumer = AMQP.newConsumer(connection, ConsumerParameters(exchangeParameters, "@barack_obama", actor {
val obamaConsumer = AMQP.newConsumer(connection, ConsumerParameters("@barack_obama", actor {
case Delivery(payload, _, _, _, _) => log.info("@barack_obama received message from: %s", new String(payload))
}))
}, None, Some(exchangeParameters)))
val producer = AMQP.newProducer(connection, ProducerParameters(exchangeParameters))
val producer = AMQP.newProducer(connection, ProducerParameters(Some(exchangeParameters)))
producer ! Message("@jonas_boner: I'm going surfing".getBytes, "")
}
@ -101,17 +101,17 @@ object ExampleSession {
// defaults to amqp://guest:guest@localhost:5672/
val connection = AMQP.newConnection()
val exchangeParameters = ExchangeParameters("my_topic_exchange", ExchangeType.Topic)
val exchangeParameters = ExchangeParameters("my_topic_exchange", Topic)
val bushConsumer = AMQP.newConsumer(connection, ConsumerParameters(exchangeParameters, "@george_bush", actor {
val bushConsumer = AMQP.newConsumer(connection, ConsumerParameters("@george_bush", actor {
case Delivery(payload, _, _, _, _) => log.info("@george_bush received message from: %s", new String(payload))
}))
}, None, Some(exchangeParameters)))
val obamaConsumer = AMQP.newConsumer(connection, ConsumerParameters(exchangeParameters, "@barack_obama", actor {
val obamaConsumer = AMQP.newConsumer(connection, ConsumerParameters("@barack_obama", actor {
case Delivery(payload, _, _, _, _) => log.info("@barack_obama received message from: %s", new String(payload))
}))
}, None, Some(exchangeParameters)))
val producer = AMQP.newProducer(connection, ProducerParameters(exchangeParameters))
val producer = AMQP.newProducer(connection, ProducerParameters(Some(exchangeParameters)))
producer ! Message("@jonas_boner: You still suck!!".getBytes, "@george_bush")
producer ! Message("@jonas_boner: Yes I can!".getBytes, "@barack_obama")
}
@ -135,14 +135,14 @@ object ExampleSession {
case Restarting => // not used, sent when channel or connection fails and initiates a restart
case Stopped => log.info("Channel callback: Stopped")
}
val exchangeParameters = ExchangeParameters("my_callback_exchange", ExchangeType.Direct)
val exchangeParameters = ExchangeParameters("my_callback_exchange", Direct)
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val consumer = AMQP.newConsumer(connection, ConsumerParameters(exchangeParameters, "callback.routing", actor {
val consumer = AMQP.newConsumer(connection, ConsumerParameters("callback.routing", actor {
case _ => () // not used
}, channelParameters = Some(channelParameters)))
}, None, Some(exchangeParameters), channelParameters = Some(channelParameters)))
val producer = AMQP.newProducer(connection, ProducerParameters(exchangeParameters))
val producer = AMQP.newProducer(connection, ProducerParameters(Some(exchangeParameters)))
// Wait until both channels (producer & consumer) are started before stopping the connection
channelCountdown.await(2, TimeUnit.SECONDS)
@ -155,15 +155,15 @@ object ExampleSession {
val exchangeName = "easy.string"
// listen by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
// exchange = optional exchangeName
// routingKey = provided routingKey or <exchangeName>.request
// queueName = <routingKey>.in
AMQP.newStringConsumer(connection, exchangeName, message => println("Received message: "+message))
AMQP.newStringConsumer(connection, message => println("Received message: "+message), Some(exchangeName))
// send by default to:
// exchange = exchangeName
// routingKey = <exchange>.request
val producer = AMQP.newStringProducer(connection, exchangeName)
val producer = AMQP.newStringProducer(connection, Some(exchangeName))
producer.send("This shit is easy!")
}
@ -177,9 +177,9 @@ object ExampleSession {
log.info("Received "+message)
}
AMQP.newProtobufConsumer(connection, exchangeName, protobufMessageHandler)
AMQP.newProtobufConsumer(connection, protobufMessageHandler _, Some(exchangeName))
val producerClient = AMQP.newProtobufProducer[AddressProtocol](connection, exchangeName)
val producerClient = AMQP.newProtobufProducer[AddressProtocol](connection, Some(exchangeName))
producerClient.send(AddressProtocol.newBuilder.setHostname("akkarocks.com").setPort(1234).build)
}
@ -187,7 +187,7 @@ object ExampleSession {
val connection = AMQP.newConnection()
val exchangeParameters = ExchangeParameters("my_rpc_exchange", ExchangeType.Topic)
val exchangeName = "my_rpc_exchange"
/** Server */
val serverFromBinary = new FromBinary[String] {
@ -200,8 +200,8 @@ object ExampleSession {
def requestHandler(request: String) = 3
val rpcServer = RPC.newRpcServer[String,Int](connection, exchangeParameters, "rpc.in.key", rpcServerSerializer,
requestHandler, queueName = Some("rpc.in.key.queue"))
val rpcServer = RPC.newRpcServer[String,Int](connection, exchangeName, "rpc.in.key", rpcServerSerializer,
requestHandler _, queueName = Some("rpc.in.key.queue"))
/** Client */
@ -213,7 +213,7 @@ object ExampleSession {
}
val rpcClientSerializer = new RpcClientSerializer[String, Int](clientToBinary, clientFromBinary)
val rpcClient = RPC.newRpcClient[String,Int](connection, exchangeParameters, "rpc.in.key", rpcClientSerializer)
val rpcClient = RPC.newRpcClient[String,Int](connection, exchangeName, "rpc.in.key", rpcClientSerializer)
val response = (rpcClient !! "rpc_request")
log.info("Response: " + response)

View file

@ -5,17 +5,19 @@
package se.scalablesolutions.akka.amqp
sealed trait ExchangeType
object ExchangeType {
case object Direct extends ExchangeType {
override def toString = "direct"
}
case object Topic extends ExchangeType {
override def toString = "topic"
}
case object Fanout extends ExchangeType {
override def toString = "fanout"
}
case object Match extends ExchangeType {
override def toString = "match"
}
case object Direct extends ExchangeType {
def getInstance() = this // Needed for Java API usage
override def toString = "direct"
}
case object Topic extends ExchangeType {
def getInstance() = this // Needed for Java API usage
override def toString = "topic"
}
case object Fanout extends ExchangeType {
def getInstance() = this // Needed for Java API usage
override def toString = "fanout"
}
case object Match extends ExchangeType {
def getInstance() = this // Needed for Java API usage
override def toString = "match"
}

View file

@ -10,13 +10,10 @@ import se.scalablesolutions.akka.actor.Actor
import Actor._
import com.rabbitmq.client.{ShutdownSignalException, Channel, ShutdownListener}
import scala.PartialFunction
import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ChannelParameters}
import se.scalablesolutions.akka.amqp.AMQP._
abstract private[amqp] class FaultTolerantChannelActor(
exchangeParameters: ExchangeParameters, channelParameters: Option[ChannelParameters]) extends Actor {
import exchangeParameters._
exchangeParameters: Option[ExchangeParameters], channelParameters: Option[ChannelParameters]) extends Actor {
protected[amqp] var channel: Option[Channel] = None
log.info("%s is started", toString)
@ -64,12 +61,16 @@ abstract private[amqp] class FaultTolerantChannelActor(
protected def setupChannel(ch: Channel)
private def setupChannelInternal(ch: Channel) = if (channel.isEmpty) {
if (exchangeName != "") {
if (exchangePassive) {
ch.exchangeDeclarePassive(exchangeName)
} else {
ch.exchangeDeclare(exchangeName, exchangeType.toString, exchangeDurable, exchangeAutoDelete, JavaConversions.asMap(configurationArguments))
}
exchangeParameters.foreach {
params =>
import params._
exchangeDeclaration match {
case PassiveDeclaration => ch.exchangeDeclarePassive(exchangeName)
case ActiveDeclaration(durable, autoDelete, _) =>
ch.exchangeDeclare(exchangeName, exchangeType.toString, durable, autoDelete, JavaConversions.asMap(configurationArguments))
case NoActionDeclaration => // ignore
}
}
ch.addShutdownListener(new ShutdownListener {
def shutdownCompleted(cause: ShutdownSignalException) = {

View file

@ -8,18 +8,16 @@ import java.util.{TimerTask, Timer}
import java.io.IOException
import com.rabbitmq.client._
import se.scalablesolutions.akka.amqp.AMQP.ConnectionParameters
import se.scalablesolutions.akka.actor.{Exit, Actor}
import se.scalablesolutions.akka.config.ScalaConfig.{Permanent, LifeCycle}
import se.scalablesolutions.akka.config.ScalaConfig.{Permanent}
import se.scalablesolutions.akka.config.OneForOneStrategy
import se.scalablesolutions.akka.actor.{Exit, Actor}
private[amqp] class FaultTolerantConnectionActor(connectionParameters: ConnectionParameters) extends Actor {
import connectionParameters._
self.id = "amqp-connection-%s".format(host)
self.lifeCycle = Permanent
self.trapExit = List(classOf[Throwable])
self.faultHandler = Some(OneForOneStrategy(5, 5000))
self.faultHandler = OneForOneStrategy(List(classOf[Throwable]))
val reconnectionTimer = new Timer("%s-timer".format(self.id))
@ -70,8 +68,9 @@ private[amqp] class FaultTolerantConnectionActor(connectionParameters: Connectio
}
})
log.info("Successfully (re)connected to AMQP Server %s:%s [%s]", host, port, self.id)
log.debug("Sending new channel to %d already linked actors", self.linkedActorsAsList.size)
self.linkedActorsAsList.foreach(_ ! conn.createChannel)
log.debug("Sending new channel to %d already linked actors", self.linkedActors.size)
import scala.collection.JavaConversions._
self.linkedActors.values.iterator.foreach(_ ! conn.createChannel)
notifyCallback(Connected)
}
} catch {

View file

@ -7,14 +7,14 @@ package se.scalablesolutions.akka.amqp
import com.rabbitmq.client._
import se.scalablesolutions.akka.amqp.AMQP.ProducerParameters
import se.scalablesolutions.akka.AkkaException
private[amqp] class ProducerActor(producerParameters: ProducerParameters)
extends FaultTolerantChannelActor(
producerParameters.exchangeParameters, producerParameters.channelParameters) {
import producerParameters._
import exchangeParameters._
val exchangeName = exchangeParameters.flatMap(params => Some(params.exchangeName))
producerId.foreach(id => self.id = id)
@ -22,7 +22,7 @@ private[amqp] class ProducerActor(producerParameters: ProducerParameters)
case message@Message(payload, routingKey, mandatory, immediate, properties) if channel.isDefined => {
log.debug("Sending message [%s]", message)
channel.foreach(_.basicPublish(exchangeName, routingKey, mandatory, immediate, properties.getOrElse(null), payload))
channel.foreach(_.basicPublish(exchangeName.getOrElse(""), routingKey, mandatory, immediate, properties.getOrElse(null), payload))
}
case message@Message(payload, routingKey, mandatory, immediate, properties) => {
log.warning("Unable to send message [%s]", message)
@ -55,9 +55,6 @@ private[amqp] class ProducerActor(producerParameters: ProducerParameters)
override def toString =
"AMQP.Poducer[id= "+ self.id +
", exchange=" + exchangeName +
", exchangeType=" + exchangeType +
", durable=" + exchangeDurable +
", autoDelete=" + exchangeAutoDelete + "]"
", exchangeParameters=" + exchangeParameters + "]"
}

View file

@ -5,33 +5,93 @@ import com.google.protobuf.Message
import se.scalablesolutions.akka.actor.{Actor, ActorRef}
import Actor._
import se.scalablesolutions.akka.amqp._
import reflect.Manifest
import se.scalablesolutions.akka.japi
object RPC {
// Needed for Java API usage
def newRpcClient[O, I](connection: ActorRef,
exchangeParameters: ExchangeParameters,
exchangeName: String,
routingKey: String,
serializer: RpcClientSerializer[O, I]): ActorRef = {
newRpcClient(connection, exchangeName, routingKey, serializer, None)
}
// Needed for Java API usage
def newRpcClient[O, I](connection: ActorRef,
exchangeName: String,
routingKey: String,
serializer: RpcClientSerializer[O, I],
channelParameters: ChannelParameters): ActorRef = {
newRpcClient(connection, exchangeName, routingKey, serializer, Some(channelParameters))
}
def newRpcClient[O, I](connection: ActorRef,
exchangeName: String,
routingKey: String,
serializer: RpcClientSerializer[O, I],
channelParameters: Option[ChannelParameters] = None): ActorRef = {
val rpcActor: ActorRef = actorOf(new RpcClientActor[O, I](
exchangeParameters, routingKey, serializer, channelParameters))
ExchangeParameters(exchangeName, exchangeDeclaration = PassiveDeclaration), routingKey, serializer, channelParameters))
connection.startLink(rpcActor)
rpcActor ! Start
rpcActor
}
// Needed for Java API usage
def newRpcServer[I, O](connection: ActorRef,
exchangeParameters: ExchangeParameters,
exchangeName: String,
routingKey: String,
serializer: RpcServerSerializer[I, O],
requestHandler: japi.Function[I,O]): RpcServerHandle = {
newRpcServer(connection, exchangeName, routingKey, serializer, requestHandler.apply _)
}
// Needed for Java API usage
def newRpcServer[I, O](connection: ActorRef,
exchangeName: String,
routingKey: String,
serializer: RpcServerSerializer[I, O],
requestHandler: Function[I,O],
queueName: String): RpcServerHandle = {
newRpcServer(connection, exchangeName, routingKey, serializer, requestHandler.apply _, Some(queueName))
}
// Needed for Java API usage
def newRpcServer[I, O](connection: ActorRef,
exchangeName: String,
routingKey: String,
serializer: RpcServerSerializer[I, O],
requestHandler: japi.Function[I,O],
channelParameters: ChannelParameters): RpcServerHandle = {
newRpcServer(connection, exchangeName, routingKey, serializer, requestHandler.apply _, None, Some(channelParameters))
}
// Needed for Java API usage
def newRpcServer[I, O](connection: ActorRef,
exchangeName: String,
routingKey: String,
serializer: RpcServerSerializer[I, O],
requestHandler: japi.Function[I,O],
queueName: String,
channelParameters: ChannelParameters): RpcServerHandle = {
newRpcServer(connection, exchangeName, routingKey, serializer, requestHandler.apply _, Some(queueName), Some(channelParameters))
}
def newRpcServer[I, O](connection: ActorRef,
exchangeName: String,
routingKey: String,
serializer: RpcServerSerializer[I, O],
requestHandler: I => O,
queueName: Option[String] = None,
channelParameters: Option[ChannelParameters] = None): RpcServerHandle = {
val producer = newProducer(connection, ProducerParameters(
ExchangeParameters("", ExchangeType.Direct), channelParameters = channelParameters))
val producer = newProducer(connection, ProducerParameters(channelParameters = channelParameters))
val rpcServer = actorOf(new RpcServerActor[I, O](producer, serializer, requestHandler))
val consumer = newConsumer(connection, ConsumerParameters(exchangeParameters, routingKey, rpcServer,
channelParameters = channelParameters, selfAcknowledging = false, queueName = queueName))
val consumer = newConsumer(connection, ConsumerParameters(routingKey, rpcServer,
exchangeParameters = Some(ExchangeParameters(exchangeName)), channelParameters = channelParameters,
selfAcknowledging = false, queueName = queueName))
RpcServerHandle(producer, consumer)
}
@ -51,10 +111,28 @@ object RPC {
* RPC convenience
*/
class RpcClient[O, I](client: ActorRef){
// Needed for Java API usage
def call(request: O): Option[I] = {
call(request, 5000)
}
def call(request: O, timeout: Long = 5000): Option[I] = {
(client.!!(request, timeout)).as[I]
}
// Needed for Java API usage
def callAsync(request: O, responseHandler: japi.Procedure[I]): Unit = {
callAsync(request, 5000, responseHandler)
}
// Needed for Java API usage
def callAsync(request: O, timeout: Long, responseHandler: japi.Procedure[I]): Unit = {
callAsync(request, timeout){
case Some(response) => responseHandler.apply(response)
}
}
def callAsync(request: O, timeout: Long = 5000)(responseHandler: PartialFunction[Option[I],Unit]) = {
spawn {
val result = call(request, timeout)
@ -64,14 +142,49 @@ object RPC {
def stop = client.stop
}
// Needed for Java API usage
def newProtobufRpcServer[I <: Message, O <: Message](
connection: ActorRef,
exchange: String,
exchangeName: String,
requestHandler: japi.Function[I,O],
resultClazz: Class[I]): RpcServerHandle = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcServer(connection, exchangeName, requestHandler.apply _)
}
// Needed for Java API usage
def newProtobufRpcServer[I <: Message, O <: Message](
connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[I,O],
routingKey: String,
resultClazz: Class[I]): RpcServerHandle = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcServer(connection, exchangeName, requestHandler.apply _, Some(routingKey))
}
// Needed for Java API usage
def newProtobufRpcServer[I <: Message, O <: Message](
connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[I,O],
routingKey: String,
queueName: String,
resultClazz: Class[I]): RpcServerHandle = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcServer(connection, exchangeName, requestHandler.apply _, Some(routingKey), Some(queueName))
}
def newProtobufRpcServer[I <: Message, O <: Message](
connection: ActorRef,
exchangeName: String,
requestHandler: I => O,
routingKey: Option[String] = None,
queueName: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true)(implicit manifest: Manifest[I]): RpcServerHandle = {
queueName: Option[String] = None)(implicit manifest: Manifest[I]): RpcServerHandle = {
val serializer = new RpcServerSerializer[I, O](
new FromBinary[I] {
@ -82,16 +195,34 @@ object RPC {
def toBinary(t: O) = t.toByteArray
})
startServer(connection, exchange, requestHandler, routingKey, queueName, durable, autoDelete, serializer)
startServer(connection, exchangeName, requestHandler, routingKey, queueName, serializer)
}
// Needed for Java API usage
def newProtobufRpcClient[O <: Message, I <: Message](
connection: ActorRef,
exchangeName: String,
resultClazz: Class[I]): RpcClient[O, I] = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcClient(connection, exchangeName, None)
}
// Needed for Java API usage
def newProtobufRpcClient[O <: Message, I <: Message](
connection: ActorRef,
exchangeName: String,
routingKey: String,
resultClazz: Class[I]): RpcClient[O, I] = {
implicit val manifest = Manifest.classType[I](resultClazz)
newProtobufRpcClient(connection, exchangeName, Some(routingKey))
}
def newProtobufRpcClient[O <: Message, I <: Message](
connection: ActorRef,
exchange: String,
routingKey: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true,
passive: Boolean = true)(implicit manifest: Manifest[I]): RpcClient[O, I] = {
exchangeName: String,
routingKey: Option[String] = None)(implicit manifest: Manifest[I]): RpcClient[O, I] = {
val serializer = new RpcClientSerializer[O, I](
@ -103,16 +234,38 @@ object RPC {
}
})
startClient(connection, exchange, routingKey, durable, autoDelete, passive, serializer)
startClient(connection, exchangeName, routingKey, serializer)
}
// Needed for Java API usage
def newStringRpcServer(connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[String,String]): RpcServerHandle = {
newStringRpcServer(connection, exchangeName, requestHandler.apply _)
}
// Needed for Java API usage
def newStringRpcServer(connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[String,String],
routingKey: String): RpcServerHandle = {
newStringRpcServer(connection, exchangeName, requestHandler.apply _, Some(routingKey))
}
// Needed for Java API usage
def newStringRpcServer(connection: ActorRef,
exchangeName: String,
requestHandler: japi.Function[String,String],
routingKey: String,
queueName: String): RpcServerHandle = {
newStringRpcServer(connection, exchangeName, requestHandler.apply _, Some(routingKey), Some(queueName))
}
def newStringRpcServer(connection: ActorRef,
exchange: String,
exchangeName: String,
requestHandler: String => String,
routingKey: Option[String] = None,
queueName: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true): RpcServerHandle = {
queueName: Option[String] = None): RpcServerHandle = {
val serializer = new RpcServerSerializer[String, String](
new FromBinary[String] {
@ -123,15 +276,25 @@ object RPC {
def toBinary(t: String) = t.getBytes
})
startServer(connection, exchange, requestHandler, routingKey, queueName, durable, autoDelete, serializer)
startServer(connection, exchangeName, requestHandler, routingKey, queueName, serializer)
}
// Needed for Java API usage
def newStringRpcClient(connection: ActorRef,
exchange: String): RpcClient[String, String] = {
newStringRpcClient(connection, exchange, None)
}
// Needed for Java API usage
def newStringRpcClient(connection: ActorRef,
exchange: String,
routingKey: String): RpcClient[String, String] = {
newStringRpcClient(connection, exchange, Some(routingKey))
}
def newStringRpcClient(connection: ActorRef,
exchange: String,
routingKey: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true,
passive: Boolean = true): RpcClient[String, String] = {
routingKey: Option[String] = None): RpcClient[String, String] = {
val serializer = new RpcClientSerializer[String, String](
@ -143,40 +306,31 @@ object RPC {
}
})
startClient(connection, exchange, routingKey, durable, autoDelete, passive, serializer)
startClient(connection, exchange, routingKey, serializer)
}
private def startClient[O, I](connection: ActorRef,
exchange: String,
exchangeName: String,
routingKey: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true,
passive: Boolean = true,
serializer: RpcClientSerializer[O, I]): RpcClient[O, I] = {
val exchangeParameters = ExchangeParameters(exchange, ExchangeType.Topic,
exchangeDurable = durable, exchangeAutoDelete = autoDelete, exchangePassive = passive)
val rKey = routingKey.getOrElse("%s.request".format(exchange))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName))
val client = newRpcClient(connection, exchangeParameters, rKey, serializer)
val client = newRpcClient(connection, exchangeName, rKey, serializer)
new RpcClient(client)
}
private def startServer[I, O](connection: ActorRef,
exchange: String,
exchangeName: String,
requestHandler: I => O,
routingKey: Option[String] = None,
queueName: Option[String] = None,
durable: Boolean = false,
autoDelete: Boolean = true,
serializer: RpcServerSerializer[I, O]): RpcServerHandle = {
val exchangeParameters = ExchangeParameters(exchange, ExchangeType.Topic,
exchangeDurable = durable, exchangeAutoDelete = autoDelete)
val rKey = routingKey.getOrElse("%s.request".format(exchange))
val rKey = routingKey.getOrElse("%s.request".format(exchangeName))
val qName = queueName.getOrElse("%s.in".format(rKey))
newRpcServer(connection, exchangeParameters, rKey, serializer, requestHandler, queueName = Some(qName))
newRpcServer(connection, exchangeName, rKey, serializer, requestHandler, Some(qName))
}
}

View file

@ -13,7 +13,7 @@ class RpcClientActor[I,O](
routingKey: String,
serializer: RpcClientSerializer[I,O],
channelParameters: Option[ChannelParameters] = None)
extends FaultTolerantChannelActor(exchangeParameters, channelParameters) {
extends FaultTolerantChannelActor(Some(exchangeParameters), channelParameters) {
import exchangeParameters._

View file

@ -14,10 +14,10 @@ import org.scalatest.matchers.MustMatchers
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPConnectionRecoveryTest extends JUnitSuite with MustMatchers {
class AMQPConnectionRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def connectionAndRecovery = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def connectionAndRecovery = AMQPTest.withCleanEndState {
val connectedLatch = new StandardLatch
val reconnectingLatch = new StandardLatch

View file

@ -15,15 +15,15 @@ import se.scalablesolutions.akka.amqp.AMQP._
import org.scalatest.junit.JUnitSuite
import se.scalablesolutions.akka.actor.Actor._
class AMQPConsumerChannelRecoveryTest extends JUnitSuite with MustMatchers {
class AMQPConsumerChannelRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerChannelRecovery = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def consumerChannelRecovery = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection(ConnectionParameters(initReconnectDelay = 50))
try {
val producer = AMQP.newProducer(connection, ProducerParameters(
ExchangeParameters("text_exchange", ExchangeType.Direct)))
Some(ExchangeParameters("text_exchange"))))
val consumerStartedLatch = new StandardLatch
val consumerRestartedLatch = new StandardLatch
@ -40,11 +40,11 @@ class AMQPConsumerChannelRecoveryTest extends JUnitSuite with MustMatchers {
}
val payloadLatch = new StandardLatch
val consumerExchangeParameters = ExchangeParameters("text_exchange", ExchangeType.Direct)
val consumerExchangeParameters = ExchangeParameters("text_exchange")
val consumerChannelParameters = ChannelParameters(channelCallback = Some(consumerChannelCallback))
val consumer = AMQP.newConsumer(connection, ConsumerParameters(consumerExchangeParameters, "non.interesting.routing.key", actor {
val consumer = AMQP.newConsumer(connection, ConsumerParameters("non.interesting.routing.key", actor {
case Delivery(payload, _, _, _, _) => payloadLatch.open
}, channelParameters = Some(consumerChannelParameters)))
}, exchangeParameters = Some(consumerExchangeParameters), channelParameters = Some(consumerChannelParameters)))
consumerStartedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
val listenerLatch = new StandardLatch

View file

@ -15,10 +15,10 @@ import org.scalatest.junit.JUnitSuite
import se.scalablesolutions.akka.actor.{Actor, ActorRef}
import Actor._
class AMQPConsumerConnectionRecoveryTest extends JUnitSuite with MustMatchers {
class AMQPConsumerConnectionRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerConnectionRecovery = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def consumerConnectionRecovery = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection(ConnectionParameters(initReconnectDelay = 50))
try {
@ -38,7 +38,7 @@ class AMQPConsumerConnectionRecoveryTest extends JUnitSuite with MustMatchers {
val channelParameters = ChannelParameters(channelCallback = Some(producerChannelCallback))
val producer = AMQP.newProducer(connection, ProducerParameters(
ExchangeParameters("text_exchange", ExchangeType.Direct), channelParameters = Some(channelParameters)))
Some(ExchangeParameters("text_exchange")), channelParameters = Some(channelParameters)))
producerStartedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
@ -58,11 +58,11 @@ class AMQPConsumerConnectionRecoveryTest extends JUnitSuite with MustMatchers {
val payloadLatch = new StandardLatch
val consumerExchangeParameters = ExchangeParameters("text_exchange", ExchangeType.Direct)
val consumerExchangeParameters = ExchangeParameters("text_exchange")
val consumerChannelParameters = ChannelParameters(channelCallback = Some(consumerChannelCallback))
val consumer = AMQP.newConsumer(connection, ConsumerParameters(consumerExchangeParameters, "non.interesting.routing.key", actor {
val consumer = AMQP.newConsumer(connection, ConsumerParameters("non.interesting.routing.key", actor {
case Delivery(payload, _, _, _, _) => payloadLatch.open
}, channelParameters = Some(consumerChannelParameters)))
}, exchangeParameters = Some(consumerExchangeParameters), channelParameters = Some(consumerChannelParameters)))
consumerStartedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
val listenerLatch = new StandardLatch

View file

@ -10,14 +10,14 @@ import se.scalablesolutions.akka.amqp._
import org.junit.Test
import se.scalablesolutions.akka.actor.ActorRef
import java.util.concurrent.{CountDownLatch, TimeUnit}
import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ConsumerParameters, ChannelParameters, ProducerParameters}
import org.multiverse.api.latches.StandardLatch
import org.scalatest.junit.JUnitSuite
import se.scalablesolutions.akka.amqp.AMQP._
class AMQPConsumerManualAcknowledgeTest extends JUnitSuite with MustMatchers {
class AMQPConsumerManualAcknowledgeTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessageManualAcknowledge = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def consumerMessageManualAcknowledge = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
try {
val countDown = new CountDownLatch(2)
@ -26,13 +26,13 @@ class AMQPConsumerManualAcknowledgeTest extends JUnitSuite with MustMatchers {
case Restarting => ()
case Stopped => ()
}
val exchangeParameters = ExchangeParameters("text_exchange",ExchangeType.Direct)
val exchangeParameters = ExchangeParameters("text_exchange")
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val failLatch = new StandardLatch
val acknowledgeLatch = new StandardLatch
var deliveryTagCheck: Long = -1
val consumer:ActorRef = AMQP.newConsumer(connection, ConsumerParameters(exchangeParameters, "manual.ack.this", actor {
val consumer:ActorRef = AMQP.newConsumer(connection, ConsumerParameters("manual.ack.this", actor {
case Delivery(payload, _, deliveryTag, _, sender) => {
if (!failLatch.isOpen) {
failLatch.open
@ -43,10 +43,12 @@ class AMQPConsumerManualAcknowledgeTest extends JUnitSuite with MustMatchers {
}
}
case Acknowledged(deliveryTag) => if (deliveryTagCheck == deliveryTag) acknowledgeLatch.open
}, queueName = Some("self.ack.queue"), selfAcknowledging = false, queueAutoDelete = false, channelParameters = Some(channelParameters)))
}, queueName = Some("self.ack.queue"), exchangeParameters = Some(exchangeParameters),
selfAcknowledging = false, channelParameters = Some(channelParameters),
queueDeclaration = ActiveDeclaration(autoDelete = false)))
val producer = AMQP.newProducer(connection,
ProducerParameters(exchangeParameters, channelParameters = Some(channelParameters)))
ProducerParameters(Some(exchangeParameters), channelParameters = Some(channelParameters)))
countDown.await(2, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "manual.ack.this")

View file

@ -14,10 +14,10 @@ import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ConsumerParamete
import org.multiverse.api.latches.StandardLatch
import org.scalatest.junit.JUnitSuite
class AMQPConsumerManualRejectTest extends JUnitSuite with MustMatchers {
class AMQPConsumerManualRejectTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessageManualAcknowledge = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def consumerMessageManualAcknowledge = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
try {
val countDown = new CountDownLatch(2)
@ -27,19 +27,20 @@ class AMQPConsumerManualRejectTest extends JUnitSuite with MustMatchers {
case Restarting => restartingLatch.open
case Stopped => ()
}
val exchangeParameters = ExchangeParameters("text_exchange",ExchangeType.Direct)
val exchangeParameters = ExchangeParameters("text_exchange")
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val rejectedLatch = new StandardLatch
val consumer:ActorRef = AMQP.newConsumer(connection, ConsumerParameters(exchangeParameters, "manual.reject.this", actor {
val consumer:ActorRef = AMQP.newConsumer(connection, ConsumerParameters("manual.reject.this", actor {
case Delivery(payload, _, deliveryTag, _, sender) => {
sender.foreach(_ ! Reject(deliveryTag))
}
case Rejected(deliveryTag) => rejectedLatch.open
}, queueName = Some("self.reject.queue"), selfAcknowledging = false, queueAutoDelete = false, channelParameters = Some(channelParameters)))
}, queueName = Some("self.reject.queue"), exchangeParameters = Some(exchangeParameters),
selfAcknowledging = false, channelParameters = Some(channelParameters)))
val producer = AMQP.newProducer(connection,
ProducerParameters(exchangeParameters, channelParameters = Some(channelParameters)))
ProducerParameters(Some(exchangeParameters), channelParameters = Some(channelParameters)))
countDown.await(2, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "manual.reject.this")

View file

@ -1,48 +0,0 @@
package se.scalablesolutions.akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import se.scalablesolutions.akka.amqp._
import org.multiverse.api.latches.StandardLatch
import se.scalablesolutions.akka.actor.Actor._
import org.scalatest.matchers.MustMatchers
import java.util.concurrent.{CountDownLatch, TimeUnit}
import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ConsumerParameters, ChannelParameters, ProducerParameters}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPConsumerMessageTest extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
try {
val countDown = new CountDownLatch(2)
val channelCallback = actor {
case Started => countDown.countDown
case Restarting => ()
case Stopped => ()
}
val exchangeParameters = ExchangeParameters("text_exchange",ExchangeType.Direct)
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val payloadLatch = new StandardLatch
val consumer = AMQP.newConsumer(connection, ConsumerParameters(exchangeParameters, "non.interesting.routing.key", actor {
case Delivery(payload, _, _, _, _) => payloadLatch.open
}, channelParameters = Some(channelParameters)))
val producer = AMQP.newProducer(connection,
ProducerParameters(exchangeParameters, channelParameters = Some(channelParameters)))
countDown.await(2, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "non.interesting.routing.key")
payloadLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
} finally {
connection.stop
}
}
}

View file

@ -0,0 +1,43 @@
package se.scalablesolutions.akka.amqp.test
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
import se.scalablesolutions.akka.amqp._
import org.multiverse.api.latches.StandardLatch
import se.scalablesolutions.akka.actor.Actor._
import org.scalatest.matchers.MustMatchers
import java.util.concurrent.{CountDownLatch, TimeUnit}
import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ConsumerParameters, ChannelParameters, ProducerParameters}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPConsumerMessageTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
val countDown = new CountDownLatch(2)
val channelCallback = actor {
case Started => countDown.countDown
case Restarting => ()
case Stopped => ()
}
val exchangeParameters = ExchangeParameters("text_exchange")
val channelParameters = ChannelParameters(channelCallback = Some(channelCallback))
val payloadLatch = new StandardLatch
val consumer = AMQP.newConsumer(connection, ConsumerParameters("non.interesting.routing.key", actor {
case Delivery(payload, _, _, _, _) => payloadLatch.open
}, exchangeParameters = Some(exchangeParameters), channelParameters = Some(channelParameters)))
val producer = AMQP.newProducer(connection,
ProducerParameters(Some(exchangeParameters), channelParameters = Some(channelParameters)))
countDown.await(2, TimeUnit.SECONDS) must be (true)
producer ! Message("some_payload".getBytes, "non.interesting.routing.key")
payloadLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)
}
}

View file

@ -14,10 +14,10 @@ import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ChannelParameter
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPProducerChannelRecoveryTest extends JUnitSuite with MustMatchers {
class AMQPProducerChannelRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def producerChannelRecovery = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def producerChannelRecovery = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection(ConnectionParameters(initReconnectDelay = 50))
@ -40,7 +40,7 @@ class AMQPProducerChannelRecoveryTest extends JUnitSuite with MustMatchers {
val channelParameters = ChannelParameters(channelCallback = Some(producerCallback))
val producerParameters = ProducerParameters(
ExchangeParameters("text_exchange", ExchangeType.Direct), channelParameters = Some(channelParameters))
Some(ExchangeParameters("text_exchange")), channelParameters = Some(channelParameters))
val producer = AMQP.newProducer(connection, producerParameters)
startedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)

View file

@ -14,10 +14,10 @@ import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ChannelParameter
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPProducerConnectionRecoveryTest extends JUnitSuite with MustMatchers {
class AMQPProducerConnectionRecoveryTestIntegration extends JUnitSuite with MustMatchers {
@Test
def producerConnectionRecovery = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def producerConnectionRecovery = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection(ConnectionParameters(initReconnectDelay = 50))
try {
@ -39,7 +39,7 @@ class AMQPProducerConnectionRecoveryTest extends JUnitSuite with MustMatchers {
val channelParameters = ChannelParameters(channelCallback = Some(producerCallback))
val producerParameters = ProducerParameters(
ExchangeParameters("text_exchange", ExchangeType.Direct), channelParameters = Some(channelParameters))
Some(ExchangeParameters("text_exchange")), channelParameters = Some(channelParameters))
val producer = AMQP.newProducer(connection, producerParameters)
startedLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)

View file

@ -16,10 +16,10 @@ import se.scalablesolutions.akka.amqp.AMQP.{ExchangeParameters, ProducerParamete
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPProducerMessageTest extends JUnitSuite with MustMatchers {
class AMQPProducerMessageTestIntegration extends JUnitSuite with MustMatchers {
@Test
def producerMessage = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def producerMessage = AMQPTest.withCleanEndState {
val connection: ActorRef = AMQP.newConnection()
try {
@ -30,7 +30,7 @@ class AMQPProducerMessageTest extends JUnitSuite with MustMatchers {
}
}
val producerParameters = ProducerParameters(
ExchangeParameters("text_exchange", ExchangeType.Direct), returnListener = Some(returnListener))
Some(ExchangeParameters("text_exchange")), returnListener = Some(returnListener))
val producer = AMQP.newProducer(connection, producerParameters)

View file

@ -12,10 +12,10 @@ import java.util.concurrent.TimeUnit
import se.scalablesolutions.akka.amqp.rpc.RPC
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol
class AMQPProtobufProducerConsumerTest extends JUnitSuite with MustMatchers {
class AMQPProtobufProducerConsumerTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
@ -29,9 +29,9 @@ class AMQPProtobufProducerConsumerTest extends JUnitSuite with MustMatchers {
assert(response.getHostname == request.getHostname.reverse)
responseLatch.open
}
AMQP.newProtobufConsumer(connection, "", responseHandler, Some("proto.reply.key"))
AMQP.newProtobufConsumer(connection, responseHandler _, None, Some("proto.reply.key"))
val producer = AMQP.newProtobufProducer[AddressProtocol](connection, "protoexchange")
val producer = AMQP.newProtobufProducer[AddressProtocol](connection, Some("protoexchange"))
producer.send(request, Some("proto.reply.key"))
responseLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)

View file

@ -14,10 +14,10 @@ import se.scalablesolutions.akka.amqp.AMQP._
import org.scalatest.junit.JUnitSuite
import org.junit.Test
class AMQPRpcClientServerTest extends JUnitSuite with MustMatchers {
class AMQPRpcClientServerTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
@ -28,7 +28,7 @@ class AMQPRpcClientServerTest extends JUnitSuite with MustMatchers {
case Stopped => ()
}
val exchangeParameters = ExchangeParameters("text_topic_exchange", ExchangeType.Topic)
val exchangeName = "text_topic_exchange"
val channelParameters = ChannelParameters(channelCallback
= Some(channelCallback))
@ -41,8 +41,8 @@ class AMQPRpcClientServerTest extends JUnitSuite with MustMatchers {
def requestHandler(request: String) = 3
val rpcServer = RPC.newRpcServer[String, Int](connection, exchangeParameters, "rpc.routing", rpcServerSerializer,
requestHandler, channelParameters = Some(channelParameters))
val rpcServer = RPC.newRpcServer[String, Int](connection, exchangeName, "rpc.routing", rpcServerSerializer,
requestHandler _, channelParameters = Some(channelParameters))
val rpcClientSerializer = new RpcClientSerializer[String, Int](
new ToBinary[String] {
@ -51,7 +51,7 @@ class AMQPRpcClientServerTest extends JUnitSuite with MustMatchers {
def fromBinary(bytes: Array[Byte]) = bytes.head.toInt
})
val rpcClient = RPC.newRpcClient[String, Int](connection, exchangeParameters, "rpc.routing", rpcClientSerializer,
val rpcClient = RPC.newRpcClient[String, Int](connection, exchangeName, "rpc.routing", rpcClientSerializer,
channelParameters = Some(channelParameters))
countDown.await(2, TimeUnit.SECONDS) must be(true)

View file

@ -12,10 +12,10 @@ import se.scalablesolutions.akka.amqp.rpc.RPC
import org.multiverse.api.latches.StandardLatch
import java.util.concurrent.TimeUnit
class AMQPRpcProtobufTest extends JUnitSuite with MustMatchers {
class AMQPRpcProtobufTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()

View file

@ -11,14 +11,14 @@ import se.scalablesolutions.akka.amqp.rpc.RPC
import org.multiverse.api.latches.StandardLatch
import java.util.concurrent.TimeUnit
class AMQPRpcStringTest extends JUnitSuite with MustMatchers {
class AMQPRpcStringTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
RPC.newStringRpcServer(connection, "stringservice", requestHandler)
RPC.newStringRpcServer(connection, "stringservice", requestHandler _)
val protobufClient = RPC.newStringRpcClient(connection, "stringservice")

View file

@ -11,16 +11,16 @@ import org.multiverse.api.latches.StandardLatch
import java.util.concurrent.TimeUnit
import se.scalablesolutions.akka.amqp.rpc.RPC
class AMQPStringProducerConsumerTest extends JUnitSuite with MustMatchers {
class AMQPStringProducerConsumerTestIntegration extends JUnitSuite with MustMatchers {
@Test
def consumerMessage = if (AMQPTest.enabled) AMQPTest.withCleanEndState {
def consumerMessage = AMQPTest.withCleanEndState {
val connection = AMQP.newConnection()
val responseLatch = new StandardLatch
RPC.newStringRpcServer(connection, "stringexchange", requestHandler)
RPC.newStringRpcServer(connection, "stringexchange", requestHandler _)
val request = "somemessage"
@ -29,9 +29,9 @@ class AMQPStringProducerConsumerTest extends JUnitSuite with MustMatchers {
assert(response == request.reverse)
responseLatch.open
}
AMQP.newStringConsumer(connection, "", responseHandler, Some("string.reply.key"))
AMQP.newStringConsumer(connection, responseHandler _, None, Some("string.reply.key"))
val producer = AMQP.newStringProducer(connection, "stringexchange")
val producer = AMQP.newStringProducer(connection, Some("stringexchange"))
producer.send(request, Some("string.reply.key"))
responseLatch.tryAwait(2, TimeUnit.SECONDS) must be (true)

View file

@ -5,15 +5,18 @@
package se.scalablesolutions.akka.amqp.test
import se.scalablesolutions.akka.amqp.AMQP
object AMQPTest {
def enabled = false
object AMQPTest {
def withCleanEndState(action: => Unit) {
try {
action
} finally {
AMQP.shutdownAll
try {
action
} finally {
AMQP.shutdownAll
}
} catch {
case e => println(e)
}
}
}

View file

@ -1,5 +1,5 @@
/**
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
* Copyright (C) 2009-2010 Scalable Solutions AB <http://scalablesolutions.se>
*/
package se.scalablesolutions.akka.camel
@ -10,6 +10,7 @@ import org.apache.camel.{ProducerTemplate, CamelContext}
import org.apache.camel.impl.DefaultCamelContext
import se.scalablesolutions.akka.camel.component.TypedActorComponent
import se.scalablesolutions.akka.japi.{Option => JOption}
import se.scalablesolutions.akka.util.Logging
/**
@ -22,8 +23,8 @@ trait CamelContextLifecycle extends Logging {
// TODO: enforce correct state transitions
// valid: init -> start -> stop -> init ...
private var _context: CamelContext = _
private var _template: ProducerTemplate = _
private var _context: Option[CamelContext] = None
private var _template: Option[ProducerTemplate] = None
private var _initialized = false
private var _started = false
@ -40,24 +41,64 @@ trait CamelContextLifecycle extends Logging {
private[camel] var typedActorRegistry: Map[String, AnyRef] = _
/**
* Returns the managed CamelContext.
* Returns <code>Some(CamelContext)</code> (containing the current CamelContext)
* if <code>CamelContextLifecycle</code> has been initialized, otherwise <code>None</code>.
*/
protected def context: CamelContext = _context
def context: Option[CamelContext] = _context
/**
* Returns the managed ProducerTemplate.
* Returns <code>Some(ProducerTemplate)</code> (containing the current ProducerTemplate)
* if <code>CamelContextLifecycle</code> has been initialized, otherwise <code>None</code>.
*/
protected def template: ProducerTemplate = _template
def template: Option[ProducerTemplate] = _template
/**
* Sets the managed CamelContext.
* Returns <code>Some(CamelContext)</code> (containing the current CamelContext)
* if <code>CamelContextLifecycle</code> has been initialized, otherwise <code>None</code>.
* <p>
* Java API.
*/
protected def context_= (context: CamelContext) { _context = context }
def getContext: JOption[CamelContext] = context
/**
* Sets the managed ProducerTemplate.
* Returns <code>Some(ProducerTemplate)</code> (containing the current ProducerTemplate)
* if <code>CamelContextLifecycle</code> has been initialized, otherwise <code>None</code>.
* <p>
* Java API.
*/
protected def template_= (template: ProducerTemplate) { _template = template }
def getTemplate: JOption[ProducerTemplate] = template
/**
* Returns the current <code>CamelContext</code> if this <code>CamelContextLifecycle</code>
* has been initialized, otherwise throws an <code>IllegalStateException</code>.
*/
def mandatoryContext =
if (context.isDefined) context.get
else throw new IllegalStateException("no current CamelContext")
/**
* Returns the current <code>ProducerTemplate</code> if this <code>CamelContextLifecycle</code>
* has been initialized, otherwise throws an <code>IllegalStateException</code>.
*/
def mandatoryTemplate =
if (template.isDefined) template.get
else throw new IllegalStateException("no current ProducerTemplate")
/**
* Returns the current <code>CamelContext</code> if this <code>CamelContextLifecycle</code>
* has been initialized, otherwise throws an <code>IllegalStateException</code>.
* <p>
* Java API.
*/
def getMandatoryContext = mandatoryContext
/**
* Returns the current <code>ProducerTemplate</code> if this <code>CamelContextLifecycle</code>
* has been initialized, otherwise throws an <code>IllegalStateException</code>.
* <p>
* Java API.
*/
def getMandatoryTemplate = mandatoryTemplate
def initialized = _initialized
def started = _started
@ -66,21 +107,31 @@ trait CamelContextLifecycle extends Logging {
* Starts the CamelContext and an associated ProducerTemplate.
*/
def start = {
context.start
template.start
_started = true
log.info("Camel context started")
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 = {
template.stop
context.stop
_initialized = false
_started = false
log.info("Camel context stopped")
for {
t <- template
c <- context
} {
t.stop
c.stop
_started = false
_initialized = false
log.info("Camel context stopped")
}
}
/**
@ -98,10 +149,13 @@ trait CamelContextLifecycle extends Logging {
def init(context: CamelContext) {
this.typedActorComponent = new TypedActorComponent
this.typedActorRegistry = typedActorComponent.typedActorRegistry
this.context = context
this.context.setStreamCaching(true)
this.context.addComponent(TypedActorComponent.InternalSchema, typedActorComponent)
this.template = context.createProducerTemplate
context.setStreamCaching(true)
context.addComponent(TypedActorComponent.InternalSchema, typedActorComponent)
this._context = Some(context)
this._template = Some(context.createProducerTemplate)
_initialized = true
log.info("Camel context initialized")
}
@ -111,6 +165,38 @@ trait CamelContextLifecycle extends Logging {
* Manages a global CamelContext and an associated ProducerTemplate.
*/
object CamelContextManager extends CamelContextLifecycle {
override def context: CamelContext = super.context
override def template: ProducerTemplate = super.template
// -----------------------------------------------------
// 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
* <p>
* Java API.
*/
override def getContext: JOption[CamelContext] = super.getContext
/**
* see CamelContextLifecycle.getTemplate
* <p>
* Java API.
*/
override def getTemplate: JOption[ProducerTemplate] = super.getTemplate
/**
* see CamelContextLifecycle.getMandatoryContext
* <p>
* Java API.
*/
override def getMandatoryContext = super.getMandatoryContext
/**
* see CamelContextLifecycle.getMandatoryTemplate
* <p>
* Java API.
*/
override def getMandatoryTemplate = super.getMandatoryTemplate
}

View file

@ -10,7 +10,8 @@ import org.apache.camel.CamelContext
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.actor.{AspectInitRegistry, ActorRegistry}
import se.scalablesolutions.akka.config.Config._
import se.scalablesolutions.akka.util.{Bootable, Logging}
import se.scalablesolutions.akka.japi.{Option => JOption}
import se.scalablesolutions.akka.util.{Logging, Bootable}
/**
* Publishes (untyped) consumer actors and typed consumer actors via Camel endpoints. Actors
@ -73,7 +74,7 @@ trait CamelService extends Bootable with Logging {
// Register this instance as current CamelService and return it
CamelServiceManager.register(this)
CamelServiceManager.service
CamelServiceManager.mandatoryService
}
/**
@ -136,16 +137,37 @@ object CamelServiceManager {
* @see CamelService#stop
* @see CamelService#onUnload
*/
def stopCamelService = service.stop
def stopCamelService = for (s <- service) s.stop
/**
* Returns the current CamelService.
*
* @throws IllegalStateException if there's no current CamelService.
* Returns <code>Some(CamelService)</code> if this <code>CamelService</code>
* has been started, <code>None</code> otherwise.
*/
def service =
def service = _current
/**
* Returns the current <code>CamelService</code> if <code>CamelService</code>
* has been started, otherwise throws an <code>IllegalStateException</code>.
* <p>
* Java API
*/
def getService: JOption[CamelService] = CamelServiceManager.service
/**
* Returns <code>Some(CamelService)</code> (containing the current CamelService)
* if this <code>CamelService</code>has been started, <code>None</code> otherwise.
*/
def mandatoryService =
if (_current.isDefined) _current.get
else throw new IllegalStateException("no current CamelService")
else throw new IllegalStateException("co current Camel service")
/**
* Returns <code>Some(CamelService)</code> (containing the current CamelService)
* if this <code>CamelService</code>has been started, <code>None</code> otherwise.
* <p>
* Java API
*/
def getMandatoryService = mandatoryService
private[camel] def register(service: CamelService) =
if (_current.isDefined) throw new IllegalStateException("current CamelService already registered")

View file

@ -23,7 +23,7 @@ private[camel] object ConsumerPublisher extends Logging {
* Creates a route to the registered consumer actor.
*/
def handleConsumerRegistered(event: ConsumerRegistered) {
CamelContextManager.context.addRoutes(new ConsumerActorRoute(event.uri, event.uuid, event.blocking))
CamelContextManager.mandatoryContext.addRoutes(new ConsumerActorRoute(event.uri, event.uuid, event.blocking))
log.info("published actor %s at endpoint %s" format (event.actorRef, event.uri))
}
@ -31,7 +31,7 @@ private[camel] object ConsumerPublisher extends Logging {
* Stops the route to the already un-registered consumer actor.
*/
def handleConsumerUnregistered(event: ConsumerUnregistered) {
CamelContextManager.context.stopRoute(event.uuid.toString)
CamelContextManager.mandatoryContext.stopRoute(event.uuid.toString)
log.info("unpublished actor %s from endpoint %s" format (event.actorRef, event.uri))
}
@ -43,7 +43,7 @@ private[camel] object ConsumerPublisher extends Logging {
val objectId = "%s_%s" format (event.init.actorRef.uuid, targetMethod)
CamelContextManager.typedActorRegistry.put(objectId, event.typedActor)
CamelContextManager.context.addRoutes(new ConsumerMethodRoute(event.uri, objectId, targetMethod))
CamelContextManager.mandatoryContext.addRoutes(new ConsumerMethodRoute(event.uri, objectId, targetMethod))
log.info("published method %s of %s at endpoint %s" format (targetMethod, event.typedActor, event.uri))
}
@ -55,7 +55,7 @@ private[camel] object ConsumerPublisher extends Logging {
val objectId = "%s_%s" format (event.init.actorRef.uuid, targetMethod)
CamelContextManager.typedActorRegistry.remove(objectId)
CamelContextManager.context.stopRoute(objectId)
CamelContextManager.mandatoryContext.stopRoute(objectId)
log.info("unpublished method %s of %s from endpoint %s" format (targetMethod, event.typedActor, event.uri))
}
}

View file

@ -24,7 +24,7 @@ case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
* @see CamelContextManager.
*/
def bodyAs[T](clazz: Class[T]): T =
CamelContextManager.context.getTypeConverter.mandatoryConvertTo[T](clazz, body)
CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](clazz, body)
/**
* Returns the body of the message converted to the type <code>T</code>. Conversion is done
@ -35,7 +35,7 @@ case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
* @see CamelContextManager.
*/
def bodyAs[T](implicit m: Manifest[T]): T =
CamelContextManager.context.getTypeConverter.mandatoryConvertTo[T](m.erasure.asInstanceOf[Class[T]], body)
CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](m.erasure.asInstanceOf[Class[T]], body)
/**
* Returns those headers from this message whose name is contained in <code>names</code>.
@ -53,14 +53,14 @@ case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
* <code>NoSuchElementException</code> if the header doesn't exist.
*/
def headerAs[T](name: String)(implicit m: Manifest[T]): T =
CamelContextManager.context.getTypeConverter.mandatoryConvertTo[T](m.erasure.asInstanceOf[Class[T]], header(name))
CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](m.erasure.asInstanceOf[Class[T]], header(name))
/**
* Returns the header with given <code>name</code> converted to type given by the <code>clazz</code>
* argument. Throws <code>NoSuchElementException</code> if the header doesn't exist.
*/
def headerAs[T](name: String, clazz: Class[T]): T =
CamelContextManager.context.getTypeConverter.mandatoryConvertTo[T](clazz, header(name))
CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](clazz, header(name))
/**
* Creates a Message with a new <code>body</code> using a <code>transformer</code> function.

View file

@ -27,7 +27,7 @@ trait ProducerSupport { this: Actor =>
* <code>Endpoint</code> object resolved from the current CamelContext with
* <code>endpointUri</code>.
*/
private lazy val endpoint = CamelContextManager.context.getEndpoint(endpointUri)
private lazy val endpoint = CamelContextManager.mandatoryContext.getEndpoint(endpointUri)
/**
* <code>SendProcessor</code> for producing messages to <code>endpoint</code>.

View file

@ -13,6 +13,6 @@ public class SampleUntypedForwardingProducer extends UntypedProducerActor {
public void onReceiveAfterProduce(Object message) {
Message msg = (Message)message;
String body = msg.bodyAs(String.class);
CamelContextManager.template().sendBody("direct:forward-test-1", body);
CamelContextManager.getMandatoryTemplate().sendBody("direct:forward-test-1", body);
}
}

View file

@ -6,22 +6,30 @@ import org.scalatest.junit.JUnitSuite
class CamelContextLifecycleTest extends JUnitSuite with CamelContextLifecycle {
@Test def shouldManageCustomCamelContext {
assert(context === null)
assert(template === null)
assert(context === None)
assert(template === None)
intercept[IllegalStateException] { mandatoryContext }
intercept[IllegalStateException] { mandatoryTemplate }
val ctx = new TestCamelContext
assert(ctx.isStreamCaching === false)
init(ctx)
assert(context.isStreamCaching === true)
assert(!context.asInstanceOf[TestCamelContext].isStarted)
// In Camel 2.3 CamelComtext.createProducerTemplate starts
// the template before returning it (wasn't started in 2.2)
assert(template.asInstanceOf[DefaultProducerTemplate].isStarted)
assert(mandatoryContext.isStreamCaching === true)
assert(!mandatoryContext.asInstanceOf[TestCamelContext].isStarted)
assert(mandatoryTemplate.asInstanceOf[DefaultProducerTemplate].isStarted)
start
assert(context.asInstanceOf[TestCamelContext].isStarted)
assert(template.asInstanceOf[DefaultProducerTemplate].isStarted)
assert(mandatoryContext.asInstanceOf[TestCamelContext].isStarted)
assert(mandatoryTemplate.asInstanceOf[DefaultProducerTemplate].isStarted)
stop
assert(!context.asInstanceOf[TestCamelContext].isStarted)
assert(!template.asInstanceOf[DefaultProducerTemplate].isStarted)
assert(!mandatoryContext.asInstanceOf[TestCamelContext].isStarted)
assert(!mandatoryTemplate.asInstanceOf[DefaultProducerTemplate].isStarted)
}
class TestCamelContext extends DefaultCamelContext

View file

@ -10,19 +10,22 @@ import se.scalablesolutions.akka.actor.ActorRegistry
*/
class CamelServiceManagerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
override def afterAll = ActorRegistry.shutdownAll
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.service must be theSameInstanceAs (service)
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
intercept[IllegalStateException] { CamelServiceManager.service }
CamelServiceManager.service must be (None)
}
}
}
@ -32,13 +35,13 @@ class CamelServiceManagerTest extends WordSpec with BeforeAndAfterAll with MustM
"a CamelService instance has been started externally" must {
"have registered the started CamelService instance" in {
service.start
CamelServiceManager.service must be theSameInstanceAs (service)
CamelServiceManager.mandatoryService must be theSameInstanceAs (service)
}
}
"the current CamelService instance has been stopped externally" must {
"have unregistered the current CamelService instance" in {
service.stop
intercept[IllegalStateException] { CamelServiceManager.service }
CamelServiceManager.service must be (None)
}
}
}
@ -54,10 +57,6 @@ class CamelServiceManagerTest extends WordSpec with BeforeAndAfterAll with MustM
"only allow the current CamelService instance to be stopped" in {
intercept[IllegalStateException] { CamelServiceFactory.createCamelService.stop }
}
"ensure that the current CamelService instance has been actually started" in {
CamelServiceManager.stopCamelService
intercept[IllegalStateException] { CamelServiceManager.stopCamelService }
}
}
}
}

View file

@ -14,7 +14,7 @@ import se.scalablesolutions.akka.actor._
* @author Martin Krasser
*/
class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
import CamelContextManager.template
import CamelContextManager.mandatoryTemplate
import ConsumerTest._
var service: CamelService = _
@ -45,12 +45,12 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
val consumer = actorOf(new TestConsumer("direct:publish-test-2"))
"started before starting the CamelService" must {
"support an in-out message exchange via its endpoint" in {
template.requestBody("direct:publish-test-1", "msg1") must equal ("received msg1")
mandatoryTemplate.requestBody("direct:publish-test-1", "msg1") must equal ("received msg1")
}
}
"not started" must {
"not have an associated endpoint in the CamelContext" in {
CamelContextManager.context.hasEndpoint("direct:publish-test-2") must be (null)
CamelContextManager.mandatoryContext.hasEndpoint("direct:publish-test-2") must be (null)
}
}
"started" must {
@ -58,10 +58,10 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
val latch = service.expectEndpointActivationCount(1)
consumer.start
latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
template.requestBody("direct:publish-test-2", "msg2") must equal ("received msg2")
mandatoryTemplate.requestBody("direct:publish-test-2", "msg2") must equal ("received msg2")
}
"have an associated endpoint in the CamelContext" in {
CamelContextManager.context.hasEndpoint("direct:publish-test-2") must not be (null)
CamelContextManager.mandatoryContext.hasEndpoint("direct:publish-test-2") must not be (null)
}
}
"stopped" must {
@ -70,7 +70,7 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
consumer.stop
latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
intercept[CamelExecutionException] {
template.requestBody("direct:publish-test-2", "msg2")
mandatoryTemplate.requestBody("direct:publish-test-2", "msg2")
}
}
}
@ -83,9 +83,9 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
val latch = service.expectEndpointActivationCount(3)
actor = TypedActor.newInstance(classOf[SampleTypedConsumer], classOf[SampleTypedConsumerImpl])
latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
template.requestBodyAndHeader("direct:m2", "x", "test", "y") must equal ("m2: x y")
template.requestBodyAndHeader("direct:m3", "x", "test", "y") must equal ("m3: x y")
template.requestBodyAndHeader("direct:m4", "x", "test", "y") must equal ("m4: x y")
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 {
@ -94,13 +94,13 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
TypedActor.stop(actor)
latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
intercept[CamelExecutionException] {
template.requestBodyAndHeader("direct:m2", "x", "test", "y")
mandatoryTemplate.requestBodyAndHeader("direct:m2", "x", "test", "y")
}
intercept[CamelExecutionException] {
template.requestBodyAndHeader("direct:m3", "x", "test", "y")
mandatoryTemplate.requestBodyAndHeader("direct:m3", "x", "test", "y")
}
intercept[CamelExecutionException] {
template.requestBodyAndHeader("direct:m4", "x", "test", "y")
mandatoryTemplate.requestBodyAndHeader("direct:m4", "x", "test", "y")
}
}
}
@ -113,8 +113,8 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
val latch = service.expectEndpointActivationCount(2)
actor = TypedActor.newInstance(classOf[TestTypedConsumer], classOf[TestTypedConsumerImpl])
latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
template.requestBody("direct:publish-test-3", "x") must equal ("foo: x")
template.requestBody("direct:publish-test-4", "x") must equal ("bar: x")
mandatoryTemplate.requestBody("direct:publish-test-3", "x") must equal ("foo: x")
mandatoryTemplate.requestBody("direct:publish-test-4", "x") must equal ("bar: x")
}
}
"stopped" must {
@ -123,10 +123,10 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
TypedActor.stop(actor)
latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
intercept[CamelExecutionException] {
template.requestBody("direct:publish-test-3", "x")
mandatoryTemplate.requestBody("direct:publish-test-3", "x")
}
intercept[CamelExecutionException] {
template.requestBody("direct:publish-test-4", "x")
mandatoryTemplate.requestBody("direct:publish-test-4", "x")
}
}
}
@ -139,7 +139,7 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
val latch = service.expectEndpointActivationCount(1)
consumer.start
latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
template.requestBodyAndHeader("direct:test-untyped-consumer", "x", "test", "y") must equal ("x y")
mandatoryTemplate.requestBodyAndHeader("direct:test-untyped-consumer", "x", "test", "y") must equal ("x y")
}
}
"stopped" must {
@ -148,7 +148,7 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
consumer.stop
latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
intercept[CamelExecutionException] {
template.sendBodyAndHeader("direct:test-untyped-consumer", "blah", "test", "blub")
mandatoryTemplate.sendBodyAndHeader("direct:test-untyped-consumer", "blah", "test", "blub")
}
}
}
@ -162,7 +162,7 @@ class ConsumerTest extends WordSpec with BeforeAndAfterAll with MustMatchers {
latch.await(5000, TimeUnit.MILLISECONDS) must be (true)
try {
template.requestBody("direct:publish-test-5", "msg3")
mandatoryTemplate.requestBody("direct:publish-test-5", "msg3")
fail("expected TimoutException not thrown")
} catch {
case e => {

View file

@ -14,7 +14,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
override protected def beforeAll = {
ActorRegistry.shutdownAll
CamelContextManager.init
CamelContextManager.context.addRoutes(new TestRoute)
CamelContextManager.mandatoryContext.addRoutes(new TestRoute)
CamelContextManager.start
}
@ -239,7 +239,7 @@ class ProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with Before
}
}
private def mockEndpoint = CamelContextManager.context.getEndpoint("mock:mock", classOf[MockEndpoint])
private def mockEndpoint = CamelContextManager.mandatoryContext.getEndpoint("mock:mock", classOf[MockEndpoint])
}
object ProducerFeatureTest {

View file

@ -45,12 +45,12 @@ class RemoteConsumerTest extends FeatureSpec with BeforeAndAfterAll with GivenWh
val consumer = actorOf[RemoteConsumer].start
when("remote consumer publication is triggered")
var latch = service.expectEndpointActivationCount(1)
var latch = mandatoryService.expectEndpointActivationCount(1)
consumer !! "init"
assert(latch.await(5000, TimeUnit.MILLISECONDS))
then("the published consumer is accessible via its endpoint URI")
val response = CamelContextManager.template.requestBody("direct:remote-consumer", "test")
val response = CamelContextManager.mandatoryTemplate.requestBody("direct:remote-consumer", "test")
assert(response === "remote actor: test")
}
}
@ -61,12 +61,12 @@ class RemoteConsumerTest extends FeatureSpec with BeforeAndAfterAll with GivenWh
val consumer = TypedActor.newRemoteInstance(classOf[SampleRemoteTypedConsumer], classOf[SampleRemoteTypedConsumerImpl], host, port)
when("remote typed consumer publication is triggered")
var latch = service.expectEndpointActivationCount(1)
var latch = mandatoryService.expectEndpointActivationCount(1)
consumer.foo("init")
assert(latch.await(5000, TimeUnit.MILLISECONDS))
then("the published method is accessible via its endpoint URI")
val response = CamelContextManager.template.requestBody("direct:remote-typed-consumer", "test")
val response = CamelContextManager.mandatoryTemplate.requestBody("direct:remote-typed-consumer", "test")
assert(response === "remote typed actor: test")
}
}
@ -77,12 +77,12 @@ class RemoteConsumerTest extends FeatureSpec with BeforeAndAfterAll with GivenWh
val consumer = UntypedActor.actorOf(classOf[SampleRemoteUntypedConsumer]).start
when("remote untyped consumer publication is triggered")
var latch = service.expectEndpointActivationCount(1)
var latch = mandatoryService.expectEndpointActivationCount(1)
consumer.sendRequestReply(Message("init", Map("test" -> "init")))
assert(latch.await(5000, TimeUnit.MILLISECONDS))
then("the published untyped consumer is accessible via its endpoint URI")
val response = CamelContextManager.template.requestBodyAndHeader("direct:remote-untyped-consumer", "a", "test", "b")
val response = CamelContextManager.mandatoryTemplate.requestBodyAndHeader("direct:remote-untyped-consumer", "a", "test", "b")
assert(response === "a b")
}
}

View file

@ -14,7 +14,7 @@ class UntypedProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with
override protected def beforeAll = {
ActorRegistry.shutdownAll
CamelContextManager.init
CamelContextManager.context.addRoutes(new TestRoute)
CamelContextManager.mandatoryContext.addRoutes(new TestRoute)
CamelContextManager.start
}
@ -78,7 +78,7 @@ class UntypedProducerFeatureTest extends FeatureSpec with BeforeAndAfterAll with
}
private def mockEndpoint = CamelContextManager.context.getEndpoint("mock:mock", classOf[MockEndpoint])
private def mockEndpoint = CamelContextManager.mandatoryContext.getEndpoint("mock:mock", classOf[MockEndpoint])
}
object UntypedProducerFeatureTest {

View file

@ -18,7 +18,7 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
override protected def beforeAll = {
ActorRegistry.shutdownAll
CamelContextManager.init
CamelContextManager.context.addRoutes(new TestRoute)
CamelContextManager.mandatoryContext.addRoutes(new TestRoute)
CamelContextManager.start
}
@ -30,12 +30,12 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
}
feature("Communicate with an actor via an actor:uuid endpoint") {
import CamelContextManager.template
import CamelContextManager.mandatoryTemplate
scenario("one-way communication") {
val actor = actorOf[Tester1].start
val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
template.sendBody("actor:uuid:%s" format actor.uuid, "Martin")
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")
@ -43,36 +43,36 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
scenario("two-way communication") {
val actor = actorOf[Tester2].start
assert(template.requestBody("actor:uuid:%s" format actor.uuid, "Martin") === "Hello Martin")
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] {
template.requestBody("actor:uuid:%s?blocking=true" format actor.uuid, "Martin")
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")
template.requestBody("direct:failure-test-1", "whatever")
mandatoryTemplate.requestBody("direct:failure-test-1", "whatever")
mockEndpoint.assertIsSatisfied
}
scenario("two-way communication via a custom route with exception") {
mockEndpoint.expectedBodiesReceived("whatever")
template.requestBody("direct:failure-test-2", "whatever")
mandatoryTemplate.requestBody("direct:failure-test-2", "whatever")
mockEndpoint.assertIsSatisfied
}
}
feature("Communicate with an actor via an actor:id endpoint") {
import CamelContextManager.template
import CamelContextManager.mandatoryTemplate
scenario("one-way communication") {
val actor = actorOf[Tester1].start
val latch = (actor !! SetExpectedMessageCount(1)).as[CountDownLatch].get
template.sendBody("actor:%s" format actor.id, "Martin")
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")
@ -80,17 +80,17 @@ class ActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with
scenario("two-way communication") {
val actor = actorOf[Tester2].start
assert(template.requestBody("actor:%s" format actor.id, "Martin") === "Hello Martin")
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(template.requestBody("direct:custom-id-test-1", "Martin") === "Received Martin")
assert(template.requestBody("direct:custom-id-test-2", "Martin") === "Received Martin")
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.context.getEndpoint("mock:mock", classOf[MockEndpoint])
private def mockEndpoint = CamelContextManager.mandatoryContext.getEndpoint("mock:mock", classOf[MockEndpoint])
}
object ActorComponentFeatureTest {

View file

@ -13,7 +13,7 @@ import se.scalablesolutions.akka.camel._
*/
class TypedActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll with BeforeAndAfterEach {
import TypedActorComponentFeatureTest._
import CamelContextManager.template
import CamelContextManager.mandatoryTemplate
override protected def beforeAll = {
val typedActor = TypedActor.newInstance(classOf[SampleTypedActor], classOf[SampleTypedActorImpl]) // not a consumer
@ -24,7 +24,7 @@ class TypedActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll
registry.put("ta", typedActor)
CamelContextManager.init(new DefaultCamelContext(registry))
CamelContextManager.context.addRoutes(new CustomRouteBuilder)
CamelContextManager.mandatoryContext.addRoutes(new CustomRouteBuilder)
CamelContextManager.start
// Internal registration
@ -41,19 +41,19 @@ class TypedActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll
import ExchangePattern._
scenario("two-way communication with method returning String") {
val result1 = template.requestBodyAndHeader("%s:tc?method=m2" format InternalSchema, "x", "test", "y")
val result2 = template.requestBodyAndHeader("%s:tc?method=m4" format InternalSchema, "x", "test", "y")
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 = template.requestBodyAndHeader("%s:tc?method=m5" format InternalSchema, "x", "test", "y")
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 = template.send("%s:tc?method=m2" format InternalSchema, InOnly, new Processor {
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")
@ -65,7 +65,7 @@ class TypedActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll
}
scenario("one-way communication with method returning void") {
val result = template.send("%s:tc?method=m5" format InternalSchema, InOnly, new Processor {
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")
@ -81,19 +81,19 @@ class TypedActorComponentFeatureTest extends FeatureSpec with BeforeAndAfterAll
feature("Communicate with an internally-registered typed actor using typed-actor endpoint URIs") {
scenario("communication not possible") {
intercept[ResolveEndpointFailedException] {
template.requestBodyAndHeader("typed-actor:tc?method=m2", "x", "test", "y")
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 = template.requestBody("typed-actor:ta?method=foo", "test")
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 = template.requestBody("direct:test", "test")
val result = mandatoryTemplate.requestBody("direct:test", "test")
assert(result === "foo: test")
}
}

View file

@ -30,7 +30,7 @@ import java.util.concurrent.atomic.AtomicLong
import scala.collection.mutable.{HashSet, HashMap}
import scala.reflect.BeanProperty
import se.scalablesolutions.akka.actor._
import se.scalablesolutions.akka.util.{Address, ListenerManagement, Logging, Duration}
import se.scalablesolutions.akka.util._
/**
* Life-cycle events for RemoteClient.
@ -200,56 +200,52 @@ class RemoteClient private[akka] (
private val remoteAddress = new InetSocketAddress(hostname, port)
//FIXME rewrite to a wrapper object (minimize volatile access and maximize encapsulation)
@volatile private[remote] var isRunning = false
@volatile private var bootstrap: ClientBootstrap = _
@volatile private[remote] var connection: ChannelFuture = _
@volatile private[remote] var openChannels: DefaultChannelGroup = _
@volatile private var timer: HashedWheelTimer = _
private[remote] val runSwitch = new Switch()
private[remote] def isRunning = runSwitch.isOn
private val reconnectionTimeWindow = Duration(config.getInt(
"akka.remote.client.reconnection-time-window", 600), TIME_UNIT).toMillis
@volatile private var reconnectionTimeWindowStart = 0L
def connect = synchronized {
if (!isRunning) {
openChannels = new DefaultChannelGroup(classOf[RemoteClient].getName)
timer = new HashedWheelTimer
bootstrap = new ClientBootstrap(
new NioClientSocketChannelFactory(
Executors.newCachedThreadPool,Executors.newCachedThreadPool
)
def connect = runSwitch switchOn {
openChannels = new DefaultChannelGroup(classOf[RemoteClient].getName)
timer = new HashedWheelTimer
bootstrap = new ClientBootstrap(
new NioClientSocketChannelFactory(
Executors.newCachedThreadPool,Executors.newCachedThreadPool
)
bootstrap.setPipelineFactory(new RemoteClientPipelineFactory(name, futures, supervisors, bootstrap, remoteAddress, timer, this))
bootstrap.setOption("tcpNoDelay", true)
bootstrap.setOption("keepAlive", true)
connection = bootstrap.connect(remoteAddress)
log.info("Starting remote client connection to [%s:%s]", hostname, port)
// Wait until the connection attempt succeeds or fails.
val channel = connection.awaitUninterruptibly.getChannel
openChannels.add(channel)
if (!connection.isSuccess) {
notifyListeners(RemoteClientError(connection.getCause, this))
log.error(connection.getCause, "Remote client connection to [%s:%s] has failed", hostname, port)
}
notifyListeners(RemoteClientStarted(this))
isRunning = true
)
bootstrap.setPipelineFactory(new RemoteClientPipelineFactory(name, futures, supervisors, bootstrap, remoteAddress, timer, this))
bootstrap.setOption("tcpNoDelay", true)
bootstrap.setOption("keepAlive", true)
connection = bootstrap.connect(remoteAddress)
log.info("Starting remote client connection to [%s:%s]", hostname, port)
// Wait until the connection attempt succeeds or fails.
val channel = connection.awaitUninterruptibly.getChannel
openChannels.add(channel)
if (!connection.isSuccess) {
notifyListeners(RemoteClientError(connection.getCause, this))
log.error(connection.getCause, "Remote client connection to [%s:%s] has failed", hostname, port)
}
notifyListeners(RemoteClientStarted(this))
}
def shutdown = synchronized {
def shutdown = runSwitch switchOff {
log.info("Shutting down %s", name)
if (isRunning) {
isRunning = false
notifyListeners(RemoteClientShutdown(this))
timer.stop
timer = null
openChannels.close.awaitUninterruptibly
openChannels = null
bootstrap.releaseExternalResources
bootstrap = null
connection = null
log.info("%s has been shut down", name)
}
notifyListeners(RemoteClientShutdown(this))
timer.stop
timer = null
openChannels.close.awaitUninterruptibly
openChannels = null
bootstrap.releaseExternalResources
bootstrap = null
connection = null
log.info("%s has been shut down", name)
}
@deprecated("Use addListener instead")
@ -423,7 +419,7 @@ class RemoteClientHandler(
}
}
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = if (client.isRunning) {
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = client.runSwitch ifOn {
if (client.isWithinReconnectionTimeWindow) {
timer.newTimeout(new TimerTask() {
def run(timeout: Timeout) = {

View file

@ -66,7 +66,8 @@ object RemoteNode extends RemoteServer
*
* @author <a href="http://jonasboner.com">Jonas Bon&#233;r</a>
*/
object RemoteServer {
object
RemoteServer {
val UUID_PREFIX = "uuid:"
val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost")
val PORT = config.getInt("akka.remote.server.port", 9999)
@ -572,6 +573,29 @@ class RemoteServerHandler(
server.typedActorsByUuid().get(uuid)
}
private def findActorByIdOrUuid(id: String, uuid: String) : ActorRef = {
var actorRefOrNull = if (id.startsWith(UUID_PREFIX)) {
findActorByUuid(id.substring(UUID_PREFIX.length))
} else {
findActorById(id)
}
if (actorRefOrNull eq null) {
actorRefOrNull = findActorByUuid(uuid)
}
actorRefOrNull
}
private def findTypedActorByIdOrUuid(id: String, uuid: String) : AnyRef = {
var actorRefOrNull = if (id.startsWith(UUID_PREFIX)) {
findTypedActorByUuid(id.substring(UUID_PREFIX.length))
} else {
findTypedActorById(id)
}
if (actorRefOrNull eq null) {
actorRefOrNull = findTypedActorByUuid(uuid)
}
actorRefOrNull
}
/**
* Creates a new instance of the actor with name, uuid and timeout specified as arguments.
@ -587,11 +611,7 @@ class RemoteServerHandler(
val name = actorInfo.getTarget
val timeout = actorInfo.getTimeout
val actorRefOrNull = if (id.startsWith(UUID_PREFIX)) {
findActorByUuid(id.substring(UUID_PREFIX.length))
} else {
findActorById(id)
}
val actorRefOrNull = findActorByIdOrUuid(id, uuidFrom(uuid.getHigh,uuid.getLow).toString)
if (actorRefOrNull eq null) {
try {
@ -603,7 +623,7 @@ class RemoteServerHandler(
actorRef.id = id
actorRef.timeout = timeout
actorRef.remoteAddress = None
server.actors.put(id, actorRef) // register by id
server.actorsByUuid.put(actorRef.uuid.toString, actorRef) // register by uuid
actorRef
} catch {
case e =>
@ -618,11 +638,7 @@ class RemoteServerHandler(
val uuid = actorInfo.getUuid
val id = actorInfo.getId
val typedActorOrNull = if (id.startsWith(UUID_PREFIX)) {
findTypedActorByUuid(id.substring(UUID_PREFIX.length))
} else {
findTypedActorById(id)
}
val typedActorOrNull = findTypedActorByIdOrUuid(id, uuidFrom(uuid.getHigh,uuid.getLow).toString)
if (typedActorOrNull eq null) {
val typedActorInfo = actorInfo.getTypedActorInfo
@ -639,7 +655,7 @@ class RemoteServerHandler(
val newInstance = TypedActor.newInstance(
interfaceClass, targetClass.asInstanceOf[Class[_ <: TypedActor]], actorInfo.getTimeout).asInstanceOf[AnyRef]
server.typedActors.put(id, newInstance) // register by id
server.typedActors.put(uuidFrom(uuid.getHigh,uuid.getLow).toString, newInstance) // register by uuid
newInstance
} catch {
case e =>

View file

@ -244,13 +244,13 @@ object RemoteActorSerialization {
val host = homeAddress.getHostName
val port = homeAddress.getPort
Actor.log.debug("Register serialized Actor [%s] as remote @ [%s:%s]", actorClass.getName, host, port)
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(actorClass.getName)
.setActorClassname(actorClassName)
.setHomeAddress(AddressProtocol.newBuilder.setHostname(host).setPort(port).build)
.setTimeout(timeout)
.build
@ -286,9 +286,9 @@ object RemoteActorSerialization {
case ActorType.TypedActor => actorInfoBuilder.setActorType(TYPED_ACTOR)
}
val actorInfo = actorInfoBuilder.build
val requestUuid = newUuid
val requestBuilder = RemoteRequestProtocol.newBuilder
.setUuid(UuidProtocol.newBuilder.setHigh(uuid.getTime).setLow(uuid.getClockSeqAndNode).build)
.setUuid(UuidProtocol.newBuilder.setHigh(requestUuid.getTime).setLow(requestUuid.getClockSeqAndNode).build)
.setMessage(MessageSerializer.serialize(message))
.setActorInfo(actorInfo)
.setIsOneWay(isOneWay)

View file

@ -56,6 +56,15 @@ object ClientInitiatedRemoteActorSpec {
SendOneWayAndReplySenderActor.latch.countDown
}
}
class MyActorCustomConstructor extends Actor {
var prefix = "default-"
var count = 0
def receive = {
case "incrPrefix" => count += 1; prefix = "" + count + "-"
case msg: String => self.reply(prefix + msg)
}
}
}
class ClientInitiatedRemoteActorSpec extends JUnitSuite {
@ -123,6 +132,19 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
actor.stop
}
@Test
def shouldSendBangBangMessageAndReceiveReplyConcurrently = {
val actors = (1 to 10).
map(num => {
val a = actorOf[RemoteActorSpecActorBidirectional]
a.makeRemote(HOSTNAME, PORT1)
a.start
}).toList
actors.map(_ !!! "Hello").
foreach(future => assert("World" === future.await.result.asInstanceOf[Option[String]].get))
actors.foreach(_.stop)
}
@Test
def shouldSendAndReceiveRemoteException {
implicit val timeout = 500000000L
@ -137,6 +159,26 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
assert("Expected exception; to test fault-tolerance" === e.getMessage())
}
actor.stop
}
}
@Test
def shouldRegisterActorByUuid {
val actor1 = actorOf[MyActorCustomConstructor]
actor1.makeRemote(HOSTNAME, PORT1)
actor1.start
actor1 ! "incrPrefix"
assert((actor1 !! "test").get === "1-test")
actor1 ! "incrPrefix"
assert((actor1 !! "test").get === "2-test")
val actor2 = actorOf[MyActorCustomConstructor]
actor2.makeRemote(HOSTNAME, PORT1)
actor2.start
assert((actor2 !! "test").get === "default-test")
actor1.stop
actor2.stop
}
}

View file

@ -40,7 +40,7 @@ class Boot {
// Use a custom Camel context and a custom touter builder
CamelContextManager.init(new DefaultCamelContext(registry))
CamelContextManager.context.addRoutes(new CustomRouteBuilder)
CamelContextManager.mandatoryContext.addRoutes(new CustomRouteBuilder)
val producer = actorOf[Producer1]
val mediator = actorOf(new Transformer(producer))

View file

@ -12,7 +12,7 @@ import se.scalablesolutions.akka.camel._
* @author Martin Krasser
*/
object StandaloneApplication extends Application {
import CamelContextManager.context
import CamelContextManager._
import CamelServiceManager._
// 'externally' register typed actors
@ -21,15 +21,15 @@ object StandaloneApplication extends Application {
// customize CamelContext
CamelContextManager.init(new DefaultCamelContext(registry))
CamelContextManager.context.addRoutes(new StandaloneApplicationRoute)
CamelContextManager.mandatoryContext.addRoutes(new StandaloneApplicationRoute)
startCamelService
// access 'externally' registered typed actors
assert("hello msg1" == context.createProducerTemplate.requestBody("direct:test", "msg1"))
assert("hello msg1" == mandatoryContext.createProducerTemplate.requestBody("direct:test", "msg1"))
// set expectations on upcoming endpoint activation
val activation = service.expectEndpointActivationCount(1)
val activation = mandatoryService.expectEndpointActivationCount(1)
// 'internally' register typed actor (requires CamelService)
TypedActor.newInstance(classOf[TypedConsumer2], classOf[TypedConsumer2Impl])
@ -39,7 +39,7 @@ object StandaloneApplication extends Application {
// access 'internally' (automatically) registered typed-actors
// (see @consume annotation value at TypedConsumer2.foo method)
assert("default: msg3" == context.createProducerTemplate.requestBody("direct:default", "msg3"))
assert("default: msg3" == mandatoryContext.createProducerTemplate.requestBody("direct:default", "msg3"))
stopCamelService
@ -60,7 +60,7 @@ object StandaloneSpringApplication extends Application {
val appctx = new ClassPathXmlApplicationContext("/context-standalone.xml")
// access 'externally' registered typed actors with typed-actor component
assert("hello msg3" == template.requestBody("direct:test3", "msg3"))
assert("hello msg3" == mandatoryTemplate.requestBody("direct:test3", "msg3"))
appctx.close
@ -86,7 +86,7 @@ object StandaloneJmsApplication extends Application {
startCamelService
// Expect two consumer endpoints to be activated
val completion = service.expectEndpointActivationCount(2)
val completion = mandatoryService.expectEndpointActivationCount(2)
val jmsUri = "jms:topic:test"
// Wire publisher and consumer using a JMS topic
@ -104,7 +104,7 @@ object StandaloneJmsApplication extends Application {
// Send 10 messages to JMS topic directly
for(i <- 1 to 10) {
CamelContextManager.template.sendBody(jmsUri, "Camel rocks (%d)" format i)
CamelContextManager.mandatoryTemplate.sendBody(jmsUri, "Camel rocks (%d)" format i)
}
stopCamelService

View file

@ -50,7 +50,7 @@ object HttpConcurrencyTestStress {
val workers = for (i <- 1 to 8) yield actorOf[HttpServerWorker].start
val balancer = loadBalancerActor(new CyclicIterator(workers.toList))
val completion = service.expectEndpointActivationCount(1)
val completion = service.get.expectEndpointActivationCount(1)
val server = actorOf(new HttpServerActor(balancer)).start
completion.await
}

View file

@ -170,9 +170,7 @@ trait RedisChatStorageFactory { this: Actor =>
* Chat server. Manages sessions and redirects all other messages to the Session for the client.
*/
trait ChatServer extends Actor {
self.faultHandler = Some(OneForOneStrategy(5, 5000))
self.trapExit = List(classOf[Exception])
self.faultHandler = OneForOneStrategy(List(classOf[Exception]),5, 5000)
val storage: ActorRef
log.info("Chat server is starting up...")

View file

@ -21,8 +21,8 @@ class CamelServiceSpringFeatureTest extends FeatureSpec with BeforeAndAfterEach
import CamelContextManager._
scenario("with a custom CamelContext and access a registered typed actor") {
val appctx = new ClassPathXmlApplicationContext("/appContextCamelServiceCustom.xml")
assert(context.isInstanceOf[SpringCamelContext])
assert("hello sample" === template.requestBody("direct:test", "sample"))
assert(mandatoryContext.isInstanceOf[SpringCamelContext])
assert("hello sample" === mandatoryTemplate.requestBody("direct:test", "sample"))
appctx.close
}
@ -32,10 +32,10 @@ class CamelServiceSpringFeatureTest extends FeatureSpec with BeforeAndAfterEach
val registry = new SimpleRegistry
registry.put("custom", TypedActor.newInstance(classOf[SampleBeanIntf], classOf[SampleBean]))
// set custom registry in DefaultCamelContext
assert(context.isInstanceOf[DefaultCamelContext])
context.asInstanceOf[DefaultCamelContext].setRegistry(registry)
assert(mandatoryContext.isInstanceOf[DefaultCamelContext])
mandatoryContext.asInstanceOf[DefaultCamelContext].setRegistry(registry)
// access registered typed actor
assert("hello sample" === template.requestBody("typed-actor:custom?method=foo", "sample"))
assert("hello sample" === mandatoryTemplate.requestBody("typed-actor:custom?method=foo", "sample"))
appctx.close
}
}

View file

@ -665,13 +665,12 @@ object TypedActor extends Logging {
* @param trapExceptions array of exceptions that should be handled by the supervisor
*/
def link(supervisor: AnyRef, supervised: AnyRef,
handler: FaultHandlingStrategy, trapExceptions: Array[Class[_ <: Throwable]]) = {
handler: FaultHandlingStrategy) = {
val supervisorActor = actorFor(supervisor).getOrElse(
throw new IllegalActorStateException("Can't link when the supervisor is not an Typed Actor"))
val supervisedActor = actorFor(supervised).getOrElse(
throw new IllegalActorStateException("Can't link when the supervised is not an Typed Actor"))
supervisorActor.trapExit = trapExceptions.toList
supervisorActor.faultHandler = Some(handler)
supervisorActor.faultHandler = handler
supervisorActor.link(supervisedActor)
}
@ -688,18 +687,6 @@ object TypedActor extends Logging {
supervisorActor.unlink(supervisedActor)
}
/**
* Sets the trap exit for the given supervisor Typed Actor.
* @param supervisor the supervisor Typed Actor
* @param trapExceptions array of exceptions that should be handled by the supervisor
*/
def trapExit(supervisor: AnyRef, trapExceptions: Array[Class[_ <: Throwable]]) = {
val supervisorActor = actorFor(supervisor).getOrElse(
throw new IllegalActorStateException("Can't set trap exceptions when the supervisor is not an Typed Actor"))
supervisorActor.trapExit = trapExceptions.toList
this
}
/**
* Sets the fault handling strategy for the given supervisor Typed Actor.
* @param supervisor the supervisor Typed Actor
@ -708,7 +695,7 @@ object TypedActor extends Logging {
def faultHandler(supervisor: AnyRef, handler: FaultHandlingStrategy) = {
val supervisorActor = actorFor(supervisor).getOrElse(
throw new IllegalActorStateException("Can't set fault handler when the supervisor is not an Typed Actor"))
supervisorActor.faultHandler = Some(handler)
supervisorActor.faultHandler = handler
this
}

View file

@ -87,7 +87,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft
SamplePojoImpl.reset
val pojo = TypedActor.newInstance(classOf[SimpleJavaPojo], classOf[SimpleJavaPojoImpl])
val supervisor = TypedActor.newInstance(classOf[SimpleJavaPojo], classOf[SimpleJavaPojoImpl])
link(supervisor, pojo, OneForOneStrategy(3, 2000), Array(classOf[Throwable]))
link(supervisor, pojo, OneForOneStrategy(Array(classOf[Throwable]), 3, 2000))
pojo.throwException
Thread.sleep(500)
SimpleJavaPojoImpl._pre should be(true)

View file

@ -435,12 +435,14 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
class AkkaAMQPProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
val commons_io = Dependencies.commons_io
val rabbit = Dependencies.rabbit
val protobuf = Dependencies.protobuf
val protobuf = Dependencies.protobuf
// testing
val junit = Dependencies.junit
val multiverse = Dependencies.multiverse
val scalatest = Dependencies.scalatest
val junit = Dependencies.junit
val multiverse = Dependencies.multiverse
val scalatest = Dependencies.scalatest
override def testOptions = createTestFilter( _.endsWith("Test") )
}
// -------------------------------------------------------------------------------------------------------------------