diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala
index 015a4a4b40..8253bb6d10 100644
--- a/akka-actor/src/main/scala/actor/ActorRef.scala
+++ b/akka-actor/src/main/scala/actor/ActorRef.scala
@@ -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.
- *
- *
- * Trap all exceptions:
- *
- * getContext().setTrapExit(new Class[]{Throwable.class});
- *
- *
- * Trap specific exceptions only:
- *
- * getContext().setTrapExit(new Class[]{MyApplicationException.class, MyApplicationError.class});
- *
- */
- 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.
*
* Can be one of:
*
- * getContext().setFaultHandler(new AllForOneStrategy(maxNrOfRetries, withinTimeRange));
+ * getContext().setFaultHandler(new AllForOneStrategy(new Class[]{Throwable.class},maxNrOfRetries, withinTimeRange));
*
* Or:
*
- * getContext().setFaultHandler(new OneForOneStrategy(maxNrOfRetries, withinTimeRange));
+ * getContext().setFaultHandler(new OneForOneStrategy(new Class[]{Throwable.class},maxNrOfRetries, withinTimeRange));
*
*/
- 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.
*
- * 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.
*
- * 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'.
*
@@ -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.
- *
*
- * Set trapExit to the list of exception classes that the actor should be able to trap
- * from the actor it is supervising. When the supervising actor throws these exceptions
- * then they will trigger a restart.
- *
- *
- * Trap no exceptions:
- *
- * trapExit = Nil
- *
- *
- * Trap all exceptions:
- *
- * trapExit = List(classOf[Throwable])
- *
- *
- * Trap specific exceptions only:
- *
- * trapExit = List(classOf[MyApplicationException], classOf[MyApplicationError])
- *
- */
- @volatile
- var trapExit: List[Class[_ <: Throwable]] = Nil
-
- /**
- * User overridable callback/setting.
- *
- * If 'trapExit' is set for the actor to act as supervisor, then a faultHandler must be defined.
+ * Don't forget to supply a List of exception types to intercept (trapExit)
*
* Can be one of:
*
- * faultHandler = Some(AllForOneStrategy(maxNrOfRetries, withinTimeRange))
+ * faultHandler = AllForOneStrategy(trapExit = List(classOf[Exception]),maxNrOfRetries, withinTimeRange)
*
* Or:
*
- * faultHandler = Some(OneForOneStrategy(maxNrOfRetries, withinTimeRange))
+ * faultHandler = OneForOneStrategy(trapExit = List(classOf[Exception]),maxNrOfRetries, withinTimeRange)
*
*/
@volatile
- var faultHandler: Option[FaultHandlingStrategy] = None
+ @BeanProperty
+ var faultHandler: FaultHandlingStrategy = NoFaultHandlingStrategy
/**
* The reference sender Actor of the last received message.
diff --git a/akka-actor/src/main/scala/actor/ActorRegistry.scala b/akka-actor/src/main/scala/actor/ActorRegistry.scala
index f4ef69f353..41bff91132 100644
--- a/akka-actor/src/main/scala/actor/ActorRegistry.scala
+++ b/akka-actor/src/main/scala/actor/ActorRegistry.scala
@@ -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.
diff --git a/akka-actor/src/main/scala/actor/Agent.scala b/akka-actor/src/main/scala/actor/Agent.scala
index e5b00d4f5e..6b9385ca4e 100644
--- a/akka-actor/src/main/scala/actor/Agent.scala
+++ b/akka-actor/src/main/scala/actor/Agent.scala
@@ -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
diff --git a/akka-actor/src/main/scala/actor/Supervisor.scala b/akka-actor/src/main/scala/actor/Supervisor.scala
index 28d883e5ea..ba559e6945 100644
--- a/akka-actor/src/main/scala/actor/Supervisor.scala
+++ b/akka-actor/src/main/scala/actor/Supervisor.scala
@@ -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 Jonas Bonér
*/
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 Jonas Bonér
*/
-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
diff --git a/akka-actor/src/main/scala/config/SupervisionConfig.scala b/akka-actor/src/main/scala/config/SupervisionConfig.scala
index 43a5e21395..12202f5d9d 100644
--- a/akka-actor/src/main/scala/config/SupervisionConfig.scala
+++ b/akka-actor/src/main/scala/config/SupervisionConfig.scala
@@ -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.
diff --git a/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala b/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala
index 329682de52..56face4b6b 100644
--- a/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala
+++ b/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala
@@ -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.
diff --git a/akka-actor/src/main/scala/japi/JavaAPI.scala b/akka-actor/src/main/scala/japi/JavaAPI.scala
new file mode 100644
index 0000000000..7e79fe8184
--- /dev/null
+++ b/akka-actor/src/main/scala/japi/JavaAPI.scala
@@ -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 Option
+ * are either instances of case class Some or it is case
+ * object None.
+ *
+ * Java API
+ */
+sealed abstract class Option[A] extends java.lang.Iterable[A] {
+ import scala.collection.JavaConversions._
+
+ def get: A
+ def isEmpty: Boolean
+ def isDefined = !isEmpty
+ def asScala: scala.Option[A]
+ def iterator = if (isEmpty) Iterator.empty else Iterator.single(get)
+}
+
+object Option {
+ /**
+ * Option factory that creates Some
+ */
+ def some[A](v: A): Option[A] = Some(v)
+
+ /**
+ * Option factory that creates None
+ */
+ def none[A] = None.asInstanceOf[Option[A]]
+
+ /**
+ * Option factory that creates None if
+ * v is null, Some(v) otherwise.
+ */
+ def option[A](v: A): Option[A] = if (v == null) none else some(v)
+
+ /**
+ * Class Some[A] represents existing values of type
+ * A.
+ */
+ final case class Some[A](v: A) extends Option[A] {
+ def get = v
+ def isEmpty = false
+ def asScala = scala.Some(v)
+ }
+
+ /**
+ * This case object represents non-existent values.
+ */
+ private case object None extends Option[Nothing] {
+ def get = throw new NoSuchElementException("None.get")
+ def isEmpty = true
+ def asScala = scala.None
+ }
+
+ implicit def java2ScalaOption[A](o: Option[A]): scala.Option[A] = o.asScala
+ implicit def scala2JavaOption[A](o: scala.Option[A]): Option[A] = option(o.get)
+}
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/util/JavaAPI.scala b/akka-actor/src/main/scala/util/JavaAPI.scala
deleted file mode 100644
index 099082595d..0000000000
--- a/akka-actor/src/main/scala/util/JavaAPI.scala
+++ /dev/null
@@ -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
-}
diff --git a/akka-actor/src/main/scala/util/LockUtil.scala b/akka-actor/src/main/scala/util/LockUtil.scala
index 3d1261e468..6df0695f03 100644
--- a/akka-actor/src/main/scala/util/LockUtil.scala
+++ b/akka-actor/src/main/scala/util/LockUtil.scala
@@ -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
}
\ No newline at end of file
diff --git a/akka-actor/src/test/java/se/scalablesolutions/akka/japi/JavaAPITestBase.java b/akka-actor/src/test/java/se/scalablesolutions/akka/japi/JavaAPITestBase.java
new file mode 100644
index 0000000000..af00530593
--- /dev/null
+++ b/akka-actor/src/test/java/se/scalablesolutions/akka/japi/JavaAPITestBase.java
@@ -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 o = Option.some("abc");
+ assertFalse(o.isEmpty());
+ assertTrue(o.isDefined());
+ assertEquals("abc", o.get());
+ }
+
+ @Test public void shouldCreateNone() {
+ Option o1 = Option.none();
+ assertTrue(o1.isEmpty());
+ assertFalse(o1.isDefined());
+
+ Option 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());
+ }
+}
diff --git a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
index b9fa238963..7fb9bd6ffe 100644
--- a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
+++ b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
@@ -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)
}
}
diff --git a/akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala
index ffc9dbd860..b1f8af27c0 100644
--- a/akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala
+++ b/akka-actor/src/test/scala/actor/supervisor/SupervisorHierarchySpec.scala
@@ -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
diff --git a/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala b/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala
index f7d1752ded..d7390a0d43 100644
--- a/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala
+++ b/akka-actor/src/test/scala/actor/supervisor/SupervisorSpec.scala
@@ -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)
diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
index 9942c5812e..352ade75f0 100644
--- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
+++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
@@ -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)
}
}
diff --git a/akka-actor/src/test/scala/japi/JavaAPITest.scala b/akka-actor/src/test/scala/japi/JavaAPITest.scala
new file mode 100644
index 0000000000..721342b7af
--- /dev/null
+++ b/akka-actor/src/test/scala/japi/JavaAPITest.scala
@@ -0,0 +1,5 @@
+package se.scalablesolutions.akka.japi
+
+import org.scalatest.junit.JUnitSuite
+
+class JavaAPITest extends JavaAPITestBase with JUnitSuite
\ No newline at end of file
diff --git a/akka-amqp/src/main/java/se/scalablesolutions/akka/amqp/ExampleSessionJava.java b/akka-amqp/src/main/java/se/scalablesolutions/akka/amqp/ExampleSessionJava.java
new file mode 100644
index 0000000000..398feb17ce
--- /dev/null
+++ b/akka-amqp/src/main/java/se/scalablesolutions/akka/amqp/ExampleSessionJava.java
@@ -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 .request
+ // queueName = .in
+ Procedure procedure = new Procedure() {
+ public void apply(String message) {
+ System.out.println("### >> Received message: " + message);
+ }
+ };
+ AMQP.newStringConsumer(connection, procedure, exchangeName);
+
+ // send by default to:
+ // exchange = exchangeName
+ // routingKey = .request
+ AMQP.ProducerClient producer = AMQP.newStringProducer(connection, exchangeName);
+
+ producer.send("This shit is easy!");
+ }
+
+ public void easyProtobufProducerConsumer() {
+
+ ActorRef connection = AMQP.newConnection();
+
+ String exchangeName = "easy.protobuf";
+
+ Procedure procedure = new Procedure() {
+ public void apply(RemoteProtocol.AddressProtocol message) {
+ System.out.println("### >> Received message: " + message);
+ }
+ };
+
+ AMQP.newProtobufConsumer(connection, procedure, exchangeName, RemoteProtocol.AddressProtocol.class);
+
+ AMQP.ProducerClient 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 = .request
+ // queueName = .in
+ RPC.newStringRpcServer(connection, exchangeName, new Function() {
+ public String apply(String request) {
+ System.out.println("### >> Got request: " + request);
+ return "Response to: '" + request + "'";
+ }
+ });
+
+ // send by default to:
+ // exchange = exchangeName
+ // routingKey = .request
+ RPC.RpcClient stringRpcClient = RPC.newStringRpcClient(connection, exchangeName);
+
+ Option response = stringRpcClient.call("AMQP Rocks!");
+ System.out.println("### >> Got response: " + response);
+
+ final StandardLatch standardLatch = new StandardLatch();
+ stringRpcClient.callAsync("AMQP is dead easy", new Procedure() {
+ 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() {
+ public RemoteProtocol.AddressProtocol apply(RemoteProtocol.AddressProtocol request) {
+ return RemoteProtocol.AddressProtocol.newBuilder().setHostname(request.getHostname()).setPort(request.getPort()).build();
+ }
+ }, RemoteProtocol.AddressProtocol.class);
+
+ RPC.RpcClient protobufRpcClient =
+ RPC.newProtobufRpcClient(connection, exchangeName, RemoteProtocol.AddressProtocol.class);
+
+ scala.Option 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);
+ }
+}
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQP.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQP.scala
index cd73d27e03..73389f910b 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQP.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQP.scala
@@ -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
}
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQPMessage.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQPMessage.scala
index 2b53a73d08..7f1ef053de 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQPMessage.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/AMQPMessage.scala
@@ -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,
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala
index 0ca9046093..e5ff20cbb4 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala
@@ -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 + "]"
}
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala
index ecb3029444..00756aa959 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala
@@ -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 = .request
+ // exchange = optional exchangeName
+ // routingKey = provided routingKey or .request
// queueName = .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 = .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)
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExchangeType.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExchangeType.scala
index 1fa6e0543f..2c35a017e4 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExchangeType.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExchangeType.scala
@@ -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"
}
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala
index 4d642df554..6617c62a44 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala
@@ -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) = {
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala
index bf5a192299..16ec8db389 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala
@@ -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 {
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ProducerActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ProducerActor.scala
index 3551ffa276..bb3448f8b5 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ProducerActor.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ProducerActor.scala
@@ -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 + "]"
}
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RPC.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RPC.scala
index 5c50fd4670..ed0f8be7e1 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RPC.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RPC.scala
@@ -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))
}
}
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala
index 10596e393f..90fe3ac66a 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala
@@ -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._
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConnectionRecoveryTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala
similarity index 91%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConnectionRecoveryTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala
index f9d30227f0..abd1e4c498 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConnectionRecoveryTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConnectionRecoveryTestIntegration.scala
@@ -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
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerChannelRecoveryTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala
similarity index 83%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerChannelRecoveryTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala
index 31a90c8200..03d315187a 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerChannelRecoveryTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerChannelRecoveryTestIntegration.scala
@@ -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
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerConnectionRecoveryTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala
similarity index 85%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerConnectionRecoveryTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala
index 50c078a13a..90889d8dc4 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerConnectionRecoveryTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerConnectionRecoveryTestIntegration.scala
@@ -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
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualAcknowledgeTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala
similarity index 74%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualAcknowledgeTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala
index 011f287636..cfb5c920d4 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualAcknowledgeTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualAcknowledgeTestIntegration.scala
@@ -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")
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualRejectTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala
similarity index 77%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualRejectTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala
index cdbfe4028c..f0b21d1286 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualRejectTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerManualRejectTestIntegration.scala
@@ -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")
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerMessageTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerMessageTest.scala
deleted file mode 100644
index 88661de58d..0000000000
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerMessageTest.scala
+++ /dev/null
@@ -1,48 +0,0 @@
-package se.scalablesolutions.akka.amqp.test
-
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-
-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
- }
- }
-}
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala
new file mode 100644
index 0000000000..baa6b4e551
--- /dev/null
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPConsumerMessageTestIntegration.scala
@@ -0,0 +1,43 @@
+package se.scalablesolutions.akka.amqp.test
+
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+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)
+ }
+}
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerChannelRecoveryTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala
similarity index 86%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerChannelRecoveryTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala
index e0ede02de3..81b9c29945 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerChannelRecoveryTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerChannelRecoveryTestIntegration.scala
@@ -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)
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerConnectionRecoveryTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala
similarity index 86%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerConnectionRecoveryTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala
index ad756ff5f0..9e74bbaf46 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerConnectionRecoveryTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerConnectionRecoveryTestIntegration.scala
@@ -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)
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerMessageTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerMessageTestIntegration.scala
similarity index 83%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerMessageTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerMessageTestIntegration.scala
index 7d485b1b8f..dfc3ccd294 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerMessageTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProducerMessageTestIntegration.scala
@@ -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)
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProtobufProducerConsumerTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala
similarity index 82%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProtobufProducerConsumerTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala
index fdc147210a..1502d1e40b 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProtobufProducerConsumerTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPProtobufProducerConsumerTestIntegration.scala
@@ -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)
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcClientServerTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala
similarity index 80%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcClientServerTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala
index 1bb487f51c..b4f2a49939 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcClientServerTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcClientServerTestIntegration.scala
@@ -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)
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcProtobufTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala
similarity index 91%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcProtobufTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala
index c86dd9bf81..1e4383e8d7 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcProtobufTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcProtobufTestIntegration.scala
@@ -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()
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcStringTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcStringTestIntegration.scala
similarity index 88%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcStringTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcStringTestIntegration.scala
index 999100d0da..fb36af74ab 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcStringTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPRpcStringTestIntegration.scala
@@ -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")
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPStringProducerConsumerTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala
similarity index 74%
rename from akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPStringProducerConsumerTest.scala
rename to akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala
index f490b605ae..a9de971815 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPStringProducerConsumerTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPStringProducerConsumerTestIntegration.scala
@@ -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)
diff --git a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPTest.scala b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPTest.scala
index 0c993d6815..2a35df0a77 100644
--- a/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPTest.scala
+++ b/akka-amqp/src/test/scala/se/scalablesolutions/akka/amqp/test/AMQPTest.scala
@@ -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)
}
}
}
diff --git a/akka-camel/src/main/scala/CamelContextLifecycle.scala b/akka-camel/src/main/scala/CamelContextLifecycle.scala
index 32bb386dee..3996cd7baf 100644
--- a/akka-camel/src/main/scala/CamelContextLifecycle.scala
+++ b/akka-camel/src/main/scala/CamelContextLifecycle.scala
@@ -1,5 +1,5 @@
/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
+ * Copyright (C) 2009-2010 Scalable Solutions AB
*/
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 Some(CamelContext) (containing the current CamelContext)
+ * if CamelContextLifecycle has been initialized, otherwise None.
*/
- protected def context: CamelContext = _context
+ def context: Option[CamelContext] = _context
/**
- * Returns the managed ProducerTemplate.
+ * Returns Some(ProducerTemplate) (containing the current ProducerTemplate)
+ * if CamelContextLifecycle has been initialized, otherwise None.
*/
- protected def template: ProducerTemplate = _template
+ def template: Option[ProducerTemplate] = _template
/**
- * Sets the managed CamelContext.
+ * Returns Some(CamelContext) (containing the current CamelContext)
+ * if CamelContextLifecycle has been initialized, otherwise None.
+ *
+ * Java API.
*/
- protected def context_= (context: CamelContext) { _context = context }
+ def getContext: JOption[CamelContext] = context
/**
- * Sets the managed ProducerTemplate.
+ * Returns Some(ProducerTemplate) (containing the current ProducerTemplate)
+ * if CamelContextLifecycle has been initialized, otherwise None.
+ *
+ * Java API.
*/
- protected def template_= (template: ProducerTemplate) { _template = template }
+ def getTemplate: JOption[ProducerTemplate] = template
+
+ /**
+ * Returns the current CamelContext if this CamelContextLifecycle
+ * has been initialized, otherwise throws an IllegalStateException.
+ */
+ def mandatoryContext =
+ if (context.isDefined) context.get
+ else throw new IllegalStateException("no current CamelContext")
+
+ /**
+ * Returns the current ProducerTemplate if this CamelContextLifecycle
+ * has been initialized, otherwise throws an IllegalStateException.
+ */
+ def mandatoryTemplate =
+ if (template.isDefined) template.get
+ else throw new IllegalStateException("no current ProducerTemplate")
+
+ /**
+ * Returns the current CamelContext if this CamelContextLifecycle
+ * has been initialized, otherwise throws an IllegalStateException.
+ *
+ * Java API.
+ */
+ def getMandatoryContext = mandatoryContext
+
+ /**
+ * Returns the current ProducerTemplate if this CamelContextLifecycle
+ * has been initialized, otherwise throws an IllegalStateException.
+ *
+ * Java API.
+ */
+ def getMandatoryTemplate = mandatoryTemplate
def initialized = _initialized
def started = _started
@@ -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
+ *
+ * Java API.
+ */
+ override def getContext: JOption[CamelContext] = super.getContext
+
+ /**
+ * see CamelContextLifecycle.getTemplate
+ *
+ * Java API.
+ */
+ override def getTemplate: JOption[ProducerTemplate] = super.getTemplate
+
+ /**
+ * see CamelContextLifecycle.getMandatoryContext
+ *
+ * Java API.
+ */
+ override def getMandatoryContext = super.getMandatoryContext
+
+ /**
+ * see CamelContextLifecycle.getMandatoryTemplate
+ *
+ * Java API.
+ */
+ override def getMandatoryTemplate = super.getMandatoryTemplate
}
diff --git a/akka-camel/src/main/scala/CamelService.scala b/akka-camel/src/main/scala/CamelService.scala
index 22b536e383..d53ff07dec 100644
--- a/akka-camel/src/main/scala/CamelService.scala
+++ b/akka-camel/src/main/scala/CamelService.scala
@@ -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 Some(CamelService) if this CamelService
+ * has been started, None otherwise.
*/
- def service =
+ def service = _current
+
+ /**
+ * Returns the current CamelService if CamelService
+ * has been started, otherwise throws an IllegalStateException.
+ *
+ * Java API
+ */
+ def getService: JOption[CamelService] = CamelServiceManager.service
+
+ /**
+ * Returns Some(CamelService) (containing the current CamelService)
+ * if this CamelServicehas been started, None otherwise.
+ */
+ def mandatoryService =
if (_current.isDefined) _current.get
- else throw new IllegalStateException("no current CamelService")
+ else throw new IllegalStateException("co current Camel service")
+
+ /**
+ * Returns Some(CamelService) (containing the current CamelService)
+ * if this CamelServicehas been started, None otherwise.
+ *
+ * Java API
+ */
+ def getMandatoryService = mandatoryService
private[camel] def register(service: CamelService) =
if (_current.isDefined) throw new IllegalStateException("current CamelService already registered")
diff --git a/akka-camel/src/main/scala/ConsumerPublisher.scala b/akka-camel/src/main/scala/ConsumerPublisher.scala
index dfc1510ea6..f1bb5d7ab3 100644
--- a/akka-camel/src/main/scala/ConsumerPublisher.scala
+++ b/akka-camel/src/main/scala/ConsumerPublisher.scala
@@ -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))
}
}
diff --git a/akka-camel/src/main/scala/Message.scala b/akka-camel/src/main/scala/Message.scala
index d5cfd23faf..7c503009e8 100644
--- a/akka-camel/src/main/scala/Message.scala
+++ b/akka-camel/src/main/scala/Message.scala
@@ -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 T. 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 names.
@@ -53,14 +53,14 @@ case class Message(val body: Any, val headers: Map[String, Any] = Map.empty) {
* NoSuchElementException if the header doesn't exist.
*/
def headerAs[T](name: String)(implicit m: Manifest[T]): T =
- CamelContextManager.context.getTypeConverter.mandatoryConvertTo[T](m.erasure.asInstanceOf[Class[T]], header(name))
+ CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](m.erasure.asInstanceOf[Class[T]], header(name))
/**
* Returns the header with given name converted to type given by the clazz
* argument. Throws NoSuchElementException if the header doesn't exist.
*/
def headerAs[T](name: String, clazz: Class[T]): T =
- CamelContextManager.context.getTypeConverter.mandatoryConvertTo[T](clazz, header(name))
+ CamelContextManager.mandatoryContext.getTypeConverter.mandatoryConvertTo[T](clazz, header(name))
/**
* Creates a Message with a new body using a transformer function.
diff --git a/akka-camel/src/main/scala/Producer.scala b/akka-camel/src/main/scala/Producer.scala
index 3313c1d0be..2924590c9e 100644
--- a/akka-camel/src/main/scala/Producer.scala
+++ b/akka-camel/src/main/scala/Producer.scala
@@ -27,7 +27,7 @@ trait ProducerSupport { this: Actor =>
* Endpoint object resolved from the current CamelContext with
* endpointUri.
*/
- private lazy val endpoint = CamelContextManager.context.getEndpoint(endpointUri)
+ private lazy val endpoint = CamelContextManager.mandatoryContext.getEndpoint(endpointUri)
/**
* SendProcessor for producing messages to endpoint.
diff --git a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java
index e909947de8..bfa34f42e5 100644
--- a/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java
+++ b/akka-camel/src/test/java/se/scalablesolutions/akka/camel/SampleUntypedForwardingProducer.java
@@ -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);
}
}
diff --git a/akka-camel/src/test/scala/CamelContextLifecycleTest.scala b/akka-camel/src/test/scala/CamelContextLifecycleTest.scala
index cf558ec8d9..6e6889c295 100644
--- a/akka-camel/src/test/scala/CamelContextLifecycleTest.scala
+++ b/akka-camel/src/test/scala/CamelContextLifecycleTest.scala
@@ -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
diff --git a/akka-camel/src/test/scala/CamelServiceManagerTest.scala b/akka-camel/src/test/scala/CamelServiceManagerTest.scala
index fd15ce7154..712ffec70b 100644
--- a/akka-camel/src/test/scala/CamelServiceManagerTest.scala
+++ b/akka-camel/src/test/scala/CamelServiceManagerTest.scala
@@ -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 }
- }
}
}
}
diff --git a/akka-camel/src/test/scala/ConsumerTest.scala b/akka-camel/src/test/scala/ConsumerTest.scala
index 2a2cc0b11f..0af8aec7d5 100644
--- a/akka-camel/src/test/scala/ConsumerTest.scala
+++ b/akka-camel/src/test/scala/ConsumerTest.scala
@@ -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 => {
diff --git a/akka-camel/src/test/scala/ProducerFeatureTest.scala b/akka-camel/src/test/scala/ProducerFeatureTest.scala
index a27e05a54f..5f31bcbe1c 100644
--- a/akka-camel/src/test/scala/ProducerFeatureTest.scala
+++ b/akka-camel/src/test/scala/ProducerFeatureTest.scala
@@ -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 {
diff --git a/akka-camel/src/test/scala/RemoteConsumerTest.scala b/akka-camel/src/test/scala/RemoteConsumerTest.scala
index afba2011d5..2218aac25a 100644
--- a/akka-camel/src/test/scala/RemoteConsumerTest.scala
+++ b/akka-camel/src/test/scala/RemoteConsumerTest.scala
@@ -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")
}
}
diff --git a/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala b/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala
index c8a0bd8542..0d268785b6 100644
--- a/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala
+++ b/akka-camel/src/test/scala/UntypedProducerFeatureTest.scala
@@ -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 {
diff --git a/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala b/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala
index 331f2c23b6..cc9f750aae 100644
--- a/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala
+++ b/akka-camel/src/test/scala/component/ActorComponentFeatureTest.scala
@@ -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 {
diff --git a/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala b/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala
index 37352eb154..e1f169187a 100644
--- a/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala
+++ b/akka-camel/src/test/scala/component/TypedActorComponentFeatureTest.scala
@@ -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")
}
}
diff --git a/akka-remote/src/main/scala/remote/RemoteClient.scala b/akka-remote/src/main/scala/remote/RemoteClient.scala
index 2694c9aee8..e39b83a503 100644
--- a/akka-remote/src/main/scala/remote/RemoteClient.scala
+++ b/akka-remote/src/main/scala/remote/RemoteClient.scala
@@ -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) = {
diff --git a/akka-remote/src/main/scala/remote/RemoteServer.scala b/akka-remote/src/main/scala/remote/RemoteServer.scala
index 27b9af15a2..deb3c05b87 100644
--- a/akka-remote/src/main/scala/remote/RemoteServer.scala
+++ b/akka-remote/src/main/scala/remote/RemoteServer.scala
@@ -66,7 +66,8 @@ object RemoteNode extends RemoteServer
*
* @author Jonas Bonér
*/
-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 =>
diff --git a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
index 421217e10b..6ebc624e62 100644
--- a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
+++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
@@ -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)
diff --git a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
index d39b58d41d..ba550dc2aa 100644
--- a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
@@ -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
+ }
}
diff --git a/akka-samples/akka-sample-camel/src/main/scala/Boot.scala b/akka-samples/akka-sample-camel/src/main/scala/Boot.scala
index 461ba31d83..fc6afc856a 100644
--- a/akka-samples/akka-sample-camel/src/main/scala/Boot.scala
+++ b/akka-samples/akka-sample-camel/src/main/scala/Boot.scala
@@ -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))
diff --git a/akka-samples/akka-sample-camel/src/main/scala/StandaloneApplication.scala b/akka-samples/akka-sample-camel/src/main/scala/StandaloneApplication.scala
index c86295da57..2ecccb1e02 100644
--- a/akka-samples/akka-sample-camel/src/main/scala/StandaloneApplication.scala
+++ b/akka-samples/akka-sample-camel/src/main/scala/StandaloneApplication.scala
@@ -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
diff --git a/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala b/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala
index 3813463601..76cbc58a8b 100644
--- a/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala
+++ b/akka-samples/akka-sample-camel/src/test/scala/HttpConcurrencyTestStress.scala
@@ -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
}
diff --git a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala
index e5b60b364f..b65f833763 100644
--- a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala
+++ b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala
@@ -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...")
diff --git a/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala b/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala
index e8b0d727c3..246ad88f37 100644
--- a/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala
+++ b/akka-spring/src/test/scala/CamelServiceSpringFeatureTest.scala
@@ -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
}
}
diff --git a/akka-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
index 48e0d5a405..8b9cc2034a 100644
--- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala
+++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
@@ -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
}
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
index 781537d5a9..f2903adf03 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
@@ -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)
diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala
index cfbd2e4312..81688ccf82 100644
--- a/project/build/AkkaProject.scala
+++ b/project/build/AkkaProject.scala
@@ -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") )
}
// -------------------------------------------------------------------------------------------------------------------