+ * import Actor._
+ *
+ * val a = actor {
+ * case msg => ... // handle message
+ * }
+ *
+ */
def actor(body: PartialFunction[Any, Unit]): Actor = new Actor() {
start
def receive = body
}
+ /**
+ * Use to create an anonymous event-driven actor with both an init block and a message loop block.
+ * The actor is started when created.
+ * Example:
+ *
+ * import Actor._
+ *
+ * val a = actor {
+ * ... // init stuff
+ * } receive {
+ * case msg => ... // handle message
+ * }
+ *
+ *
+ */
+ def actor[A](body: => Unit) = {
+ def handler[A](body: Unit) = new {
+ def receive(handler: PartialFunction[Any, Unit]) = new Actor() {
+ start
+ body
+ def receive = handler
+ }
+ }
+ handler(body)
+ }
+
+ /**
+ * Use to create an anonymous event-driven actor with a body but no message loop block.
+ *
+ * This actor can not respond to any messages but can be used as a simple way to
+ * spawn a lightweight thread to process some task.
+ *
+ * The actor is started when created.
+ * Example:
+ *
+ * import Actor._
+ *
+ * spawn {
+ * ... // do stuff
+ * }
+ *
+ */
+ def spawn(body: => Unit): Actor = new Actor() {
+ start
+ body
+ def receive = {
+ case _ => throw new IllegalArgumentException("Actors created with 'actor(body: => Unit)' do not respond to messages.")
+ }
+ }
+
+ /**
+ * Use to create an anonymous event-driven actor with a life-cycle configuration.
+ * The actor is started when created.
+ * Example:
+ *
+ * import Actor._
+ *
+ * val a = actor(LifeCycle(Temporary)) {
+ * case msg => ... // handle message
+ * }
+ *
+ */
def actor(lifeCycleConfig: LifeCycle)(body: PartialFunction[Any, Unit]): Actor = new Actor() {
- lifeCycle = Some(lifeCycleConfig)
+ lifeCycle = lifeCycleConfig
start
def receive = body
}
+
+ /**
+ * Use to create an anonymous event-driven actor with both an init block and a message loop block
+ * as well as a life-cycle configuration.
+ * The actor is started when created.
+ * Example:
+ *
+ * import Actor._
+ *
+ * val a = actor(LifeCycle(Temporary)) {
+ * ... // init stuff
+ * } receive {
+ * case msg => ... // handle message
+ * }
+ *
+ */
+ def actor[A](lifeCycleConfig: LifeCycle)(body: => Unit) = {
+ def handler[A](body: Unit) = new {
+ def receive(handler: PartialFunction[Any, Unit]) = new Actor() {
+ lifeCycle = lifeCycleConfig
+ start
+ body
+ def receive = handler
+ }
+ }
+ handler(body)
+ }
}
/**
* Actor base trait that should be extended by or mixed to create an Actor with the semantics of the 'Actor Model':
* http://en.wikipedia.org/wiki/Actor_model
- *
+ *
+ * An actor has a well-defined (non-cyclic) life-cycle.
+ * + * => NEW (newly created actor) - can't receive messages (yet) + * => STARTED (when 'start' is invoked) - can receive messages + * => SHUT DOWN (when 'exit' is invoked) - can't do anything + *+ * * @author Jonas Bonér */ trait Actor extends Logging with TransactionManagement { ActorRegistry.register(this) + + implicit protected val self: Actor = this // FIXME http://www.assembla.com/spaces/akka/tickets/56-Change-UUID-generation-for-the-TransactionManagement-trait - val uuid = Uuid.newUuid.toString - + private[akka] var _uuid = Uuid.newUuid.toString + def uuid = _uuid + + // ==================================== // private fields + // ==================================== + @volatile private var _isRunning: Boolean = false + @volatile private var _isShutDown: Boolean = false private var _hotswap: Option[PartialFunction[Any, Unit]] = None private var _config: Option[AnyRef] = None private val _remoteFlagLock = new ReadWriteLock @@ -89,12 +232,28 @@ trait Actor extends Logging with TransactionManagement { private[akka] var _mailbox: MessageQueue = _ private[akka] var _supervisor: Option[Actor] = None + // ==================================== + // protected fields + // ==================================== + /** - * This field should normally not be touched by user code, which should instead use the 'reply' method. + * The 'sender' field holds the sender of the message currently being processed. + * + * If the sender was an actor then it is defined as 'Some(senderActor)' and + * if the sender was of some other instance then it is defined as 'None'. + * + * This sender reference can be used together with the '!' method for request/reply + * message exchanges and which is in many ways better than using the '!!' method + * which will make the sender wait for a reply using a *blocking* future. + */ + protected[this] var sender: Option[Actor] = None + + /** + * The 'senderFuture' field should normally not be touched by user code, which should instead use the 'reply' method. * But it can be used for advanced use-cases when one might want to store away the future and * resolve it later and/or somewhere else. */ - protected var senderFuture: Option[CompletableFutureResult] = None + protected[this] var senderFuture: Option[CompletableFutureResult] = None // ==================================== // ==== USER CALLBACKS TO OVERRIDE ==== @@ -119,32 +278,20 @@ trait Actor extends Logging with TransactionManagement { /** * User overridable callback/setting. - * - * User can (and is encouraged to) override the default configuration (newEventBasedThreadPoolDispatcher) - * so it fits the specific use-case that the actor is used for. * - * It is beneficial to have actors share the same dispatcher, easily +100 actors can share the same. + * The default dispatcher is the Dispatchers.globalEventBasedThreadPoolDispatcher. + * This means that all actors will share the same event-driven thread-pool based dispatcher. * - * But if you are running many many actors then it can be a good idea to have split them up in terms of - * dispatcher sharing. + * You can override it so it fits the specific use-case that the actor is used for. + * See the se.scalablesolutions.akka.dispatch.Dispatchers class for the different + * dispatchers available. * - * Default is that all actors that are created and spawned from within this actor is sharing the same - * dispatcher as its creator. - *
- * val dispatcher = Dispatchers.newEventBasedThreadPoolDispatcher - * dispatcher - * .withNewThreadPoolWithBoundedBlockingQueue(100) - * .setCorePoolSize(16) - * .setMaxPoolSize(128) - * .setKeepAliveTimeInMillis(60000) - * .setRejectionPolicy(new CallerRunsPolicy) - * .buildThreadPool - *+ * The default is also that all actors that are created and spawned from within this actor + * is sharing the same dispatcher as its creator. */ protected[akka] var messageDispatcher: MessageDispatcher = { - val dispatcher = Dispatchers.newEventBasedThreadPoolDispatcher(getClass.getName) + val dispatcher = Dispatchers.globalEventBasedThreadPoolDispatcher _mailbox = dispatcher.messageQueue - dispatcher.registerHandler(this, new ActorMessageInvoker(this)) dispatcher } @@ -183,7 +330,7 @@ trait Actor extends Logging with TransactionManagement { * * Defines the life-cycle for a supervised actor. Default is 'LifeCycle(Permanent)' but can be overridden. */ - @volatile var lifeCycle: Option[LifeCycle] = Some(LifeCycle(Permanent)) + @volatile var lifeCycle: LifeCycle = LifeCycle(Permanent) /** * User overridable callback/setting. @@ -201,7 +348,7 @@ trait Actor extends Logging with TransactionManagement { * * Example code: *
- * def receive: PartialFunction[Any, Unit] = {
+ * def receive = {
* case Ping =>
* println("got a ping")
* reply("pong")
@@ -263,34 +410,82 @@ trait Actor extends Logging with TransactionManagement {
/**
* Starts up the actor and its message queue.
*/
- def start = synchronized {
+ def start: Actor = synchronized {
+ if (_isShutDown) throw new IllegalStateException("Can't restart an actor that have been shut down with 'exit'")
if (!_isRunning) {
+ dispatcher.registerHandler(this, new ActorMessageInvoker(this))
messageDispatcher.start
_isRunning = true
//if (isTransactional) this !! TransactionalInit
}
log.info("[%s] has started", toString)
+ this
}
/**
- * Stops the actor and its message queue.
+ * Shuts down the actor its dispatcher and message queue.
+ * Delegates to 'stop'
+ */
+ protected def exit = stop
+
+ /**
+ * Shuts down the actor its dispatcher and message queue.
*/
def stop = synchronized {
if (_isRunning) {
- dispatcher.unregisterHandler(this)
- if (dispatcher.isInstanceOf[ThreadBasedDispatcher]) dispatcher.shutdown
- // FIXME: Need to do reference count to know if EventBasedThreadPoolDispatcher and EventBasedSingleThreadDispatcher can be shut down
+ messageDispatcher.unregisterHandler(this)
+ if (messageDispatcher.canBeShutDown) messageDispatcher.shutdown // shut down in the dispatcher's references is zero
_isRunning = false
+ _isShutDown = true
shutdown
}
}
/**
* Sends a one-way asynchronous message. E.g. fire-and-forget semantics.
+ *
+ *
+ * If invoked from within an actor then the actor reference is implicitly passed on as the implicit 'sender' argument.
+ *
+ *
+ * This actor 'sender' reference is then available in the receiving actor in the 'sender' member variable.
+ *
+ * actor ! message
+ *
+ *
+ *
+ * If invoked from within a *non* Actor instance then either add this import to resolve the implicit argument:
+ *
+ * import Actor.Sender._
+ * actor ! message
+ *
+ *
+ * Or pass in the implicit argument explicitly:
+ *
+ * actor.!(message)(this)
+ *
+ *
+ * Or use the 'send(..)' method;
+ *
+ * actor.send(message)
+ *
*/
- def !(message: AnyRef) =
- if (_isRunning) postMessageToMailbox(message)
- else throw new IllegalStateException("Actor has not been started, you need to invoke 'actor.start' before using it")
+ def !(message: AnyRef)(implicit sender: AnyRef) = {
+ val from = if (sender != null && sender.isInstanceOf[Actor]) Some(sender.asInstanceOf[Actor])
+ else None
+ if (_isRunning) postMessageToMailbox(message, from)
+ else throw new IllegalStateException(
+ "Actor has not been started, you need to invoke 'actor.start' before using it")
+ }
+
+ /**
+ * Same as the '!' method but does not take an implicit sender as second parameter.
+ */
+ def send(message: AnyRef) = {
+ if (_isRunning) postMessageToMailbox(message, None)
+ else throw new IllegalStateException(
+ "Actor has not been started, you need to invoke 'actor.start' before using it")
+ }
/**
* Sends a message asynchronously and waits on a future for a reply message.
@@ -299,6 +494,8 @@ trait Actor extends Logging with TransactionManagement {
* or until the timeout expires (which will return None). E.g. send-and-receive-eventually semantics.
*
* NOTE:
+ * Use this method with care. In most cases it is better to use '!' together with the 'sender' member field to
+ * implement request/response message exchanges.
* If you are sending messages using !! then you have to use reply(..)
* to send a reply message to the original sender. If not then the sender will block until the timeout expires.
*/
@@ -314,7 +511,8 @@ trait Actor extends Logging with TransactionManagement {
else None
}
getResultOrThrowException(future)
- } else throw new IllegalStateException("Actor has not been started, you need to invoke 'actor.start' before using it")
+ } else throw new IllegalStateException(
+ "Actor has not been started, you need to invoke 'actor.start' before using it")
/**
* Sends a message asynchronously and waits on a future for a reply message.
@@ -323,40 +521,47 @@ trait Actor extends Logging with TransactionManagement {
* or until the timeout expires (which will return None). E.g. send-and-receive-eventually semantics.
*
* NOTE:
+ * Use this method with care. In most cases it is better to use '!' together with the 'sender' member field to
+ * implement request/response message exchanges.
* If you are sending messages using !! then you have to use reply(..)
* to send a reply message to the original sender. If not then the sender will block until the timeout expires.
*/
def !: Option[T] = !
/**
- * Sends a message asynchronously, but waits on a future indefinitely. E.g. emulates a synchronous call.
- *
- * NOTE:
- * Should be used with care (almost never), since very dangerous (will block a thread indefinitely if no reply).
+ * This method is evil and have been removed. Use '!!' with a timeout instead.
*/
- def !?[T](message: AnyRef): T = if (_isRunning) {
- val future = postMessageToMailboxAndCreateFutureResultWithTimeout(message, 0)
- future.awaitBlocking
- getResultOrThrowException(future).get
- } else throw new IllegalStateException("Actor has not been started, you need to invoke 'actor.start' before using it")
+ def !?[T](message: AnyRef): T = throw new UnsupportedOperationException(
+ "'!?' is evil and have been removed. Use '!!' with a timeout instead")
/**
* Use reply(..) to reply with a message to the original sender of the message currently
* being processed.
- *
- * NOTE:
- * Does only work together with the actor !! method and/or active objects not annotated
- * with @oneway.
*/
- protected[this] def reply(message: AnyRef) = senderFuture match {
- case None => throw new IllegalStateException(
- "\n\tNo sender in scope, can't reply. " +
- "\n\tHave you used the '!' message send or the '@oneway' active object annotation? " +
- "\n\tIf so, switch to '!!' (or remove '@oneway') which passes on an implicit future that will be bound by the argument passed to 'reply'." )
- case Some(future) => future.completeWithResult(message)
+ protected[this] def reply(message: AnyRef) = {
+ sender match {
+ case Some(senderActor) =>
+ senderActor ! message
+ case None =>
+ senderFuture match {
+ case None =>
+ throw new IllegalStateException(
+ "\n\tNo sender in scope, can't reply. " +
+ "\n\tYou have probably used the '!' method to either; " +
+ "\n\t\t1. Send a message to a remote actor" +
+ "\n\t\t2. Send a message from an instance that is *not* an actor" +
+ "\n\t\t3. Send a message to an Active Object annotated with the '@oneway' annotation? " +
+ "\n\tIf so, switch to '!!' (or remove '@oneway') which passes on an implicit future that will be bound by the argument passed to 'reply'." )
+ case Some(future) =>
+ future.completeWithResult(message)
+ }
+ }
}
- def dispatcher = messageDispatcher
+ /**
+ * Get the dispatcher for this actor.
+ */
+ def dispatcher = synchronized { messageDispatcher }
/**
* Sets the dispatcher for this actor. Needs to be invoked before the actor is started.
@@ -366,7 +571,8 @@ trait Actor extends Logging with TransactionManagement {
messageDispatcher = dispatcher
_mailbox = messageDispatcher.messageQueue
messageDispatcher.registerHandler(this, new ActorMessageInvoker(this))
- } else throw new IllegalArgumentException("Can not swap dispatcher for " + toString + " after it has been started")
+ } else throw new IllegalArgumentException(
+ "Can not swap dispatcher for " + toString + " after it has been started")
}
/**
@@ -392,7 +598,8 @@ trait Actor extends Logging with TransactionManagement {
*
*/
def makeTransactionRequired = synchronized {
- if (_isRunning) throw new IllegalArgumentException("Can not make actor transaction required after it has been started")
+ if (_isRunning) throw new IllegalArgumentException(
+ "Can not make actor transaction required after it has been started")
else isTransactionRequiresNew = true
}
@@ -409,10 +616,12 @@ trait Actor extends Logging with TransactionManagement {
protected[this] def link(actor: Actor) = {
if (_isRunning) {
_linkedActors.add(actor)
- if (actor._supervisor.isDefined) throw new IllegalStateException("Actor can only have one supervisor [" + actor + "], e.g. link(actor) fails")
+ if (actor._supervisor.isDefined) throw new IllegalStateException(
+ "Actor can only have one supervisor [" + actor + "], e.g. link(actor) fails")
actor._supervisor = Some(this)
log.debug("Linking actor [%s] to actor [%s]", actor, this)
- } else throw new IllegalStateException("Actor has not been started, you need to invoke 'actor.start' before using it")
+ } else throw new IllegalStateException(
+ "Actor has not been started, you need to invoke 'actor.start' before using it")
}
/**
@@ -426,7 +635,8 @@ trait Actor extends Logging with TransactionManagement {
_linkedActors.remove(actor)
actor._supervisor = None
log.debug("Unlinking actor [%s] from actor [%s]", actor, this)
- } else throw new IllegalStateException("Actor has not been started, you need to invoke 'actor.start' before using it")
+ } else throw new IllegalStateException(
+ "Actor has not been started, you need to invoke 'actor.start' before using it")
}
/**
@@ -508,12 +718,13 @@ trait Actor extends Logging with TransactionManagement {
// ==== IMPLEMENTATION DETAILS ====
// ================================
- private def postMessageToMailbox(message: AnyRef): Unit = _remoteFlagLock.withReadLock { // the price you pay for being able to make an actor remote at runtime
+ private def postMessageToMailbox(message: AnyRef, sender: Option[Actor]): Unit = _remoteFlagLock.withReadLock { // the price you pay for being able to make an actor remote at runtime
if (_remoteAddress.isDefined) {
val requestBuilder = RemoteRequest.newBuilder
.setId(RemoteRequestIdFactory.nextId)
.setTarget(this.getClass.getName)
.setTimeout(timeout)
+ .setUuid(uuid)
.setIsActor(true)
.setIsOneWay(true)
.setIsEscaped(false)
@@ -522,17 +733,19 @@ trait Actor extends Logging with TransactionManagement {
RemoteProtocolBuilder.setMessage(message, requestBuilder)
RemoteClient.clientFor(_remoteAddress.get).send(requestBuilder.build)
} else {
- val handle = new MessageInvocation(this, message, None, currentTransaction.get)
+ val handle = new MessageInvocation(this, message, None, sender, currentTransaction.get)
handle.send
}
}
- private def postMessageToMailboxAndCreateFutureResultWithTimeout(message: AnyRef, timeout: Long): CompletableFutureResult = _remoteFlagLock.withReadLock { // the price you pay for being able to make an actor remote at runtime
+ private def postMessageToMailboxAndCreateFutureResultWithTimeout(message: AnyRef, timeout: Long):
+ CompletableFutureResult = _remoteFlagLock.withReadLock { // the price you pay for being able to make an actor remote at runtime
if (_remoteAddress.isDefined) {
val requestBuilder = RemoteRequest.newBuilder
.setId(RemoteRequestIdFactory.nextId)
.setTarget(this.getClass.getName)
.setTimeout(timeout)
+ .setUuid(uuid)
.setIsActor(true)
.setIsOneWay(false)
.setIsEscaped(false)
@@ -541,10 +754,11 @@ trait Actor extends Logging with TransactionManagement {
if (id.isDefined) requestBuilder.setSupervisorUuid(id.get)
val future = RemoteClient.clientFor(_remoteAddress.get).send(requestBuilder.build)
if (future.isDefined) future.get
- else throw new IllegalStateException("Expected a future from remote call to actor " + toString)
+ else throw new IllegalStateException(
+ "Expected a future from remote call to actor " + toString)
} else {
val future = new DefaultCompletableFutureResult(timeout)
- val handle = new MessageInvocation(this, message, Some(future), currentTransaction.get)
+ val handle = new MessageInvocation(this, message, Some(future), None, currentTransaction.get)
handle.send
future
}
@@ -554,25 +768,32 @@ trait Actor extends Logging with TransactionManagement {
* Callback for the dispatcher. E.g. single entry point to the user code and all protected[this] methods
*/
private[akka] def invoke(messageHandle: MessageInvocation) = synchronized {
- if (TransactionManagement.isTransactionalityEnabled) transactionalDispatch(messageHandle)
- else dispatch(messageHandle)
+ try {
+ if (TransactionManagement.isTransactionalityEnabled) transactionalDispatch(messageHandle)
+ else dispatch(messageHandle)
+ } catch {
+ case e =>
+ log.error(e, "Could not invoke actor [%s]", this)
+ throw e
+ }
}
private def dispatch[T](messageHandle: MessageInvocation) = {
setTransaction(messageHandle.tx)
val message = messageHandle.message //serializeMessage(messageHandle.message)
- val future = messageHandle.future
+ senderFuture = messageHandle.future
+ sender = messageHandle.sender
+
try {
- senderFuture = future
if (base.isDefinedAt(message)) base(message) // invoke user actor's receive partial function
else throw new IllegalArgumentException("No handler matching message [" + message + "] in " + toString)
} catch {
case e =>
+ log.error(e, "Could not invoke actor [%s]", this)
// FIXME to fix supervisor restart of remote actor for oneway calls, inject a supervisor proxy that can send notification back to client
if (_supervisor.isDefined) _supervisor.get ! Exit(this, e)
- if (future.isDefined) future.get.completeWithException(this, e)
- else e.printStackTrace
+ if (senderFuture.isDefined) senderFuture.get.completeWithException(this, e)
} finally {
clearTransaction
}
@@ -582,23 +803,25 @@ trait Actor extends Logging with TransactionManagement {
setTransaction(messageHandle.tx)
val message = messageHandle.message //serializeMessage(messageHandle.message)
- val future = messageHandle.future
+ senderFuture = messageHandle.future
+ sender = messageHandle.sender
def proceed = {
try {
incrementTransaction
if (base.isDefinedAt(message)) base(message) // invoke user actor's receive partial function
- else throw new IllegalArgumentException("Actor " + toString + " could not process message [" + message + "] since no matching 'case' clause in its 'receive' method could be found")
+ else throw new IllegalArgumentException(
+ "Actor " + toString + " could not process message [" + message + "]" +
+ "\n\tsince no matching 'case' clause in its 'receive' method could be found")
} finally {
decrementTransaction
}
}
try {
- senderFuture = future
if (isTransactionRequiresNew && !isTransactionInScope) {
if (senderFuture.isEmpty) throw new StmException(
- "\n\tCan't continue transaction in a one-way fire-forget message send" +
+ "Can't continue transaction in a one-way fire-forget message send" +
"\n\tE.g. using Actor '!' method or Active Object 'void' method" +
"\n\tPlease use the Actor '!!', '!?' methods or Active Object method with non-void return type")
atomic {
@@ -607,13 +830,9 @@ trait Actor extends Logging with TransactionManagement {
} else proceed
} catch {
case e =>
- e.printStackTrace
-
- if (future.isDefined) future.get.completeWithException(this, e)
- else e.printStackTrace
-
+ log.error(e, "Could not invoke actor [%s]", this)
+ if (senderFuture.isDefined) senderFuture.get.completeWithException(this, e)
clearTransaction // need to clear currentTransaction before call to supervisor
-
// FIXME to fix supervisor restart of remote actor for oneway calls, inject a supervisor proxy that can send notification back to client
if (_supervisor.isDefined) _supervisor.get ! Exit(this, e)
} finally {
@@ -632,8 +851,7 @@ trait Actor extends Logging with TransactionManagement {
case HotSwap(code) => _hotswap = code
case Restart(reason) => restart(reason)
case Exit(dead, reason) => handleTrapExit(dead, reason)
- case Kill(killer) => throw new ActorKilledException(this, killer)
-// case TransactionalInit => initTransactionalState
+ case Kill => throw new ActorKilledException(this)
}
private[this] def handleTrapExit(dead: Actor, reason: Throwable): Unit = {
@@ -644,7 +862,9 @@ trait Actor extends Logging with TransactionManagement {
case AllForOneStrategy(maxNrOfRetries, withinTimeRange) => restartLinkedActors(reason)
case OneForOneStrategy(maxNrOfRetries, withinTimeRange) => dead.restart(reason)
}
- } else throw new IllegalStateException("No 'faultHandler' defined for actor with the 'trapExit' member field set to non-empty list of exception classes - can't proceed " + toString)
+ } else throw new IllegalStateException(
+ "No 'faultHandler' defined for an actor with the 'trapExit' member field defined " +
+ "\n\tto non-empty list of exception classes - can't proceed " + toString)
} else {
if (_supervisor.isDefined) _supervisor.get ! Exit(dead, reason) // if 'trapExit' is not defined then pass the Exit on
}
@@ -653,8 +873,7 @@ trait Actor extends Logging with TransactionManagement {
private[this] def restartLinkedActors(reason: AnyRef) = {
_linkedActors.toArray.toList.asInstanceOf[List[Actor]].foreach { actor =>
actor.lifeCycle match {
- case None => throw new IllegalStateException("Actor [" + actor.id + "] does not have a life-cycle defined.")
- case Some(LifeCycle(scope, _)) => {
+ case LifeCycle(scope, _) => {
scope match {
case Permanent =>
actor.restart(reason)
diff --git a/akka-actors/src/main/scala/actor/Scheduler.scala b/akka-actors/src/main/scala/actor/Scheduler.scala
index df4d41ff40..9b9ee8bc7e 100644
--- a/akka-actors/src/main/scala/actor/Scheduler.scala
+++ b/akka-actors/src/main/scala/actor/Scheduler.scala
@@ -28,13 +28,13 @@ case class SchedulerException(msg: String, e: Throwable) extends RuntimeExceptio
* which is licensed under the Apache 2 License.
*/
class ScheduleActor(val receiver: Actor, val future: ScheduledFuture[AnyRef]) extends Actor with Logging {
- lifeCycle = Some(LifeCycle(Permanent))
+ lifeCycle = LifeCycle(Permanent)
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case UnSchedule =>
Scheduler.stopSupervising(this)
future.cancel(true)
- stop
+ exit
}
}
@@ -69,7 +69,7 @@ object Scheduler extends Actor {
service.shutdown
}
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case _ => {} // ignore all messages
}
}
diff --git a/akka-actors/src/main/scala/actor/Supervisor.scala b/akka-actors/src/main/scala/actor/Supervisor.scala
index 429677d6f0..ec65fbfec7 100644
--- a/akka-actors/src/main/scala/actor/Supervisor.scala
+++ b/akka-actors/src/main/scala/actor/Supervisor.scala
@@ -12,19 +12,6 @@ import se.scalablesolutions.akka.dispatch.Dispatchers
import java.util.concurrent.ConcurrentHashMap
-import scala.collection.mutable.HashMap
-
-/**
- * Messages that the supervisor responds to and returns.
- *
- * @author Jonas Bonér
- */
-sealed abstract class SupervisorMessage
-case object StartSupervisor extends SupervisorMessage
-case object StopSupervisor extends SupervisorMessage
-case class ConfigureSupervisor(config: SupervisorConfig, factory: SupervisorFactory) extends SupervisorMessage
-case object ConfigSupervisorSuccess extends SupervisorMessage
-
sealed abstract class FaultHandlingStrategy
case class AllForOneStrategy(maxNrOfRetries: Int, withinTimeRange: Int) extends FaultHandlingStrategy
case class OneForOneStrategy(maxNrOfRetries: Int, withinTimeRange: Int) extends FaultHandlingStrategy
@@ -34,42 +21,32 @@ case class OneForOneStrategy(maxNrOfRetries: Int, withinTimeRange: Int) extends
* * Example usage: *
- * class MySupervisorFactory extends SupervisorFactory {
- *
- * override protected def getSupervisorConfig: SupervisorConfig = {
- * SupervisorConfig(
- * RestartStrategy(OneForOne, 3, 10),
- * Supervise(
- * myFirstActor,
- * LifeCycle(Permanent))
- * ::
- * Supervise(
- * mySecondActor,
- * LifeCycle(Permanent))
- * :: Nil)
- * }
- * }
- *
- *
- * Then create a concrete factory in which we mix in support for the specific implementation of the Service we want to use.
- *
- * - * object factory extends MySupervisorFactory + * val factory = SupervisorFactory( + * SupervisorConfig( + * RestartStrategy(OneForOne, 3, 10), + * Supervise( + * myFirstActor, + * LifeCycle(Permanent)) :: + * Supervise( + * mySecondActor, + * LifeCycle(Permanent)) :: + * Nil)) ** * Then create a new Supervisor tree with the concrete Services we have defined. * *
- * val supervisor = factory.newSupervisor - * supervisor ! Start // start up all managed servers + * val supervisor = factory.newInstance + * supervisor.start // start up all managed servers ** * @author Jonas Bonér */ -abstract class SupervisorFactory extends Logging { - def newSupervisor: Supervisor = newSupervisorFor(getSupervisorConfig) +class SupervisorFactory(val config: SupervisorConfig) extends Logging { - def newSupervisorFor(config: SupervisorConfig): Supervisor = config match { + def newInstance: Supervisor = newInstanceFor(config) + + def newInstanceFor(config: SupervisorConfig): Supervisor = config match { case SupervisorConfig(restartStrategy, _) => val supervisor = create(restartStrategy) supervisor.start @@ -77,12 +54,6 @@ abstract class SupervisorFactory extends Logging { supervisor } - /** - * To be overridden by concrete factory. - * Should return the SupervisorConfig for the supervisor. - */ - protected def getSupervisorConfig: SupervisorConfig - protected def create(strategy: RestartStrategy): Supervisor = strategy match { case RestartStrategy(scheme, maxNrOfRetries, timeRange) => scheme match { @@ -92,21 +63,27 @@ abstract class SupervisorFactory extends Logging { } } +object SupervisorFactory { + def apply(config: SupervisorConfig) = new SupervisorFactory(config) +} + /** * NOTE: * - * The supervisor class is only used for the configuration system when configuring supervisor hierarchies declaratively. - * Should not be used in development. Instead wire the actors together using 'link', 'spawnLink' etc. and set the 'trapExit' - * flag in the actors that should trap error signals and trigger restart. + * The supervisor class is only used for the configuration system when configuring supervisor + * hierarchies declaratively. Should not be used as part of the regular programming API. Instead + * wire the actors together using 'link', 'spawnLink' etc. and set the 'trapExit' flag in the + * actors that should trap error signals and trigger restart. * * See the ScalaDoc for the SupervisorFactory for an example on how to declaratively wire up actors. * * @author Jonas Bonér */ -class Supervisor private[akka] (handler: FaultHandlingStrategy) extends Actor with Logging with Configurator { +sealed class Supervisor private[akka] (handler: FaultHandlingStrategy) + extends Actor with Logging with Configurator { trapExit = List(classOf[Throwable]) faultHandler = Some(handler) - //dispatcher = Dispatchers.newThreadBasedDispatcher(this) + dispatcher = Dispatchers.newThreadBasedDispatcher(this) val actors = new ConcurrentHashMap[String, Actor] @@ -116,37 +93,39 @@ class Supervisor private[akka] (handler: FaultHandlingStrategy) extends Actor wi def isDefined(clazz: Class[_]): Boolean = actors.containsKey(clazz.getName) - def startSupervisor = { + override def start = synchronized { ConfiguratorRepository.registerConfigurator(this) - actors.values.toArray.toList.foreach(println) - start - this ! StartSupervisor + _linkedActors.toArray.toList.asInstanceOf[List[Actor]].foreach { actor => + actor.start + log.info("Starting actor: %s", actor) + } + super[Actor].start } - def stopSupervisor = this ! StopSupervisor + override def stop = synchronized { + super[Actor].stop + _linkedActors.toArray.toList.asInstanceOf[List[Actor]].foreach { actor => + actor.stop + log.info("Shutting actor down: %s", actor) + } + log.info("Stopping supervisor: %s", this) + } - protected def receive: PartialFunction[Any, Unit] = { - case StartSupervisor => - _linkedActors.toArray.toList.asInstanceOf[List[Actor]].foreach { actor => actor.start; log.info("Starting actor: %s", actor) } - - case StopSupervisor => - _linkedActors.toArray.toList.asInstanceOf[List[Actor]].foreach { actor => actor.stop; log.info("Stopping actor: %s", actor) } - log.info("Stopping supervisor: %s", this) - stop + def receive = { + case unknown => throw new IllegalArgumentException("Supervisor does not respond to any messages. Unknown message [" + unknown + "]") } def configure(config: SupervisorConfig, factory: SupervisorFactory) = config match { case SupervisorConfig(_, servers) => servers.map(server => server match { - case Supervise(actor, lifecycle) => + case Supervise(actor, lifeCycle) => actors.put(actor.getClass.getName, actor) - actor.lifeCycle = Some(lifecycle) + actor.lifeCycle = lifeCycle startLink(actor) case SupervisorConfig(_, _) => // recursive configuration - val supervisor = factory.newSupervisorFor(server.asInstanceOf[SupervisorConfig]) - supervisor ! StartSupervisor + factory.newInstanceFor(server.asInstanceOf[SupervisorConfig]).start // FIXME what to do with recursively supervisors? }) } diff --git a/akka-actors/src/main/scala/config/ActiveObjectGuiceConfigurator.scala b/akka-actors/src/main/scala/config/ActiveObjectGuiceConfigurator.scala index 2a0d454d22..04f90c331d 100644 --- a/akka-actors/src/main/scala/config/ActiveObjectGuiceConfigurator.scala +++ b/akka-actors/src/main/scala/config/ActiveObjectGuiceConfigurator.scala @@ -133,7 +133,7 @@ private[akka] class ActiveObjectGuiceConfigurator extends ActiveObjectConfigurat supervisor = Some(ActiveObject.supervise(restartStrategy, supervised)) //camelContext.addComponent(AKKA_CAMEL_ROUTING_SCHEME, new ActiveObjectComponent(this)) //camelContext.start - supervisor.get.startSupervisor + supervisor.get.start ConfiguratorRepository.registerConfigurator(this) this } diff --git a/akka-actors/src/main/scala/dispatch/Dispatchers.scala b/akka-actors/src/main/scala/dispatch/Dispatchers.scala index ed4bd6d704..1209efe5c8 100644 --- a/akka-actors/src/main/scala/dispatch/Dispatchers.scala +++ b/akka-actors/src/main/scala/dispatch/Dispatchers.scala @@ -39,7 +39,9 @@ import se.scalablesolutions.akka.actor.Actor * @author Jonas Bonér */ object Dispatchers { - + + object globalEventBasedThreadPoolDispatcher extends EventBasedThreadPoolDispatcher("global:eventbased:dispatcher") + /** * Creates an event based dispatcher serving multiple (millions) of actors through a thread pool. * Has a fluent builder interface for configuring its semantics. diff --git a/akka-actors/src/main/scala/dispatch/MessageDispatcherBase.scala b/akka-actors/src/main/scala/dispatch/MessageDispatcherBase.scala index 4516dcda6e..52fe26601b 100644 --- a/akka-actors/src/main/scala/dispatch/MessageDispatcherBase.scala +++ b/akka-actors/src/main/scala/dispatch/MessageDispatcherBase.scala @@ -29,6 +29,10 @@ abstract class MessageDispatcherBase(val name: String) extends MessageDispatcher messageHandlers.remove(key) } + def canBeShutDown: Boolean = guard.synchronized { + messageHandlers.isEmpty + } + def shutdown = if (active) { active = false selectorThread.interrupt diff --git a/akka-actors/src/main/scala/dispatch/Reactor.scala b/akka-actors/src/main/scala/dispatch/Reactor.scala index 36be55a391..36a1dfb989 100644 --- a/akka-actors/src/main/scala/dispatch/Reactor.scala +++ b/akka-actors/src/main/scala/dispatch/Reactor.scala @@ -25,6 +25,7 @@ trait MessageDispatcher { def messageQueue: MessageQueue def registerHandler(key: AnyRef, handler: MessageInvoker) def unregisterHandler(key: AnyRef) + def canBeShutDown: Boolean def start def shutdown } @@ -39,6 +40,7 @@ trait MessageDemultiplexer { class MessageInvocation(val receiver: Actor, val message: AnyRef, val future: Option[CompletableFutureResult], + val sender: Option[Actor], val tx: Option[Transaction]) { if (receiver == null) throw new IllegalArgumentException("receiver is null") if (message == null) throw new IllegalArgumentException("message is null") @@ -65,6 +67,10 @@ class MessageInvocation(val receiver: Actor, } override def toString(): String = synchronized { - "MessageInvocation[message = " + message + ", receiver = " + receiver + ", future = " + future + ", tx = " + tx + "]" + "MessageInvocation[message = " + message + + ", receiver = " + receiver + + ", sender = " + sender + + ", future = " + future + + ", tx = " + tx + "]" } } diff --git a/akka-actors/src/main/scala/dispatch/ThreadBasedDispatcher.scala b/akka-actors/src/main/scala/dispatch/ThreadBasedDispatcher.scala index a8fc5ada6b..86c7e0ed09 100644 --- a/akka-actors/src/main/scala/dispatch/ThreadBasedDispatcher.scala +++ b/akka-actors/src/main/scala/dispatch/ThreadBasedDispatcher.scala @@ -39,6 +39,8 @@ class ThreadBasedDispatcher private[akka] (val name: String, val messageHandler: selectorThread.start } + def canBeShutDown = true + def shutdown = if (active) { active = false selectorThread.interrupt diff --git a/akka-actors/src/main/scala/nio/LzfCompression.scala b/akka-actors/src/main/scala/nio/LzfCompression.scala new file mode 100644 index 0000000000..d69e9856ee --- /dev/null +++ b/akka-actors/src/main/scala/nio/LzfCompression.scala @@ -0,0 +1,29 @@ +/** + * Copyright (C) 2009 Scalable Solutions. + */ + +package se.scalablesolutions.akka.nio + +import org.h2.compress.{LZFInputStream, LZFOutputStream} + +import org.jboss.netty.channel.{Channel, ChannelHandlerContext, ChannelPipelineCoverage} +import org.jboss.netty.buffer.{ChannelBufferOutputStream, ChannelBufferInputStream, ChannelBuffer} +import org.jboss.netty.handler.codec.oneone.{OneToOneEncoder, OneToOneDecoder}; + +@ChannelPipelineCoverage("all") +class LzfDecoder extends OneToOneDecoder { + override protected def decode(ctx: ChannelHandlerContext, channel: Channel, message: AnyRef) = { + if (!(message.isInstanceOf[ChannelBuffer])) message + else { + new LZFInputStream(new ChannelBufferInputStream(message.asInstanceOf[ChannelBuffer])) + } + } +} + +@ChannelPipelineCoverage("all") +class LzfEncoder extends OneToOneEncoder { + override protected def encode(ctx: ChannelHandlerContext, channel: Channel, message: AnyRef) = { + if (!(message.isInstanceOf[ChannelBuffer])) message + else new LZFOutputStream(new ChannelBufferOutputStream(message.asInstanceOf[ChannelBuffer])) + } +} diff --git a/akka-actors/src/main/scala/nio/RemoteClient.scala b/akka-actors/src/main/scala/nio/RemoteClient.scala index c25782fe31..3743699b3f 100644 --- a/akka-actors/src/main/scala/nio/RemoteClient.scala +++ b/akka-actors/src/main/scala/nio/RemoteClient.scala @@ -6,23 +6,24 @@ package se.scalablesolutions.akka.nio import scala.collection.mutable.HashMap -import protobuf.RemoteProtocol.{RemoteRequest, RemoteReply} +import se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.{RemoteRequest, RemoteReply} import se.scalablesolutions.akka.actor.{Exit, Actor} import se.scalablesolutions.akka.dispatch.{DefaultCompletableFutureResult, CompletableFutureResult} import se.scalablesolutions.akka.util.Logging import se.scalablesolutions.akka.Config.config -import org.jboss.netty.bootstrap.ClientBootstrap import org.jboss.netty.channel._ import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory +import org.jboss.netty.bootstrap.ClientBootstrap import org.jboss.netty.handler.codec.frame.{LengthFieldBasedFrameDecoder, LengthFieldPrepender} +import org.jboss.netty.handler.codec.compression.{ZlibDecoder, ZlibEncoder} import org.jboss.netty.handler.codec.protobuf.{ProtobufDecoder, ProtobufEncoder} import org.jboss.netty.handler.timeout.ReadTimeoutHandler import org.jboss.netty.util.{TimerTask, Timeout, HashedWheelTimer} import java.net.InetSocketAddress import java.util.concurrent.{TimeUnit, Executors, ConcurrentMap, ConcurrentHashMap} - + /** * @author Jonas Bonér */ @@ -79,8 +80,7 @@ class RemoteClient(hostname: String, port: Int) extends Logging { // Wait until the connection attempt succeeds or fails. connection.awaitUninterruptibly if (!connection.isSuccess) { - log.error("Remote connection to [%s:%s] has failed due to [%s]", hostname, port, connection.getCause) - connection.getCause.printStackTrace + log.error(connection.getCause, "Remote connection to [%s:%s] has failed", hostname, port) } isRunning = true } @@ -128,8 +128,18 @@ class RemoteClientPipelineFactory(name: String, def getPipeline: ChannelPipeline = { val pipeline = Channels.pipeline() pipeline.addLast("timeout", new ReadTimeoutHandler(RemoteClient.TIMER, RemoteClient.READ_TIMEOUT)) + RemoteServer.COMPRESSION_SCHEME match { + case "zlib" => pipeline.addLast("zlibDecoder", new ZlibDecoder) + //case "lzf" => pipeline.addLast("lzfDecoder", new LzfDecoder) + case _ => {} // no compression + } pipeline.addLast("frameDecoder", new LengthFieldBasedFrameDecoder(1048576, 0, 4, 0, 4)) pipeline.addLast("protobufDecoder", new ProtobufDecoder(RemoteReply.getDefaultInstance)) + RemoteServer.COMPRESSION_SCHEME match { + case "zlib" => pipeline.addLast("zlibEncoder", new ZlibEncoder(RemoteServer.ZLIB_COMPRESSION_LEVEL)) + //case "lzf" => pipeline.addLast("lzfEncoder", new LzfEncoder) + case _ => {} // no compression + } pipeline.addLast("frameEncoder", new LengthFieldPrepender(4)) pipeline.addLast("protobufEncoder", new ProtobufEncoder()) pipeline.addLast("handler", new RemoteClientHandler(name, futures, supervisors, bootstrap)) @@ -146,9 +156,11 @@ class RemoteClientHandler(val name: String, val supervisors: ConcurrentMap[String, Actor], val bootstrap: ClientBootstrap) extends SimpleChannelUpstreamHandler with Logging { + import Actor.Sender.Self override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = { - if (event.isInstanceOf[ChannelStateEvent] && event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) { + if (event.isInstanceOf[ChannelStateEvent] && + event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) { log.debug(event.toString) } super.handleUpstream(ctx, event) @@ -199,8 +211,7 @@ class RemoteClientHandler(val name: String, log.debug("Remote client disconnected from [%s]", ctx.getChannel.getRemoteAddress); override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { - log.error("Unexpected exception from downstream in remote client: %s", event.getCause) - event.getCause.printStackTrace + log.error(event.getCause, "Unexpected exception from downstream in remote client") event.getChannel.close } diff --git a/akka-actors/src/main/scala/nio/RemoteServer.scala b/akka-actors/src/main/scala/nio/RemoteServer.scala index 0c89ac6246..5a542268c8 100755 --- a/akka-actors/src/main/scala/nio/RemoteServer.scala +++ b/akka-actors/src/main/scala/nio/RemoteServer.scala @@ -18,17 +18,67 @@ import org.jboss.netty.channel._ import org.jboss.netty.channel.socket.nio.NioServerSocketChannelFactory import org.jboss.netty.handler.codec.frame.{LengthFieldBasedFrameDecoder, LengthFieldPrepender} import org.jboss.netty.handler.codec.protobuf.{ProtobufDecoder, ProtobufEncoder} +import org.jboss.netty.handler.codec.compression.{ZlibEncoder, ZlibDecoder} /** + * Use this object if you need a single remote server on a specific node. + * + *
+ * RemoteNode.start + *+ * + * If you need to create more than one, then you can use the RemoteServer: + * + *
+ * val server = new RemoteServer + * server.start + *+ * * @author Jonas Bonér */ -object RemoteServer extends Logging { +object RemoteNode extends RemoteServer + +/** + * This object holds configuration variables. + * + * @author Jonas Bonér + */ +object RemoteServer { val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost") val PORT = config.getInt("akka.remote.server.port", 9999) - val CONNECTION_TIMEOUT_MILLIS = config.getInt("akka.remote.server.connection-timeout", 1000) - private var hostname = HOSTNAME - private var port = PORT + val CONNECTION_TIMEOUT_MILLIS = config.getInt("akka.remote.server.connection-timeout", 1000) + + val COMPRESSION_SCHEME = config.getString("akka.remote.compression-scheme", "zlib") + val ZLIB_COMPRESSION_LEVEL = { + val level = config.getInt("akka.remote.zlib-compression-level", 6) + if (level < 1 && level > 9) throw new IllegalArgumentException( + "zlib compression level has to be within 1-9, with 1 being fastest and 9 being the most compressed") + level + } +} + +/** + * Use this class if you need a more than one remote server on a specific node. + * + *
+ * val server = new RemoteServer + * server.start + *+ * + * If you need to create more than one, then you can use the RemoteServer: + * + *
+ * RemoteNode.start + *+ * + * @author Jonas Bonér + */ +class RemoteServer extends Logging { + val name = "RemoteServer@" + hostname + ":" + port + + private var hostname = RemoteServer.HOSTNAME + private var port = RemoteServer.PORT @volatile private var isRunning = false @volatile private var isConfigured = false @@ -39,13 +89,11 @@ object RemoteServer extends Logging { private val bootstrap = new ServerBootstrap(factory) - def name = "RemoteServer@" + hostname + ":" + port - def start: Unit = start(None) - def start(loader: Option[ClassLoader]): Unit = start(HOSTNAME, PORT, loader) + def start(loader: Option[ClassLoader]): Unit = start(hostname, port, loader) - def start(hostname: String, port: Int): Unit = start(hostname, port, None) + def start(_hostname: String, _port: Int): Unit = start(_hostname, _port, None) def start(_hostname: String, _port: Int, loader: Option[ClassLoader]): Unit = synchronized { if (!isRunning) { @@ -53,29 +101,47 @@ object RemoteServer extends Logging { port = _port log.info("Starting remote server at [%s:%s]", hostname, port) bootstrap.setPipelineFactory(new RemoteServerPipelineFactory(name, loader)) + // FIXME make these RemoteServer options configurable bootstrap.setOption("child.tcpNoDelay", true) bootstrap.setOption("child.keepAlive", true) bootstrap.setOption("child.reuseAddress", true) - bootstrap.setOption("child.connectTimeoutMillis", CONNECTION_TIMEOUT_MILLIS) + bootstrap.setOption("child.connectTimeoutMillis", RemoteServer.CONNECTION_TIMEOUT_MILLIS) bootstrap.bind(new InetSocketAddress(hostname, port)) isRunning = true } } + + def shutdown = { + bootstrap.releaseExternalResources + } } /** * @author Jonas Bonér */ -class RemoteServerPipelineFactory(name: String, loader: Option[ClassLoader]) extends ChannelPipelineFactory { +class RemoteServerPipelineFactory(name: String, loader: Option[ClassLoader]) + extends ChannelPipelineFactory { + import RemoteServer._ + def getPipeline: ChannelPipeline = { - val p = Channels.pipeline() - p.addLast("frameDecoder", new LengthFieldBasedFrameDecoder(1048576, 0, 4, 0, 4)) - p.addLast("protobufDecoder", new ProtobufDecoder(RemoteRequest.getDefaultInstance)) - p.addLast("frameEncoder", new LengthFieldPrepender(4)) - p.addLast("protobufEncoder", new ProtobufEncoder) - p.addLast("handler", new RemoteServerHandler(name, loader)) - p + val pipeline = Channels.pipeline() + RemoteServer.COMPRESSION_SCHEME match { + case "zlib" => pipeline.addLast("zlibDecoder", new ZlibDecoder) + //case "lzf" => pipeline.addLast("lzfDecoder", new LzfDecoder) + case _ => {} // no compression + } + pipeline.addLast("frameDecoder", new LengthFieldBasedFrameDecoder(1048576, 0, 4, 0, 4)) + pipeline.addLast("protobufDecoder", new ProtobufDecoder(RemoteRequest.getDefaultInstance)) + RemoteServer.COMPRESSION_SCHEME match { + case "zlib" => pipeline.addLast("zlibEncoder", new ZlibEncoder(RemoteServer.ZLIB_COMPRESSION_LEVEL)) + //case "lzf" => pipeline.addLast("lzfEncoder", new LzfEncoder) + case _ => {} // no compression + } + pipeline.addLast("frameEncoder", new LengthFieldPrepender(4)) + pipeline.addLast("protobufEncoder", new ProtobufEncoder) + pipeline.addLast("handler", new RemoteServerHandler(name, loader)) + pipeline } } @@ -83,12 +149,16 @@ class RemoteServerPipelineFactory(name: String, loader: Option[ClassLoader]) ext * @author Jonas Bonér */ @ChannelPipelineCoverage { val value = "all" } -class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassLoader]) extends SimpleChannelUpstreamHandler with Logging { +class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassLoader]) + extends SimpleChannelUpstreamHandler with Logging { + val AW_PROXY_PREFIX = "$$ProxiedByAW".intern + private val activeObjects = new ConcurrentHashMap[String, AnyRef] private val actors = new ConcurrentHashMap[String, Actor] override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = { - if (event.isInstanceOf[ChannelStateEvent] && event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) { + if (event.isInstanceOf[ChannelStateEvent] && + event.asInstanceOf[ChannelStateEvent].getState != ChannelState.INTEREST_OPS) { log.debug(event.toString) } super.handleUpstream(ctx, event) @@ -96,13 +166,15 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL override def messageReceived(ctx: ChannelHandlerContext, event: MessageEvent) = { val message = event.getMessage - if (message == null) throw new IllegalStateException("Message in remote MessageEvent is null: " + event) - if (message.isInstanceOf[RemoteRequest]) handleRemoteRequest(message.asInstanceOf[RemoteRequest], event.getChannel) + if (message == null) throw new IllegalStateException( + "Message in remote MessageEvent is null: " + event) + if (message.isInstanceOf[RemoteRequest]) { + handleRemoteRequest(message.asInstanceOf[RemoteRequest], event.getChannel) + } } override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = { - log.error("Unexpected exception from remote downstream: %s", event.getCause) - event.getCause.printStackTrace + log.error(event.getCause, "Unexpected exception from remote downstream") event.getChannel.close } @@ -114,10 +186,12 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL private def dispatchToActor(request: RemoteRequest, channel: Channel) = { log.debug("Dispatching to remote actor [%s]", request.getTarget) - val actor = createActor(request.getTarget, request.getTimeout) + val actor = createActor(request.getTarget, request.getUuid, request.getTimeout) actor.start val message = RemoteProtocolBuilder.getMessage(request) - if (request.getIsOneWay) actor ! message + if (request.getIsOneWay) { + actor.send(message) + } else { try { val resultOrNone = actor !! message @@ -133,8 +207,7 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL channel.write(replyMessage) } catch { case e: Throwable => - log.error("Could not invoke remote actor [%s] due to: %s", request.getTarget, e) - e.printStackTrace + log.error(e, "Could not invoke remote actor [%s]", request.getTarget) val replyBuilder = RemoteReply.newBuilder .setId(request.getId) .setException(e.getClass.getName + "$" + e.getMessage) @@ -157,7 +230,8 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL //continueTransaction(request) try { - val messageReceiver = activeObject.getClass.getDeclaredMethod(request.getMethod, unescapedArgClasses: _*) + val messageReceiver = activeObject.getClass.getDeclaredMethod( + request.getMethod, unescapedArgClasses: _*) if (request.getIsOneWay) messageReceiver.invoke(activeObject, unescapedArgs: _*) else { val result = messageReceiver.invoke(activeObject, unescapedArgs: _*) @@ -173,8 +247,7 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL } } catch { case e: InvocationTargetException => - log.error("Could not invoke remote active object [%s :: %s] due to: %s", request.getMethod, request.getTarget, e.getCause) - e.getCause.printStackTrace + log.error(e.getCause, "Could not invoke remote active object [%s :: %s]", request.getMethod, request.getTarget) val replyBuilder = RemoteReply.newBuilder .setId(request.getId) .setException(e.getCause.getClass.getName + "$" + e.getCause.getMessage) @@ -184,8 +257,7 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL val replyMessage = replyBuilder.build channel.write(replyMessage) case e: Throwable => - log.error("Could not invoke remote active object [%s :: %s] due to: %s", request.getMethod, request.getTarget, e) - e.printStackTrace + log.error(e.getCause, "Could not invoke remote active object [%s :: %s]", request.getMethod, request.getTarget) val replyBuilder = RemoteReply.newBuilder .setId(request.getId) .setException(e.getClass.getName + "$" + e.getMessage) @@ -216,9 +288,9 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL val escapedArgs = for (i <- 0 until args.size) { val arg = args(i) - if (arg.isInstanceOf[String] && arg.asInstanceOf[String].startsWith("$$ProxiedByAW")) { + if (arg.isInstanceOf[String] && arg.asInstanceOf[String].startsWith(AW_PROXY_PREFIX)) { val argString = arg.asInstanceOf[String] - val proxyName = argString.replace("$$ProxiedByAW", "") //argString.substring(argString.indexOf("$$ProxiedByAW"), argString.length) + val proxyName = argString.replace(AW_PROXY_PREFIX, "") //argString.substring(argString.indexOf("$$ProxiedByAW"), argString.length) val activeObject = createActiveObject(proxyName, timeout) unescapedArgs(i) = activeObject unescapedArgClasses(i) = Class.forName(proxyName) @@ -242,29 +314,28 @@ class RemoteServerHandler(val name: String, val applicationLoader: Option[ClassL newInstance } catch { case e => - log.debug("Could not create remote active object instance due to: %s", e) - e.printStackTrace + log.error(e, "Could not create remote active object instance") throw e } } else activeObjectOrNull } - private def createActor(name: String, timeout: Long): Actor = { - val actorOrNull = actors.get(name) + private def createActor(name: String, uuid: String, timeout: Long): Actor = { + val actorOrNull = actors.get(uuid) if (actorOrNull == null) { try { - log.info("Creating a new remote actor [%s]", name) + log.info("Creating a new remote actor [%s:%s]", name, uuid) val clazz = if (applicationLoader.isDefined) applicationLoader.get.loadClass(name) else Class.forName(name) val newInstance = clazz.newInstance.asInstanceOf[Actor] + newInstance._uuid = uuid newInstance.timeout = timeout newInstance._remoteAddress = None - actors.put(name, newInstance) + actors.put(uuid, newInstance) newInstance } catch { case e => - log.debug("Could not create remote actor instance due to: %s", e) - e.printStackTrace + log.error(e, "Could not create remote actor object instance") throw e } } else actorOrNull diff --git a/akka-actors/src/main/scala/stm/DataFlowVariable.scala b/akka-actors/src/main/scala/stm/DataFlowVariable.scala index ec47f0983f..44a40f50af 100644 --- a/akka-actors/src/main/scala/stm/DataFlowVariable.scala +++ b/akka-actors/src/main/scala/stm/DataFlowVariable.scala @@ -24,11 +24,11 @@ object DataFlow { thread } - def thread[MessageType, ReturnType](body: MessageType => ReturnType) = + def thread[MessageType, ReturnType](body: MessageType => ReturnType) = new ReactiveEventBasedThread(body).start private class IsolatedEventBasedThread(body: => Unit) extends Actor { - def act = loop { + def act = loop { react { case 'start => body case 'exit => exit() @@ -37,7 +37,7 @@ object DataFlow { } private class ReactiveEventBasedThread[MessageType, ReturnType](body: MessageType => ReturnType) extends Actor { - def act = loop { + def act = loop { react { case 'exit => exit() case message => sender ! body(message.asInstanceOf[MessageType]) @@ -48,7 +48,7 @@ object DataFlow { /** * @author Jonas Bonér */ - sealed class DataFlowVariable[T] { + sealed class DataFlowVariable[T] { private sealed abstract class DataFlowVariableMessage private case class Set[T](value: T) extends DataFlowVariableMessage @@ -73,7 +73,7 @@ object DataFlow { private class Out[T](dataFlow: DataFlowVariable[T]) extends Actor { var reader: Option[OutputChannel[Any]] = None def act = loop { react { - case Get => + case Get => val ref = dataFlow.value.get if (ref.isDefined) reply(ref.get) else reader = Some(sender) case Set(v) => if (reader.isDefined) reader.get ! v @@ -380,5 +380,3 @@ object Test5 extends Application { //System.gc } - - diff --git a/akka-actors/src/main/scala/stm/Transaction.scala b/akka-actors/src/main/scala/stm/Transaction.scala index 20ea9ede9f..d34661c37a 100644 --- a/akka-actors/src/main/scala/stm/Transaction.scala +++ b/akka-actors/src/main/scala/stm/Transaction.scala @@ -12,7 +12,7 @@ import se.scalablesolutions.akka.util.Logging import org.multiverse.api.{Stm, Transaction => MultiverseTransaction} import org.multiverse.api.GlobalStmInstance.getGlobalStmInstance -import org.multiverse.utils.ThreadLocalTransaction._ +import org.multiverse.api.ThreadLocalTransaction._ import org.multiverse.templates.OrElseTemplate import scala.collection.mutable.HashMap @@ -21,25 +21,37 @@ class NoTransactionInScopeException extends RuntimeException class TransactionRetryException(message: String) extends RuntimeException(message) /** - * Example of atomic transaction management using high-order functions: + * Example of atomic transaction management using the atomic block: *
* import se.scalablesolutions.akka.stm.Transaction._
+ *
* atomic {
* .. // do something within a transaction
* }
*
*
- * Example of Run-OrElse transaction management.
+ * Example of atomic transaction management using atomic block with retry count:
*
* import se.scalablesolutions.akka.stm.Transaction._
- * run {
+ *
+ * atomic(maxNrOfRetries) {
+ * .. // do something within a transaction
+ * }
+ *
+ *
+ * Example of atomically-orElse transaction management.
+ * Which is a good way to reduce contention and transaction collisions.
+ *
+ * import se.scalablesolutions.akka.stm.Transaction._
+ *
+ * atomically {
* .. // try to do something
* } orElse {
* .. // if transaction clashes try do do something else to minimize contention
* }
*
*
- * Example of atomic transaction management using for comprehensions (monadic usage):
+ * Example of atomic transaction management using for comprehensions (monadic):
*
* * import se.scalablesolutions.akka.stm.Transaction._ @@ -52,7 +64,7 @@ class TransactionRetryException(message: String) extends RuntimeException(messag * } ** - * Example of using Transaction and TransactionalRef in for comprehensions (monadic usage): + * Example of using Transaction and TransactionalRef in for comprehensions (monadic): * *
* // For example, if you have a List with TransactionalRef
@@ -79,19 +91,33 @@ class TransactionRetryException(message: String) extends RuntimeException(messag
object Transaction extends TransactionManagement {
val idFactory = new AtomicLong(-1L)
- // -- monadic api --------------------------
+ /**
+ * See ScalaDoc on class.
+ */
def map[T](f: Transaction => T): T = atomic { f(getTransactionInScope) }
+ /**
+ * See ScalaDoc on class.
+ */
def flatMap[T](f: Transaction => T): T = atomic { f(getTransactionInScope) }
+ /**
+ * See ScalaDoc on class.
+ */
def foreach(f: Transaction => Unit): Unit = atomic { f(getTransactionInScope) }
- // -- atomic block --------------------------
- private[akka] def atomically[T](body: => T): T = new AtomicTemplate[T](
+ /**
+ * Creates a "pure" STM atomic transaction and by-passes all transactions hooks such as persistence etc.
+ * Only for internal usage.
+ */
+ private[akka] def pureAtomic[T](body: => T): T = new AtomicTemplate[T](
getGlobalStmInstance, "akka", false, false, TransactionManagement.MAX_NR_OF_RETRIES) {
def execute(mtx: MultiverseTransaction): T = body
}.execute()
+ /**
+ * See ScalaDoc on class.
+ */
def atomic[T](body: => T): T = new AtomicTemplate[T](
getGlobalStmInstance, "akka", false, false, TransactionManagement.MAX_NR_OF_RETRIES) {
def execute(mtx: MultiverseTransaction): T = body
@@ -106,6 +132,9 @@ object Transaction extends TransactionManagement {
}
}.execute()
+ /**
+ * See ScalaDoc on class.
+ */
def atomic[T](retryCount: Int)(body: => T): T = {
new AtomicTemplate[T](getGlobalStmInstance, "akka", false, false, retryCount) {
def execute(mtx: MultiverseTransaction): T = body
@@ -121,6 +150,9 @@ object Transaction extends TransactionManagement {
}.execute
}
+ /**
+ * See ScalaDoc on class.
+ */
def atomicReadOnly[T](retryCount: Int)(body: => T): T = {
new AtomicTemplate[T](getGlobalStmInstance, "akka", false, true, retryCount) {
def execute(mtx: MultiverseTransaction): T = body
@@ -136,6 +168,9 @@ object Transaction extends TransactionManagement {
}.execute
}
+ /**
+ * See ScalaDoc on class.
+ */
def atomicReadOnly[T](body: => T): T = {
new AtomicTemplate[T](true) {
def execute(mtx: MultiverseTransaction): T = body
@@ -151,12 +186,19 @@ object Transaction extends TransactionManagement {
}.execute
}
- // -- Run-OrElse --------------------------
- def run[A](orBody: => A) = elseBody(orBody)
- def elseBody[A](orBody: => A) = new {
- def orElse(elseBody: => A) = new OrElseTemplate[A] {
- def run(t: MultiverseTransaction) = orBody
- def orelserun(t: MultiverseTransaction) = elseBody
+ /**
+ * See ScalaDoc on class.
+ */
+ def atomically[A](firstBody: => A) = elseBody(firstBody)
+
+ /**
+ * Should only be used together with atomically to form atomically-orElse constructs.
+ * See ScalaDoc on class.
+ */
+ def elseBody[A](firstBody: => A) = new {
+ def orElse(secondBody: => A) = new OrElseTemplate[A] {
+ def run(t: MultiverseTransaction) = firstBody
+ def orelserun(t: MultiverseTransaction) = secondBody
}.execute()
}
}
@@ -176,7 +218,7 @@ object Transaction extends TransactionManagement {
// --- public methods ---------
def commit = synchronized {
- atomically {
+ pureAtomic {
persistentStateMap.values.foreach(_.commit)
TransactionManagement.clearTransaction
}
diff --git a/akka-actors/src/main/scala/stm/TransactionManagement.scala b/akka-actors/src/main/scala/stm/TransactionManagement.scala
index 06e65d0ceb..6a3ee9abf8 100644
--- a/akka-actors/src/main/scala/stm/TransactionManagement.scala
+++ b/akka-actors/src/main/scala/stm/TransactionManagement.scala
@@ -10,7 +10,7 @@ import se.scalablesolutions.akka.util.Logging
import scala.collection.mutable.HashSet
-import org.multiverse.utils.ThreadLocalTransaction._
+import org.multiverse.api.ThreadLocalTransaction._
class StmException(msg: String) extends RuntimeException(msg)
diff --git a/akka-actors/src/main/scala/stm/TransactionalState.scala b/akka-actors/src/main/scala/stm/TransactionalState.scala
index de3b97e794..bd89a41224 100644
--- a/akka-actors/src/main/scala/stm/TransactionalState.scala
+++ b/akka-actors/src/main/scala/stm/TransactionalState.scala
@@ -74,7 +74,7 @@ object TransactionalRef {
* @author Jonas Bonér
*/
class TransactionalRef[T] extends Transactional {
- import org.multiverse.utils.ThreadLocalTransaction._
+ import org.multiverse.api.ThreadLocalTransaction._
private[this] val ref: Ref[T] = atomic { new Ref }
diff --git a/akka-actors/src/test/scala/ActorFireForgetRequestReplyTest.scala b/akka-actors/src/test/scala/ActorFireForgetRequestReplyTest.scala
new file mode 100644
index 0000000000..2804e588d8
--- /dev/null
+++ b/akka-actors/src/test/scala/ActorFireForgetRequestReplyTest.scala
@@ -0,0 +1,53 @@
+package se.scalablesolutions.akka.actor
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+
+object state {
+ var s = "NIL"
+}
+
+class ReplyActor extends Actor {
+ def receive = {
+ case "Send" => reply("Reply")
+ case "SendImplicit" => sender.get ! "ReplyImplicit"
+ }
+}
+
+class SenderActor(replyActor: Actor) extends Actor {
+ def receive = {
+ case "Init" => replyActor ! "Send"
+ case "Reply" => state.s = "Reply"
+ case "InitImplicit" => replyActor ! "SendImplicit"
+ case "ReplyImplicit" => state.s = "ReplyImplicit"
+ }
+}
+
+class ActorFireForgetRequestReplyTest extends JUnitSuite {
+
+ @Test
+ def shouldReplyToBangMessageUsingReply = {
+ import Actor.Sender.Self
+
+ val replyActor = new ReplyActor
+ replyActor.start
+ val senderActor = new SenderActor(replyActor)
+ senderActor.start
+ senderActor ! "Init"
+ Thread.sleep(1000)
+ assert("Reply" === state.s)
+ }
+
+ @Test
+ def shouldReplyToBangMessageUsingImplicitSender = {
+ import Actor.Sender.Self
+
+ val replyActor = new ReplyActor
+ replyActor.start
+ val senderActor = new SenderActor(replyActor)
+ senderActor.start
+ senderActor ! "InitImplicit"
+ Thread.sleep(1000)
+ assert("ReplyImplicit" === state.s)
+ }
+}
diff --git a/akka-actors/src/test/scala/EventBasedSingleThreadActorTest.scala b/akka-actors/src/test/scala/EventBasedSingleThreadActorTest.scala
index d8a3917293..e556a1a724 100644
--- a/akka-actors/src/test/scala/EventBasedSingleThreadActorTest.scala
+++ b/akka-actors/src/test/scala/EventBasedSingleThreadActorTest.scala
@@ -8,12 +8,14 @@ import org.junit.Test
import se.scalablesolutions.akka.dispatch.Dispatchers
class EventBasedSingleThreadActorTest extends JUnitSuite {
+ import Actor.Sender.Self
+
private val unit = TimeUnit.MILLISECONDS
class TestActor extends Actor {
dispatcher = Dispatchers.newEventBasedSingleThreadDispatcher(uuid)
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "Hello" =>
reply("World")
case "Failure" =>
@@ -25,7 +27,7 @@ class EventBasedSingleThreadActorTest extends JUnitSuite {
implicit val timeout = 5000L
var oneWay = "nada"
val actor = new Actor {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "OneWay" => oneWay = "received"
}
}
@@ -40,7 +42,7 @@ class EventBasedSingleThreadActorTest extends JUnitSuite {
implicit val timeout = 5000L
val actor = new TestActor
actor.start
- val result: String = actor !? "Hello"
+ val result: String = (actor !! ("Hello", 10000)).get
assert("World" === result)
actor.stop
}
diff --git a/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala b/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala
index af62f475e0..649d95f4d2 100644
--- a/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala
+++ b/akka-actors/src/test/scala/EventBasedSingleThreadDispatcherTest.scala
@@ -50,9 +50,9 @@ class EventBasedSingleThreadDispatcherTest extends JUnitSuite {
internalTestMessagesDispatchedToHandlersAreExecutedInFIFOOrder
}
- val key1 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
- val key2 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
- val key3 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
+ val key1 = new Actor { def receive = { case _ => {}} }
+ val key2 = new Actor { def receive = { case _ => {}} }
+ val key3 = new Actor { def receive = { case _ => {}} }
private def internalTestMessagesDispatchedToTheSameHandlerAreExecutedSequentially: Unit = {
val guardLock = new ReentrantLock
@@ -61,7 +61,7 @@ class EventBasedSingleThreadDispatcherTest extends JUnitSuite {
dispatcher.registerHandler(key1, new TestMessageHandle(handleLatch))
dispatcher.start
for (i <- 0 until 100) {
- dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None, None))
}
assert(handleLatch.await(5, TimeUnit.SECONDS))
assert(!threadingIssueDetected.get)
@@ -73,8 +73,8 @@ class EventBasedSingleThreadDispatcherTest extends JUnitSuite {
dispatcher.registerHandler(key1, new TestMessageHandle(handleLatch))
dispatcher.registerHandler(key2, new TestMessageHandle(handleLatch))
dispatcher.start
- dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None))
- dispatcher.messageQueue.append(new MessageInvocation(key2, new Object, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key2, new Object, None, None, None))
assert(handleLatch.await(5, TimeUnit.SECONDS))
assert(!threadingIssueDetected.get)
}
@@ -106,8 +106,8 @@ class EventBasedSingleThreadDispatcherTest extends JUnitSuite {
})
dispatcher.start
for (i <- 0 until 100) {
- dispatcher.messageQueue.append(new MessageInvocation(key1, new java.lang.Integer(i), None, None))
- dispatcher.messageQueue.append(new MessageInvocation(key2, new java.lang.Integer(i), None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key1, new java.lang.Integer(i), None, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key2, new java.lang.Integer(i), None, None, None))
}
assert(handleLatch.await(5, TimeUnit.SECONDS))
assert(!threadingIssueDetected.get)
diff --git a/akka-actors/src/test/scala/EventBasedThreadPoolActorTest.scala b/akka-actors/src/test/scala/EventBasedThreadPoolActorTest.scala
index 168a20ff9c..2d90145810 100644
--- a/akka-actors/src/test/scala/EventBasedThreadPoolActorTest.scala
+++ b/akka-actors/src/test/scala/EventBasedThreadPoolActorTest.scala
@@ -6,10 +6,12 @@ import org.scalatest.junit.JUnitSuite
import org.junit.Test
class EventBasedThreadPoolActorTest extends JUnitSuite {
+ import Actor.Sender.Self
+
private val unit = TimeUnit.MILLISECONDS
class TestActor extends Actor {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "Hello" =>
reply("World")
case "Failure" =>
@@ -21,7 +23,7 @@ class EventBasedThreadPoolActorTest extends JUnitSuite {
implicit val timeout = 5000L
var oneWay = "nada"
val actor = new Actor {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "OneWay" => oneWay = "received"
}
}
@@ -36,7 +38,7 @@ class EventBasedThreadPoolActorTest extends JUnitSuite {
implicit val timeout = 5000L
val actor = new TestActor
actor.start
- val result: String = actor !? "Hello"
+ val result: String = (actor !! ("Hello", 10000)).get
assert("World" === result)
actor.stop
}
diff --git a/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala b/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala
index 7391c348d9..f8c0107d05 100644
--- a/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala
+++ b/akka-actors/src/test/scala/EventBasedThreadPoolDispatcherTest.scala
@@ -13,9 +13,9 @@ import se.scalablesolutions.akka.actor.Actor
class EventBasedThreadPoolDispatcherTest extends JUnitSuite {
private var threadingIssueDetected: AtomicBoolean = null
- val key1 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
- val key2 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
- val key3 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
+ val key1 = new Actor { def receive = { case _ => {}} }
+ val key2 = new Actor { def receive = { case _ => {}} }
+ val key3 = new Actor { def receive = { case _ => {}} }
@Before
def setUp = {
@@ -67,7 +67,7 @@ class EventBasedThreadPoolDispatcherTest extends JUnitSuite {
})
dispatcher.start
for (i <- 0 until 10) {
- dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None, None))
}
assert(handleLatch.await(5, TimeUnit.SECONDS))
assert(!threadingIssueDetected.get)
@@ -109,10 +109,10 @@ class EventBasedThreadPoolDispatcherTest extends JUnitSuite {
}
})
dispatcher.start
- dispatcher.messageQueue.append(new MessageInvocation(key1, "Sending Message 1", None, None))
- dispatcher.messageQueue.append(new MessageInvocation(key1, "Sending Message 1.1", None, None))
- dispatcher.messageQueue.append(new MessageInvocation(key2, "Sending Message 2", None, None))
- dispatcher.messageQueue.append(new MessageInvocation(key2, "Sending Message 2.2", None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key1, "Sending Message 1", None, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key1, "Sending Message 1.1", None, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key2, "Sending Message 2", None, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key2, "Sending Message 2.2", None, None, None))
handlersBarrier.await(5, TimeUnit.SECONDS)
assert(!threadingIssueDetected.get)
@@ -151,8 +151,8 @@ class EventBasedThreadPoolDispatcherTest extends JUnitSuite {
})
dispatcher.start
for (i <- 0 until 100) {
- dispatcher.messageQueue.append(new MessageInvocation(key1, new java.lang.Integer(i), None, None))
- dispatcher.messageQueue.append(new MessageInvocation(key2, new java.lang.Integer(i), None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key1, new java.lang.Integer(i), None, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key2, new java.lang.Integer(i), None, None, None))
}
assert(handleLatch.await(5, TimeUnit.SECONDS))
assert(!threadingIssueDetected.get)
diff --git a/akka-actors/src/test/scala/InMemoryActorTest.scala b/akka-actors/src/test/scala/InMemoryActorTest.scala
index 83c105d69f..cd06b80d0a 100644
--- a/akka-actors/src/test/scala/InMemoryActorTest.scala
+++ b/akka-actors/src/test/scala/InMemoryActorTest.scala
@@ -30,7 +30,7 @@ class InMemStatefulActor extends Actor {
private lazy val vectorState = TransactionalState.newVector[String]
private lazy val refState = TransactionalState.newRef[String]
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case GetMapState(key) =>
reply(mapState.get(key).get)
case GetVectorSize =>
@@ -79,7 +79,7 @@ class InMemStatefulActor extends Actor {
@serializable
class InMemFailerActor extends Actor {
makeTransactionRequired
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "Failure" =>
throw new RuntimeException("expected")
}
diff --git a/akka-actors/src/test/scala/RemoteActorTest.scala b/akka-actors/src/test/scala/RemoteActorTest.scala
index a187c3f16a..3e614bd42c 100644
--- a/akka-actors/src/test/scala/RemoteActorTest.scala
+++ b/akka-actors/src/test/scala/RemoteActorTest.scala
@@ -3,22 +3,23 @@ package se.scalablesolutions.akka.actor
import java.util.concurrent.TimeUnit
import junit.framework.TestCase
-import se.scalablesolutions.akka.nio.{RemoteServer, RemoteClient}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
+import se.scalablesolutions.akka.nio.{RemoteNode, RemoteServer, RemoteClient}
+
object Global {
var oneWay = "nada"
}
class RemoteActorSpecActorUnidirectional extends Actor {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "OneWay" =>
Global.oneWay = "received"
}
}
class RemoteActorSpecActorBidirectional extends Actor {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "Hello" =>
reply("World")
case "Failure" =>
@@ -27,10 +28,12 @@ class RemoteActorSpecActorBidirectional extends Actor {
}
class RemoteActorTest extends JUnitSuite {
+ import Actor.Sender.Self
+
akka.Config.config
new Thread(new Runnable() {
def run = {
- RemoteServer.start
+ RemoteNode.start
}
}).start
Thread.sleep(1000)
@@ -49,17 +52,6 @@ class RemoteActorTest extends JUnitSuite {
actor.stop
}
- @Test
- def shouldSendReplySync = {
- implicit val timeout = 500000000L
- val actor = new RemoteActorSpecActorBidirectional
- actor.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
- actor.start
- val result: String = actor !? "Hello"
- assert("World" === result)
- actor.stop
- }
-
@Test
def shouldSendReplyAsync = {
implicit val timeout = 500000000L
diff --git a/akka-actors/src/test/scala/RemoteSupervisorTest.scala b/akka-actors/src/test/scala/RemoteSupervisorTest.scala
index 5aefa0d0a1..0fef2a2e6e 100644
--- a/akka-actors/src/test/scala/RemoteSupervisorTest.scala
+++ b/akka-actors/src/test/scala/RemoteSupervisorTest.scala
@@ -5,26 +5,28 @@
package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.serialization.BinaryString
-import se.scalablesolutions.akka.nio.{RemoteClient, RemoteServer}
import se.scalablesolutions.akka.config.ScalaConfig._
+import se.scalablesolutions.akka.nio.{RemoteNode, RemoteClient, RemoteServer}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
object Log {
var messageLog: String = ""
- var oneWayLog: String = ""
+ var oneWayLog: String = ""
}
+
/**
* @author Jonas Bonér
*/
-class RemoteSupervisorTest extends JUnitSuite {
+class RemoteSupervisorTest extends JUnitSuite {
+ import Actor.Sender.Self
akka.Config.config
new Thread(new Runnable() {
- def run = {
- RemoteServer.start
- }
+ def run = {
+ RemoteNode.start
+ }
}).start
Thread.sleep(1000)
@@ -35,7 +37,7 @@ class RemoteSupervisorTest extends JUnitSuite {
@Test def shouldStartServer = {
Log.messageLog = ""
val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
+ sup.start
expect("pong") {
(pingpong1 !! BinaryString("Ping")).getOrElse("nil")
@@ -45,7 +47,7 @@ class RemoteSupervisorTest extends JUnitSuite {
@Test def shouldKillSingleActorOneForOne = {
Log.messageLog = ""
val sup = getSingleActorOneForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong1 !! BinaryString("Die")
@@ -59,7 +61,7 @@ class RemoteSupervisorTest extends JUnitSuite {
@Test def shouldCallKillCallSingleActorOneForOne = {
Log.messageLog = ""
val sup = getSingleActorOneForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 !! BinaryString("Ping")).getOrElse("nil")
@@ -87,7 +89,7 @@ class RemoteSupervisorTest extends JUnitSuite {
@Test def shouldKillSingleActorAllForOne = {
Log.messageLog = ""
val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong1 !! BinaryString("Die")
@@ -101,7 +103,7 @@ class RemoteSupervisorTest extends JUnitSuite {
@Test def shouldCallKillCallSingleActorAllForOne = {
Log.messageLog = ""
val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 !! BinaryString("Ping")).getOrElse("nil")
@@ -129,7 +131,7 @@ class RemoteSupervisorTest extends JUnitSuite {
@Test def shouldKillMultipleActorsOneForOne = {
Log.messageLog = ""
val sup = getMultipleActorsOneForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong3 !! BinaryString("Die")
@@ -143,7 +145,7 @@ class RemoteSupervisorTest extends JUnitSuite {
def tesCallKillCallMultipleActorsOneForOne = {
Log.messageLog = ""
val sup = getMultipleActorsOneForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 !! BinaryString("Ping")).getOrElse("nil")
@@ -187,7 +189,7 @@ class RemoteSupervisorTest extends JUnitSuite {
@Test def shouldKillMultipleActorsAllForOne = {
Log.messageLog = ""
val sup = getMultipleActorsAllForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong2 !! BinaryString("Die")
@@ -201,7 +203,7 @@ class RemoteSupervisorTest extends JUnitSuite {
def tesCallKillCallMultipleActorsAllForOne = {
Log.messageLog = ""
val sup = getMultipleActorsAllForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 !! BinaryString("Ping")).getOrElse("nil")
@@ -246,7 +248,7 @@ class RemoteSupervisorTest extends JUnitSuite {
@Test def shouldOneWayKillSingleActorOneForOne = {
Logg.messageLog = ""
val sup = getSingleActorOneForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
pingpong1 ! BinaryString("Die")
Thread.sleep(500)
@@ -258,7 +260,7 @@ class RemoteSupervisorTest extends JUnitSuite {
@Test def shouldOneWayCallKillCallSingleActorOneForOne = {
Logg.messageLog = ""
val sup = getSingleActorOneForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
pingpong1 ! OneWay
Thread.sleep(500)
@@ -277,172 +279,172 @@ class RemoteSupervisorTest extends JUnitSuite {
}
}
*/
-
+
/*
- @Test def shouldOneWayKillSingleActorAllForOne = {
- Logg.messageLog = ""
- val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
- Thread.sleep(500)
- intercept[RuntimeException] {
- pingpong1 ! BinaryString("Die")
- }
- Thread.sleep(500)
- expect("DIE") {
- Logg.messageLog
- }
- }
+ @Test def shouldOneWayKillSingleActorAllForOne = {
+ Logg.messageLog = ""
+ val sup = getSingleActorAllForOneSupervisor
+ sup.start
+ Thread.sleep(500)
+ intercept[RuntimeException] {
+ pingpong1 ! BinaryString("Die")
+ }
+ Thread.sleep(500)
+ expect("DIE") {
+ Logg.messageLog
+ }
+ }
- @Test def shouldOneWayCallKillCallSingleActorAllForOne = {
- Logg.messageLog = ""
- val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
- Thread.sleep(500)
- expect("pong") {
- (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("ping") {
- Logg.messageLog
- }
- intercept[RuntimeException] {
- pingpong1 ! BinaryString("Die")
- }
- Thread.sleep(500)
- expect("pingDIE") {
- Logg.messageLog
- }
- expect("pong") {
- (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pingDIEping") {
- Logg.messageLog
- }
- }
+ @Test def shouldOneWayCallKillCallSingleActorAllForOne = {
+ Logg.messageLog = ""
+ val sup = getSingleActorAllForOneSupervisor
+ sup.start
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("ping") {
+ Logg.messageLog
+ }
+ intercept[RuntimeException] {
+ pingpong1 ! BinaryString("Die")
+ }
+ Thread.sleep(500)
+ expect("pingDIE") {
+ Logg.messageLog
+ }
+ expect("pong") {
+ (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pingDIEping") {
+ Logg.messageLog
+ }
+ }
- @Test def shouldOneWayKillMultipleActorsOneForOne = {
- Logg.messageLog = ""
- val sup = getMultipleActorsOneForOneConf
- sup ! StartSupervisor
- Thread.sleep(500)
- intercept[RuntimeException] {
- pingpong3 ! BinaryString("Die")
- }
- Thread.sleep(500)
- expect("DIE") {
- Logg.messageLog
- }
- }
+ @Test def shouldOneWayKillMultipleActorsOneForOne = {
+ Logg.messageLog = ""
+ val sup = getMultipleActorsOneForOneConf
+ sup.start
+ Thread.sleep(500)
+ intercept[RuntimeException] {
+ pingpong3 ! BinaryString("Die")
+ }
+ Thread.sleep(500)
+ expect("DIE") {
+ Logg.messageLog
+ }
+ }
- def tesOneWayCallKillCallMultipleActorsOneForOne = {
- Logg.messageLog = ""
- val sup = getMultipleActorsOneForOneConf
- sup ! StartSupervisor
- Thread.sleep(500)
- expect("pong") {
- (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pong") {
- (pingpong2 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pong") {
- (pingpong3 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pingpingping") {
- Logg.messageLog
- }
- intercept[RuntimeException] {
- pingpong2 ! BinaryString("Die")
- }
- Thread.sleep(500)
- expect("pingpingpingDIE") {
- Logg.messageLog
- }
- expect("pong") {
- (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pong") {
- (pingpong2 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pong") {
- (pingpong3 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pingpingpingDIEpingpingping") {
- Logg.messageLog
- }
- }
+ def tesOneWayCallKillCallMultipleActorsOneForOne = {
+ Logg.messageLog = ""
+ val sup = getMultipleActorsOneForOneConf
+ sup.start
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong2 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong3 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pingpingping") {
+ Logg.messageLog
+ }
+ intercept[RuntimeException] {
+ pingpong2 ! BinaryString("Die")
+ }
+ Thread.sleep(500)
+ expect("pingpingpingDIE") {
+ Logg.messageLog
+ }
+ expect("pong") {
+ (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong2 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong3 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pingpingpingDIEpingpingping") {
+ Logg.messageLog
+ }
+ }
- @Test def shouldOneWayKillMultipleActorsAllForOne = {
- Logg.messageLog = ""
- val sup = getMultipleActorsAllForOneConf
- sup ! StartSupervisor
- Thread.sleep(500)
- intercept[RuntimeException] {
- pingpong2 ! BinaryString("Die")
- }
- Thread.sleep(500)
- expect("DIEDIEDIE") {
- Logg.messageLog
- }
- }
+ @Test def shouldOneWayKillMultipleActorsAllForOne = {
+ Logg.messageLog = ""
+ val sup = getMultipleActorsAllForOneConf
+ sup.start
+ Thread.sleep(500)
+ intercept[RuntimeException] {
+ pingpong2 ! BinaryString("Die")
+ }
+ Thread.sleep(500)
+ expect("DIEDIEDIE") {
+ Logg.messageLog
+ }
+ }
- def tesOneWayCallKillCallMultipleActorsAllForOne = {
- Logg.messageLog = ""
- val sup = getMultipleActorsAllForOneConf
- sup ! StartSupervisor
- Thread.sleep(500)
- expect("pong") {
- pingpong1 ! BinaryString("Ping")
- }
- Thread.sleep(500)
- expect("pong") {
- (pingpong2 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pong") {
- (pingpong3 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pingpingping") {
- Logg.messageLog
- }
- intercept[RuntimeException] {
- pingpong2 ! BinaryString("Die")
- }
- Thread.sleep(500)
- expect("pingpingpingDIEDIEDIE") {
- Logg.messageLog
- }
- expect("pong") {
- (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pong") {
- (pingpong2 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pong") {
- (pingpong3 ! BinaryString("Ping")).getOrElse("nil")
- }
- Thread.sleep(500)
- expect("pingpingpingDIEDIEDIEpingpingping") {
- Logg.messageLog
- }
- }
- */
+ def tesOneWayCallKillCallMultipleActorsAllForOne = {
+ Logg.messageLog = ""
+ val sup = getMultipleActorsAllForOneConf
+ sup.start
+ Thread.sleep(500)
+ expect("pong") {
+ pingpong1 ! BinaryString("Ping")
+ }
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong2 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong3 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pingpingping") {
+ Logg.messageLog
+ }
+ intercept[RuntimeException] {
+ pingpong2 ! BinaryString("Die")
+ }
+ Thread.sleep(500)
+ expect("pingpingpingDIEDIEDIE") {
+ Logg.messageLog
+ }
+ expect("pong") {
+ (pingpong1 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong2 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pong") {
+ (pingpong3 ! BinaryString("Ping")).getOrElse("nil")
+ }
+ Thread.sleep(500)
+ expect("pingpingpingDIEDIEDIEpingpingping") {
+ Logg.messageLog
+ }
+ }
+ */
/*
@Test def shouldNestedSupervisorsTerminateFirstLevelActorAllForOne = {
Logg.messageLog = ""
val sup = getNestedSupervisorsAllForOneConf
- sup ! StartSupervisor
+ sup.start
intercept[RuntimeException] {
pingpong1 !! BinaryString("Die")
}
@@ -467,34 +469,29 @@ class RemoteSupervisorTest extends JUnitSuite {
pingpong1 = new RemotePingPong1Actor
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
- SupervisorConfig(
- RestartStrategy(AllForOne, 3, 100),
- Supervise(
- pingpong1,
- LifeCycle(Permanent))
- :: Nil)
- }
- }
- factory.newSupervisor
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ RestartStrategy(AllForOne, 3, 100),
+ Supervise(
+ pingpong1,
+ LifeCycle(Permanent))
+ :: Nil))
+
+ factory.newInstance
}
def getSingleActorOneForOneSupervisor: Supervisor = {
pingpong1 = new RemotePingPong1Actor
pingpong1.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
- SupervisorConfig(
- RestartStrategy(OneForOne, 3, 100),
- Supervise(
- pingpong1,
- LifeCycle(Permanent))
- :: Nil)
- }
- }
- factory.newSupervisor
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ RestartStrategy(OneForOne, 3, 100),
+ Supervise(
+ pingpong1,
+ LifeCycle(Permanent))
+ :: Nil))
+ factory.newInstance
}
def getMultipleActorsAllForOneConf: Supervisor = {
@@ -505,25 +502,22 @@ class RemoteSupervisorTest extends JUnitSuite {
pingpong3 = new RemotePingPong3Actor
pingpong3.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
- SupervisorConfig(
- RestartStrategy(AllForOne, 3, 100),
- Supervise(
- pingpong1,
- LifeCycle(Permanent))
- ::
- Supervise(
- pingpong2,
- LifeCycle(Permanent))
- ::
- Supervise(
- pingpong3,
- LifeCycle(Permanent))
- :: Nil)
- }
- }
- factory.newSupervisor
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ RestartStrategy(AllForOne, 3, 100),
+ Supervise(
+ pingpong1,
+ LifeCycle(Permanent))
+ ::
+ Supervise(
+ pingpong2,
+ LifeCycle(Permanent))
+ ::
+ Supervise(
+ pingpong3,
+ LifeCycle(Permanent))
+ :: Nil))
+ factory.newInstance
}
def getMultipleActorsOneForOneConf: Supervisor = {
@@ -534,25 +528,22 @@ class RemoteSupervisorTest extends JUnitSuite {
pingpong3 = new RemotePingPong3Actor
pingpong3.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
- SupervisorConfig(
- RestartStrategy(OneForOne, 3, 100),
- Supervise(
- pingpong1,
- LifeCycle(Permanent))
- ::
- Supervise(
- pingpong2,
- LifeCycle(Permanent))
- ::
- Supervise(
- pingpong3,
- LifeCycle(Permanent))
- :: Nil)
- }
- }
- factory.newSupervisor
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ RestartStrategy(OneForOne, 3, 100),
+ Supervise(
+ pingpong1,
+ LifeCycle(Permanent))
+ ::
+ Supervise(
+ pingpong2,
+ LifeCycle(Permanent))
+ ::
+ Supervise(
+ pingpong3,
+ LifeCycle(Permanent))
+ :: Nil))
+ factory.newInstance
}
def getNestedSupervisorsAllForOneConf: Supervisor = {
@@ -563,34 +554,30 @@ class RemoteSupervisorTest extends JUnitSuite {
pingpong3 = new RemotePingPong3Actor
pingpong3.makeRemote(RemoteServer.HOSTNAME, RemoteServer.PORT)
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
- SupervisorConfig(
- RestartStrategy(AllForOne, 3, 100),
- Supervise(
- pingpong1,
- LifeCycle(Permanent))
- ::
- SupervisorConfig(
- RestartStrategy(AllForOne, 3, 100),
- Supervise(
- pingpong2,
- LifeCycle(Permanent))
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ RestartStrategy(AllForOne, 3, 100),
+ Supervise(
+ pingpong1,
+ LifeCycle(Permanent))
::
- Supervise(
- pingpong3,
- LifeCycle(Permanent))
- :: Nil)
- :: Nil)
- }
- }
- factory.newSupervisor
- }
-
+ SupervisorConfig(
+ RestartStrategy(AllForOne, 3, 100),
+ Supervise(
+ pingpong2,
+ LifeCycle(Permanent))
+ ::
+ Supervise(
+ pingpong3,
+ LifeCycle(Permanent))
+ :: Nil)
+ :: Nil))
+ factory.newInstance
+ }
}
@serializable class RemotePingPong1Actor extends Actor {
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case BinaryString("Ping") =>
Log.messageLog += "ping"
reply("pong")
@@ -601,26 +588,28 @@ class RemoteSupervisorTest extends JUnitSuite {
case BinaryString("Die") =>
throw new RuntimeException("DIE")
}
+
override protected def postRestart(reason: AnyRef, config: Option[AnyRef]) {
Log.messageLog += reason.asInstanceOf[Exception].getMessage
}
}
@serializable class RemotePingPong2Actor extends Actor {
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case BinaryString("Ping") =>
Log.messageLog += "ping"
reply("pong")
case BinaryString("Die") =>
throw new RuntimeException("DIE")
}
+
override protected def postRestart(reason: AnyRef, config: Option[AnyRef]) {
Log.messageLog += reason.asInstanceOf[Exception].getMessage
}
}
@serializable class RemotePingPong3Actor extends Actor {
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case BinaryString("Ping") =>
Log.messageLog += "ping"
reply("pong")
diff --git a/akka-actors/src/test/scala/SchedulerTest.scala b/akka-actors/src/test/scala/SchedulerTest.scala
index 029872a295..383e1f5206 100644
--- a/akka-actors/src/test/scala/SchedulerTest.scala
+++ b/akka-actors/src/test/scala/SchedulerTest.scala
@@ -11,14 +11,14 @@ class SchedulerTest extends JUnitSuite {
var count = 0
case object Tick
val actor = new Actor() {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Tick => count += 1
}}
actor.start
Thread.sleep(1000)
Scheduler.schedule(actor, Tick, 0L, 1L, TimeUnit.SECONDS)
Thread.sleep(5000)
- Scheduler.shutdown
+ Scheduler.stop
assert(count > 0)
}
}
\ No newline at end of file
diff --git a/akka-actors/src/test/scala/SupervisorTest.scala b/akka-actors/src/test/scala/SupervisorTest.scala
index 7bba332e69..b143ef21e7 100644
--- a/akka-actors/src/test/scala/SupervisorTest.scala
+++ b/akka-actors/src/test/scala/SupervisorTest.scala
@@ -4,7 +4,7 @@
package se.scalablesolutions.akka.actor
-import config.ScalaConfig._
+import se.scalablesolutions.akka.config.ScalaConfig._
import org.scalatest.junit.JUnitSuite
import org.junit.Test
@@ -13,6 +13,8 @@ import org.junit.Test
* @author Jonas Bonér
*/
class SupervisorTest extends JUnitSuite {
+ import Actor.Sender.Self
+
var messageLog: String = ""
var oneWayLog: String = ""
@@ -24,7 +26,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldStartServer = {
messageLog = ""
val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
+ sup.start
expect("pong") {
(pingpong1 !! Ping).getOrElse("nil")
@@ -34,7 +36,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldKillSingleActorOneForOne = {
messageLog = ""
val sup = getSingleActorOneForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong1 !! Die
@@ -48,7 +50,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldCallKillCallSingleActorOneForOne = {
messageLog = ""
val sup = getSingleActorOneForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 !! Ping).getOrElse("nil")
@@ -76,7 +78,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldKillSingleActorAllForOne = {
messageLog = ""
val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong1 !! Die
@@ -90,7 +92,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldCallKillCallSingleActorAllForOne = {
messageLog = ""
val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 !! Ping).getOrElse("nil")
@@ -118,7 +120,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldKillMultipleActorsOneForOne = {
messageLog = ""
val sup = getMultipleActorsOneForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong3 !! Die
@@ -132,7 +134,7 @@ class SupervisorTest extends JUnitSuite {
def tesCallKillCallMultipleActorsOneForOne = {
messageLog = ""
val sup = getMultipleActorsOneForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 !! Ping).getOrElse("nil")
@@ -176,7 +178,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldKillMultipleActorsAllForOne = {
messageLog = ""
val sup = getMultipleActorsAllForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong2 !! Die
@@ -190,7 +192,7 @@ class SupervisorTest extends JUnitSuite {
def tesCallKillCallMultipleActorsAllForOne = {
messageLog = ""
val sup = getMultipleActorsAllForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 !! Ping).getOrElse("nil")
@@ -234,7 +236,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldOneWayKillSingleActorOneForOne = {
messageLog = ""
val sup = getSingleActorOneForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
pingpong1 ! Die
Thread.sleep(500)
@@ -246,7 +248,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldOneWayCallKillCallSingleActorOneForOne = {
messageLog = ""
val sup = getSingleActorOneForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
pingpong1 ! OneWay
Thread.sleep(500)
@@ -269,7 +271,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldOneWayKillSingleActorAllForOne = {
messageLog = ""
val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong1 ! Die
@@ -283,7 +285,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldOneWayCallKillCallSingleActorAllForOne = {
messageLog = ""
val sup = getSingleActorAllForOneSupervisor
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 ! Ping).getOrElse("nil")
@@ -311,7 +313,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldOneWayKillMultipleActorsOneForOne = {
messageLog = ""
val sup = getMultipleActorsOneForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong3 ! Die
@@ -325,7 +327,7 @@ class SupervisorTest extends JUnitSuite {
def tesOneWayCallKillCallMultipleActorsOneForOne = {
messageLog = ""
val sup = getMultipleActorsOneForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
(pingpong1 ! Ping).getOrElse("nil")
@@ -369,7 +371,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldOneWayKillMultipleActorsAllForOne = {
messageLog = ""
val sup = getMultipleActorsAllForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
intercept[RuntimeException] {
pingpong2 ! Die
@@ -383,7 +385,7 @@ class SupervisorTest extends JUnitSuite {
def tesOneWayCallKillCallMultipleActorsAllForOne = {
messageLog = ""
val sup = getMultipleActorsAllForOneConf
- sup ! StartSupervisor
+ sup.start
Thread.sleep(500)
expect("pong") {
pingpong1 ! Ping
@@ -429,7 +431,7 @@ class SupervisorTest extends JUnitSuite {
@Test def shouldNestedSupervisorsTerminateFirstLevelActorAllForOne = {
messageLog = ""
val sup = getNestedSupervisorsAllForOneConf
- sup ! StartSupervisor
+ sup.start
intercept[RuntimeException] {
pingpong1 !! Die
}
@@ -453,33 +455,27 @@ class SupervisorTest extends JUnitSuite {
pingpong1 = new PingPong1Actor
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
+ val factory = SupervisorFactory(
SupervisorConfig(
RestartStrategy(AllForOne, 3, 100),
Supervise(
pingpong1,
LifeCycle(Permanent))
- :: Nil)
- }
- }
- factory.newSupervisor
+ :: Nil))
+ factory.newInstance
}
def getSingleActorOneForOneSupervisor: Supervisor = {
pingpong1 = new PingPong1Actor
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
+ val factory = SupervisorFactory(
SupervisorConfig(
RestartStrategy(OneForOne, 3, 100),
Supervise(
pingpong1,
LifeCycle(Permanent))
- :: Nil)
- }
- }
- factory.newSupervisor
+ :: Nil))
+ factory.newInstance
}
def getMultipleActorsAllForOneConf: Supervisor = {
@@ -487,8 +483,7 @@ class SupervisorTest extends JUnitSuite {
pingpong2 = new PingPong2Actor
pingpong3 = new PingPong3Actor
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
+ val factory = SupervisorFactory(
SupervisorConfig(
RestartStrategy(AllForOne, 3, 100),
Supervise(
@@ -502,10 +497,8 @@ class SupervisorTest extends JUnitSuite {
Supervise(
pingpong3,
LifeCycle(Permanent))
- :: Nil)
- }
- }
- factory.newSupervisor
+ :: Nil))
+ factory.newInstance
}
def getMultipleActorsOneForOneConf: Supervisor = {
@@ -513,8 +506,7 @@ class SupervisorTest extends JUnitSuite {
pingpong2 = new PingPong2Actor
pingpong3 = new PingPong3Actor
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
+ val factory = SupervisorFactory(
SupervisorConfig(
RestartStrategy(OneForOne, 3, 100),
Supervise(
@@ -528,10 +520,8 @@ class SupervisorTest extends JUnitSuite {
Supervise(
pingpong3,
LifeCycle(Permanent))
- :: Nil)
- }
- }
- factory.newSupervisor
+ :: Nil))
+ factory.newInstance
}
def getNestedSupervisorsAllForOneConf: Supervisor = {
@@ -539,8 +529,7 @@ class SupervisorTest extends JUnitSuite {
pingpong2 = new PingPong2Actor
pingpong3 = new PingPong3Actor
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
+ val factory = SupervisorFactory(
SupervisorConfig(
RestartStrategy(AllForOne, 3, 100),
Supervise(
@@ -557,14 +546,12 @@ class SupervisorTest extends JUnitSuite {
pingpong3,
LifeCycle(Permanent))
:: Nil)
- :: Nil)
- }
- }
- factory.newSupervisor
+ :: Nil))
+ factory.newInstance
}
class PingPong1Actor extends Actor {
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Ping =>
messageLog += "ping"
reply("pong")
@@ -581,7 +568,7 @@ class SupervisorTest extends JUnitSuite {
}
class PingPong2Actor extends Actor {
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Ping =>
messageLog += "ping"
reply("pong")
@@ -594,7 +581,7 @@ class SupervisorTest extends JUnitSuite {
}
class PingPong3Actor extends Actor {
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Ping =>
messageLog += "ping"
reply("pong")
diff --git a/akka-actors/src/test/scala/ThreadBasedActorTest.scala b/akka-actors/src/test/scala/ThreadBasedActorTest.scala
index 6d30ec58db..ead74068d1 100644
--- a/akka-actors/src/test/scala/ThreadBasedActorTest.scala
+++ b/akka-actors/src/test/scala/ThreadBasedActorTest.scala
@@ -8,12 +8,14 @@ import org.junit.Test
import se.scalablesolutions.akka.dispatch.Dispatchers
class ThreadBasedActorTest extends JUnitSuite {
+ import Actor.Sender.Self
+
private val unit = TimeUnit.MILLISECONDS
class TestActor extends Actor {
dispatcher = Dispatchers.newThreadBasedDispatcher(this)
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "Hello" =>
reply("World")
case "Failure" =>
@@ -25,7 +27,7 @@ class ThreadBasedActorTest extends JUnitSuite {
implicit val timeout = 5000L
var oneWay = "nada"
val actor = new Actor {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "OneWay" => oneWay = "received"
}
}
@@ -40,7 +42,7 @@ class ThreadBasedActorTest extends JUnitSuite {
implicit val timeout = 5000L
val actor = new TestActor
actor.start
- val result: String = actor !? "Hello"
+ val result: String = (actor !! ("Hello", 10000)).get
assert("World" === result)
actor.stop
}
diff --git a/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala b/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala
index e5f4a6f1d4..b434762b37 100644
--- a/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala
+++ b/akka-actors/src/test/scala/ThreadBasedDispatcherTest.scala
@@ -13,9 +13,9 @@ import se.scalablesolutions.akka.actor.Actor
class ThreadBasedDispatcherTest extends JUnitSuite {
private var threadingIssueDetected: AtomicBoolean = null
- val key1 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
- val key2 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
- val key3 = new Actor { def receive: PartialFunction[Any, Unit] = { case _ => {}} }
+ val key1 = new Actor { def receive = { case _ => {}} }
+ val key2 = new Actor { def receive = { case _ => {}} }
+ val key3 = new Actor { def receive = { case _ => {}} }
class TestMessageHandle(handleLatch: CountDownLatch) extends MessageInvoker {
val guardLock: Lock = new ReentrantLock
@@ -57,7 +57,7 @@ class ThreadBasedDispatcherTest extends JUnitSuite {
val dispatcher = new ThreadBasedDispatcher("name", new TestMessageHandle(handleLatch))
dispatcher.start
for (i <- 0 until 100) {
- dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key1, new Object, None, None, None))
}
assert(handleLatch.await(5, TimeUnit.SECONDS))
assert(!threadingIssueDetected.get)
@@ -78,7 +78,7 @@ class ThreadBasedDispatcherTest extends JUnitSuite {
})
dispatcher.start
for (i <- 0 until 100) {
- dispatcher.messageQueue.append(new MessageInvocation(key1, new Integer(i), None, None))
+ dispatcher.messageQueue.append(new MessageInvocation(key1, new Integer(i), None, None, None))
}
assert(handleLatch.await(5, TimeUnit.SECONDS))
assert(!threadingIssueDetected.get)
diff --git a/akka-amqp/pom.xml b/akka-amqp/pom.xml
index 5199ed0dc3..113b6917b1 100644
--- a/akka-amqp/pom.xml
+++ b/akka-amqp/pom.xml
@@ -17,24 +17,19 @@
akka-util
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-actors
- se.scalablesolutions.akka
- 0.6
-
-
+ ${project.groupId}
+ ${project.version}
+
+
com.rabbitmq
rabbitmq-client
0.9.1
-
- commons-io
- commons-io
- 1.4
-
diff --git a/akka-amqp/src/main/scala/AMQP.scala b/akka-amqp/src/main/scala/AMQP.scala
index 6dddd7a377..462e65f854 100644
--- a/akka-amqp/src/main/scala/AMQP.scala
+++ b/akka-amqp/src/main/scala/AMQP.scala
@@ -31,10 +31,8 @@ import java.io.IOException
* val consumer = AMQP.newConsumer(params, hostname, port, exchange, ExchangeType.Direct, Serializer.ScalaJSON, None, 100)
*
- * consumer ! MessageConsumerListener(queue, routingKey, new Actor() {
- * def receive: PartialFunction[Any, Unit] = {
- * case Message(payload, _, _, _, _) => log.debug("Received message: %s", payload)
- * }
+ * consumer ! MessageConsumerListener(queue, routingKey, actor {
+ * case Message(payload, _, _, _, _) => log.debug("Received message: %s", payload)
* })
*
* val producer = AMQP.newProducer(params, hostname, port, exchange, Serializer.ScalaJSON, None, None, 100)
@@ -43,15 +41,113 @@ import java.io.IOException
*
* @author Jonas Bonér
*/
-object AMQP extends Actor {
- private val connections = new ConcurrentHashMap[FaultTolerantConnectionActor, FaultTolerantConnectionActor]
- faultHandler = Some(OneForOneStrategy(5, 5000))
- trapExit = List(classOf[Throwable])
- start
+object AMQP {
+ private val supervisor = new AMQPSupervisor
+
+ def newProducer(
+ config: ConnectionParameters,
+ hostname: String,
+ port: Int,
+ exchangeName: String,
+ returnListener: Option[ReturnListener],
+ shutdownListener: Option[ShutdownListener],
+ initReconnectDelay: Long) =
+ supervisor.newProducer(
+ config, hostname, port, exchangeName, returnListener, shutdownListener, initReconnectDelay)
+
+ def newConsumer(
+ config: ConnectionParameters,
+ hostname: String,
+ port: Int,
+ exchangeName: String,
+ exchangeType: ExchangeType,
+ shutdownListener: Option[ShutdownListener],
+ initReconnectDelay: Long,
+ passive: Boolean,
+ durable: Boolean,
+ configurationArguments: Map[String, AnyRef]) =
+ supervisor.newConsumer(
+ config, hostname, port, exchangeName, exchangeType,
+ shutdownListener, initReconnectDelay, passive, durable, configurationArguments)
+
+ def stopConnection(connection: FaultTolerantConnectionActor) = supervisor.stopConnection(connection)
+
+ /**
+ * @author Jonas Bonér
+ */
+ class AMQPSupervisor extends Actor {
+ private val connections = new ConcurrentHashMap[FaultTolerantConnectionActor, FaultTolerantConnectionActor]
+
+ faultHandler = Some(OneForOneStrategy(5, 5000))
+ trapExit = List(classOf[Throwable])
+ start
+
+ def newProducer(
+ config: ConnectionParameters,
+ hostname: String,
+ port: Int,
+ exchangeName: String,
+ returnListener: Option[ReturnListener],
+ shutdownListener: Option[ShutdownListener],
+ initReconnectDelay: Long): Producer = {
+ val producer = new Producer(
+ new ConnectionFactory(config),
+ hostname, port,
+ exchangeName,
+ returnListener,
+ shutdownListener,
+ initReconnectDelay)
+ startLink(producer)
+ producer
+ }
+
+ def newConsumer(
+ config: ConnectionParameters,
+ hostname: String,
+ port: Int,
+ exchangeName: String,
+ exchangeType: ExchangeType,
+ shutdownListener: Option[ShutdownListener],
+ initReconnectDelay: Long,
+ passive: Boolean,
+ durable: Boolean,
+ configurationArguments: Map[String, AnyRef]): Consumer = {
+ val consumer = new Consumer(
+ new ConnectionFactory(config),
+ hostname, port,
+ exchangeName,
+ exchangeType,
+ shutdownListener,
+ initReconnectDelay,
+ passive,
+ durable,
+ configurationArguments)
+ startLink(consumer)
+ consumer
+ }
+
+ def stopConnection(connection: FaultTolerantConnectionActor) = {
+ connection ! Stop
+ unlink(connection)
+ connections.remove(connection)
+ }
+
+ override def shutdown = {
+ connections.values.asScala.foreach(_ ! Stop)
+ exit
+ }
+
+ def receive = {
+ case _ => {} // ignore all messages
+ }
+ }
sealed trait AMQPMessage
private[akka] trait InternalAMQPMessage extends AMQPMessage
+ /**
+ * @author Jonas Bonér
+ */
class Message(val payload: Array[Byte],
val routingKey: String,
val mandatory: Boolean,
@@ -59,12 +155,15 @@ object AMQP extends Actor {
val properties: RabbitMQ.BasicProperties) extends AMQPMessage {
override def toString(): String =
"Message[payload=" + payload +
- ", routingKey=" + routingKey +
- ", mandatory=" + mandatory +
- ", immediate=" + immediate +
- ", properties=" + properties + "]"
+ ", routingKey=" + routingKey +
+ ", mandatory=" + mandatory +
+ ", immediate=" + immediate +
+ ", properties=" + properties + "]"
}
+ /**
+ * @author Jonas Bonér
+ */
object Message {
def unapply(message: Message): Option[Tuple5[AnyRef, String, Boolean, Boolean, RabbitMQ.BasicProperties]] =
Some((message.payload, message.routingKey, message.mandatory, message.immediate, message.properties))
@@ -76,28 +175,31 @@ object AMQP extends Actor {
new Message(payload, routingKey, false, false, null)
}
- case class MessageConsumerListener(queueName: String,
- routingKey: String,
- isUsingExistingQueue: Boolean,
+ /**
+ * @author Jonas Bonér
+ */
+ case class MessageConsumerListener(queueName: String,
+ routingKey: String,
+ isUsingExistingQueue: Boolean,
actor: Actor) extends AMQPMessage {
- def this(queueName: String, routingKey: String, actor: Actor) = this(queueName, routingKey, false, actor)
-
+ def this(queueName: String, routingKey: String, actor: Actor) = this (queueName, routingKey, false, actor)
+
private[akka] var tag: Option[String] = None
- override def toString() =
+ override def toString() =
"MessageConsumerListener[actor=" + actor +
- ", queue=" + queueName +
- ", routingKey=" + routingKey +
- ", tag=" + tag +
- ", isUsingExistingQueue=" + isUsingExistingQueue + "]"
+ ", queue=" + queueName +
+ ", routingKey=" + routingKey +
+ ", tag=" + tag +
+ ", isUsingExistingQueue=" + isUsingExistingQueue + "]"
- def toString(exchangeName: String) =
+ def toString(exchangeName: String) =
"MessageConsumerListener[actor=" + actor +
- ", exchange=" + exchangeName +
- ", queue=" + queueName +
- ", routingKey=" + routingKey +
- ", tag=" + tag +
- ", isUsingExistingQueue=" + isUsingExistingQueue + "]"
+ ", exchange=" + exchangeName +
+ ", queue=" + queueName +
+ ", routingKey=" + routingKey +
+ ", tag=" + tag +
+ ", isUsingExistingQueue=" + isUsingExistingQueue + "]"
/**
* Hash code should only be based on on queue name and routing key.
@@ -114,31 +216,32 @@ object AMQP extends Actor {
*/
override def equals(that: Any): Boolean = synchronized {
that != null &&
- that.isInstanceOf[MessageConsumerListener] &&
- that.asInstanceOf[MessageConsumerListener].queueName== queueName &&
- that.asInstanceOf[MessageConsumerListener].routingKey == routingKey
+ that.isInstanceOf[MessageConsumerListener] &&
+ that.asInstanceOf[MessageConsumerListener].queueName == queueName &&
+ that.asInstanceOf[MessageConsumerListener].routingKey == routingKey
}
}
object MessageConsumerListener {
- def apply(queueName: String, routingKey: String, actor: Actor) = new MessageConsumerListener(queueName, routingKey, false, actor)
+ def apply(queueName: String, routingKey: String, actor: Actor) =
+ new MessageConsumerListener(queueName, routingKey, false, actor)
}
-
+
case object Stop extends AMQPMessage
-
+
private[akka] case class UnregisterMessageConsumerListener(consumer: MessageConsumerListener) extends InternalAMQPMessage
-
+
private[akka] case class Reconnect(delay: Long) extends InternalAMQPMessage
-
+
private[akka] case class Failure(cause: Throwable) extends InternalAMQPMessage
-
+
private[akka] class MessageNotDeliveredException(
- val message: String,
- val replyCode: Int,
- val replyText: String,
- val exchange: String,
- val routingKey: String,
- val properties: RabbitMQ.BasicProperties,
- val body: Array[Byte]) extends RuntimeException(message)
+ val message: String,
+ val replyCode: Int,
+ val replyText: String,
+ val exchange: String,
+ val routingKey: String,
+ val properties: RabbitMQ.BasicProperties,
+ val body: Array[Byte]) extends RuntimeException(message)
sealed trait ExchangeType
object ExchangeType {
@@ -156,87 +259,32 @@ object AMQP extends Actor {
}
}
- def newProducer(
- config: ConnectionParameters,
- hostname: String,
- port: Int,
- exchangeName: String,
- returnListener: Option[ReturnListener],
- shutdownListener: Option[ShutdownListener],
- initReconnectDelay: Long): Producer = {
- val producer = new Producer(
- new ConnectionFactory(config),
- hostname, port,
- exchangeName,
- returnListener,
- shutdownListener,
- initReconnectDelay)
- startLink(producer)
- producer
- }
-
- def newConsumer(
- config: ConnectionParameters,
- hostname: String,
- port: Int,
- exchangeName: String,
- exchangeType: ExchangeType,
- shutdownListener: Option[ShutdownListener],
- initReconnectDelay: Long,
- passive: Boolean,
- durable: Boolean,
- configurationArguments: Map[String, AnyRef]): Consumer = {
- val consumer = new Consumer(
- new ConnectionFactory(config),
- hostname, port,
- exchangeName,
- exchangeType,
- shutdownListener,
- initReconnectDelay,
- passive,
- durable,
- configurationArguments)
- startLink(consumer)
- consumer
- }
-
- def stopConnection(connection: FaultTolerantConnectionActor) = {
- connection ! Stop
- unlink(connection)
- connections.remove(connection)
- }
-
- override def shutdown = {
- connections.values.asScala.foreach(_ ! Stop)
- stop
- }
-
/**
* @author Jonas Bonér
*/
- class Producer private[amqp] (
- val connectionFactory: ConnectionFactory,
- val hostname: String,
- val port: Int,
- val exchangeName: String,
- val returnListener: Option[ReturnListener],
- val shutdownListener: Option[ShutdownListener],
- val initReconnectDelay: Long)
- extends FaultTolerantConnectionActor {
-
+ class Producer private[amqp](
+ val connectionFactory: ConnectionFactory,
+ val hostname: String,
+ val port: Int,
+ val exchangeName: String,
+ val returnListener: Option[ReturnListener],
+ val shutdownListener: Option[ShutdownListener],
+ val initReconnectDelay: Long)
+ extends FaultTolerantConnectionActor {
setupChannel
log.info("AMQP.Producer [%s] is started", toString)
def newRpcClient(routingKey: String): RpcClient = new RpcClient(channel, exchangeName, routingKey)
-
+
def receive = {
- case message @ Message(payload, routingKey, mandatory, immediate, properties) =>
+ case message@Message(payload, routingKey, mandatory, immediate, properties) =>
log.debug("Sending message [%s]", message)
- channel.basicPublish(exchangeName, routingKey, mandatory, immediate, properties, payload.asInstanceOf[Array[Byte]])
+ channel.basicPublish(
+ exchangeName, routingKey, mandatory, immediate, properties, payload.asInstanceOf[Array[Byte]])
case Stop =>
disconnect
- stop
+ exit
}
protected def setupChannel = {
@@ -246,18 +294,18 @@ object AMQP extends Actor {
case Some(listener) => channel.setReturnListener(listener)
case None => channel.setReturnListener(new ReturnListener() {
def handleBasicReturn(
- replyCode: Int,
- replyText: String,
- exchange: String,
- routingKey: String,
- properties: RabbitMQ.BasicProperties,
- body: Array[Byte]) = {
+ replyCode: Int,
+ replyText: String,
+ exchange: String,
+ routingKey: String,
+ properties: RabbitMQ.BasicProperties,
+ body: Array[Byte]) = {
throw new MessageNotDeliveredException(
"Could not deliver message [" + body +
- "] with reply code [" + replyCode +
- "] with reply text [" + replyText +
- "] and routing key [" + routingKey +
- "] to exchange [" + exchange + "]",
+ "] with reply code [" + replyCode +
+ "] with reply text [" + replyText +
+ "] and routing key [" + routingKey +
+ "] to exchange [" + exchange + "]",
replyCode, replyText, exchange, routingKey, properties, body)
}
})
@@ -267,25 +315,26 @@ object AMQP extends Actor {
override def toString(): String =
"AMQP.Producer[hostname=" + hostname +
- ", port=" + port +
- ", exchange=" + exchangeName + "]"
+ ", port=" + port +
+ ", exchange=" + exchangeName + "]"
}
/**
* @author Jonas Bonér
*/
- class Consumer private[amqp] (
- val connectionFactory: ConnectionFactory,
- val hostname: String,
- val port: Int,
- val exchangeName: String,
- val exchangeType: ExchangeType,
- val shutdownListener: Option[ShutdownListener],
- val initReconnectDelay: Long,
- val passive: Boolean,
- val durable: Boolean,
- val configurationArguments: Map[java.lang.String, Object])
- extends FaultTolerantConnectionActor { self: Consumer =>
+ class Consumer private[amqp](
+ val connectionFactory: ConnectionFactory,
+ val hostname: String,
+ val port: Int,
+ val exchangeName: String,
+ val exchangeType: ExchangeType,
+ val shutdownListener: Option[ShutdownListener],
+ val initReconnectDelay: Long,
+ val passive: Boolean,
+ val durable: Boolean,
+ val configurationArguments: Map[java.lang.String, Object])
+ extends FaultTolerantConnectionActor {
+ consumer: Consumer =>
faultHandler = Some(OneForOneStrategy(5, 5000))
trapExit = List(classOf[Throwable])
@@ -302,7 +351,7 @@ object AMQP extends Actor {
body(requestBody, replyProperties)
}
}
- }
+ }
def receive = {
case listener: MessageConsumerListener =>
@@ -312,32 +361,34 @@ object AMQP extends Actor {
case UnregisterMessageConsumerListener(listener) =>
unregisterListener(listener)
-
- case Reconnect(delay) =>
+
+ case Reconnect(delay) =>
reconnect(delay)
- case Failure(cause) =>
- log.error(cause, "")
+ case Failure(cause) =>
+ log.error(cause, "Error in AMQP consumer")
throw cause
- case Stop =>
+ case Stop =>
listeners.elements.toList.map(_._2).foreach(unregisterListener(_))
disconnect
- stop
+ exit
- case message: Message =>
- handleIllegalMessage("AMQP.Consumer [" + this + "] can't be used to send messages, ignoring message [" + message + "]")
+ case message: Message =>
+ handleIllegalMessage(
+ "AMQP.Consumer [" + this + "] can't be used to send messages, ignoring message [" + message + "]")
- case unknown =>
- handleIllegalMessage("Unknown message [" + unknown + "] to AMQP Consumer [" + this + "]")
+ case unknown =>
+ handleIllegalMessage(
+ "Unknown message [" + unknown + "] to AMQP Consumer [" + this + "]")
}
protected def setupChannel = {
connection = connectionFactory.newConnection(hostname, port)
channel = connection.createChannel
channel.exchangeDeclare(exchangeName.toString, exchangeType.toString,
- passive, durable,
- configurationArguments.asJava)
+ passive, durable,
+ configurationArguments.asJava)
listeners.elements.toList.map(_._2).foreach(registerListener)
if (shutdownListener.isDefined) connection.addShutdownListener(shutdownListener.get)
}
@@ -352,12 +403,12 @@ object AMQP extends Actor {
}
log.debug("Binding new queue for MessageConsumerListener [%s]", listener.queueName)
- channel.queueBind(listener.queueName, exchangeName, listener.routingKey)
+ channel.queueBind(listener.queueName, exchangeName, listener.routingKey)
val listenerTag = channel.basicConsume(listener.queueName, true, new DefaultConsumer(channel) with Logging {
- override def handleDelivery(tag: String,
- envelope: Envelope,
- properties: RabbitMQ.BasicProperties,
+ override def handleDelivery(tag: String,
+ envelope: Envelope,
+ properties: RabbitMQ.BasicProperties,
payload: Array[Byte]) {
try {
val mandatory = false // FIXME: where to find out if it's mandatory?
@@ -368,23 +419,28 @@ object AMQP extends Actor {
log.debug("Acking message with delivery tag [%s]", deliveryTag)
channel.basicAck(deliveryTag, false)
} catch {
- case cause =>
- log.error("Delivery of message to MessageConsumerListener [%s] failed due to [%s]", listener.toString(exchangeName), cause.toString)
- self ! Failure(cause) // pass on and re-throw exception in consumer actor to trigger restart and reconnect
+ case cause =>
+ log.error(
+ cause, "Delivery of message to MessageConsumerListener [%s] failed",
+ listener.toString(exchangeName))
+ consumer ! Failure(cause) // pass on and re-throw exception in consumer actor to trigger restart and reconnect
}
}
override def handleShutdownSignal(listenerTag: String, signal: ShutdownSignalException) = {
def hasTag(listener: MessageConsumerListener, listenerTag: String): Boolean = {
- if (listener.tag.isEmpty) throw new IllegalStateException("MessageConsumerListener [" + listener + "] does not have a tag")
+ if (listener.tag.isEmpty) throw new IllegalStateException(
+ "MessageConsumerListener [" + listener + "] does not have a tag")
listener.tag.get == listenerTag
}
listeners.elements.toList.map(_._2).find(hasTag(_, listenerTag)) match {
- case None => log.error("Could not find message listener for tag [%s]; can't shut listener down", listenerTag)
+ case None => log.error(
+ "Could not find message listener for tag [%s]; can't shut listener down", listenerTag)
case Some(listener) =>
- log.warning("MessageConsumerListener [%s] is being shutdown by [%s] due to [%s]",
- listener.toString(exchangeName), signal.getReference, signal.getReason)
- self ! UnregisterMessageConsumerListener(listener)
+ log.warning(
+ "MessageConsumerListener [%s] is being shutdown by [%s] due to [%s]",
+ listener.toString(exchangeName), signal.getReference, signal.getReason)
+ consumer ! UnregisterMessageConsumerListener(listener)
}
}
})
@@ -393,11 +449,15 @@ object AMQP extends Actor {
private def unregisterListener(listener: MessageConsumerListener) = {
listeners.get(listener) match {
- case None => log.warning("Can't unregister message consumer listener [%s]; no such listener", listener.toString(exchangeName))
+ case None => log.warning(
+ "Can't unregister message consumer listener [%s]; no such listener",
+ listener.toString(exchangeName))
case Some(listener) =>
listeners - listener
listener.tag match {
- case None => log.warning("Can't unregister message consumer listener [%s]; no listener tag", listener.toString(exchangeName))
+ case None => log.warning(
+ "Can't unregister message consumer listener [%s]; no listener tag",
+ listener.toString(exchangeName))
case Some(tag) =>
channel.basicCancel(tag)
unlink(listener.actor)
@@ -406,24 +466,25 @@ object AMQP extends Actor {
}
}
}
-
+
private def handleIllegalMessage(errorMessage: String) = {
log.error(errorMessage)
throw new IllegalArgumentException(errorMessage)
}
-
+
override def toString(): String =
"AMQP.Consumer[hostname=" + hostname +
- ", port=" + port +
- ", exchange=" + exchangeName +
- ", type=" + exchangeType +
- ", passive=" + passive +
- ", durable=" + durable + "]"
+ ", port=" + port +
+ ", exchange=" + exchangeName +
+ ", type=" + exchangeType +
+ ", passive=" + passive +
+ ", durable=" + durable + "]"
}
+ /**
+ * @author Jonas Bonér
+ */
trait FaultTolerantConnectionActor extends Actor {
- lifeCycle = Some(LifeCycle(Permanent))
-
val reconnectionTimer = new Timer
var connection: Connection = _
@@ -437,29 +498,32 @@ object AMQP extends Actor {
protected def setupChannel
- def createQueue: String = channel.queueDeclare("", false, false, true, true, null).getQueue
+ def createQueue: String =
+ channel.queueDeclare("", false, false, true, true, null).getQueue
- def createQueue(name: String) = channel.queueDeclare(name, false, false, true, true, null).getQueue
+ def createQueue(name: String) =
+ channel.queueDeclare(name, false, false, true, true, null).getQueue
- def createQueue(name: String, durable: Boolean) = channel.queueDeclare(name, false, durable, true, true, null).getQueue
+ def createQueue(name: String, durable: Boolean) =
+ channel.queueDeclare(name, false, durable, true, true, null).getQueue
- def createBindQueue: String = {
+ def createBindQueue: String = {
val name = createQueue
channel.queueBind(name, exchangeName, name)
name
}
- def createBindQueue(name: String) {
+ def createBindQueue(name: String) {
createQueue(name)
channel.queueBind(name, exchangeName, name)
}
- def createBindQueue(name: String, durable: Boolean) {
+ def createBindQueue(name: String, durable: Boolean) {
channel.queueDeclare(name, durable)
channel.queueBind(name, exchangeName, name)
}
- def deleteQueue(name: String) { channel.queueDelete(name) }
+ def deleteQueue(name: String) {channel.queueDelete(name)}
protected def disconnect = {
try {
@@ -494,10 +558,7 @@ object AMQP extends Actor {
}
override def preRestart(reason: AnyRef, config: Option[AnyRef]) = disconnect
+
override def postRestart(reason: AnyRef, config: Option[AnyRef]) = reconnect(initReconnectDelay)
}
-
- def receive = {
- case _ => {} // ignore all messages
- }
}
diff --git a/akka-amqp/src/main/scala/ExampleSession.scala b/akka-amqp/src/main/scala/ExampleSession.scala
index 3054080b8f..158dbe46d0 100644
--- a/akka-amqp/src/main/scala/ExampleSession.scala
+++ b/akka-amqp/src/main/scala/ExampleSession.scala
@@ -5,6 +5,7 @@
package se.scalablesolutions.akka.amqp
import se.scalablesolutions.akka.actor.Actor
+import se.scalablesolutions.akka.actor.Actor.Sender.Self
import com.rabbitmq.client.ConnectionParameters
@@ -30,7 +31,7 @@ object ExampleSession {
def direct = {
val consumer = AMQP.newConsumer(CONFIG, HOSTNAME, PORT, IM, ExchangeType.Direct, None, 100, false, false, Map[String, AnyRef]())
consumer ! MessageConsumerListener("@george_bush", "direct", new Actor() {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Message(payload, _, _, _, _) => log.info("@george_bush received message from: %s", new String(payload.asInstanceOf[Array[Byte]]))
}
})
@@ -41,12 +42,12 @@ object ExampleSession {
def fanout = {
val consumer = AMQP.newConsumer(CONFIG, HOSTNAME, PORT, CHAT, ExchangeType.Fanout, None, 100, false, false, Map[String, AnyRef]())
consumer ! MessageConsumerListener("@george_bush", "", new Actor() {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Message(payload, _, _, _, _) => log.info("@george_bush received message from: %s", new String(payload.asInstanceOf[Array[Byte]]))
}
})
consumer ! MessageConsumerListener("@barack_obama", "", new Actor() {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Message(payload, _, _, _, _) => log.info("@barack_obama received message from: %s", new String(payload.asInstanceOf[Array[Byte]]))
}
})
diff --git a/akka-camel/pom.xml b/akka-camel/pom.xml
index 797b8fef53..a862e8f64f 100644
--- a/akka-camel/pom.xml
+++ b/akka-camel/pom.xml
@@ -19,13 +19,13 @@
akka-util
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-actors
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
org.apache.camel
diff --git a/akka-fun-test-java/pom.xml b/akka-fun-test-java/pom.xml
index 0cba842396..3a0fc927eb 100644
--- a/akka-fun-test-java/pom.xml
+++ b/akka-fun-test-java/pom.xml
@@ -17,8 +17,8 @@
akka-kernel
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
com.sun.grizzly
diff --git a/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java b/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java
index 1d70324a1a..26c6747eca 100644
--- a/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java
+++ b/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemNestedStateTest.java
@@ -17,7 +17,7 @@ public class InMemNestedStateTest extends TestCase {
final private ActiveObjectConfigurator conf = new ActiveObjectConfigurator();
- protected void setUp() {
+ public InMemNestedStateTest() {
conf.configure(
new RestartStrategy(new AllForOne(), 3, 5000),
new Component[]{
@@ -26,7 +26,7 @@ public class InMemNestedStateTest extends TestCase {
new Component(InMemStatefulNested.class, new LifeCycle(new Permanent()), 10000000),
new Component(InMemFailer.class, new LifeCycle(new Permanent()), 1000)
//new Component("inmem-clasher", InMemClasher.class, InMemClasherImpl.class, new LifeCycle(new Permanent()), 100000)
- }).inject().supervise();
+ }).supervise();
Config.config();
InMemStateful stateful = conf.getInstance(InMemStateful.class);
stateful.init();
@@ -34,10 +34,6 @@ public class InMemNestedStateTest extends TestCase {
nested.init();
}
- protected void tearDown() {
- conf.stop();
- }
-
public void testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess() throws Exception {
InMemStateful stateful = conf.getInstance(InMemStateful.class);
stateful.setMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "init"); // set init state
diff --git a/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemoryStateTest.java b/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemoryStateTest.java
index 3dbe92ca75..e29e8ef81f 100644
--- a/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemoryStateTest.java
+++ b/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/InMemoryStateTest.java
@@ -9,7 +9,9 @@ import junit.framework.TestCase;
import se.scalablesolutions.akka.Config;
import se.scalablesolutions.akka.config.*;
import se.scalablesolutions.akka.config.ActiveObjectConfigurator;
+
import static se.scalablesolutions.akka.config.JavaConfig.*;
+
import se.scalablesolutions.akka.actor.*;
import se.scalablesolutions.akka.Kernel;
@@ -18,27 +20,23 @@ public class InMemoryStateTest extends TestCase {
final private ActiveObjectConfigurator conf = new ActiveObjectConfigurator();
- protected void setUp() {
+ public InMemoryStateTest() {
Config.config();
conf.configure(
new RestartStrategy(new AllForOne(), 3, 5000),
new Component[]{
new Component(InMemStateful.class,
new LifeCycle(new Permanent()),
- //new RestartCallbacks("preRestart", "postRestart")),
- 10000),
+ //new RestartCallbacks("preRestart", "postRestart")),
+ 10000),
new Component(InMemFailer.class,
new LifeCycle(new Permanent()),
- 10000)
- }).inject().supervise();
- InMemStateful stateful = conf.getInstance(InMemStateful.class);
- stateful.init();
+ 10000)
+ }).supervise();
+ InMemStateful stateful = conf.getInstance(InMemStateful.class);
+ stateful.init();
}
- protected void tearDown() {
- conf.stop();
- }
-
public void testMapShouldNotRollbackStateForStatefulServerInCaseOfSuccess() {
InMemStateful stateful = conf.getInstance(InMemStateful.class);
stateful.setMapState("testShouldNotRollbackStateForStatefulServerInCaseOfSuccess", "init"); // set init state
diff --git a/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/RemoteInMemoryStateTest.java b/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/RemoteInMemoryStateTest.java
index 62050ea03e..de2c1dbd41 100644
--- a/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/RemoteInMemoryStateTest.java
+++ b/akka-fun-test-java/src/test/java/se/scalablesolutions/akka/api/RemoteInMemoryStateTest.java
@@ -7,7 +7,8 @@ package se.scalablesolutions.akka.api;
import se.scalablesolutions.akka.Config;
import se.scalablesolutions.akka.actor.ActiveObject;
import se.scalablesolutions.akka.config.ActiveObjectConfigurator;
-import se.scalablesolutions.akka.nio.RemoteServer;
+import se.scalablesolutions.akka.nio.RemoteNode;
+
import junit.framework.TestCase;
public class RemoteInMemoryStateTest extends TestCase {
@@ -16,7 +17,7 @@ public class RemoteInMemoryStateTest extends TestCase {
static {
new Thread(new Runnable() {
public void run() {
- RemoteServer.start();
+ RemoteNode.start();
}
}).start();
try { Thread.currentThread().sleep(1000); } catch (Exception e) {}
diff --git a/akka-kernel/pom.xml b/akka-kernel/pom.xml
index 9ed31850e0..583a32c897 100755
--- a/akka-kernel/pom.xml
+++ b/akka-kernel/pom.xml
@@ -19,40 +19,40 @@
akka-actors
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-persistence
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-rest
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-amqp
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-camel
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
-
+
akka-security
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
org.scala-lang
scala-library
- 2.7.5
+ ${scala.version}
org.codehaus.aspectwerkz
@@ -69,25 +69,15 @@
configgy
1.4
-
- org.guiceyfruit
- guiceyfruit-core
- 2.0
-
org.apache.camel
camel-core
2.0-SNAPSHOT
-
- org.guiceyfruit
- guice-core
- 2.0-beta-4
-
org.jboss.netty
netty
- 3.1.0.GA
+ 3.2.0.ALPHA1
org.apache
@@ -165,39 +155,12 @@
cassandra
0.4.1
-
- com.facebook
- thrift
- 1.0
-
commons-pool
commons-pool
1.5.1
-
-
- org.slf4j
- slf4j-log4j12
- 1.4.3
-
-
- org.slf4j
- slf4j-api
- 1.4.3
-
-
- log4j
- log4j
- 1.2.13
-
-
- commons-logging
- commons-logging
- 1.0.4
-
-
com.sun.grizzly
@@ -249,32 +212,32 @@
- org.apache.maven.plugins
- maven-shade-plugin
- 1.2.1
-
-
- install
-
- shade
-
-
-
-
- junit:junit
-
-
-
-
-
-
- se.scalablesolutions.akka.Kernel
-
-
-
-
-
-
+ org.apache.maven.plugins
+ maven-shade-plugin
+ 1.2.1
+
+
+ install
+
+ shade
+
+
+
+
+ junit:junit
+
+
+
+
+
+
+ se.scalablesolutions.akka.Kernel
+
+
+
+
+
+
maven-antrun-plugin
@@ -282,8 +245,8 @@
install
-
+
diff --git a/akka-kernel/src/main/scala/Kernel.scala b/akka-kernel/src/main/scala/Kernel.scala
index 69813fdd18..88eef6b34d 100644
--- a/akka-kernel/src/main/scala/Kernel.scala
+++ b/akka-kernel/src/main/scala/Kernel.scala
@@ -12,7 +12,7 @@ import javax.ws.rs.core.UriBuilder
import java.io.File
import java.net.URLClassLoader
-import se.scalablesolutions.akka.nio.RemoteServer
+import se.scalablesolutions.akka.nio.RemoteNode
import se.scalablesolutions.akka.util.Logging
/**
@@ -54,7 +54,7 @@ object Kernel extends Logging {
def startRemoteService = {
// FIXME manage remote serve thread for graceful shutdown
val remoteServerThread = new Thread(new Runnable() {
- def run = RemoteServer.start(applicationLoader)
+ def run = RemoteNode.start(applicationLoader)
}, "Akka Remote Service")
remoteServerThread.start
}
@@ -94,7 +94,7 @@ object Kernel extends Logging {
val DEPLOY = HOME.get + "/deploy"
val DEPLOY_DIR = new File(DEPLOY)
if (!DEPLOY_DIR.exists) {
- log.error("Could not find a deploy directory at [" + DEPLOY + "]")
+ log.error("Could not find a deploy directory at [%s]", DEPLOY)
System.exit(-1)
}
val toDeploy = for (f <- DEPLOY_DIR.listFiles().toArray.toList.asInstanceOf[List[File]]) yield f.toURL
diff --git a/akka-persistence/pom.xml b/akka-persistence/pom.xml
index 7c3c992079..91a0079611 100644
--- a/akka-persistence/pom.xml
+++ b/akka-persistence/pom.xml
@@ -17,13 +17,13 @@
akka-util
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-actors
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
@@ -49,6 +49,20 @@
commons-pool
1.5.1
+
+
+
+ org.scalatest
+ scalatest
+ 1.0
+ test
+
+
+ junit
+ junit
+ 4.5
+ test
+
diff --git a/akka-persistence/src/main/scala/CassandraStorage.scala b/akka-persistence/src/main/scala/CassandraStorage.scala
index 0769053909..74f06d3a9d 100644
--- a/akka-persistence/src/main/scala/CassandraStorage.scala
+++ b/akka-persistence/src/main/scala/CassandraStorage.scala
@@ -10,8 +10,6 @@ import se.scalablesolutions.akka.serialization.Serializer
import se.scalablesolutions.akka.Config.config
import org.apache.cassandra.service._
-import org.apache.thrift.transport._
-import org.apache.thrift.protocol._
/**
* @author Jonas Bonér
@@ -92,7 +90,7 @@ object CassandraStorage extends MapStorage
else None
} catch {
case e =>
- e.printStackTrace
+ log.error(e, "Could not retreive Ref from storage")
None
}
}
@@ -111,7 +109,7 @@ object CassandraStorage extends MapStorage
}
}
- // FIXME implement
+ // FIXME implement insertVectorStorageEntriesFor
def insertVectorStorageEntriesFor(name: String, elements: List[AnyRef]) = {
throw new UnsupportedOperationException("insertVectorStorageEntriesFor for CassandraStorage is not implemented yet")
}
@@ -189,7 +187,7 @@ object CassandraStorage extends MapStorage
else None
} catch {
case e =>
- e.printStackTrace
+ log.error(e, "Could not retreive Map from storage")
None
}
}
@@ -205,11 +203,8 @@ object CassandraStorage extends MapStorage
}
}
-
- def getMapStorageSizeFor(name: String): Int = {
- sessions.withSession {
- _ |# (name, MAP_COLUMN_PARENT)
- }
+ def getMapStorageSizeFor(name: String): Int = sessions.withSession {
+ _ |# (name, MAP_COLUMN_PARENT)
}
def removeMapStorageFor(name: String): Unit = removeMapStorageFor(name, null)
@@ -234,225 +229,3 @@ object CassandraStorage extends MapStorage
columns.map(column => (column.getColumn.name, serializer.in(column.getColumn.value, None)))
}
}
-
-/**
- * NOTE: requires command line options:
- *
- * -Dcassandra -Dstorage-config=config/ -Dpidfile=akka.pid
- *
- * @author Jonas Bonér
- *
-object EmbeddedCassandraStorage extends Logging {
-val KEYSPACE = "akka"
-val MAP_COLUMN_FAMILY = "map"
-val VECTOR_COLUMN_FAMILY = "vector"
-val REF_COLUMN_FAMILY = "ref:item"
-
-val IS_ASCENDING = true
-
-val RUN_THRIFT_SERVICE = akka.akka.config.getBool("akka.storage.cassandra.thrift-server.service", false)
-val CONSISTENCY_LEVEL = {
-if (akka.akka.config.getBool("akka.storage.cassandra.blocking", true)) 0
-else 1 }
-
-@volatile private[this] var isRunning = false
-private[this] val serializer: Serializer = {
-akka.akka.config.getString("akka.storage.cassandra.storage-format", "java") match {
-case "scala-json" => Serializer.ScalaJSON
-case "java-json" => Serializer.JavaJSON
-case "protobuf" => Serializer.Protobuf
-case "java" => Serializer.Java
-case "sbinary" => throw new UnsupportedOperationException("SBinary serialization protocol is not yet supported for storage")
-case "avro" => throw new UnsupportedOperationException("Avro serialization protocol is not yet supported for storage")
-case unknown => throw new UnsupportedOperationException("Unknown storage serialization protocol [" + unknown + "]")
-}
-}
-
-// TODO: is this server thread-safe or needed to be wrapped up in an actor?
-private[this] val server = classOf[CassandraServer].newInstance.asInstanceOf[CassandraServer]
-
-private[this] var thriftServer: CassandraThriftServer = _
-
-def start = synchronized {
-if (!isRunning) {
-try {
-server.start
-log.info("Cassandra persistent storage has started up successfully");
-} catch {
-case e =>
-log.error("Could not start up Cassandra persistent storage")
-throw e
-}
-if (RUN_THRIFT_SERVICE) {
-thriftServer = new CassandraThriftServer(server)
-thriftServer.start
-}
-isRunning
-}
-}
-
-def stop = if (isRunning) {
-//server.storageService.shutdown
-if (RUN_THRIFT_SERVICE) thriftServer.stop
-}
-
-// ===============================================================
-// For Ref
-// ===============================================================
-
-def insertRefStorageFor(name: String, element: AnyRef) = {
-server.insert(
-KEYSPACE,
-name,
-REF_COLUMN_FAMILY,
-element,
-System.currentTimeMillis,
-CONSISTENCY_LEVEL)
-}
-
-def getRefStorageFor(name: String): Option[AnyRef] = {
-try {
-val column = server.get_column(KEYSPACE, name, REF_COLUMN_FAMILY)
-Some(serializer.in(column.value, None))
-} catch {
-case e =>
-e.printStackTrace
-None }
-}
-
-// ===============================================================
-// For Vector
-// ===============================================================
-
-def insertVectorStorageEntryFor(name: String, element: AnyRef) = {
-server.insert(
-KEYSPACE,
-name,
-VECTOR_COLUMN_FAMILY + ":" + getVectorStorageSizeFor(name),
-element,
-System.currentTimeMillis,
-CONSISTENCY_LEVEL)
-}
-
-def getVectorStorageEntryFor(name: String, index: Int): AnyRef = {
-try {
-val column = server.get_column(KEYSPACE, name, VECTOR_COLUMN_FAMILY + ":" + index)
-serializer.in(column.value, None)
-} catch {
-case e =>
-e.printStackTrace
-throw new Predef.NoSuchElementException(e.getMessage)
-}
-}
-
-def getVectorStorageRangeFor(name: String, start: Int, count: Int): List[AnyRef] =
-server.get_slice(KEYSPACE, name, VECTOR_COLUMN_FAMILY, IS_ASCENDING, count)
-.toArray.toList.asInstanceOf[List[Tuple2[String, AnyRef]]].map(tuple => tuple._2)
-
-def getVectorStorageSizeFor(name: String): Int =
-server.get_column_count(KEYSPACE, name, VECTOR_COLUMN_FAMILY)
-
-// ===============================================================
-// For Map
-// ===============================================================
-
-def insertMapStorageEntryFor(name: String, key: String, value: AnyRef) = {
-server.insert(
-KEYSPACE, name,
-MAP_COLUMN_FAMILY + ":" + key,
-serializer.out(value),
-System.currentTimeMillis,
-CONSISTENCY_LEVEL)
-}
-
-def insertMapStorageEntriesFor(name: String, entries: List[Tuple2[String, AnyRef]]) = {
-import java.util.{ Map, HashMap, List, ArrayList }
-val columns: Map[String, List[column_t]] = new HashMap
-for (entry <- entries) {
-val cls: List[column_t] = new ArrayList
-cls.add(new column_t(entry._1, serializer.out(entry._2), System.currentTimeMillis))
-columns.put(MAP_COLUMN_FAMILY, cls)
-}
-server.batch_insert(new BatchMutation(
-KEYSPACE, name,
-columns),
-CONSISTENCY_LEVEL)
-}
-
-def getMapStorageEntryFor(name: String, key: AnyRef): Option[AnyRef] = {
-try {
-val column = server.get_column(KEYSPACE, name, MAP_COLUMN_FAMILY + ":" + key)
-Some(serializer.in(column.value, None))
-} catch {
-case e =>
-e.printStackTrace
-None
-}
-}
-
-def getMapStorageFor(name: String): List[Tuple2[String, AnyRef]] = {
-val columns = server.get_columns_since(KEYSPACE, name, MAP_COLUMN_FAMILY, -1)
-.toArray.toList.asInstanceOf[List[org.apache.cassandra.service.column_t]]
-for {
-column <- columns
-col = (column.columnName, serializer.in(column.value, None))
-} yield col
-}
-
-def getMapStorageSizeFor(name: String): Int =
-server.get_column_count(KEYSPACE, name, MAP_COLUMN_FAMILY)
-
-def removeMapStorageFor(name: String) =
-server.remove(KEYSPACE, name, MAP_COLUMN_FAMILY, System.currentTimeMillis, CONSISTENCY_LEVEL)
-
-def getMapStorageRangeFor(name: String, start: Int, count: Int): List[Tuple2[String, AnyRef]] = {
-server.get_slice(KEYSPACE, name, MAP_COLUMN_FAMILY, IS_ASCENDING, count)
-.toArray.toList.asInstanceOf[List[Tuple2[String, AnyRef]]]
-}
-}
-
-
-class CassandraThriftServer(server: CassandraServer) extends Logging {
-case object Start
-case object Stop
-
-private[this] val serverEngine: TThreadPoolServer = try {
-val pidFile = akka.akka.config.getString("akka.storage.cassandra.thrift-server.pidfile", "akka.pid")
-if (pidFile != null) new File(pidFile).deleteOnExit();
-val listenPort = DatabaseDescriptor.getThriftPort
-
-val processor = new Cassandra.Processor(server)
-val tServerSocket = new TServerSocket(listenPort)
-val tProtocolFactory = new TBinaryProtocol.Factory
-
-val options = new TThreadPoolServer.Options
-options.minWorkerThreads = 64
-new TThreadPoolServer(new TProcessorFactory(processor),
-tServerSocket,
-new TTransportFactory,
-new TTransportFactory,
-tProtocolFactory,
-tProtocolFactory,
-options)
-} catch {
-case e =>
-log.error("Could not start up Cassandra thrift service")
-throw e
-}
-
-import scala.actors.Actor._
-private[this] val serverDaemon = actor {
-receive {
-case Start =>
-serverEngine.serve
-log.info("Cassandra thrift service has starting up successfully")
-case Stop =>
-log.info("Cassandra thrift service is shutting down...")
-serverEngine.stop
-}
-}
-
-def start = serverDaemon ! Start
-def stop = serverDaemon ! Stop
-}
- */
diff --git a/akka-persistence/src/main/scala/MongoStorage.scala b/akka-persistence/src/main/scala/MongoStorage.scala
index f9f566c92f..8fd7a0c4b5 100644
--- a/akka-persistence/src/main/scala/MongoStorage.scala
+++ b/akka-persistence/src/main/scala/MongoStorage.scala
@@ -90,7 +90,12 @@ object MongoStorage extends MapStorage with VectorStorage with RefStorage with L
case None =>
case Some(dbo) => {
val orig = dbo.get(VALUE).asInstanceOf[DBObject].toMap
- orig.remove(key.asInstanceOf[String])
+ if (key.isInstanceOf[List[_]]) {
+ val keys = key.asInstanceOf[List[_]]
+ keys.foreach(k => orig.remove(k.asInstanceOf[String]))
+ } else {
+ orig.remove(key.asInstanceOf[String])
+ }
// remove existing reference
removeMapStorageFor(name)
@@ -287,4 +292,4 @@ object MongoStorage extends MapStorage with VectorStorage with RefStorage with L
Some(serializer.in[AnyRef](dbo.get(VALUE).asInstanceOf[Array[Byte]]))
}
}
-}
\ No newline at end of file
+}
diff --git a/akka-persistence/src/main/scala/PersistentState.scala b/akka-persistence/src/main/scala/PersistentState.scala
index f659c00f14..3a63ea16f9 100644
--- a/akka-persistence/src/main/scala/PersistentState.scala
+++ b/akka-persistence/src/main/scala/PersistentState.scala
@@ -4,6 +4,7 @@
package se.scalablesolutions.akka.state
+import util.Logging
import se.scalablesolutions.akka.stm.TransactionManagement.currentTransaction
import se.scalablesolutions.akka.collection._
@@ -59,9 +60,9 @@ object PersistentState {
*
* @author Jonas Bonér
*/
-trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Transactional with Committable {
+trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Transactional with Committable with Logging {
protected val newAndUpdatedEntries = TransactionalState.newMap[AnyRef, AnyRef]
- protected val removedEntries = TransactionalState.newMap[AnyRef, AnyRef]
+ protected val removedEntries = TransactionalState.newVector[AnyRef]
protected val shouldClearOnCommit = TransactionalRef[Boolean]()
// to be concretized in subclasses
@@ -91,7 +92,7 @@ trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Tr
def remove(key: AnyRef) = {
register
- removedEntries.remove(key)
+ removedEntries.add(key)
}
def slice(start: Option[AnyRef], count: Int): List[Tuple2[AnyRef, AnyRef]] = slice(start, None, count)
@@ -114,7 +115,9 @@ trait PersistentMap extends scala.collection.mutable.Map[AnyRef, AnyRef] with Tr
} catch { case e: Exception => 0 }
override def get(key: AnyRef): Option[AnyRef] = {
- if (newAndUpdatedEntries.contains(key)) newAndUpdatedEntries.get(key)
+ if (newAndUpdatedEntries.contains(key)) {
+ newAndUpdatedEntries.get(key)
+ }
else try {
storage.getMapStorageEntryFor(uuid, key)
} catch { case e: Exception => None }
diff --git a/akka-persistence/src/test/scala/AllTest.scala b/akka-persistence/src/test/scala/AllTest.scala
index 60374da92d..2e9bc78178 100644
--- a/akka-persistence/src/test/scala/AllTest.scala
+++ b/akka-persistence/src/test/scala/AllTest.scala
@@ -1,6 +1,6 @@
package se.scalablesolutions.akka
-import akka.state.{MongoStorageSpec, MongoPersistentActorSpec, CassandraPersistentActorSpec}
+import se.scalablesolutions.akka.state.{MongoStorageSpec, MongoPersistentActorSpec, CassandraPersistentActorSpec}
import junit.framework.Test
import junit.framework.TestCase
import junit.framework.TestSuite
diff --git a/akka-persistence/src/test/scala/CassandraPersistentActorSpec.scala b/akka-persistence/src/test/scala/CassandraPersistentActorSpec.scala
index 28fe92651b..305763eba3 100644
--- a/akka-persistence/src/test/scala/CassandraPersistentActorSpec.scala
+++ b/akka-persistence/src/test/scala/CassandraPersistentActorSpec.scala
@@ -35,7 +35,7 @@ class CassandraPersistentActor extends Actor {
private lazy val vectorState: PersistentVector = PersistentState.newVector(CassandraStorageConfig())
private lazy val refState: PersistentRef = PersistentState.newRef(CassandraStorageConfig())
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case GetMapState(key) =>
reply(mapState.get(key).get)
case GetVectorSize =>
@@ -67,7 +67,7 @@ class CassandraPersistentActor extends Actor {
@serializable class PersistentFailerActor extends Actor {
makeTransactionRequired
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case "Failure" =>
throw new RuntimeException("expected")
}
diff --git a/akka-persistence/src/test/scala/MongoPersistentActorSpec.scala b/akka-persistence/src/test/scala/MongoPersistentActorSpec.scala
index 4fc18e8967..10bf943dbb 100644
--- a/akka-persistence/src/test/scala/MongoPersistentActorSpec.scala
+++ b/akka-persistence/src/test/scala/MongoPersistentActorSpec.scala
@@ -1,12 +1,15 @@
package se.scalablesolutions.akka.state
-import se.scalablesolutions.akka.actor.Actor
import junit.framework.TestCase
+
import org.junit.{Test, Before}
import org.junit.Assert._
-import _root_.dispatch.json._
+
+import _root_.dispatch.json.{JsNumber, JsValue}
import _root_.dispatch.json.Js._
+import se.scalablesolutions.akka.actor.Actor
+
/**
* A persistent actor based on MongoDB storage.
*
@@ -28,9 +31,10 @@ case object LogSize
class BankAccountActor extends Actor {
makeTransactionRequired
-
- private lazy val accountState: PersistentMap = PersistentState.newMap(MongoStorageConfig())
- private lazy val txnLog: PersistentVector = PersistentState.newVector(MongoStorageConfig())
+ private val accountState =
+ PersistentState.newMap(MongoStorageConfig())
+ private val txnLog =
+ PersistentState.newVector(MongoStorageConfig())
def receive: PartialFunction[Any, Unit] = {
// check balance
@@ -41,10 +45,12 @@ class BankAccountActor extends Actor {
// debit amount: can fail
case Debit(accountNo, amount, failer) =>
txnLog.add("Debit:" + accountNo + " " + amount)
+
val m: BigInt =
accountState.get(accountNo) match {
+ case Some(JsNumber(n)) =>
+ BigInt(n.asInstanceOf[BigDecimal].intValue)
case None => 0
- case Some(v) => v.asInstanceOf[BigInt]
}
accountState.put(accountNo, (m - amount))
if (amount > m)
@@ -55,10 +61,11 @@ class BankAccountActor extends Actor {
// demonstrates true rollback even if multiple puts have been done
case MultiDebit(accountNo, amounts, failer) =>
txnLog.add("MultiDebit:" + accountNo + " " + amounts.map(_.intValue).foldLeft(0)(_ + _))
+
val m: BigInt =
accountState.get(accountNo) match {
+ case Some(JsNumber(n)) => BigInt(n.toString)
case None => 0
- case Some(v) => BigInt(v.asInstanceOf[String])
}
var bal: BigInt = 0
amounts.foreach {amount =>
@@ -71,10 +78,12 @@ class BankAccountActor extends Actor {
// credit amount
case Credit(accountNo, amount) =>
txnLog.add("Credit:" + accountNo + " " + amount)
+
val m: BigInt =
accountState.get(accountNo) match {
+ case Some(JsNumber(n)) =>
+ BigInt(n.asInstanceOf[BigDecimal].intValue)
case None => 0
- case Some(v) => v.asInstanceOf[BigInt]
}
accountState.put(accountNo, (m + amount))
reply(m + amount)
@@ -93,19 +102,20 @@ class MongoPersistentActorSpec extends TestCase {
failer.start
bactor !! Credit("a-123", 5000)
bactor !! Debit("a-123", 3000, failer)
- val b = (bactor !! Balance("a-123"))
- assertTrue(b.isDefined)
- assertEquals(BigInt(2000), b.get)
+
+ val JsNumber(b) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
+ assertEquals(BigInt(2000), BigInt(b.intValue))
bactor !! Credit("a-123", 7000)
- val b1 = (bactor !! Balance("a-123"))
- assertTrue(b1.isDefined)
- assertEquals(BigInt(9000), b1.get)
+
+ val JsNumber(b1) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
+ assertEquals(BigInt(9000), BigInt(b1.intValue))
bactor !! Debit("a-123", 8000, failer)
- val b2 = (bactor !! Balance("a-123"))
- assertTrue(b2.isDefined)
- assertEquals(BigInt(1000), b2.get)
+
+ val JsNumber(b2) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
+ assertEquals(BigInt(1000), BigInt(b2.intValue))
+
assertEquals(7, (bactor !! LogSize).get)
}
@@ -115,9 +125,8 @@ class MongoPersistentActorSpec extends TestCase {
bactor.start
bactor !! Credit("a-123", 5000)
- val b = (bactor !! Balance("a-123"))
- assertTrue(b.isDefined)
- assertEquals(BigInt(5000), b.get)
+ val JsNumber(b) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
+ assertEquals(BigInt(5000), BigInt(b.intValue))
val failer = new PersistentFailerActor
failer.start
@@ -126,160 +135,8 @@ class MongoPersistentActorSpec extends TestCase {
fail("should throw exception")
} catch { case e: RuntimeException => {}}
- val b1 = (bactor !! Balance("a-123"))
- assertTrue(b1.isDefined)
- assertEquals(BigInt(5000), b1.get)
-
- // should not count the failed one
- assertEquals(3, (bactor !! LogSize).get)
- }
-
- @Test
- def testUnsuccessfulMultiDebit = {
- val bactor = new BankAccountActor
- bactor.start
- bactor !! Credit("a-123", 5000)
- val b = (bactor !! Balance("a-123"))
- assertTrue(b.isDefined)
- assertEquals(BigInt(5000), b.get)
-
- val failer = new PersistentFailerActor
- failer.start
- try {
- bactor !! MultiDebit("a-123", List(500, 2000, 1000, 3000), failer)
- fail("should throw exception")
- } catch { case e: RuntimeException => {}}
-
- val b1 = (bactor !! Balance("a-123"))
- assertTrue(b1.isDefined)
- assertEquals(BigInt(5000), b1.get)
-
- // should not count the failed one
- assertEquals(3, (bactor !! LogSize).get)
- }
-}
-
-/*
-case class Balance(accountNo: String)
-case class Debit(accountNo: String, amount: BigInt, failer: Actor)
-case class MultiDebit(accountNo: String, amounts: List[BigInt], failer: Actor)
-case class Credit(accountNo: String, amount: BigInt)
-case object LogSize
-
-class BankAccountActor extends Actor {
- makeTransactionRequired
-
- private var accountState: PersistentMap = _
- private var txnLog: PersistentVector = _
- override def initializeTransactionalState = {
- accountState = PersistentState.newMap(MongoStorageConfig())
- txnLog = PersistentState.newVector(MongoStorageConfig())
- }
-
- def receive: PartialFunction[Any, Unit] = {
- // check balance
- case Balance(accountNo) =>
- txnLog.add("Balance:" + accountNo)
- reply(accountState.get(accountNo).get)
-
- // debit amount: can fail
- case Debit(accountNo, amount, failer) =>
- txnLog.add("Debit:" + accountNo + " " + amount)
- val m: BigInt =
- accountState.get(accountNo) match {
- case None => 0
- case Some(v) => {
- println("======= " + v)
- val JsNumber(n) = v.asInstanceOf[JsValue]
- BigInt(n.toString)
- }
- }
- accountState.put(accountNo, (m - amount))
- if (amount > m)
- failer !! "Failure"
- reply(m - amount)
-
- // many debits: can fail
- // demonstrates true rollback even if multiple puts have been done
- case MultiDebit(accountNo, amounts, failer) =>
- txnLog.add("MultiDebit:" + accountNo + " " + amounts.map(_.intValue).foldLeft(0)(_ + _))
- val m: BigInt =
- accountState.get(accountNo) match {
- case None => 0
- case Some(v) => BigInt(v.asInstanceOf[String])
- }
- var bal: BigInt = 0
- amounts.foreach {amount =>
- bal = bal + amount
- accountState.put(accountNo, (m - bal))
- }
- if (bal > m) failer !! "Failure"
- reply(m - bal)
-
- // credit amount
- case Credit(accountNo, amount) =>
- txnLog.add("Credit:" + accountNo + " " + amount)
- val m: BigInt =
- accountState.get(accountNo) match {
- case None => 0
- case Some(v) => {
- val JsNumber(n) = v.asInstanceOf[JsValue]
- BigInt(n.toString)
- }
- }
- accountState.put(accountNo, (m + amount))
- reply(m + amount)
-
- case LogSize =>
- reply(txnLog.length.asInstanceOf[AnyRef])
- }
-}
-
-class MongoPersistentActorSpec extends TestCase {
- @Test
- def testSuccessfulDebit = {
- val bactor = new BankAccountActor
- bactor.start
- val failer = new PersistentFailerActor
- failer.start
- bactor !! Credit("a-123", 5000)
- bactor !! Debit("a-123", 3000, failer)
- val b = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- val JsNumber(n) = b
- assertEquals(BigInt(2000), BigInt(n.toString))
-
- bactor !! Credit("a-123", 7000)
- val b1 = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- val JsNumber(n1) = b1
- assertEquals(BigInt(9000), BigInt(n1.toString))
-
- bactor !! Debit("a-123", 8000, failer)
- val b2 = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- val JsNumber(n2) = b2
- assertEquals(BigInt(1000), BigInt(n2.toString))
- assertEquals(7, (bactor !! LogSize).get)
- }
-
- @Test
- def testUnsuccessfulDebit = {
- val bactor = new BankAccountActor
- bactor.start
- bactor !! Credit("a-123", 5000)
-
- val b = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- val JsNumber(n) = b
- assertEquals(BigInt(5000), BigInt(n.toString))
-
- val failer = new PersistentFailerActor
- failer.start
- try {
- bactor !! Debit("a-123", 7000, failer)
- fail("should throw exception")
- } catch { case e: RuntimeException => {}}
-
- val b1 = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- val JsNumber(n1) = b1
- assertEquals(BigInt(5000), BigInt(n1.toString))
+ val JsNumber(b1) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
+ assertEquals(BigInt(5000), BigInt(b1.intValue))
// should not count the failed one
assertEquals(3, (bactor !! LogSize).get)
@@ -290,9 +147,9 @@ class MongoPersistentActorSpec extends TestCase {
val bactor = new BankAccountActor
bactor.start
bactor !! Credit("a-123", 5000)
- val b = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- val JsNumber(n) = b
- assertEquals(BigInt(5000), BigInt(n.toString))
+
+ val JsNumber(b) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
+ assertEquals(BigInt(5000), BigInt(b.intValue))
val failer = new PersistentFailerActor
failer.start
@@ -301,12 +158,10 @@ class MongoPersistentActorSpec extends TestCase {
fail("should throw exception")
} catch { case e: RuntimeException => {}}
- val b1 = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- val JsNumber(n1) = b1
- assertEquals(BigInt(5000), BigInt(n1.toString))
+ val JsNumber(b1) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
+ assertEquals(BigInt(5000), BigInt(b1.intValue))
// should not count the failed one
assertEquals(3, (bactor !! LogSize).get)
}
}
-*/
\ No newline at end of file
diff --git a/akka-persistence/src/test/scala/MongoStorageSpec.scala b/akka-persistence/src/test/scala/MongoStorageSpec.scala
index 89971a6c25..4adf61ced2 100644
--- a/akka-persistence/src/test/scala/MongoStorageSpec.scala
+++ b/akka-persistence/src/test/scala/MongoStorageSpec.scala
@@ -272,6 +272,16 @@ class MongoStorageSpec extends TestCase {
fail("should throw exception")
} catch { case e => {}}
+ // remove key "4"
+ MongoStorage.removeMapStorageFor("U-M1", "4")
+ assertEquals(3,
+ MongoStorage.getMapStorageSizeFor("U-M1"))
+
+ // remove key "2"
+ MongoStorage.removeMapStorageFor("U-M1", "2")
+ assertEquals(2,
+ MongoStorage.getMapStorageSizeFor("U-M1"))
+
// remove the whole stuff
MongoStorage.removeMapStorageFor("U-M1")
diff --git a/akka-rest/pom.xml b/akka-rest/pom.xml
index 069b9c6d1a..ab010b9fa9 100644
--- a/akka-rest/pom.xml
+++ b/akka-rest/pom.xml
@@ -19,13 +19,13 @@
akka-util
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-actors
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
diff --git a/akka-samples-java/pom.xml b/akka-samples-java/pom.xml
index 2a7cf2adff..9eb36bc01d 100644
--- a/akka-samples-java/pom.xml
+++ b/akka-samples-java/pom.xml
@@ -17,33 +17,33 @@
akka-util-java
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-util
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-actors
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-persistence
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-rest
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-kernel
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
javax.ws.rs
@@ -73,8 +73,8 @@
install
-
+
diff --git a/akka-samples-lift/pom.xml b/akka-samples-lift/pom.xml
index 20af39b13c..ef47434b1a 100644
--- a/akka-samples-lift/pom.xml
+++ b/akka-samples-lift/pom.xml
@@ -21,33 +21,33 @@
akka-util-java
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-util
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-actors
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-persistence
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-rest
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-kernel
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
javax.ws.rs
@@ -75,6 +75,7 @@
2.5
provided
+
junit
junit
diff --git a/akka-samples-lift/src/main/scala/akka/SimpleService.scala b/akka-samples-lift/src/main/scala/akka/SimpleService.scala
index 8f7537f5bf..b4bbd52157 100644
--- a/akka-samples-lift/src/main/scala/akka/SimpleService.scala
+++ b/akka-samples-lift/src/main/scala/akka/SimpleService.scala
@@ -29,7 +29,7 @@ class SimpleService extends Actor {
@Produces(Array("text/html"))
def count = (this !! Tick).getOrElse(Error in counter
)
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Tick => if (hasStartedTicking) {
val counter = storage.get(KEY).get.asInstanceOf[Integer].intValue
storage.put(KEY, new Integer(counter + 1))
@@ -62,7 +62,7 @@ class PersistentSimpleService extends Actor {
@Produces(Array("text/html"))
def count = (this !! Tick).getOrElse(Error in counter
)
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Tick => if (hasStartedTicking) {
val counter = storage.get(KEY).get.asInstanceOf[Integer].intValue
storage.put(KEY, new Integer(counter + 1))
diff --git a/akka-samples-lift/src/main/scala/bootstrap/liftweb/Boot.scala b/akka-samples-lift/src/main/scala/bootstrap/liftweb/Boot.scala
index ae32e277b6..553d90b277 100644
--- a/akka-samples-lift/src/main/scala/bootstrap/liftweb/Boot.scala
+++ b/akka-samples-lift/src/main/scala/bootstrap/liftweb/Boot.scala
@@ -4,7 +4,6 @@ import _root_.net.liftweb.util._
import _root_.net.liftweb.http._
import _root_.net.liftweb.sitemap._
import _root_.net.liftweb.sitemap.Loc._
-//import _root_.net.liftweb.common._
import _root_.net.liftweb.http.auth._
import Helpers._
@@ -15,9 +14,9 @@ import se.scalablesolutions.akka.util.Logging
import sample.lift.{PersistentSimpleService, SimpleService}
/**
- * A class that's instantiated early and run. It allows the application
- * to modify lift's environment
- */
+ * A class that's instantiated early and run. It allows the application
+ * to modify lift's environment
+ */
class Boot {
def boot {
// where to search snippet
@@ -37,21 +36,17 @@ class Boot {
LiftRules.passNotFoundToChain = true
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
- SupervisorConfig(
- RestartStrategy(OneForOne, 3, 100),
- Supervise(
- new SimpleService,
- LifeCycle(Permanent)) ::
- Supervise(
- new PersistentSimpleService,
- LifeCycle(Permanent)) ::
- Nil)
- }
- }
- val supervisor = factory.newSupervisor
- supervisor.startSupervisor
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ RestartStrategy(OneForOne, 3, 100),
+ Supervise(
+ new SimpleService,
+ LifeCycle(Permanent)) ::
+ Supervise(
+ new PersistentSimpleService,
+ LifeCycle(Permanent)) ::
+ Nil))
+ factory.newInstance.start
// Build SiteMap
// val entries = Menu(Loc("Home", List("index"), "Home")) :: Nil
diff --git a/akka-samples-scala/pom.xml b/akka-samples-scala/pom.xml
index 59e9948e3d..9739f3a07b 100644
--- a/akka-samples-scala/pom.xml
+++ b/akka-samples-scala/pom.xml
@@ -17,33 +17,33 @@
akka-util-java
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-util
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-actors
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-persistence
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-rest
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-kernel
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
javax.ws.rs
@@ -62,8 +62,8 @@
install
-
+
diff --git a/akka-samples-scala/src/main/scala/SimpleService.scala b/akka-samples-scala/src/main/scala/SimpleService.scala
index bbed4f9d75..361c699da7 100644
--- a/akka-samples-scala/src/main/scala/SimpleService.scala
+++ b/akka-samples-scala/src/main/scala/SimpleService.scala
@@ -18,24 +18,20 @@ import org.atmosphere.util.XSSHtmlFilter
import org.atmosphere.cpr.BroadcastFilter
class Boot {
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
- SupervisorConfig(
- RestartStrategy(OneForOne, 3, 100),
- Supervise(
- new SimpleService,
- LifeCycle(Permanent)) ::
- Supervise(
- new Chat,
- LifeCycle(Permanent)) ::
- Supervise(
- new PersistentSimpleService,
- LifeCycle(Permanent))
- :: Nil)
- }
- }
- val supervisor = factory.newSupervisor
- supervisor.startSupervisor
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ RestartStrategy(OneForOne, 3, 100),
+ Supervise(
+ new SimpleService,
+ LifeCycle(Permanent)) ::
+ Supervise(
+ new Chat,
+ LifeCycle(Permanent)) ::
+ Supervise(
+ new PersistentSimpleService,
+ LifeCycle(Permanent))
+ :: Nil))
+ factory.newInstance.start
}
/**
@@ -58,7 +54,7 @@ class SimpleService extends Actor {
@Produces(Array("text/html"))
def count = (this !! Tick).getOrElse(Error in counter )
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Tick => if (hasStartedTicking) {
val counter = storage.get(KEY).get.asInstanceOf[Integer].intValue
storage.put(KEY, new Integer(counter + 1))
@@ -91,7 +87,7 @@ class PersistentSimpleService extends Actor {
@Produces(Array("text/html"))
def count = (this !! Tick).getOrElse(Error in counter )
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Tick => if (hasStartedTicking) {
val counter = storage.get(KEY).get.asInstanceOf[Integer].intValue
storage.put(KEY, new Integer(counter + 1))
@@ -121,7 +117,7 @@ class Chat extends Actor with Logging {
s toString
}
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Chat(who, what, msg) => {
what match {
case "login" => reply("System Message__" + who + " has joined.")
diff --git a/akka-samples-security/pom.xml b/akka-samples-security/pom.xml
index d870bdac17..5a3bd9adb8 100644
--- a/akka-samples-security/pom.xml
+++ b/akka-samples-security/pom.xml
@@ -17,33 +17,33 @@
akka-kernel
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-util-java
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-util
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-actors
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-security
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-persistence
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
javax.ws.rs
@@ -51,9 +51,9 @@
1.0
- javax.annotation
- jsr250-api
- 1.0
+ javax.annotation
+ jsr250-api
+ 1.0
@@ -68,8 +68,8 @@
install
-
+
diff --git a/akka-samples-security/src/main/scala/SimpleService.scala b/akka-samples-security/src/main/scala/SimpleService.scala
index 05bbd75f7b..cca4246933 100644
--- a/akka-samples-security/src/main/scala/SimpleService.scala
+++ b/akka-samples-security/src/main/scala/SimpleService.scala
@@ -11,75 +11,69 @@ import se.scalablesolutions.akka.security.{DigestAuthenticationActor, UserInfo}
import se.scalablesolutions.akka.state.TransactionalState
class Boot {
+ val factory = SupervisorFactory(
+ SupervisorConfig(
+ RestartStrategy(OneForOne, 3, 100),
+ // Dummy implementations of all authentication actors
+ // see akka.conf to enable one of these for the AkkaSecurityFilterFactory
+ Supervise(
+ new BasicAuthenticationService,
+ LifeCycle(Permanent)) ::
+ /**
+ Supervise(
+ new DigestAuthenticationService,
+ LifeCycle(Permanent)) ::
+ Supervise(
+ new SpnegoAuthenticationService,
+ LifeCycle(Permanent)) ::
+ **/
+ Supervise(
+ new SecureTickActor,
+ LifeCycle(Permanent)):: Nil))
- object factory extends SupervisorFactory {
- override def getSupervisorConfig: SupervisorConfig = {
- SupervisorConfig(
- RestartStrategy(OneForOne, 3, 100),
- // Dummy implementations of all authentication actors
- // see akka.conf to enable one of these for the AkkaSecurityFilterFactory
- Supervise(
- new BasicAuthenticationService,
- LifeCycle(Permanent)) ::
- /**
- Supervise(
- new DigestAuthenticationService,
- LifeCycle(Permanent)) ::
- Supervise(
- new SpnegoAuthenticationService,
- LifeCycle(Permanent)) ::
- **/
- Supervise(
- new SecureTickActor,
- LifeCycle(Permanent)):: Nil)
- }
-
- }
-
- val supervisor = factory.newSupervisor
- supervisor.startSupervisor
+ val supervisor = factory.newInstance
+ supervisor.start
}
/*
* In akka.conf you can set the FQN of any AuthenticationActor of your wish, under the property name: akka.rest.authenticator
*/
class DigestAuthenticationService extends DigestAuthenticationActor {
- //If you want to have a distributed nonce-map, you can use something like below,
- //don't forget to configure your standalone Cassandra instance
- //
- //makeTransactionRequired
- //override def mkNonceMap = PersistentState.newMap(CassandraStorageConfig()).asInstanceOf[scala.collection.mutable.Map[String,Long]]
+ //If you want to have a distributed nonce-map, you can use something like below,
+ //don't forget to configure your standalone Cassandra instance
+ //
+ //makeTransactionRequired
+ //override def mkNonceMap = PersistentState.newMap(CassandraStorageConfig()).asInstanceOf[scala.collection.mutable.Map[String,Long]]
- //Use an in-memory nonce-map as default
- override def mkNonceMap = new scala.collection.mutable.HashMap[String,Long]
+ //Use an in-memory nonce-map as default
+ override def mkNonceMap = new scala.collection.mutable.HashMap[String, Long]
- //Change this to whatever you want
- override def realm = "test"
+ //Change this to whatever you want
+ override def realm = "test"
- //Dummy method that allows you to log on with whatever username with the password "bar"
- override def userInfo(username : String) : Option[UserInfo] = Some(UserInfo(username,"bar","ninja" :: "chef" :: Nil))
+ //Dummy method that allows you to log on with whatever username with the password "bar"
+ override def userInfo(username: String): Option[UserInfo] = Some(UserInfo(username, "bar", "ninja" :: "chef" :: Nil))
}
class BasicAuthenticationService extends BasicAuthenticationActor {
- //Change this to whatever you want
- override def realm = "test"
+ //Change this to whatever you want
+ override def realm = "test"
- //Dummy method that allows you to log on with whatever username
- def verify(odc : Option[BasicCredentials]) : Option[UserInfo] = odc match {
- case Some(dc) => userInfo(dc.username)
- case _ => None
- }
+ //Dummy method that allows you to log on with whatever username
+ def verify(odc: Option[BasicCredentials]): Option[UserInfo] = odc match {
+ case Some(dc) => userInfo(dc.username)
+ case _ => None
+ }
- //Dummy method that allows you to log on with whatever username with the password "bar"
- def userInfo(username : String) : Option[UserInfo] = Some(UserInfo(username,"bar","ninja" :: "chef" :: Nil))
+ //Dummy method that allows you to log on with whatever username with the password "bar"
+ def userInfo(username: String): Option[UserInfo] = Some(UserInfo(username, "bar", "ninja" :: "chef" :: Nil))
}
class SpnegoAuthenticationService extends SpnegoAuthenticationActor {
-
- def rolesFor(user: String) = "ninja" :: "chef" :: Nil
+ def rolesFor(user: String) = "ninja" :: "chef" :: Nil
}
@@ -87,16 +81,16 @@ class SpnegoAuthenticationService extends SpnegoAuthenticationActor {
* a REST Actor with class level paranoia settings to deny all access
*
* The interesting part is
- * @RolesAllowed
- * @PermitAll
- * @DenyAll
+ * @RolesAllowed
+ * @PermitAll
+ * @DenyAll
*/
import java.lang.Integer
import javax.annotation.security.{RolesAllowed, DenyAll, PermitAll}
import javax.ws.rs.{GET, Path, Produces}
+
@Path("/secureticker")
class SecureTickActor extends Actor with Logging {
-
makeTransactionRequired
case object Tick
@@ -131,11 +125,13 @@ class SecureTickActor extends Actor with Logging {
def paranoiaTick = tick
def tick = (this !! Tick) match {
- case(Some(counter)) => (Tick: {counter} )
+ case (Some(counter)) => (Tick:
+ {counter}
+ )
case _ => (Error in counter )
}
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Tick => if (hasStartedTicking) {
val counter = storage.get(KEY).get.intValue
storage.put(KEY, counter + 1)
diff --git a/akka-security/pom.xml b/akka-security/pom.xml
index 6e90607d1a..8758508fdd 100644
--- a/akka-security/pom.xml
+++ b/akka-security/pom.xml
@@ -18,32 +18,32 @@
org.scala-lang
scala-library
- 2.7.5
+ ${scala.version}
akka-actors
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-persistence
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
akka-util
- se.scalablesolutions.akka
- 0.6
+ ${project.groupId}
+ ${project.version}
- javax.annotation
- jsr250-api
- 1.0
+ javax.annotation
+ jsr250-api
+ 1.0
com.sun.jersey
@@ -56,11 +56,11 @@
1.0
- net.liftweb
- lift-util
- 1.1-M6
+ net.liftweb
+ lift-util
+ 1.1-M6
-
+
org.scalatest
@@ -80,8 +80,8 @@
1.8.0
test
-
-
+
+
diff --git a/akka-security/src/main/scala/Security.scala b/akka-security/src/main/scala/Security.scala
index da02640f36..f6f2b939a1 100644
--- a/akka-security/src/main/scala/Security.scala
+++ b/akka-security/src/main/scala/Security.scala
@@ -86,12 +86,12 @@ class AkkaSecurityFilterFactory extends ResourceFilterFactory with Logging {
override def filter(request: ContainerRequest): ContainerRequest =
rolesAllowed match {
case Some(roles) => {
- (authenticator !? Authenticate(request, roles)).asInstanceOf[AnyRef] match {
+ (authenticator !! (Authenticate(request, roles), 10000)).get.asInstanceOf[AnyRef] match {
case OK => request
case r if r.isInstanceOf[Response] =>
throw new WebApplicationException(r.asInstanceOf[Response])
case x => {
- log.error("Authenticator replied with unexpected result: ", x);
+ log.error("Authenticator replied with unexpected result [%s]", x);
throw new WebApplicationException(Response.Status.INTERNAL_SERVER_ERROR)
}
}
@@ -100,7 +100,9 @@ class AkkaSecurityFilterFactory extends ResourceFilterFactory with Logging {
}
}
- lazy val authenticatorFQN = Config.config.getString("akka.rest.authenticator").getOrElse(throw new IllegalStateException("akka.rest.authenticator"))
+ lazy val authenticatorFQN =
+ Config.config.getString("akka.rest.authenticator")
+ .getOrElse(throw new IllegalStateException("akka.rest.authenticator"))
/**
* Currently we always take the first, since there usually should be at most one authentication actor, but a round-robin
@@ -264,17 +266,18 @@ trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] {
override def receive = authenticate orElse invalidateNonces
override def unauthorized: Response = {
- val nonce = randomString(64);
+ val nonce = randomString(64)
nonceMap.put(nonce, System.currentTimeMillis)
unauthorized(nonce, "auth", randomString(64))
}
def unauthorized(nonce: String, qop: String, opaque: String): Response = {
- Response.status(401).header("WWW-Authenticate",
+ Response.status(401).header(
+ "WWW-Authenticate",
"Digest realm=\"" + realm + "\", " +
- "qop=\"" + qop + "\", " +
- "nonce=\"" + nonce + "\", " +
- "opaque=\"" + opaque + "\"").build
+ "qop=\"" + qop + "\", " +
+ "nonce=\"" + nonce + "\", " +
+ "opaque=\"" + opaque + "\"").build
}
//Tests wether the specified credentials are valid
@@ -284,9 +287,10 @@ trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials] {
val ha1 = h(auth.userName + ":" + auth.realm + ":" + user.password)
val ha2 = h(auth.method + ":" + auth.uri)
- val response = h(ha1 + ":" + auth.nonce + ":" +
- auth.nc + ":" + auth.cnonce + ":" +
- auth.qop + ":" + ha2)
+ val response = h(
+ ha1 + ":" + auth.nonce + ":" +
+ auth.nc + ":" + auth.cnonce + ":" +
+ auth.qop + ":" + ha2)
(response == auth.response) && (nonceMap.getOrElse(auth.nonce, -1) != -1)
}
@@ -365,7 +369,7 @@ trait SpnegoAuthenticationActor extends AuthenticationActor[SpnegoCredentials] {
Some(UserInfo(user, null, rolesFor(user)))
} catch {
case e: PrivilegedActionException => {
- e.printStackTrace
+ log.error(e, "Action not allowed")
return None
}
}
diff --git a/akka-security/src/test/scala/SecuritySpec.scala b/akka-security/src/test/scala/SecuritySpec.scala
index 880f26693c..feabd8f474 100644
--- a/akka-security/src/test/scala/SecuritySpec.scala
+++ b/akka-security/src/test/scala/SecuritySpec.scala
@@ -26,7 +26,7 @@ class BasicAuthenticatorSpec extends junit.framework.TestCase
@Test def testChallenge = {
val req = mock[ContainerRequest]
- val result: Response = (authenticator !? Authenticate(req, List("foo")))
+ val result: Response = (authenticator !! (Authenticate(req, List("foo")), 10000)).get
// the actor replies with a challenge for the browser
result.getStatus must equal(Response.Status.UNAUTHORIZED.getStatusCode)
@@ -41,7 +41,7 @@ class BasicAuthenticatorSpec extends junit.framework.TestCase
// fake a request authorization -> this will authorize the user
when(req.isUserInRole("chef")).thenReturn(true)
- val result: AnyRef = (authenticator !? Authenticate(req, List("chef")))
+ val result: AnyRef = (authenticator !! (Authenticate(req, List("chef")), 10000)).get
result must be(OK)
// the authenticator must have set a security context
@@ -55,7 +55,7 @@ class BasicAuthenticatorSpec extends junit.framework.TestCase
when(req.getHeaderValue("Authorization")).thenReturn("Basic " + new String(Base64.encode("foo:bar")))
when(req.isUserInRole("chef")).thenReturn(false) // this will deny access
- val result: Response = (authenticator !? Authenticate(req, List("chef")))
+ val result: Response = (authenticator !! (Authenticate(req, List("chef")), 10000)).get
result.getStatus must equal(Response.Status.FORBIDDEN.getStatusCode)
diff --git a/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.java b/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.java
index b956e77c96..950cfeb918 100644
--- a/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.java
+++ b/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.java
@@ -73,36 +73,43 @@ public final class RemoteProtocol {
public boolean hasTarget() { return hasTarget; }
public java.lang.String getTarget() { return target_; }
- // required uint64 timeout = 7;
- public static final int TIMEOUT_FIELD_NUMBER = 7;
+ // required string uuid = 7;
+ public static final int UUID_FIELD_NUMBER = 7;
+ private boolean hasUuid;
+ private java.lang.String uuid_ = "";
+ public boolean hasUuid() { return hasUuid; }
+ public java.lang.String getUuid() { return uuid_; }
+
+ // required uint64 timeout = 8;
+ public static final int TIMEOUT_FIELD_NUMBER = 8;
private boolean hasTimeout;
private long timeout_ = 0L;
public boolean hasTimeout() { return hasTimeout; }
public long getTimeout() { return timeout_; }
- // optional string supervisorUuid = 8;
- public static final int SUPERVISORUUID_FIELD_NUMBER = 8;
+ // optional string supervisorUuid = 9;
+ public static final int SUPERVISORUUID_FIELD_NUMBER = 9;
private boolean hasSupervisorUuid;
private java.lang.String supervisorUuid_ = "";
public boolean hasSupervisorUuid() { return hasSupervisorUuid; }
public java.lang.String getSupervisorUuid() { return supervisorUuid_; }
- // required bool isActor = 9;
- public static final int ISACTOR_FIELD_NUMBER = 9;
+ // required bool isActor = 10;
+ public static final int ISACTOR_FIELD_NUMBER = 10;
private boolean hasIsActor;
private boolean isActor_ = false;
public boolean hasIsActor() { return hasIsActor; }
public boolean getIsActor() { return isActor_; }
- // required bool isOneWay = 10;
- public static final int ISONEWAY_FIELD_NUMBER = 10;
+ // required bool isOneWay = 11;
+ public static final int ISONEWAY_FIELD_NUMBER = 11;
private boolean hasIsOneWay;
private boolean isOneWay_ = false;
public boolean hasIsOneWay() { return hasIsOneWay; }
public boolean getIsOneWay() { return isOneWay_; }
- // required bool isEscaped = 11;
- public static final int ISESCAPED_FIELD_NUMBER = 11;
+ // required bool isEscaped = 12;
+ public static final int ISESCAPED_FIELD_NUMBER = 12;
private boolean hasIsEscaped;
private boolean isEscaped_ = false;
public boolean hasIsEscaped() { return hasIsEscaped; }
@@ -113,6 +120,7 @@ public final class RemoteProtocol {
if (!hasProtocol) return false;
if (!hasMessage) return false;
if (!hasTarget) return false;
+ if (!hasUuid) return false;
if (!hasTimeout) return false;
if (!hasIsActor) return false;
if (!hasIsOneWay) return false;
@@ -140,20 +148,23 @@ public final class RemoteProtocol {
if (hasTarget()) {
output.writeString(6, getTarget());
}
+ if (hasUuid()) {
+ output.writeString(7, getUuid());
+ }
if (hasTimeout()) {
- output.writeUInt64(7, getTimeout());
+ output.writeUInt64(8, getTimeout());
}
if (hasSupervisorUuid()) {
- output.writeString(8, getSupervisorUuid());
+ output.writeString(9, getSupervisorUuid());
}
if (hasIsActor()) {
- output.writeBool(9, getIsActor());
+ output.writeBool(10, getIsActor());
}
if (hasIsOneWay()) {
- output.writeBool(10, getIsOneWay());
+ output.writeBool(11, getIsOneWay());
}
if (hasIsEscaped()) {
- output.writeBool(11, getIsEscaped());
+ output.writeBool(12, getIsEscaped());
}
getUnknownFields().writeTo(output);
}
@@ -188,25 +199,29 @@ public final class RemoteProtocol {
size += com.google.protobuf.CodedOutputStream
.computeStringSize(6, getTarget());
}
+ if (hasUuid()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(7, getUuid());
+ }
if (hasTimeout()) {
size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(7, getTimeout());
+ .computeUInt64Size(8, getTimeout());
}
if (hasSupervisorUuid()) {
size += com.google.protobuf.CodedOutputStream
- .computeStringSize(8, getSupervisorUuid());
+ .computeStringSize(9, getSupervisorUuid());
}
if (hasIsActor()) {
size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(9, getIsActor());
+ .computeBoolSize(10, getIsActor());
}
if (hasIsOneWay()) {
size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(10, getIsOneWay());
+ .computeBoolSize(11, getIsOneWay());
}
if (hasIsEscaped()) {
size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(11, getIsEscaped());
+ .computeBoolSize(12, getIsEscaped());
}
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
@@ -375,6 +390,9 @@ public final class RemoteProtocol {
if (other.hasTarget()) {
setTarget(other.getTarget());
}
+ if (other.hasUuid()) {
+ setUuid(other.getUuid());
+ }
if (other.hasTimeout()) {
setTimeout(other.getTimeout());
}
@@ -439,23 +457,27 @@ public final class RemoteProtocol {
setTarget(input.readString());
break;
}
- case 56: {
+ case 58: {
+ setUuid(input.readString());
+ break;
+ }
+ case 64: {
setTimeout(input.readUInt64());
break;
}
- case 66: {
+ case 74: {
setSupervisorUuid(input.readString());
break;
}
- case 72: {
+ case 80: {
setIsActor(input.readBool());
break;
}
- case 80: {
+ case 88: {
setIsOneWay(input.readBool());
break;
}
- case 88: {
+ case 96: {
setIsEscaped(input.readBool());
break;
}
@@ -584,7 +606,28 @@ public final class RemoteProtocol {
return this;
}
- // required uint64 timeout = 7;
+ // required string uuid = 7;
+ public boolean hasUuid() {
+ return result.hasUuid();
+ }
+ public java.lang.String getUuid() {
+ return result.getUuid();
+ }
+ public Builder setUuid(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasUuid = true;
+ result.uuid_ = value;
+ return this;
+ }
+ public Builder clearUuid() {
+ result.hasUuid = false;
+ result.uuid_ = getDefaultInstance().getUuid();
+ return this;
+ }
+
+ // required uint64 timeout = 8;
public boolean hasTimeout() {
return result.hasTimeout();
}
@@ -602,7 +645,7 @@ public final class RemoteProtocol {
return this;
}
- // optional string supervisorUuid = 8;
+ // optional string supervisorUuid = 9;
public boolean hasSupervisorUuid() {
return result.hasSupervisorUuid();
}
@@ -623,7 +666,7 @@ public final class RemoteProtocol {
return this;
}
- // required bool isActor = 9;
+ // required bool isActor = 10;
public boolean hasIsActor() {
return result.hasIsActor();
}
@@ -641,7 +684,7 @@ public final class RemoteProtocol {
return this;
}
- // required bool isOneWay = 10;
+ // required bool isOneWay = 11;
public boolean hasIsOneWay() {
return result.hasIsOneWay();
}
@@ -659,7 +702,7 @@ public final class RemoteProtocol {
return this;
}
- // required bool isEscaped = 11;
+ // required bool isEscaped = 12;
public boolean hasIsEscaped() {
return result.hasIsEscaped();
}
@@ -1263,17 +1306,17 @@ public final class RemoteProtocol {
java.lang.String[] descriptorData = {
"\n;se/scalablesolutions/akka/nio/protobuf" +
"/RemoteProtocol.proto\022&se.scalablesoluti" +
- "ons.akka.nio.protobuf\"\326\001\n\rRemoteRequest\022" +
+ "ons.akka.nio.protobuf\"\344\001\n\rRemoteRequest\022" +
"\n\n\002id\030\001 \002(\004\022\020\n\010protocol\030\002 \002(\r\022\017\n\007message" +
"\030\003 \002(\014\022\027\n\017messageManifest\030\004 \001(\014\022\016\n\006metho" +
- "d\030\005 \001(\t\022\016\n\006target\030\006 \002(\t\022\017\n\007timeout\030\007 \002(\004" +
- "\022\026\n\016supervisorUuid\030\010 \001(\t\022\017\n\007isActor\030\t \002(" +
- "\010\022\020\n\010isOneWay\030\n \002(\010\022\021\n\tisEscaped\030\013 \002(\010\"\247" +
- "\001\n\013RemoteReply\022\n\n\002id\030\001 \002(\004\022\020\n\010protocol\030\002" +
- " \001(\r\022\017\n\007message\030\003 \001(\014\022\027\n\017messageManifest",
- "\030\004 \001(\014\022\021\n\texception\030\005 \001(\t\022\026\n\016supervisorU" +
- "uid\030\006 \001(\t\022\017\n\007isActor\030\007 \002(\010\022\024\n\014isSuccessf" +
- "ul\030\010 \002(\010B\002H\001"
+ "d\030\005 \001(\t\022\016\n\006target\030\006 \002(\t\022\014\n\004uuid\030\007 \002(\t\022\017\n" +
+ "\007timeout\030\010 \002(\004\022\026\n\016supervisorUuid\030\t \001(\t\022\017" +
+ "\n\007isActor\030\n \002(\010\022\020\n\010isOneWay\030\013 \002(\010\022\021\n\tisE" +
+ "scaped\030\014 \002(\010\"\247\001\n\013RemoteReply\022\n\n\002id\030\001 \002(\004" +
+ "\022\020\n\010protocol\030\002 \001(\r\022\017\n\007message\030\003 \001(\014\022\027\n\017m",
+ "essageManifest\030\004 \001(\014\022\021\n\texception\030\005 \001(\t\022" +
+ "\026\n\016supervisorUuid\030\006 \001(\t\022\017\n\007isActor\030\007 \002(\010" +
+ "\022\024\n\014isSuccessful\030\010 \002(\010"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -1285,7 +1328,7 @@ public final class RemoteProtocol {
internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteRequest_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteRequest_descriptor,
- new java.lang.String[] { "Id", "Protocol", "Message", "MessageManifest", "Method", "Target", "Timeout", "SupervisorUuid", "IsActor", "IsOneWay", "IsEscaped", },
+ new java.lang.String[] { "Id", "Protocol", "Message", "MessageManifest", "Method", "Target", "Uuid", "Timeout", "SupervisorUuid", "IsActor", "IsOneWay", "IsEscaped", },
se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest.class,
se.scalablesolutions.akka.nio.protobuf.RemoteProtocol.RemoteRequest.Builder.class);
internal_static_se_scalablesolutions_akka_nio_protobuf_RemoteReply_descriptor =
diff --git a/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.proto b/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.proto
index 148b3960ae..1248339b3f 100644
--- a/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.proto
+++ b/akka-util-java/src/main/java/se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.proto
@@ -1,7 +1,7 @@
/**
* Copyright (C) 2009 Scalable Solutions.
*/
-
+
package se.scalablesolutions.akka.nio.protobuf;
/*
@@ -10,8 +10,6 @@ package se.scalablesolutions.akka.nio.protobuf;
protoc se/scalablesolutions/akka/nio/protobuf/RemoteProtocol.proto --java_out .
*/
-option optimize_for = SPEED;
-
message RemoteRequest {
required uint64 id = 1;
required uint32 protocol = 2;
@@ -19,11 +17,12 @@ message RemoteRequest {
optional bytes messageManifest = 4;
optional string method = 5;
required string target = 6;
- required uint64 timeout = 7;
- optional string supervisorUuid = 8;
- required bool isActor = 9;
- required bool isOneWay = 10;
- required bool isEscaped = 11;
+ required string uuid = 7;
+ required uint64 timeout = 8;
+ optional string supervisorUuid = 9;
+ required bool isActor = 10;
+ required bool isOneWay = 11;
+ required bool isEscaped = 12;
}
message RemoteReply {
diff --git a/akka-util-java/src/main/java/se/scalablesolutions/akka/stm/AtomicTemplate.java b/akka-util-java/src/main/java/se/scalablesolutions/akka/stm/AtomicTemplate.java
index d01d1c12a4..a693ff1248 100644
--- a/akka-util-java/src/main/java/se/scalablesolutions/akka/stm/AtomicTemplate.java
+++ b/akka-util-java/src/main/java/se/scalablesolutions/akka/stm/AtomicTemplate.java
@@ -1,16 +1,18 @@
package se.scalablesolutions.akka.stm;
-import org.multiverse.templates.AbortedException;
+import static org.multiverse.api.GlobalStmInstance.getGlobalStmInstance;
import org.multiverse.api.Stm;
+import static org.multiverse.api.ThreadLocalTransaction.getThreadLocalTransaction;
+import static org.multiverse.api.ThreadLocalTransaction.setThreadLocalTransaction;
import org.multiverse.api.Transaction;
import org.multiverse.api.TransactionStatus;
import org.multiverse.api.exceptions.CommitFailureException;
import org.multiverse.api.exceptions.LoadException;
import org.multiverse.api.exceptions.RetryError;
import org.multiverse.api.exceptions.TooManyRetriesException;
-import static org.multiverse.api.GlobalStmInstance.getGlobalStmInstance;
-import static org.multiverse.utils.ThreadLocalTransaction.getThreadLocalTransaction;
-import static org.multiverse.utils.ThreadLocalTransaction.setThreadLocalTransaction;
+import org.multiverse.templates.AbortedException;
+import org.multiverse.utils.latches.CheapLatch;
+import org.multiverse.utils.latches.Latch;
import static java.lang.String.format;
import java.util.logging.Logger;
@@ -259,7 +261,9 @@ public abstract class AtomicTemplate {
postCommit();
return result;
} catch (RetryError e) {
- t.abortAndWaitForRetry();
+ Latch latch = new CheapLatch();
+ t.abortAndRegisterRetryLatch(latch);
+ latch.awaitUninterruptible();
//since the abort is already done, no need to do it again.
abort = false;
} catch (CommitFailureException ex) {
@@ -274,7 +278,8 @@ public abstract class AtomicTemplate {
if (abort) {
t.abort();
if (reset) {
- t.restart();
+ t = t.abortAndReturnRestarted();
+ setTransaction(t);
}
}
}
@@ -333,4 +338,4 @@ public abstract class AtomicTemplate {
return (Exception) super.getCause();
}
}
-}
\ No newline at end of file
+}
diff --git a/akka-util/pom.xml b/akka-util/pom.xml
index b8645bd74d..5a7997a3af 100644
--- a/akka-util/pom.xml
+++ b/akka-util/pom.xml
@@ -18,12 +18,12 @@
org.scala-lang
scala-library
- 2.7.5
+ ${scala.version}
net.lag
configgy
- 1.3
+ 1.4
diff --git a/akka-util/src/main/scala/Config.scala b/akka-util/src/main/scala/Config.scala
index 0359a3a58a..e3e82c7af9 100644
--- a/akka-util/src/main/scala/Config.scala
+++ b/akka-util/src/main/scala/Config.scala
@@ -35,7 +35,9 @@ object Config extends Logging {
Configgy.configure(configFile)
log.info("AKKA_HOME is defined to [%s], config loaded from [%s].", HOME.get, configFile)
} catch {
- case e: ParseException => throw new IllegalStateException("'akka.conf' config file can not be found in [" + HOME + "/config/akka.conf] - aborting. Either add it in the 'config' directory or add it to the classpath.")
+ case e: ParseException => throw new IllegalStateException(
+ "'akka.conf' config file can not be found in [" + HOME + "/config/akka.conf] aborting." +
+ "\n\tEither add it in the 'config' directory or add it to the classpath.")
}
} else if (System.getProperty("akka.config", "") != "") {
val configFile = System.getProperty("akka.config", "")
@@ -43,21 +45,26 @@ object Config extends Logging {
Configgy.configure(configFile)
log.info("Config loaded from -Dakka.config=%s", configFile)
} catch {
- case e: ParseException => throw new IllegalStateException("Config could not be loaded from -Dakka.config=" + configFile)
+ case e: ParseException => throw new IllegalStateException(
+ "Config could not be loaded from -Dakka.config=" + configFile)
}
} else {
try {
Configgy.configureFromResource("akka.conf", getClass.getClassLoader)
log.info("Config loaded from the application classpath.")
} catch {
- case e: ParseException => throw new IllegalStateException("'$AKKA_HOME/config/akka.conf' could not be found and no 'akka.conf' can be found on the classpath - aborting. . Either add it in the '$AKKA_HOME/config' directory or add it to the classpath.")
+ case e: ParseException => throw new IllegalStateException(
+ "'$AKKA_HOME/config/akka.conf' could not be found" +
+ "\n\tand no 'akka.conf' can be found on the classpath - aborting." +
+ "\n\tEither add it in the '$AKKA_HOME/config' directory or add it to the classpath.")
}
}
Configgy.config
}
val CONFIG_VERSION = config.getString("akka.version", "0")
- if (VERSION != CONFIG_VERSION) throw new IllegalStateException("Akka JAR version [" + VERSION + "] is different than the provided config ('akka.conf') version [" + CONFIG_VERSION + "]")
+ if (VERSION != CONFIG_VERSION) throw new IllegalStateException(
+ "Akka JAR version [" + VERSION + "] is different than the provided config ('akka.conf') version [" + CONFIG_VERSION + "]")
val startTime = System.currentTimeMillis
def uptime = (System.currentTimeMillis - startTime) / 1000
diff --git a/akka-util/src/main/scala/Logging.scala b/akka-util/src/main/scala/Logging.scala
index 39f2afee21..feb088ac2b 100644
--- a/akka-util/src/main/scala/Logging.scala
+++ b/akka-util/src/main/scala/Logging.scala
@@ -4,11 +4,8 @@
package se.scalablesolutions.akka.util
-import java.util.logging.Level
-import net.lag.configgy.Config
import net.lag.logging.Logger
-import java.util.Date
import java.io.StringWriter;
import java.io.PrintWriter;
import java.net.InetAddress;
@@ -16,15 +13,11 @@ import java.net.UnknownHostException;
/**
* Base trait for all classes that wants to be able use the logging infrastructure.
- *
+ *
* @author Jonas Bonér
*/
trait Logging {
- @transient var log = {
- val log = Logger.get(this.getClass.getName)
- //0log.setLevel(Level.ALL)
- log
- }
+ @transient @volatile var log = Logger.get(this.getClass.getName)
}
/**
@@ -34,7 +27,7 @@ trait Logging {
* It keeps track of the exception is logged or not and also stores the unique id,
* so that it can be carried all along to the client tier and displayed to the end user.
* The end user can call up the customer support using this number.
- *
+ *
* @author Jonas Bonér
*/
class LoggableException extends Exception with Logging {
@@ -50,14 +43,14 @@ class LoggableException extends Exception with Logging {
def logException = synchronized {
if (!isLogged) {
originalException match {
- case Some(e) => log.error("Logged Exception [%s] %s", uniqueId, getStackTrace(e))
- case None => log.error("Logged Exception [%s] %s", uniqueId, getStackTrace(this))
+ case Some(e) => log.error("Logged Exception [%s] %s", uniqueId, getStackTraceAsString(e))
+ case None => log.error("Logged Exception [%s] %s", uniqueId, getStackTraceAsString(this))
}
isLogged = true
}
- }
+ }
- def getExceptionID: String = {
+ private def getExceptionID: String = {
val hostname: String = try {
InetAddress.getLocalHost.getHostName
} catch {
@@ -68,7 +61,7 @@ class LoggableException extends Exception with Logging {
hostname + "_" + System.currentTimeMillis
}
- def getStackTrace(exception: Throwable): String = {
+ private def getStackTraceAsString(exception: Throwable): String = {
val sw = new StringWriter
val pw = new PrintWriter(sw)
exception.printStackTrace(pw)
diff --git a/changes.xml b/changes.xml
index 105309c949..658b8ab550 100644
--- a/changes.xml
+++ b/changes.xml
@@ -10,13 +10,20 @@ description
see http://maven.apache.org/plugins/maven-changes-plugin/usage.html for full guide
-->
+
+
+
Akka Release Notes
-
+ Jonas Bonér
-
+
MongoDB as Akka storage backend
Transparent JSON serialization of Scala objects based on SJSON
MongoDB backed actor example
@@ -25,38 +32,47 @@ see http://maven.apache.org/plugins/maven-changes-plugin/usage.html for full gui
Support for using Scala XML tags in RESTful Actors (scala-jersey)
Support for Comet Actors using Atmosphere
Kerberos/SPNEGO support for Security module
- AMQP integration; abstracted as actors in a supervisor hierarchy. Impl AMQP 0.9.1
Rewritten STM, now integrated with Multiverse STM
Added STM API for atomic {..} and run {..} orElse {..}
Added STM retry
Complete rewrite of the persistence transaction management, now based on Unit of Work and Multiverse STM
Monadic API to TransactionalRef (use it in for-comprehension)
- Lightweight actor syntax 'actor { case _ => .. }'
+ Lightweight actor syntax using one of the Actor.actor(..) methods. F.e: 'actor { case _ => .. }'
New Scala JSON parser based on sjson
+ Added zlib compression to remote actors
+ Added implicit sender reference for fire-forget ('!') message sends
Monadic API to TransactionalRef (use it in for-comprehension)
- Smoother web app integration; just add akka.conf to WEB-INF/classes, no need for AKKA_HOME
+ Smoother web app integration; just add akka.conf to the classpath (WEB-INF/classes), no need for AKKA_HOME or -Dakka.conf=..
Modularization of distribution into a thin core (actors, remoting and STM) and the rest in submodules
JSON serialization for Java objects (using Jackson)
JSON serialization for Scala objects (using SJSON)
+ Added implementation for remote actor reconnect upon failure
Protobuf serialization for Java and Scala objects
SBinary serialization for Scala objects
Protobuf as remote protocol
+ AMQP integration; abstracted as actors in a supervisor hierarchy. Impl AMQP 0.9.1
Updated Cassandra integration and CassandraSession API to v0.4
Added CassandraSession API (with socket pooling) wrapping Cassandra's Thrift API in Scala and Java APIs
CassandraStorage is now works with external Cassandra cluster
- Removed embedded Cassandra mode
- Removed startup scripts and lib dir
- Removed 'Transient' Actors and restart timeout
ActorRegistry for retrieving Actor instances by class name and by id
SchedulerActor for scheduling periodic tasks
Now start up kernel with 'java -jar dist/akka-0.6.jar'
- Concurrent mode is now per actor basis
- Fixed dispatcher bug
- Cleaned up Maven scripts and distribution in general
Added mailing list: akka-user@googlegroups.com
Improved and restructured documentation
New URL: http://akkasource.org
- Fixed many many bugs and minor issues
+ Enhanced trapping of failures: 'trapExit = List(classOf[..], classOf[..])'
+ Upgraded to Netty 3.2, Protobuf 2.2, ScalaTest 1.0, Jersey 1.1.3, Atmosphere 0.4.1, Cassandra 0.4.1, Configgy 1.4
+ Concurrent mode is now per actor basis
+ Remote actors are now defined by their UUID (not class name)
+ Fixed dispatcher bug
+ Cleaned up Maven scripts and distribution in general
+ Fixed many many bugs and minor issues
+ Fixed inconsistencies and uglyness in Actors API
+ Removed embedded Cassandra mode
+ Removed the !? method in Actor (synchronous message send, since it's evil. Use !! with time-out instead.
+ Removed startup scripts and lib dir
+ Removed the 'Transient' life-cycle scope since to close to 'Temporary' in semantics.
+ Removed 'Transient' Actors and restart timeout
diff --git a/config/akka-reference.conf b/config/akka-reference.conf
index a4e42a267a..8ca35927e9 100644
--- a/config/akka-reference.conf
+++ b/config/akka-reference.conf
@@ -22,27 +22,26 @@
boot = ["sample.java.Boot", "sample.scala.Boot", "se.scalablesolutions.akka.security.samples.Boot"]
- timeout = 5000 # default timeout for future based invocations
- serialize-messages = off # does a deep clone of (non-primitive) messages to ensure immutability
+ timeout = 5000 # default timeout for future based invocations
+ serialize-messages = off # does a deep clone of (non-primitive) messages to ensure immutability
service = on
- max-nr-of-retries = 10
- restart-on-collision = off # (not implemented yet) if 'on' then it reschedules the transaction,
- # if 'off' then throws an exception or rollback for user to handle
- wait-for-completion = 1000 # how long time in millis a transaction should be given time to complete when a collision is detected
- wait-nr-of-times = 3 # the number of times it should check for completion of a pending transaction upon collision
- distributed = off # not implemented yet
+ distributed = off # not implemented yet
+ compression-scheme = "zlib" # Options: "zlib" (lzf to come), leave out for no compression
+ zlib-compression-level = 6 # Options: 0-9 (1 being fastest and 9 being the most compressed), default is 6
+
service = on
hostname = "localhost"
port = 9999
connection-timeout = 1000 # in millis (1 sec default)
+
reconnect-delay = 5000 # in millis (5 sec default)
read-timeout = 10000 # in millis (10 sec default)
@@ -64,6 +63,7 @@
storage-format = "scala-json" # Options: java, scala-json, java-json, protobuf
consistency-level = "QUORUM" # Options: ZERO, ONE, QUORUM, ALL
+
hostname = "127.0.0.1" # IP address or hostname of the MongoDB DB instance
port = 27017
diff --git a/docs/scaladocs-akka-actors/actor/ActiveObject.scala.html b/docs/scaladocs-akka-actors/actor/ActiveObject.scala.html
index 1e86256495..2cc8b693e2 100644
--- a/docs/scaladocs-akka-actors/actor/ActiveObject.scala.html
+++ b/docs/scaladocs-akka-actors/actor/ActiveObject.scala.html
@@ -135,7 +135,7 @@ object ActiveObject {
private[akka] def supervise(restartStrategy: RestartStrategy, components: List[Supervise]): Supervisor = {
- object factory extends SupervisorFactory {
+ val factory = SupervisorFactory {
override def getSupervisorConfig = SupervisorConfig(restartStrategy, components)
}
val supervisor = factory.newSupervisor
@@ -333,7 +333,7 @@ private[akka] class Dispatcher(val callbacks: Option[RestartCallbacks]) extends
//if (initTxState.isDefined) initTxState.get.setAccessible(true)
}
- override def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Invocation(joinPoint, isOneWay, _) =>
if (Actor.SERIALIZE_MESSAGES) serializeArguments(joinPoint)
if (isOneWay) joinPoint.proceed
diff --git a/docs/scaladocs-akka-actors/actor/Actor.scala.html b/docs/scaladocs-akka-actors/actor/Actor.scala.html
index e6dcc18edb..646df38c48 100644
--- a/docs/scaladocs-akka-actors/actor/Actor.scala.html
+++ b/docs/scaladocs-akka-actors/actor/Actor.scala.html
@@ -158,7 +158,7 @@ trait Actor extends Logging with TransactionManagement {
* <p/>
* Example code:
* <pre>
- * def receive: PartialFunction[Any, Unit] = {
+ * def receive = {
* case Ping =>
* println("got a ping")
* reply("pong")
@@ -171,7 +171,7 @@ trait Actor extends Logging with TransactionManagement {
* }
* </pre>
*/
- protected def receive: PartialFunction[Any, Unit]
+ protected def receive
/**
* User overridable callback/setting.
diff --git a/docs/scaladocs-akka-actors/actor/Scheduler.scala.html b/docs/scaladocs-akka-actors/actor/Scheduler.scala.html
index 5bca802c80..ff8e609167 100644
--- a/docs/scaladocs-akka-actors/actor/Scheduler.scala.html
+++ b/docs/scaladocs-akka-actors/actor/Scheduler.scala.html
@@ -38,7 +38,7 @@ case class SchedulerException(msg: String, e: Throwable) extends RuntimeExceptio
class ScheduleActor(val receiver: Actor, val future: ScheduledFuture[AnyRef]) extends Actor with Logging {
lifeCycleConfig = Some(LifeCycle(Permanent))
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case UnSchedule =>
Scheduler.stopSupervising(this)
future.cancel(true)
@@ -77,7 +77,7 @@ object Scheduler extends Actor {
service.shutdown
}
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case _ => {} // ignore all messages
}
}
diff --git a/docs/scaladocs-akka-actors/actor/Supervisor.scala.html b/docs/scaladocs-akka-actors/actor/Supervisor.scala.html
index 18ca63a33c..7a92cdda45 100644
--- a/docs/scaladocs-akka-actors/actor/Supervisor.scala.html
+++ b/docs/scaladocs-akka-actors/actor/Supervisor.scala.html
@@ -133,7 +133,7 @@ class Supervisor private[akka] (handler: FaultHandlingStrategy) extends Actor wi
def stopSupervisor = this ! StopSupervisor
- protected def receive: PartialFunction[Any, Unit] = {
+ protected def receive = {
case StartSupervisor =>
linkedActors.toArray.toList.asInstanceOf[List[Actor]].foreach { actor => actor.start; log.info("Starting actor: %s", actor) }
diff --git a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Actor.html b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Actor.html
index 059539ef3e..972addab96 100644
--- a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Actor.html
+++ b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Actor.html
@@ -278,7 +278,7 @@
Example code:
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Ping =>
println("got a ping")
reply("pong")
diff --git a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/ScheduleActor.html b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/ScheduleActor.html
index f4bf833de8..81436a04f7 100644
--- a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/ScheduleActor.html
+++ b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/ScheduleActor.html
@@ -287,7 +287,7 @@
Example code:
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Ping =>
println("got a ping")
reply("pong")
diff --git a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Scheduler$object.html b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Scheduler$object.html
index 8380573c7a..8d5529b56e 100644
--- a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Scheduler$object.html
+++ b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Scheduler$object.html
@@ -282,7 +282,7 @@
Example code:
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Ping =>
println("got a ping")
reply("pong")
diff --git a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Supervisor.html b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Supervisor.html
index 7a7d04b067..f19c18da1a 100644
--- a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Supervisor.html
+++ b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/actor/Supervisor.html
@@ -311,7 +311,7 @@
Example code:
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Ping =>
println("got a ping")
reply("pong")
diff --git a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServer$object.html b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServer$object.html
index 94a5b01949..ca03ada2f5 100644
--- a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServer$object.html
+++ b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServer$object.html
@@ -60,7 +60,7 @@
- Source: RemoteServer.scala(25)
+ Source: RemoteServerNode.scala(25)
diff --git a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServerHandler.html b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServerHandler.html
index 4c1fee9478..70071c45c7 100644
--- a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServerHandler.html
+++ b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServerHandler.html
@@ -60,7 +60,7 @@
- Source: RemoteServer.scala(86)
+ Source: RemoteServerNode.scala(86)
diff --git a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServerPipelineFactory.html b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServerPipelineFactory.html
index 2709a9bb02..53f1eb9b9d 100644
--- a/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServerPipelineFactory.html
+++ b/docs/scaladocs-akka-actors/se/scalablesolutions/akka/nio/RemoteServerPipelineFactory.html
@@ -60,7 +60,7 @@
- Source: RemoteServer.scala(70)
+ Source: RemoteServerNode.scala(70)
diff --git a/docs/scaladocs-akka-amqp/AMQP.scala.html b/docs/scaladocs-akka-amqp/AMQP.scala.html
index 4d30a70242..3757fa33b5 100644
--- a/docs/scaladocs-akka-amqp/AMQP.scala.html
+++ b/docs/scaladocs-akka-amqp/AMQP.scala.html
@@ -41,7 +41,7 @@ import java.io.IOException
* val consumer = AMQP.newConsumer(params, hostname, port, exchange, ExchangeType.Direct, Serializer.ScalaJSON, None, 100)
*
* consumer ! MessageConsumerListener(queue, routingKey, new Actor() {
- * def receive: PartialFunction[Any, Unit] = {
+ * def receive = {
* case Message(payload, _, _, _, _) => log.debug("Received message: %s", payload)
* }
* })
@@ -209,7 +209,7 @@ object AMQP extends Actor {
log.info("AMQP.Producer [%s] is started", toString)
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case message @ Message(payload, routingKey, mandatory, immediate, properties) =>
log.debug("Sending message [%s]", message)
channel.basicPublish(exchangeName, routingKey, mandatory, immediate, properties, serializer.out(payload))
@@ -312,7 +312,7 @@ object AMQP extends Actor {
listener.tag = Some(listenerTag)
}
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case listener: MessageConsumerListener =>
startLink(listener.actor)
listeners.put(listener, listener)
@@ -425,7 +425,7 @@ object AMQP extends Actor {
override def postRestart(reason: AnyRef, config: Option[AnyRef]) = reconnect(initReconnectDelay)
}
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case _ => {} // ignore all messages
}
}
diff --git a/docs/scaladocs-akka-amqp/ExampleSession.scala.html b/docs/scaladocs-akka-amqp/ExampleSession.scala.html
index a937fbf14f..6495493289 100644
--- a/docs/scaladocs-akka-amqp/ExampleSession.scala.html
+++ b/docs/scaladocs-akka-amqp/ExampleSession.scala.html
@@ -40,7 +40,7 @@ object ExampleSession {
def direct = {
val consumer = AMQP.newConsumer(CONFIG, HOSTNAME, PORT, IM, ExchangeType.Direct, SERIALIZER, None, 100, false, false, Map[String, AnyRef]())
consumer ! MessageConsumerListener("@george_bush", "direct", new Actor() {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Message(payload, _, _, _, _) => log.info("@george_bush received message from: %s", payload)
}
})
@@ -51,12 +51,12 @@ object ExampleSession {
def fanout = {
val consumer = AMQP.newConsumer(CONFIG, HOSTNAME, PORT, CHAT, ExchangeType.Fanout, SERIALIZER, None, 100, false, false, Map[String, AnyRef]())
consumer ! MessageConsumerListener("@george_bush", "", new Actor() {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Message(payload, _, _, _, _) => log.info("@george_bush received message from: %s", payload)
}
})
consumer ! MessageConsumerListener("@barack_obama", "", new Actor() {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Message(payload, _, _, _, _) => log.info("@barack_obama received message from: %s", payload)
}
})
diff --git a/docs/scaladocs-akka-amqp/se/scalablesolutions/akka/amqp/AMQP$object.html b/docs/scaladocs-akka-amqp/se/scalablesolutions/akka/amqp/AMQP$object.html
index 827f8ffac8..ed8087c7c1 100644
--- a/docs/scaladocs-akka-amqp/se/scalablesolutions/akka/amqp/AMQP$object.html
+++ b/docs/scaladocs-akka-amqp/se/scalablesolutions/akka/amqp/AMQP$object.html
@@ -63,7 +63,7 @@
val consumer = AMQP.newConsumer(params, hostname, port, exchange, ExchangeType.Direct, Serializer.ScalaJSON, None, 100)
consumer ! MessageConsumerListener(queue, routingKey, new Actor() {
- def receive: PartialFunction[Any, Unit] = {
+ def receive = {
case Message(payload, _, _, _, _) => log.debug("Received message: %s", payload)
}
})
diff --git a/docs/scaladocs-akka-security/Security.scala.html b/docs/scaladocs-akka-security/Security.scala.html
index ce150f8bfc..8620ec9d74 100644
--- a/docs/scaladocs-akka-security/Security.scala.html
+++ b/docs/scaladocs-akka-security/Security.scala.html
@@ -211,7 +211,7 @@ trait AuthenticationActor[C <: Credentials] extends Actor with Logging
}
}
- override def receive: PartialFunction[Any, Unit] = authenticate
+ def receive = authenticate
//returns the string value of the "Authorization"-header of the request
def auth(r : Req) = r.getHeaderValue("Authorization")
@@ -284,7 +284,7 @@ trait DigestAuthenticationActor extends AuthenticationActor[DigestCredentials]
Scheduler.schedule(this, InvalidateNonces, noncePurgeInterval, noncePurgeInterval, TimeUnit.MILLISECONDS )
//authenticate or invalidate nonces
- override def receive: PartialFunction[Any, Unit] = authenticate orElse invalidateNonces
+ def receive = authenticate orElse invalidateNonces
override def unauthorized : Response =
{
diff --git a/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/AuthenticationActor.html b/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/AuthenticationActor.html
index 9d9812df82..460dd68825 100644
--- a/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/AuthenticationActor.html
+++ b/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/AuthenticationActor.html
@@ -253,7 +253,7 @@
receive..
- override def receive
+ def receive
diff --git a/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/BasicAuthenticationActor.html b/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/BasicAuthenticationActor.html
index 3afa719cdb..2e4f01c012 100644
--- a/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/BasicAuthenticationActor.html
+++ b/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/BasicAuthenticationActor.html
@@ -251,7 +251,7 @@
receive..
- override def receive
+ def receive
diff --git a/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/DigestAuthenticationActor.html b/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/DigestAuthenticationActor.html
index 97195537fa..e33418278a 100644
--- a/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/DigestAuthenticationActor.html
+++ b/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/DigestAuthenticationActor.html
@@ -321,7 +321,7 @@
receive..
- override def receive
+ def receive
diff --git a/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/SpnegoAuthenticationActor.html b/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/SpnegoAuthenticationActor.html
index 8cf953060b..ab70c641ee 100644
--- a/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/SpnegoAuthenticationActor.html
+++ b/docs/scaladocs-akka-security/se/scalablesolutions/akka/security/SpnegoAuthenticationActor.html
@@ -276,7 +276,7 @@
receive..
- override def receive
+ def receive
diff --git a/embedded-repo/org/h2/compress/h2-lzf/1.0/h2-lzf-1.0.jar b/embedded-repo/org/h2/compress/h2-lzf/1.0/h2-lzf-1.0.jar
new file mode 100644
index 0000000000..620cfb1371
Binary files /dev/null and b/embedded-repo/org/h2/compress/h2-lzf/1.0/h2-lzf-1.0.jar differ
diff --git a/embedded-repo/org/h2/compress/h2-lzf/maven-metadata-local.xml b/embedded-repo/org/h2/compress/h2-lzf/maven-metadata-local.xml
new file mode 100644
index 0000000000..a5dcb56659
--- /dev/null
+++ b/embedded-repo/org/h2/compress/h2-lzf/maven-metadata-local.xml
@@ -0,0 +1,12 @@
+
+
+ org.h2.compress
+ h2-lzf
+ 1.0
+
+
+ 1.0
+
+ 20091122093116
+
+
diff --git a/pom.xml b/pom.xml
index c6f161a0e4..7ffd4cd08a 100755
--- a/pom.xml
+++ b/pom.xml
@@ -9,12 +9,25 @@
se.scalablesolutions.akka
0.6
2009
+ http://akkasource.org
pom
+
+ Akka implements a unique hybrid of the Actor model and Software Transactional Memory (STM).
+ Akka gives you you:
+ * Concurrency (high-level and simple).
+ * Asynchronous, non-blocking, event-driven and highly performant components.
+ * Scalability through very performant remote actors.
+ * Fault-tolerance through supervision hierarchies with “let-it-crash” semantics.
+
+
- 0.6
- se.scalablesolutions.akka
- 2.7.5
+ 2.7.7
+ UTF-8
+ 1.5
+ ${maven.compiler.source}
+ ${project.build.sourceEncoding}
+ ${project.build.sourceEncoding}
@@ -39,12 +52,6 @@
http://scalablesolutions.se
-
- scm:git:git://github.com/jboner/akka.git
- scm:git:git@github.com:jboner/akka.git
- http://github.com/jboner/akka
-
-
The Apache License, ASL Version 2.0
@@ -66,6 +73,36 @@
+
+ scm:git:git://github.com/jboner/akka.git
+ scm:git:git@github.com:jboner/akka.git
+ http://github.com/jboner/akka
+
+
+
+ assembla
+ http://assembla.com/spaces/akka/
+
+
+
+ hudson
+ http://hudson.scala-tools.org/job/akka/
+
+
+
+
+
+
+
+
+ User and Developer Discussion List
+ http://groups.google.com/group/akka-user
+ akka-user@googlegroups.com
+ akka-user+subscribe@googlegroups.com
+ akka-user+unsubscribe@googlegroups.com
+
+
+
project.embedded.module
@@ -262,7 +299,7 @@
- ${akka.version}
+ ${project.version}
@@ -348,28 +385,58 @@
+ org.apache.maven.plugins
maven-project-info-reports-plugin
+ 2.1.2
+
+
+
+ cim
+ dependencies
+ dependency-convergence
+
+ index
+ issue-tracking
+ license
+ mailing-list
+
+ plugins
+ project-team
+ scm
+ summary
+
+
+
org.scala-tools
maven-scala-plugin
- 2.9.1
+ 2.12.2
-
- -Xmx1024m
- -DpackageLinkDefs=file://${basedir}/../vscaladocs-packageLinkDefs.properties
-
-
-
-
+ ${project.build.sourceEncoding}
+
1.2-SNAPSHOT
${scala.version}
+
+ -Xmx1024m
+ -DpackageLinkDefs=file://${project.build.directory}/packageLinkDefs.properties
+
+
+
org.apache.maven.plugins
maven-changes-plugin
- 2.0-beta-3
+ 2.1