diff --git a/akka-actor/src/main/resources/logback.xml b/akka-actor/src/main/resources/logback.xml
deleted file mode 100644
index 4635396601..0000000000
--- a/akka-actor/src/main/resources/logback.xml
+++ /dev/null
@@ -1,31 +0,0 @@
-
-
-
-
-
-
-
-
-
-
-
-
-
- [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n
-
-
-
- ./logs/akka.log
-
- [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n
-
-
- ./logs/akka.log.%d{yyyy-MM-dd-HH}
-
-
-
-
-
-
-
-
diff --git a/akka-actor/src/main/scala/actor/Actor.scala b/akka-actor/src/main/scala/actor/Actor.scala
index d3e7699403..d232ca2a77 100644
--- a/akka-actor/src/main/scala/actor/Actor.scala
+++ b/akka-actor/src/main/scala/actor/Actor.scala
@@ -60,8 +60,8 @@ case object ReceiveTimeout extends LifeCycleMessage
case class MaximumNumberOfRestartsWithinTimeRangeReached(
@BeanProperty val victim: ActorRef,
- @BeanProperty val maxNrOfRetries: Int,
- @BeanProperty val withinTimeRange: Int,
+ @BeanProperty val maxNrOfRetries: Option[Int],
+ @BeanProperty val withinTimeRange: Option[Int],
@BeanProperty val lastExceptionCausingRestart: Throwable) extends LifeCycleMessage
// Exceptions for Actors
@@ -410,14 +410,14 @@ trait Actor extends Logging {
*
* Is called when an Actor is started by invoking 'actor.start'.
*/
- def init {}
+ def preStart {}
/**
* User overridable callback.
*
* Is called when 'actor.stop' is invoked.
*/
- def shutdown {}
+ def postStop {}
/**
* User overridable callback.
@@ -433,13 +433,6 @@ trait Actor extends Logging {
*/
def postRestart(reason: Throwable) {}
- /**
- * User overridable callback.
- *
- * Is called during initialization. Can be used to initialize transactional state. Will be invoked within a transaction.
- */
- def initTransactionalState {}
-
/**
* Is the actor able to handle the message passed in as arguments?
*/
diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala
index a6b42db579..4905e62670 100644
--- a/akka-actor/src/main/scala/actor/ActorRef.scala
+++ b/akka-actor/src/main/scala/actor/ActorRef.scala
@@ -73,7 +73,6 @@ trait ActorRef extends
@volatile protected[akka] var _isBeingRestarted = false
@volatile protected[akka] var _homeAddress = new InetSocketAddress(RemoteServerModule.HOSTNAME, RemoteServerModule.PORT)
@volatile protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None
- @volatile protected[akka] var startOnCreation = false
@volatile protected[akka] var registeredInRemoteNodeDuringSerialization = false
protected[akka] val guard = new ReentrantGuard
@@ -197,19 +196,10 @@ trait ActorRef extends
*/
@volatile private[akka] var _transactionFactory: Option[TransactionFactory] = None
- /**
- * This lock ensures thread safety in the dispatching: only one message can
- * be dispatched at once on the actor.
- */
- protected[akka] val dispatcherLock = new ReentrantLock
-
/**
* This is a reference to the message currently being processed by the actor
*/
- protected[akka] var _currentMessage: Option[MessageInvocation] = None
-
- protected[akka] def currentMessage_=(msg: Option[MessageInvocation]) = guard.withGuard { _currentMessage = msg }
- protected[akka] def currentMessage = guard.withGuard { _currentMessage }
+ @volatile protected[akka] var currentMessage: MessageInvocation = null
/**
* Comparison only takes uuid into account.
@@ -611,9 +601,9 @@ trait ActorRef extends
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit
- protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit
+ protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit
- protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit
+ protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit
protected[akka] def registerSupervisorAsRemoteActor: Option[String]
@@ -827,12 +817,11 @@ class LocalActorRef private[akka](
_transactionFactory = None
_isRunning = false
_isShutDown = true
- actor.shutdown
+ actor.postStop
ActorRegistry.unregister(this)
if (isRemotingEnabled) {
- remoteAddress.foreach { address =>
- RemoteClientModule.unregister(address, uuid)
- }
+ if(remoteAddress.isDefined)
+ RemoteClientModule.unregister(remoteAddress.get, uuid)
RemoteServerModule.unregister(this)
}
nullOutActorRefReferencesFor(actorInstance.get)
@@ -875,11 +864,11 @@ class LocalActorRef private[akka](
*
* To be invoked from within the actor itself.
*/
- def startLink(actorRef: ActorRef) = guard.withGuard {
+ def startLink(actorRef: ActorRef):Unit = guard.withGuard {
try {
- actorRef.start
- } finally {
link(actorRef)
+ } finally {
+ actorRef.start
}
}
@@ -888,13 +877,13 @@ class LocalActorRef private[akka](
*
* To be invoked from within the actor itself.
*/
- def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int) = guard.withGuard {
+ def startLinkRemote(actorRef: ActorRef, hostname: String, port: Int): Unit = guard.withGuard {
ensureRemotingEnabled
try {
actorRef.makeRemote(hostname, port)
- actorRef.start
- } finally {
link(actorRef)
+ } finally {
+ actorRef.start
}
}
@@ -904,9 +893,7 @@ class LocalActorRef private[akka](
* To be invoked from within the actor itself.
*/
def spawn(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard {
- val actorRef = spawnButDoNotStart(clazz)
- actorRef.start
- actorRef
+ spawnButDoNotStart(clazz).start
}
/**
@@ -930,9 +917,9 @@ class LocalActorRef private[akka](
def spawnLink(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard {
val actor = spawnButDoNotStart(clazz)
try {
- actor.start
- } finally {
link(actor)
+ } finally {
+ actor.start
}
actor
}
@@ -947,10 +934,11 @@ class LocalActorRef private[akka](
val actor = spawnButDoNotStart(clazz)
try {
actor.makeRemote(hostname, port)
- actor.start
- } finally {
link(actor)
+ } finally {
+ actor.start
}
+ actor
}
/**
@@ -980,12 +968,12 @@ class LocalActorRef private[akka](
protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit = {
joinTransaction(message)
- if (isRemotingEnabled && remoteAddress.isDefined) {
+ if (remoteAddress.isDefined && isRemotingEnabled) {
RemoteClientModule.send[Any](
message, senderOption, None, remoteAddress.get, timeout, true, this, None, ActorType.ScalaActor)
} else {
val invocation = new MessageInvocation(this, message, senderOption, None, transactionSet.get)
- invocation.send
+ dispatcher dispatch invocation
}
}
@@ -996,7 +984,7 @@ class LocalActorRef private[akka](
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
joinTransaction(message)
- if (isRemotingEnabled && remoteAddress.isDefined) {
+ if (remoteAddress.isDefined && isRemotingEnabled) {
val future = RemoteClientModule.send[T](
message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, ActorType.ScalaActor)
if (future.isDefined) future.get
@@ -1006,7 +994,7 @@ class LocalActorRef private[akka](
else new DefaultCompletableFuture[T](timeout)
val invocation = new MessageInvocation(
this, message, senderOption, Some(future.asInstanceOf[CompletableFuture[Any]]), transactionSet.get)
- invocation.send
+ dispatcher dispatch invocation
future
}
}
@@ -1018,7 +1006,7 @@ class LocalActorRef private[akka](
if (isShutdown)
Actor.log.warning("Actor [%s] is shut down,\n\tignoring message [%s]", toString, messageHandle)
else {
- currentMessage = Option(messageHandle)
+ currentMessage = messageHandle
try {
dispatch(messageHandle)
} catch {
@@ -1026,7 +1014,7 @@ class LocalActorRef private[akka](
Actor.log.error(e, "Could not invoke actor [%s]", this)
throw e
} finally {
- currentMessage = None //TODO: Don't reset this, we might want to resend the message
+ currentMessage = null //TODO: Don't reset this, we might want to resend the message
}
}
}
@@ -1049,12 +1037,18 @@ class LocalActorRef private[akka](
}
}
- protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = {
+ protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = {
if (maxNrOfRetriesCount == 0) restartsWithinTimeRangeTimestamp = System.currentTimeMillis // first time around
- maxNrOfRetriesCount += 1
+
+ val tooManyRestarts = if (maxNrOfRetries.isDefined) {
+ maxNrOfRetriesCount += 1
+ maxNrOfRetriesCount > maxNrOfRetries.get
+ } else false
+
+ val restartingHasExpired = if (withinTimeRange.isDefined)
+ (System.currentTimeMillis - restartsWithinTimeRangeTimestamp) > withinTimeRange.get
+ else false
- val tooManyRestarts = maxNrOfRetriesCount > maxNrOfRetries
- val restartingHasExpired = (System.currentTimeMillis - restartsWithinTimeRangeTimestamp) > withinTimeRange
if (tooManyRestarts || restartingHasExpired) {
val notification = MaximumNumberOfRestartsWithinTimeRangeReached(this, maxNrOfRetries, withinTimeRange, reason)
Actor.log.warning(
@@ -1092,7 +1086,7 @@ class LocalActorRef private[akka](
}
}
- protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int) = {
+ protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) = {
linkedActorsAsList.foreach { actorRef =>
actorRef.lifeCycle match {
// either permanent or none where default is permanent
@@ -1134,8 +1128,7 @@ class LocalActorRef private[akka](
failedActor.preRestart(reason)
nullOutActorRefReferencesFor(failedActor)
val freshActor = newActor
- freshActor.init
- freshActor.initTransactionalState
+ freshActor.preStart
actorInstance.set(freshActor)
if (failedActor.isInstanceOf[Proxyable])
failedActor.asInstanceOf[Proxyable].swapProxiedActor(freshActor)
@@ -1143,26 +1136,20 @@ class LocalActorRef private[akka](
freshActor.postRestart(reason)
}
- private def spawnButDoNotStart(clazz: Class[_ <: Actor]): ActorRef = guard.withGuard {
- val actorRef = Actor.actorOf(clazz.newInstance)
- if (!dispatcher.isInstanceOf[ThreadBasedDispatcher]) actorRef.dispatcher = dispatcher
- actorRef
- }
+ private def spawnButDoNotStart(clazz: Class[_ <: Actor]): ActorRef = Actor.actorOf(clazz.newInstance)
private[this] def newActor: Actor = {
Actor.actorRefInCreation.withValue(Some(this)){
isInInitialization = true
val actor = actorFactory match {
case Left(Some(clazz)) =>
- try {
- clazz.newInstance
- } catch {
- case e: InstantiationException => throw new ActorInitializationException(
- "Could not instantiate Actor due to:\n" + e +
+ import ReflectiveAccess.{createInstance,noParams,noArgs}
+ createInstance(clazz.asInstanceOf[Class[_]],noParams,noArgs).
+ getOrElse(throw new ActorInitializationException(
+ "Could not instantiate Actor" +
"\nMake sure Actor is NOT defined inside a class/trait," +
"\nif so put it outside the class/trait, f.e. in a companion object," +
- "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.")
- }
+ "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'."))
case Right(Some(factory)) =>
factory()
case _ =>
@@ -1192,7 +1179,7 @@ class LocalActorRef private[akka](
}
private def dispatch[T](messageHandle: MessageInvocation) = {
- Actor.log.trace("Invoking actor with message:\n" + messageHandle)
+ Actor.log.trace("Invoking actor with message: %s\n",messageHandle)
val message = messageHandle.message //serializeMessage(messageHandle.message)
var topLevelTransaction = false
val txSet: Option[CountDownCommitBarrier] =
@@ -1305,8 +1292,7 @@ class LocalActorRef private[akka](
}
private def initializeActorInstance = {
- actor.init // run actor init and initTransactionalState callbacks
- actor.initTransactionalState
+ actor.preStart // run actor preStart
Actor.log.trace("[%s] has started", toString)
ActorRegistry.register(this)
if (id == "N/A") id = actorClass.getName // if no name set, then use default name (class name)
@@ -1357,17 +1343,18 @@ object RemoteActorSystemMessage {
* @author Jonas Bonér
*/
private[akka] case class RemoteActorRef private[akka] (
- uuuid: String,
+ classOrServiceName: String,
val className: String,
val hostname: String,
val port: Int,
_timeout: Long,
- loader: Option[ClassLoader])
+ loader: Option[ClassLoader],
+ val actorType: ActorType = ActorType.ScalaActor)
extends ActorRef with ScalaActorRef {
ensureRemotingEnabled
- _uuid = uuuid
+ id = classOrServiceName
timeout = _timeout
start
@@ -1375,7 +1362,7 @@ private[akka] case class RemoteActorRef private[akka] (
def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]): Unit =
RemoteClientModule.send[Any](
- message, senderOption, None, remoteAddress.get, timeout, true, this, None, ActorType.ScalaActor)
+ message, senderOption, None, remoteAddress.get, timeout, true, this, None, actorType)
def postMessageToMailboxAndCreateFutureResultWithTimeout[T](
message: Any,
@@ -1383,7 +1370,7 @@ private[akka] case class RemoteActorRef private[akka] (
senderOption: Option[ActorRef],
senderFuture: Option[CompletableFuture[T]]): CompletableFuture[T] = {
val future = RemoteClientModule.send[T](
- message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, ActorType.ScalaActor)
+ message, senderOption, senderFuture, remoteAddress.get, timeout, false, this, None, actorType)
if (future.isDefined) future.get
else throw new IllegalActorStateException("Expected a future from remote call to actor " + toString)
}
@@ -1431,8 +1418,8 @@ private[akka] case class RemoteActorRef private[akka] (
protected[akka] def mailbox: AnyRef = unsupported
protected[akka] def mailbox_=(value: AnyRef):AnyRef = unsupported
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
- protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
- protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
+ protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
+ protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def linkedActors: JMap[String, ActorRef] = unsupported
protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = unsupported
@@ -1539,10 +1526,9 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
* Is defined if the message was sent from another Actor, else None.
*/
def sender: Option[ActorRef] = {
- // Five lines of map-performance-avoidance, could be just: currentMessage map { _.sender }
val msg = currentMessage
- if (msg.isEmpty) None
- else msg.get.sender
+ if (msg eq null) None
+ else msg.sender
}
/**
@@ -1550,10 +1536,9 @@ trait ScalaActorRef extends ActorRefShared { ref: ActorRef =>
* Is defined if the message was sent with sent with '!!' or '!!!', else None.
*/
def senderFuture(): Option[CompletableFuture[Any]] = {
- // Five lines of map-performance-avoidance, could be just: currentMessage map { _.senderFuture }
val msg = currentMessage
- if (msg.isEmpty) None
- else msg.get.senderFuture
+ if (msg eq null) None
+ else msg.senderFuture
}
diff --git a/akka-actor/src/main/scala/actor/ActorRegistry.scala b/akka-actor/src/main/scala/actor/ActorRegistry.scala
index e8c38f2b76..51bbfd3477 100644
--- a/akka-actor/src/main/scala/actor/ActorRegistry.scala
+++ b/akka-actor/src/main/scala/actor/ActorRegistry.scala
@@ -11,6 +11,7 @@ import java.util.concurrent.{ConcurrentSkipListSet, ConcurrentHashMap}
import java.util.{Set => JSet}
import se.scalablesolutions.akka.util.ListenerManagement
+import annotation.tailrec
/**
* Base trait for ActorRegistry events, allows listen to when an actor is added and removed from the ActorRegistry.
@@ -35,10 +36,8 @@ case class ActorUnregistered(actor: ActorRef) extends ActorRegistryEvent
*/
object ActorRegistry extends ListenerManagement {
private val actorsByUUID = new ConcurrentHashMap[String, ActorRef]
- private val actorsById = new ConcurrentHashMap[String, JSet[ActorRef]]
-
- private val Naught = Array[ActorRef]() //Nil for Arrays
-
+ private val actorsById = new Index[String,ActorRef]
+
/**
* Returns all actors in the system.
*/
@@ -108,43 +107,25 @@ object ActorRegistry extends ListenerManagement {
/**
* Finds all actors that has a specific id.
*/
- def actorsFor(id: String): Array[ActorRef] = {
- if (actorsById.containsKey(id)) {
- actorsById.get(id).toArray(Naught)
- } else Naught
- }
+ def actorsFor(id: String): Array[ActorRef] = actorsById values id
/**
* Finds the actor that has a specific UUID.
*/
- def actorFor(uuid: String): Option[ActorRef] = {
- if (actorsByUUID.containsKey(uuid)) Some(actorsByUUID.get(uuid))
- else None
- }
+ def actorFor(uuid: String): Option[ActorRef] = Option(actorsByUUID get uuid)
/**
* Registers an actor in the ActorRegistry.
*/
def register(actor: ActorRef) = {
// ID
- val id = actor.id
- if (id eq null) throw new IllegalActorStateException("Actor.id is null " + actor)
-
- val set = actorsById get id
- if (set ne null) set add actor
- else {
- val newSet = new ConcurrentSkipListSet[ActorRef]
- newSet add actor
- val oldSet = actorsById.putIfAbsent(id,newSet)
- // Parry for two simultaneous putIfAbsent(id,newSet)
- if (oldSet ne null) oldSet add actor
- }
+ actorsById.put(actor.id, actor)
// UUID
actorsByUUID.put(actor.uuid, actor)
// notify listeners
- foreachListener(_ ! ActorRegistered(actor))
+ notifyListeners(ActorRegistered(actor))
}
/**
@@ -153,13 +134,10 @@ object ActorRegistry extends ListenerManagement {
def unregister(actor: ActorRef) = {
actorsByUUID remove actor.uuid
- val set = actorsById get actor.id
- if (set ne null) set remove actor
-
- //FIXME: safely remove set if empty, leaks memory
+ actorsById.remove(actor.id,actor)
// notify listeners
- foreachListener(_ ! ActorUnregistered(actor))
+ notifyListeners(ActorUnregistered(actor))
}
/**
@@ -173,3 +151,85 @@ object ActorRegistry extends ListenerManagement {
log.info("All actors have been shut down and unregistered from ActorRegistry")
}
}
+
+class Index[K <: AnyRef,V <: AnyRef : Manifest] {
+ import scala.collection.JavaConversions._
+
+ private val Naught = Array[V]() //Nil for Arrays
+ private val container = new ConcurrentHashMap[K, JSet[V]]
+ private val emptySet = new ConcurrentSkipListSet[V]
+
+ def put(key: K, value: V) {
+
+ //Returns whether it needs to be retried or not
+ def tryPut(set: JSet[V], v: V): Boolean = {
+ set.synchronized {
+ if (set.isEmpty) true //IF the set is empty then it has been removed, so signal retry
+ else { //Else add the value to the set and signal that retry is not needed
+ set add v
+ false
+ }
+ }
+ }
+
+ @tailrec def syncPut(k: K, v: V): Boolean = {
+ var retry = false
+ val set = container get k
+ if (set ne null) retry = tryPut(set,v)
+ else {
+ val newSet = new ConcurrentSkipListSet[V]
+ newSet add v
+
+ // Parry for two simultaneous putIfAbsent(id,newSet)
+ val oldSet = container.putIfAbsent(k,newSet)
+ if (oldSet ne null)
+ retry = tryPut(oldSet,v)
+ }
+
+ if (retry) syncPut(k,v)
+ else true
+ }
+
+ syncPut(key,value)
+ }
+
+ def values(key: K) = {
+ val set: JSet[V] = container get key
+ if (set ne null) set toArray Naught
+ else Naught
+ }
+
+ def foreach(key: K)(fun: (V) => Unit) {
+ val set = container get key
+ if (set ne null)
+ set foreach fun
+ }
+
+ def findValue(key: K)(f: (V) => Boolean): Option[V] = {
+ val set = container get key
+ if (set ne null)
+ set.iterator.find(f)
+ else
+ None
+ }
+
+ def foreach(fun: (K,V) => Unit) {
+ container.entrySet foreach {
+ (e) => e.getValue.foreach(fun(e.getKey,_))
+ }
+ }
+
+ def remove(key: K, value: V) {
+ val set = container get key
+ if (set ne null) {
+ set.synchronized {
+ if (set.remove(value)) { //If we can remove the value
+ if (set.isEmpty) //and the set becomes empty
+ container.remove(key,emptySet) //We try to remove the key if it's mapped to an empty set
+ }
+ }
+ }
+ }
+
+ def clear = { foreach(remove _) }
+}
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/actor/Agent.scala b/akka-actor/src/main/scala/actor/Agent.scala
index df358cdfc4..e5b00d4f5e 100644
--- a/akka-actor/src/main/scala/actor/Agent.scala
+++ b/akka-actor/src/main/scala/actor/Agent.scala
@@ -6,7 +6,7 @@ package se.scalablesolutions.akka.actor
import se.scalablesolutions.akka.stm.Ref
import se.scalablesolutions.akka.AkkaException
-
+import se.scalablesolutions.akka.util.{ Function => JFunc, Procedure => JProc }
import java.util.concurrent.atomic.AtomicReference
import java.util.concurrent.CountDownLatch
@@ -136,6 +136,13 @@ sealed class Agent[T] private (initialValue: T) {
*/
final def apply(message: (T => T)): Unit = dispatcher ! Function(message)
+ /**
+ * Submits the provided function for execution against the internal agent's state.
+ * Java API
+ */
+ final def apply(message: JFunc[T,T]): Unit = dispatcher ! Function((t: T) => message(t))
+
+
/**
* Submits a new value to be set as the new agent's internal state.
*/
@@ -146,6 +153,12 @@ sealed class Agent[T] private (initialValue: T) {
*/
final def send(message: (T) => T): Unit = dispatcher ! Function(message)
+ /**
+ * Submits the provided function of type 'T => T' for execution against the internal agent's state.
+ * Java API
+ */
+ final def send(message: JFunc[T,T]): Unit = dispatcher ! Function((t: T) => message(t))
+
/**
* Submits a new value to be set as the new agent's internal state.
*/
@@ -159,6 +172,15 @@ sealed class Agent[T] private (initialValue: T) {
*/
final def sendProc(f: (T) => Unit): Unit = dispatcher ! Procedure(f)
+ /**
+ * Asynchronously submits a procedure of type 'T => Unit' to read the internal state.
+ * The supplied procedure will be executed on the returned internal state value. A copy
+ * of the internal state will be used, depending on the underlying effective copyStrategy.
+ * Does not change the value of the agent (this).
+ * Java API
+ */
+ final def sendProc(f: JProc[T]): Unit = dispatcher ! Procedure((t:T) => f(t))
+
/**
* Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result.
* Does not change the value of the agent (this).
@@ -177,6 +199,27 @@ sealed class Agent[T] private (initialValue: T) {
*/
final def foreach(f: (T) => Unit): Unit = f(get)
+ /**
+ * Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result.
+ * Does not change the value of the agent (this).
+ * Java API
+ */
+ final def map[B](f: JFunc[T,B]): Agent[B] = Agent(f(get))
+
+ /**
+ * Applies function with type 'T => B' to the agent's internal state and then returns a new agent with the result.
+ * Does not change the value of the agent (this).
+ * Java API
+ */
+ final def flatMap[B](f: JFunc[T,Agent[B]]): Agent[B] = Agent(f(get)())
+
+ /**
+ * Applies procedure with type T to the agent's internal state.
+ * Does not change the value of the agent (this).
+ * Java API
+ */
+ final def foreach(f: JProc[T]): Unit = f(get)
+
/**
* Closes the agents and makes it eligable for garbage collection.
*
diff --git a/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala b/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala
index dfb8541396..278a75418e 100644
--- a/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala
+++ b/akka-actor/src/main/scala/actor/BootableActorLoaderService.scala
@@ -55,8 +55,8 @@ trait BootableActorLoaderService extends Bootable with Logging {
protected def createApplicationClassLoader : Option[ClassLoader] = {
Some(
if (HOME.isDefined) {
- val CONFIG = HOME.get + "/config"
- val DEPLOY = HOME.get + "/deploy"
+ val CONFIG = HOME.getOrElse(throwNoAkkaHomeException) + "/config"
+ val DEPLOY = HOME.getOrElse(throwNoAkkaHomeException) + "/deploy"
val DEPLOY_DIR = new File(DEPLOY)
if (!DEPLOY_DIR.exists) {
log.error("Could not find a deploy directory at [%s]", DEPLOY)
diff --git a/akka-actor/src/main/scala/actor/Supervisor.scala b/akka-actor/src/main/scala/actor/Supervisor.scala
index 1af351a33d..f575cda299 100644
--- a/akka-actor/src/main/scala/actor/Supervisor.scala
+++ b/akka-actor/src/main/scala/actor/Supervisor.scala
@@ -187,7 +187,7 @@ final class SupervisorActor private[akka] (
trapExit = trapExceptions
faultHandler = Some(handler)
- override def shutdown(): Unit = shutdownLinkedActors
+ override def postStop(): Unit = shutdownLinkedActors
def receive = {
// FIXME add a way to respond to MaximumNumberOfRestartsWithinTimeRangeReached in declaratively configured Supervisor
diff --git a/akka-actor/src/main/scala/config/Config.scala b/akka-actor/src/main/scala/config/Config.scala
index 3b50d613c1..c9d9a4968b 100644
--- a/akka-actor/src/main/scala/config/Config.scala
+++ b/akka-actor/src/main/scala/config/Config.scala
@@ -64,9 +64,12 @@ object Config {
Configgy.config
} else if (HOME.isDefined) {
try {
- val configFile = HOME.get + "/config/akka.conf"
+ val configFile = HOME.getOrElse(throwNoAkkaHomeException) + "/config/akka.conf"
Configgy.configure(configFile)
- ConfigLogger.log.info("AKKA_HOME is defined as [%s], config loaded from [%s].", HOME.get, configFile)
+ ConfigLogger.log.info(
+ "AKKA_HOME is defined as [%s], config loaded from [%s].",
+ HOME.getOrElse(throwNoAkkaHomeException),
+ configFile)
} catch {
case e: ParseException => throw new ConfigurationException(
"AKKA_HOME is defined as [" + HOME.get + "] " +
@@ -86,7 +89,7 @@ object Config {
CConfig.fromString("") // default empty config
}
}
-
+
val CONFIG_VERSION = config.getString("akka.version", VERSION)
if (VERSION != CONFIG_VERSION) throw new ConfigurationException(
"Akka JAR version [" + VERSION + "] is different than the provided config ('akka.conf') version [" + CONFIG_VERSION + "]")
@@ -95,4 +98,9 @@ object Config {
val startTime = System.currentTimeMillis
def uptime = (System.currentTimeMillis - startTime) / 1000
+
+ def throwNoAkkaHomeException = throw new ConfigurationException(
+ "Akka home is not defined. Either:" +
+ "\n\t1. Define 'AKKA_HOME' environment variable pointing to the root of the Akka distribution." +
+ "\n\t2. Add the '-Dakka.home=...' option pointing to the root of the Akka distribution.")
}
diff --git a/akka-actor/src/main/scala/config/SupervisionConfig.scala b/akka-actor/src/main/scala/config/SupervisionConfig.scala
index 2f25f4ed33..d85001b5ca 100644
--- a/akka-actor/src/main/scala/config/SupervisionConfig.scala
+++ b/akka-actor/src/main/scala/config/SupervisionConfig.scala
@@ -8,8 +8,19 @@ import se.scalablesolutions.akka.actor.{ActorRef}
import se.scalablesolutions.akka.dispatch.MessageDispatcher
sealed abstract class FaultHandlingStrategy
-case class AllForOneStrategy(maxNrOfRetries: Int, withinTimeRange: Int) extends FaultHandlingStrategy
-case class OneForOneStrategy(maxNrOfRetries: Int, withinTimeRange: Int) extends FaultHandlingStrategy
+object AllForOneStrategy {
+ def apply(maxNrOfRetries: Int, withinTimeRange: Int): AllForOneStrategy =
+ AllForOneStrategy(if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
+ if (withinTimeRange < 0) None else Some(withinTimeRange))
+}
+case class AllForOneStrategy(maxNrOfRetries: Option[Int] = None, withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy
+
+object OneForOneStrategy {
+ def apply(maxNrOfRetries: Int, withinTimeRange: Int): OneForOneStrategy =
+ this(if (maxNrOfRetries < 0) None else Some(maxNrOfRetries),
+ if (withinTimeRange < 0) None else Some(withinTimeRange))
+}
+case class OneForOneStrategy(maxNrOfRetries: Option[Int] = None, withinTimeRange: Option[Int] = None) extends FaultHandlingStrategy
/**
* Configuration classes - not to be used as messages.
diff --git a/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala b/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala
index 787793dc5f..6608f6075b 100644
--- a/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala
+++ b/akka-actor/src/main/scala/dataflow/DataFlowVariable.scala
@@ -11,6 +11,7 @@ import se.scalablesolutions.akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.dispatch.CompletableFuture
import se.scalablesolutions.akka.AkkaException
+import se.scalablesolutions.akka.util.{ Function, SideEffect }
/**
* Implements Oz-style dataflow (single assignment) variables.
@@ -27,9 +28,22 @@ object DataFlow {
*/
def thread(body: => Unit): Unit = spawn(body)
+ /** Executes the supplied SideEffect in another thread
+ * JavaAPI
+ */
+ def thread(body: SideEffect): Unit = spawn(body.apply)
+
+ /** Executes the supplied function in another thread
+ */
def thread[A <: AnyRef, R <: AnyRef](body: A => R) =
actorOf(new ReactiveEventBasedThread(body)).start
+ /** Executes the supplied Function in another thread
+ * JavaAPI
+ */
+ def thread[A <: AnyRef, R <: AnyRef](body: Function[A,R]) =
+ actorOf(new ReactiveEventBasedThread(body.apply)).start
+
private class ReactiveEventBasedThread[A <: AnyRef, T <: AnyRef](body: A => T)
extends Actor {
def receive = {
@@ -91,6 +105,11 @@ object DataFlow {
"Attempt to change data flow variable (from [" + this.value.get + "] to [" + ref() + "])")
}
+ /** Sets the value of this variable (if unset) with the value of the supplied variable
+ * JavaAPI
+ */
+ def set(ref: DataFlowVariable[T]) { this << ref }
+
/** Sets the value of this variable (if unset)
*/
def <<(value: T) {
@@ -99,6 +118,16 @@ object DataFlow {
"Attempt to change data flow variable (from [" + this.value.get + "] to [" + value + "])")
}
+ /** Sets the value of this variable (if unset) with the value of the supplied variable
+ * JavaAPI
+ */
+ def set(value: T) { this << value }
+
+ /** Retrieves the value of variable
+ * throws a DataFlowVariableException if it times out
+ */
+ def get(): T = this()
+
/** Retrieves the value of variable
* throws a DataFlowVariableException if it times out
*/
diff --git a/akka-actor/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala
deleted file mode 100644
index 6bacec73be..0000000000
--- a/akka-actor/src/main/scala/dispatch/AbstractReactorBasedEventDrivenDispatcher.scala
+++ /dev/null
@@ -1,63 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-
-package se.scalablesolutions.akka.dispatch
-
-import java.util.{LinkedList, Queue, List}
-import java.util.HashMap
-
-import se.scalablesolutions.akka.actor.{Actor, ActorRef}
-
-abstract class AbstractReactorBasedEventDrivenDispatcher(val name: String) extends MessageDispatcher {
- @volatile protected var active: Boolean = false
- protected val queue = new ReactiveMessageQueue(name)
- protected val messageInvokers = new HashMap[ActorRef, ActorRef]
- protected var selectorThread: Thread = _
- protected val guard = new Object
-
- def dispatch(invocation: MessageInvocation) = queue.append(invocation)
-
- override def register(actorRef: ActorRef) = synchronized {
- messageInvokers.put(actorRef, actorRef)
- super.register(actorRef)
- }
-
- override def unregister(actorRef: ActorRef) = synchronized {
- messageInvokers.remove(actorRef)
- super.unregister(actorRef)
- }
-
- def shutdown = if (active) {
- log.debug("Shutting down %s", toString)
- active = false
- selectorThread.interrupt
- doShutdown
- }
-
- /**
- * Subclass callback. Override if additional shutdown behavior is needed.
- */
- protected def doShutdown = {}
-}
-
-class ReactiveMessageQueue(name: String) extends MessageQueue {
- private[akka] val queue: Queue[MessageInvocation] = new LinkedList[MessageInvocation]
- @volatile private var interrupted = false
-
- def append(handle: MessageInvocation) = queue.synchronized {
- queue.offer(handle)
- queue.notifyAll
- }
-
- def read(destination: List[MessageInvocation]) = queue.synchronized {
- while (queue.isEmpty && !interrupted) queue.wait
- if (!interrupted) while (!queue.isEmpty) destination.add(queue.remove)
- else interrupted = false
- }
-
- def interrupt = queue.synchronized {
- interrupted = true
- queue.notifyAll
- }
-}
diff --git a/akka-actor/src/main/scala/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/dispatch/Dispatchers.scala
index 803fd700cc..7e7904ec29 100644
--- a/akka-actor/src/main/scala/dispatch/Dispatchers.scala
+++ b/akka-actor/src/main/scala/dispatch/Dispatchers.scala
@@ -5,12 +5,11 @@
package se.scalablesolutions.akka.dispatch
import se.scalablesolutions.akka.actor.{Actor, ActorRef}
-import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.config.Config.config
import net.lag.configgy.ConfigMap
-import se.scalablesolutions.akka.util.UUID
import java.util.concurrent.ThreadPoolExecutor.{AbortPolicy, CallerRunsPolicy, DiscardOldestPolicy, DiscardPolicy}
import java.util.concurrent.TimeUnit
+import se.scalablesolutions.akka.util.{Duration, Logging, UUID}
/**
* Scala API. Dispatcher factory.
@@ -45,8 +44,14 @@ import java.util.concurrent.TimeUnit
* @author Jonas Bonér
*/
object Dispatchers extends Logging {
- val THROUGHPUT = config.getInt("akka.actor.throughput", 5)
- val MAILBOX_CAPACITY = config.getInt("akka.actor.default-dispatcher.mailbox-capacity", 1000)
+ val THROUGHPUT = config.getInt("akka.actor.throughput", 5)
+ val THROUGHPUT_DEADLINE_MS = config.getInt("akka.actor.throughput-deadline-ms",-1)
+ val MAILBOX_CAPACITY = config.getInt("akka.actor.default-dispatcher.mailbox-capacity", -1)
+ val MAILBOX_CONFIG = MailboxConfig(
+ capacity = Dispatchers.MAILBOX_CAPACITY,
+ pushTimeOut = config.getInt("akka.actor.default-dispatcher.mailbox-push-timeout-ms").map(Duration(_,TimeUnit.MILLISECONDS)),
+ blockingDequeue = false
+ )
lazy val defaultGlobalDispatcher = {
config.getConfigMap("akka.actor.default-dispatcher").flatMap(from).getOrElse(globalExecutorBasedEventDrivenDispatcher)
@@ -54,17 +59,13 @@ object Dispatchers extends Logging {
object globalHawtDispatcher extends HawtDispatcher
- object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher("global") {
+ object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher("global",THROUGHPUT,THROUGHPUT_DEADLINE_MS,MAILBOX_CONFIG) {
override def register(actor: ActorRef) = {
if (isShutdown) init
super.register(actor)
}
}
- object globalReactorBasedSingleThreadEventDrivenDispatcher extends ReactorBasedSingleThreadEventDrivenDispatcher("global")
-
- object globalReactorBasedThreadPoolEventDrivenDispatcher extends ReactorBasedThreadPoolEventDrivenDispatcher("global")
-
/**
* Creates an event-driven dispatcher based on the excellent HawtDispatch library.
*
@@ -95,7 +96,7 @@ object Dispatchers extends Logging {
*
* E.g. each actor consumes its own thread.
*/
- def newThreadBasedDispatcher(actor: ActorRef, mailboxCapacity: Int, pushTimeout: Long, pushTimeUnit: TimeUnit) = new ThreadBasedDispatcher(actor, mailboxCapacity, pushTimeout, pushTimeUnit)
+ def newThreadBasedDispatcher(actor: ActorRef, mailboxCapacity: Int, pushTimeOut: Duration) = new ThreadBasedDispatcher(actor, MailboxConfig(mailboxCapacity,Option(pushTimeOut),true))
/**
* Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
@@ -116,7 +117,15 @@ object Dispatchers extends Logging {
*
* Has a fluent builder interface for configuring its semantics.
*/
- def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, mailboxCapacity: Int) = new ExecutorBasedEventDrivenDispatcher(name, throughput, mailboxCapacity)
+ def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxCapacity: Int) = new ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineMs, mailboxCapacity)
+
+ /**
+ * Creates a executor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
+ *
+ * Has a fluent builder interface for configuring its semantics.
+ */
+ def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxCapacity: Int, pushTimeOut: Duration) = new ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineMs, MailboxConfig(mailboxCapacity,Some(pushTimeOut),false))
+
/**
* Creates a executor-based event-driven dispatcher with work stealing (TODO: better doc) serving multiple (millions) of actors through a thread pool.
@@ -132,18 +141,6 @@ object Dispatchers extends Logging {
*/
def newExecutorBasedEventDrivenWorkStealingDispatcher(name: String, mailboxCapacity: Int) = new ExecutorBasedEventDrivenWorkStealingDispatcher(name, mailboxCapacity)
- /**
- * Creates a reactor-based event-driven dispatcher serving multiple (millions) of actors through a thread pool.
- *
- * Has a fluent builder interface for configuring its semantics.
- */
- def newReactorBasedThreadPoolEventDrivenDispatcher(name: String) = new ReactorBasedThreadPoolEventDrivenDispatcher(name)
-
- /**
- * Creates a reactor-based event-driven dispatcher serving multiple (millions) of actors through a single thread.
- */
- def newReactorBasedSingleThreadEventDrivenDispatcher(name: String) = new ReactorBasedSingleThreadEventDrivenDispatcher(name)
-
/**
* Utility function that tries to load the specified dispatcher config from the akka.conf
* or else use the supplied default dispatcher
@@ -156,9 +153,8 @@ object Dispatchers extends Logging {
*
* default-dispatcher {
* type = "GlobalExecutorBasedEventDriven" # Must be one of the following, all "Global*" are non-configurable
- * # ReactorBasedSingleThreadEventDriven, (ExecutorBasedEventDrivenWorkStealing), ExecutorBasedEventDriven,
- * # ReactorBasedThreadPoolEventDriven, Hawt, GlobalReactorBasedSingleThreadEventDriven,
- * # GlobalReactorBasedThreadPoolEventDriven, GlobalExecutorBasedEventDriven, GlobalHawt
+ * # (ExecutorBasedEventDrivenWorkStealing), ExecutorBasedEventDriven,
+ * # Hawt, GlobalExecutorBasedEventDriven, GlobalHawt
* keep-alive-ms = 60000 # Keep alive time for threads
* core-pool-size-factor = 1.0 # No of core threads ... ceil(available processors * factor)
* max-pool-size-factor = 4.0 # Max no of threads ... ceil(available processors * factor)
@@ -177,23 +173,8 @@ object Dispatchers extends Logging {
def from(cfg: ConfigMap): Option[MessageDispatcher] = {
lazy val name = cfg.getString("name", UUID.newUuid.toString)
- val dispatcher: Option[MessageDispatcher] = cfg.getString("type") map {
- case "ReactorBasedSingleThreadEventDriven" => newReactorBasedSingleThreadEventDrivenDispatcher(name)
- case "ExecutorBasedEventDrivenWorkStealing" => newExecutorBasedEventDrivenWorkStealingDispatcher(name)
- case "ExecutorBasedEventDriven" => newExecutorBasedEventDrivenDispatcher(name,cfg.getInt("throughput",THROUGHPUT))
- case "ReactorBasedThreadPoolEventDriven" => newReactorBasedThreadPoolEventDrivenDispatcher(name)
- case "Hawt" => newHawtDispatcher(cfg.getBool("aggregate").getOrElse(true))
- case "GlobalReactorBasedSingleThreadEventDriven" => globalReactorBasedSingleThreadEventDrivenDispatcher
- case "GlobalReactorBasedThreadPoolEventDriven" => globalReactorBasedThreadPoolEventDrivenDispatcher
- case "GlobalExecutorBasedEventDriven" => globalExecutorBasedEventDrivenDispatcher
- case "GlobalHawt" => globalHawtDispatcher
-
- case unknown => throw new IllegalArgumentException("Unknown dispatcher type [%s]" format unknown)
- }
-
- dispatcher foreach {
- case d: ThreadPoolBuilder => d.configureIfPossible( builder => {
-
+ def threadPoolConfig(b: ThreadPoolBuilder) {
+ b.configureIfPossible( builder => {
cfg.getInt("keep-alive-ms").foreach(builder.setKeepAliveTimeInMillis(_))
cfg.getDouble("core-pool-size-factor").foreach(builder.setCorePoolSizeFromFactor(_))
cfg.getDouble("max-pool-size-factor").foreach(builder.setMaxPoolSizeFromFactor(_))
@@ -209,7 +190,37 @@ object Dispatchers extends Logging {
case x => throw new IllegalArgumentException("[%s] is not a valid rejectionPolicy!" format x)
}).foreach(builder.setRejectionPolicy(_))
})
- case _ =>
+ }
+
+ lazy val mailboxBounds: MailboxConfig = {
+ val capacity = cfg.getInt("mailbox-capacity",Dispatchers.MAILBOX_CAPACITY)
+ val timeout = cfg.getInt("mailbox-push-timeout-ms").map(Duration(_,TimeUnit.MILLISECONDS))
+ MailboxConfig(capacity,timeout,false)
+ }
+
+ val dispatcher: Option[MessageDispatcher] = cfg.getString("type") map {
+ case "ExecutorBasedEventDrivenWorkStealing" =>
+ new ExecutorBasedEventDrivenWorkStealingDispatcher(name,MAILBOX_CAPACITY,threadPoolConfig)
+
+ case "ExecutorBasedEventDriven" =>
+ new ExecutorBasedEventDrivenDispatcher(
+ name,
+ cfg.getInt("throughput",THROUGHPUT),
+ cfg.getInt("throughput-deadline-ms",THROUGHPUT_DEADLINE_MS),
+ mailboxBounds,
+ threadPoolConfig)
+
+ case "Hawt" =>
+ new HawtDispatcher(cfg.getBool("aggregate").getOrElse(true))
+
+ case "GlobalExecutorBasedEventDriven" =>
+ globalExecutorBasedEventDrivenDispatcher
+
+ case "GlobalHawt" =>
+ globalHawtDispatcher
+
+ case unknown =>
+ throw new IllegalArgumentException("Unknown dispatcher type [%s]" format unknown)
}
dispatcher
diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
index 1f8a6bfe9c..63ce310848 100644
--- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
+++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
@@ -7,7 +7,7 @@ package se.scalablesolutions.akka.dispatch
import se.scalablesolutions.akka.actor.{ActorRef, IllegalActorStateException}
import java.util.Queue
-import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue}
+import java.util.concurrent.{RejectedExecutionException, ConcurrentLinkedQueue, LinkedBlockingQueue}
/**
* Default settings are:
@@ -64,89 +64,104 @@ import java.util.concurrent.{ConcurrentLinkedQueue, LinkedBlockingQueue}
*/
class ExecutorBasedEventDrivenDispatcher(
_name: String,
- throughput: Int = Dispatchers.THROUGHPUT,
- capacity: Int = Dispatchers.MAILBOX_CAPACITY) extends MessageDispatcher with ThreadPoolBuilder {
+ val throughput: Int = Dispatchers.THROUGHPUT,
+ val throughputDeadlineMs: Int = Dispatchers.THROUGHPUT_DEADLINE_MS,
+ mailboxConfig: MailboxConfig = Dispatchers.MAILBOX_CONFIG,
+ config: (ThreadPoolBuilder) => Unit = _ => ()) extends MessageDispatcher with ThreadPoolBuilder {
- def this(_name: String, throughput: Int) = this(_name, throughput, Dispatchers.MAILBOX_CAPACITY) // Needed for Java API usage
- def this(_name: String) = this(_name, Dispatchers.THROUGHPUT) // Needed for Java API usage
+ def this(_name: String, throughput: Int, throughputDeadlineMs: Int, capacity: Int) = this(_name,throughput,throughputDeadlineMs,MailboxConfig(capacity,None,false))
+ def this(_name: String, throughput: Int) = this(_name, throughput, Dispatchers.THROUGHPUT_DEADLINE_MS, Dispatchers.MAILBOX_CAPACITY) // Needed for Java API usage
+ def this(_name: String) = this(_name,Dispatchers.THROUGHPUT, Dispatchers.THROUGHPUT_DEADLINE_MS,Dispatchers.MAILBOX_CAPACITY) // Needed for Java API usage
- mailboxCapacity = capacity
+ //FIXME remove this from ThreadPoolBuilder
+ mailboxCapacity = mailboxConfig.capacity
@volatile private var active: Boolean = false
val name = "akka:event-driven:dispatcher:" + _name
init
+ /**
+ * This is the behavior of an ExecutorBasedEventDrivenDispatchers mailbox
+ */
+ trait ExecutableMailbox extends Runnable { self: MessageQueue =>
+ final def run = {
+
+ val reschedule = try {
+ processMailbox()
+ } finally {
+ dispatcherLock.unlock()
+ }
+
+ if (reschedule || !self.isEmpty)
+ registerForExecution(self)
+ }
+
+ /**
+ * Process the messages in the mailbox
+ *
+ * @return true if the processing finished before the mailbox was empty, due to the throughput constraint
+ */
+ final def processMailbox(): Boolean = {
+ var nextMessage = self.dequeue
+ if (nextMessage ne null) {
+ val throttle = throughput > 0
+ var processedMessages = 0
+ val isDeadlineEnabled = throttle && throughputDeadlineMs > 0
+ val started = if (isDeadlineEnabled) System.currentTimeMillis else 0
+
+ do {
+ nextMessage.invoke
+
+ if(throttle) { //Will be elided when false
+ processedMessages += 1
+ if ((processedMessages >= throughput)
+ || (isDeadlineEnabled && (System.currentTimeMillis - started) >= throughputDeadlineMs)) //If we're throttled, break out
+ return !self.isEmpty
+ }
+ nextMessage = self.dequeue
+ }
+ while (nextMessage ne null)
+ }
+
+ false
+ }
+ }
+
def dispatch(invocation: MessageInvocation) = {
- getMailbox(invocation.receiver).add(invocation)
- dispatch(invocation.receiver)
+ val mbox = getMailbox(invocation.receiver)
+ mbox enqueue invocation
+ registerForExecution(mbox)
+ }
+
+ protected def registerForExecution(mailbox: MessageQueue with ExecutableMailbox): Unit = if (active) {
+ if (mailbox.dispatcherLock.tryLock()) {
+ try {
+ executor execute mailbox
+ } catch {
+ case e: RejectedExecutionException =>
+ mailbox.dispatcherLock.unlock()
+ throw e
+ }
+ }
+ } else {
+ log.warning("%s is shut down,\n\tignoring the rest of the messages in the mailbox of\n\t%s", toString, mailbox)
}
/**
* @return the mailbox associated with the actor
*/
- private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[Queue[MessageInvocation]]
+ private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[MessageQueue with ExecutableMailbox]
override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
- override def register(actorRef: ActorRef) = {
- if (actorRef.mailbox eq null ) {
- if (mailboxCapacity <= 0) actorRef.mailbox = new ConcurrentLinkedQueue[MessageInvocation]
- else actorRef.mailbox = new LinkedBlockingQueue[MessageInvocation](mailboxCapacity)
- }
- super.register(actorRef)
+ override def createMailbox(actorRef: ActorRef): AnyRef = {
+ if (mailboxCapacity > 0)
+ new DefaultBoundedMessageQueue(mailboxCapacity,mailboxConfig.pushTimeOut,blockDequeue = false) with ExecutableMailbox
+ else
+ new DefaultUnboundedMessageQueue(blockDequeue = false) with ExecutableMailbox
}
- def dispatch(receiver: ActorRef): Unit = if (active) {
-
- executor.execute(new Runnable() {
- def run = {
- var lockAcquiredOnce = false
- var finishedBeforeMailboxEmpty = false
- val lock = receiver.dispatcherLock
- val mailbox = getMailbox(receiver)
- // this do-while loop is required to prevent missing new messages between the end of the inner while
- // loop and releasing the lock
- do {
- if (lock.tryLock) {
- // Only dispatch if we got the lock. Otherwise another thread is already dispatching.
- lockAcquiredOnce = true
- try {
- finishedBeforeMailboxEmpty = processMailbox(receiver)
- } finally {
- lock.unlock
- if (finishedBeforeMailboxEmpty) dispatch(receiver)
- }
- }
- } while ((lockAcquiredOnce && !finishedBeforeMailboxEmpty && !mailbox.isEmpty))
- }
- })
- } else {
- log.warning("%s is shut down,\n\tignoring the rest of the messages in the mailbox of\n\t%s", toString, receiver)
- }
-
-
- /**
- * Process the messages in the mailbox of the given actor.
- *
- * @return true if the processing finished before the mailbox was empty, due to the throughput constraint
- */
- def processMailbox(receiver: ActorRef): Boolean = {
- var processedMessages = 0
- val mailbox = getMailbox(receiver)
- var messageInvocation = mailbox.poll
- while (messageInvocation != null) {
- messageInvocation.invoke
- processedMessages += 1
- // check if we simply continue with other messages, or reached the throughput limit
- if (throughput <= 0 || processedMessages < throughput) messageInvocation = mailbox.poll
- else {
- messageInvocation = null
- return !mailbox.isEmpty
- }
- }
- false
- }
def start = if (!active) {
log.debug("Starting up %s\n\twith throughput [%d]", toString, throughput)
@@ -160,11 +175,17 @@ class ExecutorBasedEventDrivenDispatcher(
uuids.clear
}
- def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
+ def ensureNotActive(): Unit = if (active) {
+ throw new IllegalActorStateException(
"Can't build a new thread pool for a dispatcher that is already up and running")
+ }
override def toString = "ExecutorBasedEventDrivenDispatcher[" + name + "]"
// FIXME: should we have an unbounded queue and not bounded as default ????
- private[akka] def init = withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool
+ private[akka] def init = {
+ withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
+ config(this)
+ buildThreadPool
+ }
}
diff --git a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
index 4e5d626aed..10afb1bfb6 100644
--- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
+++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcher.scala
@@ -31,7 +31,11 @@ import se.scalablesolutions.akka.actor.{Actor, ActorRef, IllegalActorStateExcept
*/
class ExecutorBasedEventDrivenWorkStealingDispatcher(
_name: String,
- capacity: Int = Dispatchers.MAILBOX_CAPACITY) extends MessageDispatcher with ThreadPoolBuilder {
+ capacity: Int = Dispatchers.MAILBOX_CAPACITY,
+ config: (ThreadPoolBuilder) => Unit = _ => ()) extends MessageDispatcher with ThreadPoolBuilder {
+
+ def this(_name: String, capacity: Int) = this(_name,capacity, _ => ())
+
mailboxCapacity = capacity
@volatile private var active: Boolean = false
@@ -52,21 +56,14 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
/**
* @return the mailbox associated with the actor
*/
- private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[Deque[MessageInvocation]]
+ private def getMailbox(receiver: ActorRef) = receiver.mailbox.asInstanceOf[Deque[MessageInvocation] with MessageQueue with Runnable]
override def mailboxSize(actorRef: ActorRef) = getMailbox(actorRef).size
def dispatch(invocation: MessageInvocation) = if (active) {
- getMailbox(invocation.receiver).add(invocation)
- executor.execute(new Runnable() {
- def run = {
- if (!tryProcessMailbox(invocation.receiver)) {
- // we are not able to process our mailbox (another thread is busy with it), so lets donate some of our mailbox
- // to another actor and then process his mailbox in stead.
- findThief(invocation.receiver).foreach( tryDonateAndProcessMessages(invocation.receiver,_) )
- }
- }
- })
+ val mbox = getMailbox(invocation.receiver)
+ mbox enqueue invocation
+ executor execute mbox
} else throw new IllegalActorStateException("Can't submit invocations to dispatcher since it's not started")
/**
@@ -75,22 +72,21 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
*
* @return true if the mailbox was processed, false otherwise
*/
- private def tryProcessMailbox(receiver: ActorRef): Boolean = {
+ private def tryProcessMailbox(mailbox: MessageQueue): Boolean = {
var lockAcquiredOnce = false
- val lock = receiver.dispatcherLock
// this do-wile loop is required to prevent missing new messages between the end of processing
// the mailbox and releasing the lock
do {
- if (lock.tryLock) {
+ if (mailbox.dispatcherLock.tryLock) {
lockAcquiredOnce = true
try {
- processMailbox(receiver)
+ processMailbox(mailbox)
} finally {
- lock.unlock
+ mailbox.dispatcherLock.unlock
}
}
- } while ((lockAcquiredOnce && !getMailbox(receiver).isEmpty))
+ } while ((lockAcquiredOnce && !mailbox.isEmpty))
lockAcquiredOnce
}
@@ -98,12 +94,11 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
/**
* Process the messages in the mailbox of the given actor.
*/
- private def processMailbox(receiver: ActorRef) = {
- val mailbox = getMailbox(receiver)
- var messageInvocation = mailbox.poll
- while (messageInvocation != null) {
+ private def processMailbox(mailbox: MessageQueue) = {
+ var messageInvocation = mailbox.dequeue
+ while (messageInvocation ne null) {
messageInvocation.invoke
- messageInvocation = mailbox.poll
+ messageInvocation = mailbox.dequeue
}
}
@@ -141,11 +136,12 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
* the thiefs dispatching lock, because in that case another thread is already processing the thiefs mailbox.
*/
private def tryDonateAndProcessMessages(receiver: ActorRef, thief: ActorRef) = {
- if (thief.dispatcherLock.tryLock) {
+ val mailbox = getMailbox(thief)
+ if (mailbox.dispatcherLock.tryLock) {
try {
- while(donateMessage(receiver, thief)) processMailbox(thief)
+ while(donateMessage(receiver, thief)) processMailbox(mailbox)
} finally {
- thief.dispatcherLock.unlock
+ mailbox.dispatcherLock.unlock
}
}
}
@@ -180,21 +176,51 @@ class ExecutorBasedEventDrivenWorkStealingDispatcher(
override def toString = "ExecutorBasedEventDrivenWorkStealingDispatcher[" + name + "]"
- private[akka] def init = withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool
+ private[akka] def init = {
+ withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
+ config(this)
+ buildThreadPool
+ }
+
+ protected override def createMailbox(actorRef: ActorRef): AnyRef = {
+ if (mailboxCapacity <= 0) {
+ new ConcurrentLinkedDeque[MessageInvocation] with MessageQueue with Runnable {
+ def enqueue(handle: MessageInvocation): Unit = this.add(handle)
+ def dequeue: MessageInvocation = this.poll()
+
+ def run = {
+ if (!tryProcessMailbox(this)) {
+ // we are not able to process our mailbox (another thread is busy with it), so lets donate some of our mailbox
+ // to another actor and then process his mailbox in stead.
+ findThief(actorRef).foreach( tryDonateAndProcessMessages(actorRef,_) )
+ }
+ }
+ }
+ }
+ else {
+ new LinkedBlockingDeque[MessageInvocation](mailboxCapacity) with MessageQueue with Runnable {
+ def enqueue(handle: MessageInvocation): Unit = this.add(handle)
+ def dequeue: MessageInvocation = this.poll()
+
+ def run = {
+ if (!tryProcessMailbox(this)) {
+ // we are not able to process our mailbox (another thread is busy with it), so lets donate some of our mailbox
+ // to another actor and then process his mailbox in stead.
+ findThief(actorRef).foreach( tryDonateAndProcessMessages(actorRef,_) )
+ }
+ }
+ }
+ }
+ }
override def register(actorRef: ActorRef) = {
verifyActorsAreOfSameType(actorRef)
- // The actor will need a ConcurrentLinkedDeque based mailbox
- if (actorRef.mailbox == null) {
- if (mailboxCapacity <= 0) actorRef.mailbox = new ConcurrentLinkedDeque[MessageInvocation]
- else actorRef.mailbox = new LinkedBlockingDeque[MessageInvocation](mailboxCapacity)
- }
- pooledActors.add(actorRef)
+ pooledActors add actorRef
super.register(actorRef)
}
override def unregister(actorRef: ActorRef) = {
- pooledActors.remove(actorRef)
+ pooledActors remove actorRef
super.unregister(actorRef)
}
diff --git a/akka-actor/src/main/scala/dispatch/Future.scala b/akka-actor/src/main/scala/dispatch/Future.scala
index 17c63bcd57..0a3cd48aa5 100644
--- a/akka-actor/src/main/scala/dispatch/Future.scala
+++ b/akka-actor/src/main/scala/dispatch/Future.scala
@@ -160,6 +160,7 @@ class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] {
if (!_completed) {
_completed = true
_result = Some(result)
+ onComplete(result)
}
} finally {
_signal.signalAll
@@ -171,6 +172,7 @@ class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] {
if (!_completed) {
_completed = true
_exception = Some(exception)
+ onCompleteException(exception)
}
} finally {
_signal.signalAll
@@ -178,4 +180,6 @@ class DefaultCompletableFuture[T](timeout: Long) extends CompletableFuture[T] {
}
private def currentTimeInNanos: Long = TIME_UNIT.toNanos(System.currentTimeMillis)
+ protected def onComplete(result: T) {}
+ protected def onCompleteException(exception: Throwable) {}
}
diff --git a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala
index e0ddf05d26..cf3f71295c 100644
--- a/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala
+++ b/akka-actor/src/main/scala/dispatch/HawtDispatcher.scala
@@ -185,16 +185,10 @@ class HawtDispatcher(val aggregate:Boolean=true, val parent:DispatchQueue=global
// TODO: figure out if this can be optional in akka
override def mailboxSize(actorRef: ActorRef) = 0
- override def register(actorRef: ActorRef) = {
- if( actorRef.mailbox == null ) {
- val queue = parent.createSerialQueue(actorRef.toString)
- if( aggregate ) {
- actorRef.mailbox = new AggregatingHawtDispatcherMailbox(queue)
- } else {
- actorRef.mailbox = new HawtDispatcherMailbox(queue)
- }
- }
- super.register(actorRef)
+ override def createMailbox(actorRef: ActorRef): AnyRef = {
+ val queue = parent.createSerialQueue(actorRef.toString)
+ if (aggregate) new AggregatingHawtDispatcherMailbox(queue)
+ else new HawtDispatcherMailbox(queue)
}
override def toString = "HawtDispatchEventDrivenDispatcher"
diff --git a/akka-actor/src/main/scala/dispatch/MessageHandling.scala b/akka-actor/src/main/scala/dispatch/MessageHandling.scala
index 395c572f0e..25a02f2603 100644
--- a/akka-actor/src/main/scala/dispatch/MessageHandling.scala
+++ b/akka-actor/src/main/scala/dispatch/MessageHandling.scala
@@ -4,14 +4,14 @@
package se.scalablesolutions.akka.dispatch
-import java.util.List
-
-import se.scalablesolutions.akka.util.{HashCode, Logging}
import se.scalablesolutions.akka.actor.{Actor, ActorRef, ActorInitializationException}
import org.multiverse.commitbarriers.CountDownCommitBarrier
import se.scalablesolutions.akka.AkkaException
-import java.util.concurrent.{ConcurrentSkipListSet}
+import java.util.{Queue, List}
+import java.util.concurrent._
+import concurrent.forkjoin.LinkedTransferQueue
+import se.scalablesolutions.akka.util.{SimpleLock, Duration, HashCode, Logging}
/**
* @author Jonas Bonér
@@ -30,8 +30,6 @@ final class MessageInvocation(val receiver: ActorRef,
"Don't call 'self ! message' in the Actor's constructor (e.g. body of the class).")
}
- def send = receiver.dispatcher.dispatch(this)
-
override def hashCode(): Int = synchronized {
var result = HashCode.SEED
result = HashCode.hash(result, receiver.actor)
@@ -63,33 +61,94 @@ class MessageQueueAppendFailedException(message: String) extends AkkaException(m
* @author Jonas Bonér
*/
trait MessageQueue {
- def append(handle: MessageInvocation)
+ val dispatcherLock = new SimpleLock
+ def enqueue(handle: MessageInvocation)
+ def dequeue(): MessageInvocation
+ def size: Int
+ def isEmpty: Boolean
+}
+
+/* Tells the dispatcher that it should create a bounded mailbox with the specified push timeout
+ * (If capacity > 0)
+ */
+case class MailboxConfig(capacity: Int, pushTimeOut: Option[Duration], blockingDequeue: Boolean) {
+
+ /**
+ * Creates a MessageQueue (Mailbox) with the specified properties
+ * bounds = whether the mailbox should be bounded (< 0 means unbounded)
+ * pushTime = only used if bounded, indicates if and how long an enqueue should block
+ * blockDequeue = whether dequeues should block or not
+ *
+ * The bounds + pushTime generates a MessageQueueAppendFailedException if enqueue times out
+ */
+ def newMailbox(bounds: Int = capacity,
+ pushTime: Option[Duration] = pushTimeOut,
+ blockDequeue: Boolean = blockingDequeue) : MessageQueue =
+ if (capacity > 0) new DefaultBoundedMessageQueue(bounds,pushTime,blockDequeue)
+ else new DefaultUnboundedMessageQueue(blockDequeue)
+}
+
+class DefaultUnboundedMessageQueue(blockDequeue: Boolean) extends LinkedBlockingQueue[MessageInvocation] with MessageQueue {
+ final def enqueue(handle: MessageInvocation) {
+ this add handle
+ }
+
+ final def dequeue(): MessageInvocation =
+ if (blockDequeue) this.take()
+ else this.poll()
+}
+
+class DefaultBoundedMessageQueue(capacity: Int, pushTimeOut: Option[Duration], blockDequeue: Boolean) extends LinkedBlockingQueue[MessageInvocation](capacity) with MessageQueue {
+ final def enqueue(handle: MessageInvocation) {
+ if (pushTimeOut.isDefined) {
+ if(!this.offer(handle,pushTimeOut.get.length,pushTimeOut.get.unit))
+ throw new MessageQueueAppendFailedException("Couldn't enqueue message " + handle + " to " + toString)
+ }
+ else {
+ this put handle
+ }
+ }
+
+ final def dequeue(): MessageInvocation =
+ if (blockDequeue) this.take()
+ else this.poll()
+
}
/**
- * @author Jonas Bonér
+ * @author Jonas Bonér
*/
trait MessageDispatcher extends Logging {
protected val uuids = new ConcurrentSkipListSet[String]
+
def dispatch(invocation: MessageInvocation)
+
def start
+
def shutdown
- def register(actorRef: ActorRef) = uuids add actorRef.uuid
+
+ def register(actorRef: ActorRef) {
+ if(actorRef.mailbox eq null)
+ actorRef.mailbox = createMailbox(actorRef)
+ uuids add actorRef.uuid
+ }
def unregister(actorRef: ActorRef) = {
uuids remove actorRef.uuid
+ actorRef.mailbox = null
if (canBeShutDown) shutdown // shut down in the dispatcher's references is zero
}
+
def canBeShutDown: Boolean = uuids.isEmpty
- def isShutdown: Boolean
- def mailboxSize(actorRef: ActorRef):Int = 0
-}
-/**
- * @author Jonas Bonér
- */
-trait MessageDemultiplexer {
- def select
- def wakeUp
- def acquireSelectedInvocations: List[MessageInvocation]
- def releaseSelectedInvocations
-}
+ def isShutdown: Boolean
+
+ /**
+ * Returns the size of the mailbox for the specified actor
+ */
+ def mailboxSize(actorRef: ActorRef):Int
+
+ /**
+ * Creates and returns a mailbox for the given actor
+ */
+ protected def createMailbox(actorRef: ActorRef): AnyRef = null
+}
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/dispatch/Queues.scala b/akka-actor/src/main/scala/dispatch/Queues.scala
deleted file mode 100644
index 5b5aa6683e..0000000000
--- a/akka-actor/src/main/scala/dispatch/Queues.scala
+++ /dev/null
@@ -1,148 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-
-package se.scalablesolutions.akka.dispatch
-
-import concurrent.forkjoin.LinkedTransferQueue
-import java.util.concurrent.{TimeUnit, Semaphore}
-import java.util.Iterator
-import se.scalablesolutions.akka.util.Logger
-
-class BoundedTransferQueue[E <: AnyRef](
- val capacity: Int,
- val pushTimeout: Long,
- val pushTimeUnit: TimeUnit)
- extends LinkedTransferQueue[E] {
- require(capacity > 0)
- require(pushTimeout > 0)
- require(pushTimeUnit ne null)
-
- protected val guard = new Semaphore(capacity)
-
- override def take(): E = {
- val e = super.take
- if (e ne null) guard.release
- e
- }
-
- override def poll(): E = {
- val e = super.poll
- if (e ne null) guard.release
- e
- }
-
- override def poll(timeout: Long, unit: TimeUnit): E = {
- val e = super.poll(timeout,unit)
- if (e ne null) guard.release
- e
- }
-
- override def remainingCapacity = guard.availablePermits
-
- override def remove(o: AnyRef): Boolean = {
- if (super.remove(o)) {
- guard.release
- true
- } else {
- false
- }
- }
-
- override def offer(e: E): Boolean = {
- if (guard.tryAcquire(pushTimeout,pushTimeUnit)) {
- val result = try {
- super.offer(e)
- } catch {
- case e => guard.release; throw e
- }
- if (!result) guard.release
- result
- } else
- false
- }
-
- override def offer(e: E, timeout: Long, unit: TimeUnit): Boolean = {
- if (guard.tryAcquire(pushTimeout,pushTimeUnit)) {
- val result = try {
- super.offer(e,timeout,unit)
- } catch {
- case e => guard.release; throw e
- }
- if (!result) guard.release
- result
- } else
- false
- }
-
- override def add(e: E): Boolean = {
- if (guard.tryAcquire(pushTimeout,pushTimeUnit)) {
- val result = try {
- super.add(e)
- } catch {
- case e => guard.release; throw e
- }
- if (!result) guard.release
- result
- } else
- false
- }
-
- override def put(e :E): Unit = {
- if (guard.tryAcquire(pushTimeout,pushTimeUnit)) {
- try {
- super.put(e)
- } catch {
- case e => guard.release; throw e
- }
- }
- }
-
- override def tryTransfer(e: E): Boolean = {
- if (guard.tryAcquire(pushTimeout,pushTimeUnit)) {
- val result = try {
- super.tryTransfer(e)
- } catch {
- case e => guard.release; throw e
- }
- if (!result) guard.release
- result
- } else
- false
- }
-
- override def tryTransfer(e: E, timeout: Long, unit: TimeUnit): Boolean = {
- if (guard.tryAcquire(pushTimeout,pushTimeUnit)) {
- val result = try {
- super.tryTransfer(e,timeout,unit)
- } catch {
- case e => guard.release; throw e
- }
- if (!result) guard.release
- result
- } else
- false
- }
-
- override def transfer(e: E): Unit = {
- if (guard.tryAcquire(pushTimeout,pushTimeUnit)) {
- try {
- super.transfer(e)
- } catch {
- case e => guard.release; throw e
- }
- }
- }
-
- override def iterator: Iterator[E] = {
- val it = super.iterator
- new Iterator[E] {
- def hasNext = it.hasNext
- def next = it.next
- def remove {
- it.remove
- guard.release //Assume remove worked if no exception was thrown
- }
- }
- }
-}
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala
deleted file mode 100644
index d0850aa830..0000000000
--- a/akka-actor/src/main/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcher.scala
+++ /dev/null
@@ -1,59 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-
-/**
- * Implements the Reactor pattern as defined in: [http://www.cs.wustl.edu/~schmidt/PDF/reactor-siemens.pdf].
- * See also this article: [http://today.java.net/cs/user/print/a/350].
- *
- * Based on code from the actorom actor framework by Sergio Bossa [http://code.google.com/p/actorom/].
- */
-package se.scalablesolutions.akka.dispatch
-
-import java.util.{LinkedList, List}
-
-class ReactorBasedSingleThreadEventDrivenDispatcher(_name: String)
- extends AbstractReactorBasedEventDrivenDispatcher("akka:event-driven:reactor:single-thread:dispatcher:" + _name) {
-
- def start = if (!active) {
- log.debug("Starting up %s", toString)
- active = true
- val messageDemultiplexer = new Demultiplexer(queue)
- selectorThread = new Thread(name) {
- override def run = {
- while (active) {
- try {
- messageDemultiplexer.select
- } catch { case e: InterruptedException => active = false }
- val selectedInvocations = messageDemultiplexer.acquireSelectedInvocations
- val iter = selectedInvocations.iterator
- while (iter.hasNext) {
- val invocation = iter.next
- val invoker = messageInvokers.get(invocation.receiver)
- if (invoker ne null) invoker.invoke(invocation)
- iter.remove
- }
- }
- }
- }
- selectorThread.start
- }
-
- def isShutdown = !active
-
- override def toString = "ReactorBasedSingleThreadEventDrivenDispatcher[" + name + "]"
-
- class Demultiplexer(private val messageQueue: ReactiveMessageQueue) extends MessageDemultiplexer {
-
- private val selectedQueue: List[MessageInvocation] = new LinkedList[MessageInvocation]
-
- def select = messageQueue.read(selectedQueue)
-
- def acquireSelectedInvocations: List[MessageInvocation] = selectedQueue
-
- def releaseSelectedInvocations = throw new UnsupportedOperationException("Demultiplexer can't release its queue")
-
- def wakeUp = throw new UnsupportedOperationException("Demultiplexer can't be woken up")
- }
-}
-
diff --git a/akka-actor/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala b/akka-actor/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala
deleted file mode 100644
index 530184d4b2..0000000000
--- a/akka-actor/src/main/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcher.scala
+++ /dev/null
@@ -1,167 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-
-package se.scalablesolutions.akka.dispatch
-
-import java.util.concurrent.locks.ReentrantLock
-
-import java.util.{HashSet, HashMap, LinkedList, List}
-
-import se.scalablesolutions.akka.actor.IllegalActorStateException
-
-/**
- * Implements the Reactor pattern as defined in: [http://www.cs.wustl.edu/~schmidt/PDF/reactor-siemens.pdf].
- * See also this article: [http://today.java.net/cs/user/print/a/350].
- *
- *
- * Default settings are:
- *
- * - withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity
- * - NR_START_THREADS = 16
- * - NR_MAX_THREADS = 128
- * - KEEP_ALIVE_TIME = 60000L // one minute
- *
- *
- *
- * The dispatcher has a fluent builder interface to build up a thread pool to suite your use-case.
- * There is a default thread pool defined but make use of the builder if you need it. Here are some examples.
- *
- *
- * Scala API.
- *
- * Example usage:
- *
- * val dispatcher = new ReactorBasedThreadPoolEventDrivenDispatcher("name")
- * dispatcher
- * .withNewThreadPoolWithBoundedBlockingQueue(100)
- * .setCorePoolSize(16)
- * .setMaxPoolSize(128)
- * .setKeepAliveTimeInMillis(60000)
- * .setRejectionPolicy(new CallerRunsPolicy)
- * .buildThreadPool
- *
- *
- *
- * Java API.
- *
- * Example usage:
- *
- * ReactorBasedThreadPoolEventDrivenDispatcher dispatcher = new ReactorBasedThreadPoolEventDrivenDispatcher("name");
- * dispatcher
- * .withNewThreadPoolWithBoundedBlockingQueue(100)
- * .setCorePoolSize(16)
- * .setMaxPoolSize(128)
- * .setKeepAliveTimeInMillis(60000)
- * .setRejectionPolicy(new CallerRunsPolicy())
- * .buildThreadPool();
- *
- *
- *
- * But the preferred way of creating dispatchers is to use
- * the {@link se.scalablesolutions.akka.dispatch.Dispatchers} factory object.
- *
- * @author Jonas Bonér
- */
-class ReactorBasedThreadPoolEventDrivenDispatcher(_name: String)
- extends AbstractReactorBasedEventDrivenDispatcher("akka:event-driven:reactor:dispatcher:" + _name)
- with ThreadPoolBuilder {
-
- private var fair = true
- private val busyActors = new HashSet[AnyRef]
- private val messageDemultiplexer = new Demultiplexer(queue)
-
- // build default thread pool
- withNewThreadPoolWithLinkedBlockingQueueWithUnboundedCapacity.buildThreadPool
-
- def start = if (!active) {
- log.debug("Starting up %s", toString)
- active = true
-
- /**
- * This dispatcher code is based on code from the actorom actor framework by Sergio Bossa
- * [http://code.google.com/p/actorom/].
- */
- selectorThread = new Thread(name) {
- override def run = {
- while (active) {
- try {
- try {
- messageDemultiplexer.select
- } catch { case e: InterruptedException => active = false }
- process(messageDemultiplexer.acquireSelectedInvocations)
- } finally {
- messageDemultiplexer.releaseSelectedInvocations
- }
- }
- }
- };
- selectorThread.start
- }
-
- override protected def doShutdown = executor.shutdownNow
-
- private def process(selectedInvocations: List[MessageInvocation]) = synchronized {
- var nrOfBusyMessages = 0
- val totalNrOfActors = messageInvokers.size
- val totalNrOfBusyActors = busyActors.size
- val invocations = selectedInvocations.iterator
- while (invocations.hasNext && totalNrOfActors > totalNrOfBusyActors && passFairnessCheck(nrOfBusyMessages)) {
- val invocation = invocations.next
- if (invocation eq null) throw new IllegalActorStateException("Message invocation is null [" + invocation + "]")
- if (!busyActors.contains(invocation.receiver)) {
- val invoker = messageInvokers.get(invocation.receiver)
- if (invoker eq null) throw new IllegalActorStateException(
- "Message invoker for invocation [" + invocation + "] is null")
- resume(invocation.receiver)
- invocations.remove
- executor.execute(new Runnable() {
- def run = {
- invoker.invoke(invocation)
- suspend(invocation.receiver)
- messageDemultiplexer.wakeUp
- }
- })
- } else nrOfBusyMessages += 1
- }
- }
-
- private def resume(actor: AnyRef) = synchronized {
- busyActors.add(actor)
- }
-
- private def suspend(actor: AnyRef) = synchronized {
- busyActors.remove(actor)
- }
-
- private def passFairnessCheck(nrOfBusyMessages: Int) = {
- if (fair) true
- else nrOfBusyMessages < 100
- }
-
- def ensureNotActive(): Unit = if (active) throw new IllegalActorStateException(
- "Can't build a new thread pool for a dispatcher that is already up and running")
-
- override def toString = "ReactorBasedThreadPoolEventDrivenDispatcher[" + name + "]"
-
- class Demultiplexer(private val messageQueue: ReactiveMessageQueue) extends MessageDemultiplexer {
- private val selectedInvocations: List[MessageInvocation] = new LinkedList[MessageInvocation]
- private val selectedInvocationsLock = new ReentrantLock
-
- def select = try {
- selectedInvocationsLock.lock
- messageQueue.read(selectedInvocations)
- } finally {
- selectedInvocationsLock.unlock
- }
-
- def acquireSelectedInvocations: List[MessageInvocation] = {
- selectedInvocationsLock.lock
- selectedInvocations
- }
-
- def releaseSelectedInvocations = selectedInvocationsLock.unlock
-
- def wakeUp = messageQueue.interrupt
- }
-}
diff --git a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala
index 589500f413..eda5a86a9e 100644
--- a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala
+++ b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala
@@ -8,8 +8,8 @@ import java.util.Queue
import se.scalablesolutions.akka.actor.{Actor, ActorRef}
import se.scalablesolutions.akka.config.Config.config
-import java.util.concurrent.{TimeUnit, LinkedBlockingQueue}
import concurrent.forkjoin.{TransferQueue, LinkedTransferQueue}
+import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, LinkedBlockingQueue}
/**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
@@ -17,10 +17,9 @@ import concurrent.forkjoin.{TransferQueue, LinkedTransferQueue}
* @author Jonas Bonér
*/
class ThreadBasedDispatcher(private val actor: ActorRef,
- val mailboxCapacity: Int = Dispatchers.MAILBOX_CAPACITY,
- val pushTimeout: Long = 10000,
- val pushTimeoutUnit: TimeUnit = TimeUnit.MILLISECONDS
+ val mailboxConfig: MailboxConfig
) extends MessageDispatcher {
+ def this(actor: ActorRef, capacity: Int) = this(actor,MailboxConfig(capacity,None,true))
def this(actor: ActorRef) = this(actor, Dispatchers.MAILBOX_CAPACITY)// For Java
private val name = actor.getClass.getName + ":" + actor.uuid
@@ -28,13 +27,8 @@ class ThreadBasedDispatcher(private val actor: ActorRef,
private var selectorThread: Thread = _
@volatile private var active: Boolean = false
- if (actor.mailbox eq null) {
- actor.mailbox = if (mailboxCapacity > 0)
- new BoundedTransferQueue[MessageInvocation](mailboxCapacity,pushTimeout,pushTimeoutUnit) with ThreadMessageQueue
- else
- new LinkedTransferQueue[MessageInvocation] with ThreadMessageQueue
- }
-
+ override def createMailbox(actorRef: ActorRef): AnyRef = mailboxConfig.newMailbox(blockDequeue = true)
+
override def register(actorRef: ActorRef) = {
if(actorRef != actor)
throw new IllegalArgumentException("Cannot register to anyone but " + actor)
@@ -42,9 +36,11 @@ class ThreadBasedDispatcher(private val actor: ActorRef,
super.register(actorRef)
}
- def mailbox = actor.mailbox.asInstanceOf[ThreadMessageQueue]
+ def mailbox = actor.mailbox.asInstanceOf[Queue[MessageInvocation] with MessageQueue]
- def dispatch(invocation: MessageInvocation) = mailbox append invocation
+ def mailboxSize(a: ActorRef) = mailbox.size
+
+ def dispatch(invocation: MessageInvocation) = mailbox enqueue invocation
def start = if (!active) {
log.debug("Starting up %s", toString)
@@ -53,7 +49,7 @@ class ThreadBasedDispatcher(private val actor: ActorRef,
override def run = {
while (active) {
try {
- actor.invoke(mailbox.next)
+ actor.invoke(mailbox.dequeue)
} catch { case e: InterruptedException => active = false }
}
}
@@ -71,16 +67,4 @@ class ThreadBasedDispatcher(private val actor: ActorRef,
}
override def toString = "ThreadBasedDispatcher[" + threadName + "]"
-}
-
-trait ThreadMessageQueue extends MessageQueue { self: TransferQueue[MessageInvocation] =>
-
- final def append(invocation: MessageInvocation): Unit = {
- if(!self.tryTransfer(invocation)) { //First, try to send the invocation to a waiting consumer
- if(!self.offer(invocation)) //If no consumer found, append it to the queue, if that fails, we're aborting
- throw new MessageQueueAppendFailedException("BlockingMessageTransferQueue transfer timed out")
- }
- }
-
- final def next: MessageInvocation = self.take
-}
+}
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala
index 9fe47d5415..5ad1b89aca 100644
--- a/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala
+++ b/akka-actor/src/main/scala/dispatch/ThreadPoolBuilder.scala
@@ -46,7 +46,7 @@ trait ThreadPoolBuilder extends Logging {
if (boundedExecutorBound > 0) {
val boundedExecutor = new BoundedExecutorDecorator(threadPoolBuilder, boundedExecutorBound)
- boundedExecutorBound = -1
+ boundedExecutorBound = -1 //Why is this here?
executor = boundedExecutor
} else {
executor = threadPoolBuilder
@@ -56,7 +56,6 @@ trait ThreadPoolBuilder extends Logging {
def withNewThreadPoolWithCustomBlockingQueue(queue: BlockingQueue[Runnable]): ThreadPoolBuilder = synchronized {
ensureNotActive
verifyNotInConstructionPhase
- inProcessOfBuilding = false
blockingQueue = queue
threadPoolBuilder = new ThreadPoolExecutor(NR_START_THREADS, NR_MAX_THREADS, KEEP_ALIVE_TIME, MILLISECONDS, queue)
this
@@ -209,7 +208,7 @@ trait ThreadPoolBuilder extends Logging {
/**
* @author Jonas Bonér
*/
- class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorService {
+ class BoundedExecutorDecorator(val executor: ExecutorService, bound: Int) extends ExecutorService with Logging {
protected val semaphore = new Semaphore(bound)
def execute(command: Runnable) = {
@@ -227,6 +226,9 @@ trait ThreadPoolBuilder extends Logging {
} catch {
case e: RejectedExecutionException =>
semaphore.release
+ case e =>
+ log.error(e,"Unexpected exception")
+ throw e
}
}
diff --git a/akka-actor/src/main/scala/routing/Iterators.scala b/akka-actor/src/main/scala/routing/Iterators.scala
index 6e73af08e4..7f21589cfa 100644
--- a/akka-actor/src/main/scala/routing/Iterators.scala
+++ b/akka-actor/src/main/scala/routing/Iterators.scala
@@ -5,6 +5,7 @@
package se.scalablesolutions.akka.routing
import se.scalablesolutions.akka.actor.ActorRef
+import scala.collection.JavaConversions._
/**
* An Iterator that is either always empty or yields an infinite number of Ts.
@@ -15,6 +16,8 @@ trait InfiniteIterator[T] extends Iterator[T]
* CyclicIterator is a round-robin style InfiniteIterator that cycles the supplied List.
*/
class CyclicIterator[T](items: List[T]) extends InfiniteIterator[T] {
+ def this(items: java.util.List[T]) = this(items.toList)
+
@volatile private[this] var current: List[T] = items
def hasNext = items != Nil
@@ -34,6 +37,7 @@ class CyclicIterator[T](items: List[T]) extends InfiniteIterator[T] {
* useful for work-stealing.
*/
class SmallestMailboxFirstIterator(items : List[ActorRef]) extends InfiniteIterator[ActorRef] {
+ def this(items: java.util.List[ActorRef]) = this(items.toList)
def hasNext = items != Nil
def next = items.reduceLeft((a1, a2) => if (a1.mailboxSize < a2.mailboxSize) a1 else a2)
diff --git a/akka-actor/src/main/scala/routing/Routers.scala b/akka-actor/src/main/scala/routing/Routers.scala
index 7f2effee29..d916e6c584 100644
--- a/akka-actor/src/main/scala/routing/Routers.scala
+++ b/akka-actor/src/main/scala/routing/Routers.scala
@@ -4,7 +4,7 @@
package se.scalablesolutions.akka.routing
-import se.scalablesolutions.akka.actor.{Actor, ActorRef}
+import se.scalablesolutions.akka.actor.{UntypedActor, Actor, ActorRef}
/**
* A Dispatcher is a trait whose purpose is to route incoming messages to actors.
@@ -26,6 +26,26 @@ trait Dispatcher { this: Actor =>
private def isSenderDefined = self.senderFuture.isDefined || self.sender.isDefined
}
+/**
+ * An UntypedDispatcher is an abstract class whose purpose is to route incoming messages to actors.
+ */
+abstract class UntypedDispatcher extends UntypedActor {
+ protected def transform(msg: Any): Any = msg
+
+ protected def route(msg: Any): ActorRef
+
+ private def isSenderDefined = self.senderFuture.isDefined || self.sender.isDefined
+
+ @throws(classOf[Exception])
+ def onReceive(msg: Any): Unit = {
+ val r = route(msg)
+ if(r eq null)
+ throw new IllegalStateException("No route for " + msg + " defined!")
+ if (isSenderDefined) r.forward(transform(msg))(someSelf)
+ else r.!(transform(msg))(None)
+ }
+}
+
/**
* A LoadBalancer is a specialized kind of Dispatcher, that is supplied an InfiniteIterator of targets
* to dispatch incoming messages to.
@@ -37,3 +57,17 @@ trait LoadBalancer extends Dispatcher { self: Actor =>
override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) )
}
+
+/**
+ * A UntypedLoadBalancer is a specialized kind of UntypedDispatcher, that is supplied an InfiniteIterator of targets
+ * to dispatch incoming messages to.
+ */
+abstract class UntypedLoadBalancer extends UntypedDispatcher {
+ protected def seq: InfiniteIterator[ActorRef]
+
+ protected def route(msg: Any) =
+ if (seq.hasNext) seq.next
+ else null
+
+ override def isDefinedAt(msg: Any) = seq.exists( _.isDefinedAt(msg) )
+}
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/util/JavaAPI.scala b/akka-actor/src/main/scala/util/JavaAPI.scala
new file mode 100644
index 0000000000..099082595d
--- /dev/null
+++ b/akka-actor/src/main/scala/util/JavaAPI.scala
@@ -0,0 +1,23 @@
+package se.scalablesolutions.akka.util
+
+/** A Function interface
+ * Used to create first-class-functions is Java (sort of)
+ * Java API
+ */
+trait Function[T,R] {
+ def apply(param: T): R
+}
+
+/** A Procedure is like a Function, but it doesn't produce a return value
+ * Java API
+ */
+trait Procedure[T] {
+ def apply(param: T): Unit
+}
+
+/**
+ * An executable piece of code that takes no parameters and doesn't return any value
+ */
+trait SideEffect {
+ def apply: Unit
+}
diff --git a/akka-actor/src/main/scala/util/ListenerManagement.scala b/akka-actor/src/main/scala/util/ListenerManagement.scala
index 0e17058380..7ad0f451f1 100644
--- a/akka-actor/src/main/scala/util/ListenerManagement.scala
+++ b/akka-actor/src/main/scala/util/ListenerManagement.scala
@@ -40,6 +40,23 @@ trait ListenerManagement extends Logging {
if (manageLifeCycleOfListeners) listener.stop
}
+ /*
+ * Returns whether there are any listeners currently
+ */
+ def hasListeners: Boolean = !listeners.isEmpty
+
+ protected def notifyListeners(message: => Any) {
+ if (hasListeners) {
+ val msg = message
+ val iterator = listeners.iterator
+ while (iterator.hasNext) {
+ val listener = iterator.next
+ if (listener.isRunning) listener ! msg
+ else log.warning("Can't notify [%s] since it is not running.", listener)
+ }
+ }
+ }
+
/**
* Execute f with each listener as argument.
*/
diff --git a/akka-actor/src/main/scala/util/LockUtil.scala b/akka-actor/src/main/scala/util/LockUtil.scala
index 885e11def7..3d1261e468 100644
--- a/akka-actor/src/main/scala/util/LockUtil.scala
+++ b/akka-actor/src/main/scala/util/LockUtil.scala
@@ -5,6 +5,7 @@
package se.scalablesolutions.akka.util
import java.util.concurrent.locks.{ReentrantReadWriteLock, ReentrantLock}
+import java.util.concurrent.atomic.AtomicBoolean
/**
* @author Jonas Bonér
@@ -58,3 +59,56 @@ class ReadWriteGuard {
}
}
+/**
+ * A very simple lock that uses CCAS (Compare Compare-And-Swap)
+ * Does not keep track of the owner and isn't Reentrant, so don't nest and try to stick to the if*-methods
+ */
+class SimpleLock {
+ val acquired = new AtomicBoolean(false)
+
+ def ifPossible(perform: () => Unit): Boolean = {
+ if (tryLock()) {
+ try {
+ perform
+ } finally {
+ unlock()
+ }
+ true
+ } else false
+ }
+
+ def ifPossibleYield[T](perform: () => T): Option[T] = {
+ if (tryLock()) {
+ try {
+ Some(perform())
+ } finally {
+ unlock()
+ }
+ } else None
+ }
+
+ def ifPossibleApply[T,R](value: T)(function: (T) => R): Option[R] = {
+ if (tryLock()) {
+ try {
+ Some(function(value))
+ } finally {
+ unlock()
+ }
+ } else None
+ }
+
+ def tryLock() = {
+ if (acquired.get) false
+ else acquired.compareAndSet(false,true)
+ }
+
+ def tryUnlock() = {
+ acquired.compareAndSet(true,false)
+ }
+
+ def locked = acquired.get
+
+ def unlock() {
+ acquired.set(false)
+ }
+}
\ No newline at end of file
diff --git a/akka-actor/src/main/scala/util/Logging.scala b/akka-actor/src/main/scala/util/Logging.scala
index f13c229653..b6ddaaa16a 100644
--- a/akka-actor/src/main/scala/util/Logging.scala
+++ b/akka-actor/src/main/scala/util/Logging.scala
@@ -99,18 +99,26 @@ class Logger(val logger: SLFLogger) {
warning(t,message(fmt,arg,argN:_*))
}
+ def warn(t: Throwable, fmt: => String, arg: Any, argN: Any*) = warning(t, fmt, arg, argN)
+
def warning(t: Throwable, msg: => String) {
if (warning_?) logger.warn(msg,t)
}
+ def warn(t: Throwable, msg: => String) = warning(t, msg)
+
def warning(fmt: => String, arg: Any, argN: Any*) {
warning(message(fmt,arg,argN:_*))
}
+ def warn(fmt: => String, arg: Any, argN: Any*) = warning(fmt, arg, argN)
+
def warning(msg: => String) {
if (warning_?) logger warn msg
}
+ def warn(msg: => String) = warning(msg)
+
//Error
def error(t: Throwable, fmt: => String, arg: Any, argN: Any*) {
error(t,message(fmt,arg,argN:_*))
diff --git a/akka-actor/src/main/scala/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/util/ReflectiveAccess.scala
index 4582304188..abccd5d9b0 100644
--- a/akka-actor/src/main/scala/util/ReflectiveAccess.scala
+++ b/akka-actor/src/main/scala/util/ReflectiveAccess.scala
@@ -62,14 +62,8 @@ object ReflectiveAccess {
def ensureRemotingEnabled = if (!isRemotingEnabled) throw new ModuleNotAvailableException(
"Can't load the remoting module, make sure that akka-remote.jar is on the classpath")
- val remoteClientObjectInstance: Option[RemoteClientObject] = {
- try {
- val clazz = loader.loadClass("se.scalablesolutions.akka.remote.RemoteClient$")
- val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*)
- ctor.setAccessible(true)
- Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[RemoteClientObject])
- } catch { case e: Exception => None }
- }
+ val remoteClientObjectInstance: Option[RemoteClientObject] =
+ getObject("se.scalablesolutions.akka.remote.RemoteClient$")
def register(address: InetSocketAddress, uuid: String) = {
ensureRemotingEnabled
@@ -126,23 +120,11 @@ object ReflectiveAccess {
def unregister(actorRef: ActorRef): Unit
}
- val remoteServerObjectInstance: Option[RemoteServerObject] = {
- try {
- val clazz = loader.loadClass("se.scalablesolutions.akka.remote.RemoteServer$")
- val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*)
- ctor.setAccessible(true)
- Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[RemoteServerObject])
- } catch { case e: Exception => None }
- }
+ val remoteServerObjectInstance: Option[RemoteServerObject] =
+ getObject("se.scalablesolutions.akka.remote.RemoteServer$")
- val remoteNodeObjectInstance: Option[RemoteNodeObject] = {
- try {
- val clazz = loader.loadClass("se.scalablesolutions.akka.remote.RemoteNode$")
- val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*)
- ctor.setAccessible(true)
- Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[RemoteNodeObject])
- } catch { case e: Exception => None }
- }
+ val remoteNodeObjectInstance: Option[RemoteNodeObject] =
+ getObject("se.scalablesolutions.akka.remote.RemoteNode$")
def registerActor(address: InetSocketAddress, uuid: String, actorRef: ActorRef) = {
ensureRemotingEnabled
@@ -177,14 +159,8 @@ object ReflectiveAccess {
def ensureTypedActorEnabled = if (!isTypedActorEnabled) throw new ModuleNotAvailableException(
"Can't load the typed actor module, make sure that akka-typed-actor.jar is on the classpath")
- val typedActorObjectInstance: Option[TypedActorObject] = {
- try {
- val clazz = loader.loadClass("se.scalablesolutions.akka.actor.TypedActor$")
- val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*)
- ctor.setAccessible(true)
- Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[TypedActorObject])
- } catch { case e: Exception => None }
- }
+ val typedActorObjectInstance: Option[TypedActorObject] =
+ getObject("se.scalablesolutions.akka.actor.TypedActor$")
def resolveFutureIfMessageIsJoinPoint(message: Any, future: Future[_]): Boolean = {
ensureTypedActorEnabled
@@ -212,18 +188,46 @@ object ReflectiveAccess {
def ensureJtaEnabled = if (!isJtaEnabled) throw new ModuleNotAvailableException(
"Can't load the typed actor module, make sure that akka-jta.jar is on the classpath")
- val transactionContainerObjectInstance: Option[TransactionContainerObject] = {
- try {
- val clazz = loader.loadClass("se.scalablesolutions.akka.actor.TransactionContainer$")
- val ctor = clazz.getDeclaredConstructor(Array[Class[_]](): _*)
- ctor.setAccessible(true)
- Some(ctor.newInstance(Array[AnyRef](): _*).asInstanceOf[TransactionContainerObject])
- } catch { case e: Exception => None }
- }
+ val transactionContainerObjectInstance: Option[TransactionContainerObject] =
+ getObject("se.scalablesolutions.akka.actor.TransactionContainer$")
def createTransactionContainer: TransactionContainer = {
ensureJtaEnabled
transactionContainerObjectInstance.get.apply.asInstanceOf[TransactionContainer]
}
}
+
+ val noParams = Array[Class[_]]()
+ val noArgs = Array[AnyRef]()
+
+ def createInstance[T](clazz: Class[_],
+ params: Array[Class[_]],
+ args: Array[AnyRef]): Option[T] = try {
+ val ctor = clazz.getDeclaredConstructor(params: _*)
+ ctor.setAccessible(true)
+ Some(ctor.newInstance(args: _*).asInstanceOf[T])
+ } catch {
+ case e: Exception => None
+ }
+
+ def createInstance[T](fqn: String,
+ params: Array[Class[_]],
+ args: Array[AnyRef],
+ classloader: ClassLoader = loader): Option[T] = try {
+ val clazz = classloader.loadClass(fqn)
+ val ctor = clazz.getDeclaredConstructor(params: _*)
+ ctor.setAccessible(true)
+ Some(ctor.newInstance(args: _*).asInstanceOf[T])
+ } catch {
+ case e: Exception => None
+ }
+
+ def getObject[T](fqn: String, classloader: ClassLoader = loader): Option[T] = try {//Obtains a reference to $MODULE$
+ val clazz = classloader.loadClass(fqn)
+ val instance = clazz.getDeclaredField("MODULE$")
+ instance.setAccessible(true)
+ Option(instance.get(null).asInstanceOf[T])
+ } catch {
+ case e: Exception => None
+ }
}
diff --git a/akka-actor/src/test/resources/logback-test.xml b/akka-actor/src/test/resources/logback-test.xml
deleted file mode 100644
index 78eae40ec4..0000000000
--- a/akka-actor/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,21 +0,0 @@
-
-
-
-
-
-
-
-
-
-
-
-
-
- [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n
-
-
-
-
-
-
-
diff --git a/akka-actor/src/test/scala/actor/actor/Bench.scala b/akka-actor/src/test/scala/actor/actor/Bench.scala
index 8e3a44f3a0..ded90edad5 100644
--- a/akka-actor/src/test/scala/actor/actor/Bench.scala
+++ b/akka-actor/src/test/scala/actor/actor/Bench.scala
@@ -78,7 +78,7 @@ object Chameneos {
var sumMeetings = 0
var numFaded = 0
- override def init = {
+ override def preStart = {
for (i <- 0 until numChameneos) actorOf(new Chameneo(self, colours(i % 3), i))
}
diff --git a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
index 5023c756e1..b9fa238963 100644
--- a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
+++ b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
@@ -42,7 +42,7 @@ class RestartStrategySpec extends JUnitSuite {
restartLatch.open
}
- override def shutdown = {
+ override def postStop = {
if (restartLatch.isOpen) {
secondRestartLatch.open
}
@@ -70,5 +70,32 @@ class RestartStrategySpec extends JUnitSuite {
}
assert(exceptionLatch.tryAwait(1, TimeUnit.SECONDS))
}
+
+ @Test
+ def slaveShouldBeImmortalWithoutMaxRestarts = {
+
+ val boss = actorOf(new Actor{
+ self.trapExit = List(classOf[Throwable])
+ self.faultHandler = Some(OneForOneStrategy(None, None))
+ protected def receive = { case _ => () }
+ }).start
+
+ val countDownLatch = new CountDownLatch(100)
+
+ val slave = actorOf(new Actor{
+
+ protected def receive = {
+ case Crash => throw new Exception("Crashing...")
+ }
+
+ override def postRestart(reason: Throwable) = {
+ countDownLatch.countDown
+ }
+ })
+
+ boss.startLink(slave)
+ (1 to 100) foreach { _ => slave ! Crash }
+ assert(countDownLatch.await(120, TimeUnit.SECONDS))
+ }
}
diff --git a/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala b/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala
index bb548b9251..81fd933cda 100644
--- a/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala
+++ b/akka-actor/src/test/scala/dispatch/DispatchersSpec.scala
@@ -28,13 +28,9 @@ object DispatchersSpec {
def ofType[T <: MessageDispatcher : Manifest]: (MessageDispatcher) => Boolean = _.getClass == manifest[T].erasure
def typesAndValidators: Map[String,(MessageDispatcher) => Boolean] = Map(
- "ReactorBasedSingleThreadEventDriven" -> ofType[ReactorBasedSingleThreadEventDrivenDispatcher],
"ExecutorBasedEventDrivenWorkStealing" -> ofType[ExecutorBasedEventDrivenWorkStealingDispatcher],
"ExecutorBasedEventDriven" -> ofType[ExecutorBasedEventDrivenDispatcher],
- "ReactorBasedThreadPoolEventDriven" -> ofType[ReactorBasedThreadPoolEventDrivenDispatcher],
"Hawt" -> ofType[HawtDispatcher],
- "GlobalReactorBasedSingleThreadEventDriven" -> instance(globalReactorBasedSingleThreadEventDrivenDispatcher),
- "GlobalReactorBasedThreadPoolEventDriven" -> instance(globalReactorBasedThreadPoolEventDrivenDispatcher),
"GlobalExecutorBasedEventDriven" -> instance(globalExecutorBasedEventDrivenDispatcher),
"GlobalHawt" -> instance(globalHawtDispatcher)
)
diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
index 9cdf43682e..3935bc9b0b 100644
--- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
+++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenDispatcherActorSpec.scala
@@ -3,9 +3,10 @@ package se.scalablesolutions.akka.actor.dispatch
import java.util.concurrent.{CountDownLatch, TimeUnit}
import org.scalatest.junit.JUnitSuite
import org.junit.Test
-import se.scalablesolutions.akka.dispatch.Dispatchers
+import se.scalablesolutions.akka.dispatch.{Dispatchers,ExecutorBasedEventDrivenDispatcher}
import se.scalablesolutions.akka.actor.Actor
import Actor._
+import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger}
object ExecutorBasedEventDrivenDispatcherActorSpec {
class TestActor extends Actor {
@@ -65,4 +66,73 @@ class ExecutorBasedEventDrivenDispatcherActorSpec extends JUnitSuite {
}
actor.stop
}
+
+ @Test def shouldRespectThroughput {
+ val throughputDispatcher = new ExecutorBasedEventDrivenDispatcher("THROUGHPUT",101,0,Dispatchers.MAILBOX_CONFIG, (e) => {
+ e.setCorePoolSize(1)
+ })
+
+ val works = new AtomicBoolean(true)
+ val latch = new CountDownLatch(100)
+ val start = new CountDownLatch(1)
+ val fastOne = actorOf(
+ new Actor {
+ self.dispatcher = throughputDispatcher
+ def receive = { case "sabotage" => works.set(false) }
+ }).start
+
+ val slowOne = actorOf(
+ new Actor {
+ self.dispatcher = throughputDispatcher
+ def receive = {
+ case "hogexecutor" => start.await
+ case "ping" => if (works.get) latch.countDown
+ }
+ }).start
+
+ slowOne ! "hogexecutor"
+ (1 to 100) foreach { _ => slowOne ! "ping"}
+ fastOne ! "sabotage"
+ start.countDown
+ val result = latch.await(3,TimeUnit.SECONDS)
+ fastOne.stop
+ slowOne.stop
+ throughputDispatcher.shutdown
+ assert(result === true)
+ }
+
+ @Test def shouldRespectThroughputDeadline {
+ val deadlineMs = 100
+ val throughputDispatcher = new ExecutorBasedEventDrivenDispatcher("THROUGHPUT",2,deadlineMs,Dispatchers.MAILBOX_CONFIG, (e) => {
+ e.setCorePoolSize(1)
+ })
+
+ val works = new AtomicBoolean(true)
+ val latch = new CountDownLatch(1)
+ val start = new CountDownLatch(1)
+ val ready = new CountDownLatch(1)
+
+ val fastOne = actorOf(
+ new Actor {
+ self.dispatcher = throughputDispatcher
+ def receive = { case "ping" => if(works.get) latch.countDown; self.stop }
+ }).start
+
+ val slowOne = actorOf(
+ new Actor {
+ self.dispatcher = throughputDispatcher
+ def receive = {
+ case "hogexecutor" => ready.countDown; start.await
+ case "ping" => works.set(false); self.stop
+ }
+ }).start
+
+ slowOne ! "hogexecutor"
+ slowOne ! "ping"
+ fastOne ! "ping"
+ assert(ready.await(5,TimeUnit.SECONDS) === true)
+ Thread.sleep(deadlineMs)
+ start.countDown
+ assert(latch.await(10,TimeUnit.SECONDS) === true)
+ }
}
diff --git a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala
index cde57a0544..3285e450c6 100644
--- a/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala
+++ b/akka-actor/src/test/scala/dispatch/ExecutorBasedEventDrivenWorkStealingDispatcherSpec.scala
@@ -5,11 +5,10 @@ import org.scalatest.junit.JUnitSuite
import org.junit.Test
-import se.scalablesolutions.akka.dispatch.Dispatchers
-
import java.util.concurrent.{TimeUnit, CountDownLatch}
import se.scalablesolutions.akka.actor.{IllegalActorStateException, Actor}
import Actor._
+import se.scalablesolutions.akka.dispatch.{MessageQueue, Dispatchers}
object ExecutorBasedEventDrivenWorkStealingDispatcherSpec {
val delayableActorDispatcher = Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher("pooled-dispatcher")
@@ -18,7 +17,7 @@ object ExecutorBasedEventDrivenWorkStealingDispatcherSpec {
class DelayableActor(name: String, delay: Int, finishedCounter: CountDownLatch) extends Actor {
self.dispatcher = delayableActorDispatcher
- var invocationCount = 0
+ @volatile var invocationCount = 0
self.id = name
def receive = {
@@ -61,10 +60,14 @@ class ExecutorBasedEventDrivenWorkStealingDispatcherSpec extends JUnitSuite with
val slow = actorOf(new DelayableActor("slow", 50, finishedCounter)).start
val fast = actorOf(new DelayableActor("fast", 10, finishedCounter)).start
+ var sentToFast = 0
+
for (i <- 1 to 100) {
// send most work to slow actor
- if (i % 20 == 0)
+ if (i % 20 == 0) {
fast ! i
+ sentToFast += 1
+ }
else
slow ! i
}
@@ -72,13 +75,18 @@ class ExecutorBasedEventDrivenWorkStealingDispatcherSpec extends JUnitSuite with
// now send some messages to actors to keep the dispatcher dispatching messages
for (i <- 1 to 10) {
Thread.sleep(150)
- if (i % 2 == 0)
+ if (i % 2 == 0) {
fast ! i
+ sentToFast += 1
+ }
else
slow ! i
}
finishedCounter.await(5, TimeUnit.SECONDS)
+ fast.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true)
+ slow.mailbox.asInstanceOf[MessageQueue].isEmpty must be(true)
+ fast.actor.asInstanceOf[DelayableActor].invocationCount must be > sentToFast
fast.actor.asInstanceOf[DelayableActor].invocationCount must be >
(slow.actor.asInstanceOf[DelayableActor].invocationCount)
slow.stop
diff --git a/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala b/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala
index 97f2e0df9d..8d4c8dedc1 100644
--- a/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala
+++ b/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala
@@ -53,7 +53,7 @@ object HawtDispatcherEchoServer {
var accept_source:DispatchSource = _
var sessions = ListBuffer[ActorRef]()
- override def init = {
+ override def preStart = {
channel = ServerSocketChannel.open();
channel.socket().bind(new InetSocketAddress(port));
channel.configureBlocking(false);
@@ -122,7 +122,7 @@ object HawtDispatcherEchoServer {
var writeCounter = 0L
var closed = false
- override def init = {
+ override def preStart = {
if(useReactorPattern) {
// Then we will be using the reactor pattern for handling IO:
@@ -154,7 +154,7 @@ object HawtDispatcherEchoServer {
println("Accepted connection from: "+remote_address);
}
- override def shutdown = {
+ override def postStop = {
closed = true
read_source.release
write_source.release
diff --git a/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala b/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala
new file mode 100644
index 0000000000..27afdbbce6
--- /dev/null
+++ b/akka-actor/src/test/scala/dispatch/MailboxConfigSpec.scala
@@ -0,0 +1,53 @@
+package se.scalablesolutions.akka.actor.dispatch
+
+import org.scalatest.junit.JUnitSuite
+import org.junit.Test
+import se.scalablesolutions.akka.actor.Actor
+import Actor._
+import java.util.concurrent.{BlockingQueue, CountDownLatch, TimeUnit}
+import se.scalablesolutions.akka.util.Duration
+import se.scalablesolutions.akka.dispatch.{MessageQueueAppendFailedException, MessageInvocation, MailboxConfig, Dispatchers}
+import java.util.concurrent.atomic.{AtomicReference}
+
+object MailboxConfigSpec {
+
+}
+
+class MailboxConfigSpec extends JUnitSuite {
+ import MailboxConfigSpec._
+
+ private val unit = TimeUnit.MILLISECONDS
+
+ @Test def shouldCreateUnboundedQueue = {
+ val m = MailboxConfig(-1,None,false)
+ assert(m.newMailbox().asInstanceOf[BlockingQueue[MessageInvocation]].remainingCapacity === Integer.MAX_VALUE)
+ }
+
+ @Test def shouldCreateBoundedQueue = {
+ val m = MailboxConfig(1,None,false)
+ assert(m.newMailbox().asInstanceOf[BlockingQueue[MessageInvocation]].remainingCapacity === 1)
+ }
+
+ @Test(expected = classOf[MessageQueueAppendFailedException]) def shouldThrowMessageQueueAppendFailedExceptionWhenTimeOutEnqueue = {
+ val m = MailboxConfig(1,Some(Duration(1,unit)),false)
+ val testActor = actorOf( new Actor { def receive = { case _ => }} )
+ val mbox = m.newMailbox()
+ (1 to 10000) foreach { i => mbox.enqueue(new MessageInvocation(testActor,i,None,None,None)) }
+ }
+
+
+ @Test def shouldBeAbleToDequeueUnblocking = {
+ val m = MailboxConfig(1,Some(Duration(1,unit)),false)
+ val mbox = m.newMailbox()
+ val latch = new CountDownLatch(1)
+ val t = new Thread { override def run = {
+ mbox.dequeue
+ latch.countDown
+ }}
+ t.start
+ val result = latch.await(5000,unit)
+ if (!result)
+ t.interrupt
+ assert(result === true)
+ }
+}
diff --git a/akka-actor/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala
deleted file mode 100644
index de9b912bf5..0000000000
--- a/akka-actor/src/test/scala/dispatch/ReactorBasedSingleThreadEventDrivenDispatcherActorSpec.scala
+++ /dev/null
@@ -1,71 +0,0 @@
-package se.scalablesolutions.akka.actor.dispatch
-
-import java.util.concurrent.{CountDownLatch, TimeUnit}
-import org.scalatest.junit.JUnitSuite
-import org.junit.Test
-
-import se.scalablesolutions.akka.dispatch.Dispatchers
-import se.scalablesolutions.akka.actor.Actor
-import Actor._
-
-object ReactorBasedSingleThreadEventDrivenDispatcherActorSpec {
- class TestActor extends Actor {
- self.dispatcher = Dispatchers.newReactorBasedSingleThreadEventDrivenDispatcher(self.uuid)
-
- def receive = {
- case "Hello" =>
- self.reply("World")
- case "Failure" =>
- throw new RuntimeException("Expected exception; to test fault-tolerance")
- }
- }
-
- object OneWayTestActor {
- val oneWay = new CountDownLatch(1)
- }
- class OneWayTestActor extends Actor {
- self.dispatcher = Dispatchers.newExecutorBasedEventDrivenDispatcher(self.uuid)
- def receive = {
- case "OneWay" => OneWayTestActor.oneWay.countDown
- }
- }
-}
-
-class ReactorBasedSingleThreadEventDrivenDispatcherActorSpec extends JUnitSuite {
- import ReactorBasedSingleThreadEventDrivenDispatcherActorSpec._
-
- private val unit = TimeUnit.MILLISECONDS
-
- @Test def shouldSendOneWay = {
- val actor = actorOf[OneWayTestActor].start
- val result = actor ! "OneWay"
- assert(OneWayTestActor.oneWay.await(1, TimeUnit.SECONDS))
- actor.stop
- }
-
- @Test def shouldSendReplySync = {
- val actor = actorOf[TestActor].start
- val result = (actor !! ("Hello", 10000)).as[String].get
- assert("World" === result)
- actor.stop
- }
-
- @Test def shouldSendReplyAsync = {
- val actor = actorOf[TestActor].start
- val result = actor !! "Hello"
- assert("World" === result.get.asInstanceOf[String])
- actor.stop
- }
-
- @Test def shouldSendReceiveException = {
- val actor = actorOf[TestActor].start
- try {
- actor !! "Failure"
- fail("Should have thrown an exception")
- } catch {
- case e =>
- assert("Expected exception; to test fault-tolerance" === e.getMessage())
- }
- actor.stop
- }
-}
diff --git a/akka-actor/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala b/akka-actor/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala
deleted file mode 100644
index 4001df8f56..0000000000
--- a/akka-actor/src/test/scala/dispatch/ReactorBasedThreadPoolEventDrivenDispatcherActorSpec.scala
+++ /dev/null
@@ -1,66 +0,0 @@
-package se.scalablesolutions.akka.actor.dispatch
-
-import java.util.concurrent.{CountDownLatch, TimeUnit}
-import org.scalatest.junit.JUnitSuite
-import org.junit.Test
-
-import se.scalablesolutions.akka.dispatch.Dispatchers
-import se.scalablesolutions.akka.actor.Actor
-import Actor._
-
-object ReactorBasedThreadPoolEventDrivenDispatcherActorSpec {
- class TestActor extends Actor {
- self.dispatcher = Dispatchers.newReactorBasedThreadPoolEventDrivenDispatcher(self.uuid)
- def receive = {
- case "Hello" =>
- self.reply("World")
- case "Failure" =>
- throw new RuntimeException("Expected exception; to test fault-tolerance")
- }
- }
-}
-
-class ReactorBasedThreadPoolEventDrivenDispatcherActorSpec extends JUnitSuite {
- import ReactorBasedThreadPoolEventDrivenDispatcherActorSpec._
-
- private val unit = TimeUnit.MILLISECONDS
-
- @Test def shouldSendOneWay {
- val oneWay = new CountDownLatch(1)
- val actor = actorOf(new Actor {
- self.dispatcher = Dispatchers.newReactorBasedThreadPoolEventDrivenDispatcher(self.uuid)
- def receive = {
- case "OneWay" => oneWay.countDown
- }
- }).start
- val result = actor ! "OneWay"
- assert(oneWay.await(1, TimeUnit.SECONDS))
- actor.stop
- }
-
- @Test def shouldSendReplySync = {
- val actor = actorOf[TestActor].start
- val result = (actor !! ("Hello", 10000)).as[String].get
- assert("World" === result)
- actor.stop
- }
-
- @Test def shouldSendReplyAsync = {
- val actor = actorOf[TestActor].start
- val result = actor !! "Hello"
- assert("World" === result.get.asInstanceOf[String])
- actor.stop
- }
-
- @Test def shouldSendReceiveException = {
- val actor = actorOf[TestActor].start
- try {
- actor !! "Failure"
- fail("Should have thrown an exception")
- } catch {
- case e =>
- assert("Expected exception; to test fault-tolerance" === e.getMessage())
- }
- actor.stop
- }
-}
diff --git a/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala b/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala
index 44cd9aade3..7ecef80e39 100644
--- a/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala
+++ b/akka-actor/src/test/scala/dispatch/ThreadBasedDispatcherSpec.scala
@@ -85,7 +85,7 @@ class ThreadBasedDispatcherSpec extends JUnitSuite {
}
assert(handleLatch.await(5, TimeUnit.SECONDS))
assert(!threadingIssueDetected.get)
- dispatcher.shutdown
+ dispatcher.postStop
}
}
*/
diff --git a/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala b/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala
index 8c9e0778ca..1fe72d6c68 100644
--- a/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala
+++ b/akka-actor/src/test/scala/misc/ActorRegistrySpec.scala
@@ -219,37 +219,35 @@ class ActorRegistrySpec extends JUnitSuite {
@Test def shouldBeAbleToRegisterActorsConcurrently {
ActorRegistry.shutdownAll
- val latch = new CountDownLatch(3)
- val barrier = new CyclicBarrier(3)
-
- def mkTestActor(i:Int) = actorOf( new Actor {
+ def mkTestActors = for(i <- (1 to 10).toList;j <- 1 to 3000) yield actorOf( new Actor {
self.id = i.toString
def receive = { case _ => }
})
- def mkTestActors = for(i <- 1 to 10;j <- 1 to 1000) yield mkTestActor(i)
+ val latch = new CountDownLatch(3)
+ val barrier = new CyclicBarrier(3)
def mkThread(actors: Iterable[ActorRef]) = new Thread {
- start
+ this.start
override def run {
barrier.await
actors foreach { _.start }
latch.countDown
}
}
+ val a1,a2,a3 = mkTestActors
+ val t1 = mkThread(a1)
+ val t2 = mkThread(a2)
+ val t3 = mkThread(a3)
- val testActors1 = mkTestActors
- val testActors2 = mkTestActors
- val testActors3 = mkTestActors
-
- mkThread(testActors1)
- mkThread(testActors2)
- mkThread(testActors3)
assert(latch.await(30,TimeUnit.SECONDS) === true)
for(i <- 1 to 10) {
- assert(ActorRegistry.actorsFor(i.toString).length === 3000)
+ val theId = i.toString
+ val actors = ActorRegistry.actorsFor(theId).toSet
+ for(a <- actors if a.id == theId) assert(actors contains a)
+ assert(actors.size === 9000)
}
}
}
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala
index d3f0acd1cf..0ca9046093 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ConsumerActor.scala
@@ -108,10 +108,10 @@ private[amqp] class ConsumerActor(consumerParameters: ConsumerParameters)
super.preRestart(reason)
}
- override def shutdown = {
+ override def postStop = {
listenerTag.foreach(tag => channel.foreach(_.basicCancel(tag)))
self.shutdownLinkedActors
- super.shutdown
+ super.postStop
}
override def toString =
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala
index f45553520d..ecb3029444 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/ExampleSession.scala
@@ -46,7 +46,7 @@ object ExampleSession {
printTopic("Happy hAkking :-)")
- // shutdown everything the amqp tree except the main AMQP supervisor
+ // postStop everything the amqp tree except the main AMQP supervisor
// all connections/consumers/producers will be stopped
AMQP.shutdownAll
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala
index 5ecae4c6d3..4d642df554 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantChannelActor.scala
@@ -103,5 +103,5 @@ abstract private[amqp] class FaultTolerantChannelActor(
closeChannel
}
- override def shutdown = closeChannel
+ override def postStop = closeChannel
}
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala
index 1e50a985be..0fd3f715b5 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/FaultTolerantConnectionActor.scala
@@ -104,9 +104,9 @@ private[amqp] class FaultTolerantConnectionActor(connectionParameters: Connectio
connectionCallback.foreach(cb => if (cb.isRunning) cb ! message)
}
- override def shutdown = {
+ override def postStop = {
reconnectionTimer.cancel
- // make sure shutdown is called on all linked actors so they can do channel cleanup before connection is killed
+ // make sure postStop is called on all linked actors so they can do channel cleanup before connection is killed
self.shutdownLinkedActors
disconnect
}
diff --git a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala
index 5c717cb8bb..10596e393f 100644
--- a/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala
+++ b/akka-amqp/src/main/scala/se/scalablesolutions/akka/amqp/rpc/RpcClientActor.scala
@@ -40,9 +40,9 @@ class RpcClientActor[I,O](
}
- override def shutdown = {
+ override def postStop = {
rpcClient.foreach(rpc => rpc.close)
- super.shutdown
+ super.postStop
}
override def toString = "AMQP.RpcClient[exchange=" +exchangeName + ", routingKey=" + routingKey+ "]"
diff --git a/akka-camel/src/main/scala/Producer.scala b/akka-camel/src/main/scala/Producer.scala
index 8764b91b4c..0be07e9737 100644
--- a/akka-camel/src/main/scala/Producer.scala
+++ b/akka-camel/src/main/scala/Producer.scala
@@ -54,10 +54,10 @@ trait ProducerSupport { this: Actor =>
def headersToCopy: Set[String] = headersToCopyDefault
/**
- * Default implementation of Actor.shutdown for freeing resources needed
+ * Default implementation of Actor.postStop for freeing resources needed
* to actually send messages to endpointUri.
*/
- override def shutdown {
+ override def postStop {
processor.stop
}
diff --git a/akka-camel/src/main/scala/component/ActorComponent.scala b/akka-camel/src/main/scala/component/ActorComponent.scala
index a5d56dd9dc..6c1c5902fa 100644
--- a/akka-camel/src/main/scala/component/ActorComponent.scala
+++ b/akka-camel/src/main/scala/component/ActorComponent.scala
@@ -247,8 +247,8 @@ private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCall
protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T](message: Any, timeout: Long, senderOption: Option[ActorRef], senderFuture: Option[CompletableFuture[T]]) = unsupported
protected[akka] def mailbox: AnyRef = unsupported
protected[akka] def mailbox_=(msg: AnyRef):AnyRef = unsupported
- protected[akka] def restart(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
- protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Int, withinTimeRange: Int): Unit = unsupported
+ protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
+ protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]): Unit = unsupported
protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable): Unit = unsupported
protected[akka] def linkedActors: JavaMap[String, ActorRef] = unsupported
protected[akka] def linkedActorsAsList: List[ActorRef] = unsupported
diff --git a/akka-core/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala b/akka-core/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala
index 3d048684cd..26fdb6e1ef 100644
--- a/akka-core/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala
+++ b/akka-core/src/test/scala/actor/supervisor/SupervisorMiscSpec.scala
@@ -26,7 +26,7 @@ class SupervisorMiscSpec extends WordSpec with MustMatchers {
}).start
val actor2 = Actor.actorOf(new Actor {
- self.dispatcher = Dispatchers.newReactorBasedSingleThreadEventDrivenDispatcher("test")
+ self.dispatcher = Dispatchers.newThreadBasedDispatcher(self)
override def postRestart(cause: Throwable) {countDownLatch.countDown}
protected def receive = {
diff --git a/akka-http/src/main/scala/AkkaBroadcaster.scala b/akka-http/src/main/scala/AkkaBroadcaster.scala
index 8f724ff445..ca5abc6f1d 100644
--- a/akka-http/src/main/scala/AkkaBroadcaster.scala
+++ b/akka-http/src/main/scala/AkkaBroadcaster.scala
@@ -9,10 +9,13 @@ import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.actor.Actor
import se.scalablesolutions.akka.dispatch.Dispatchers
-class AkkaBroadcaster extends org.atmosphere.jersey.JerseyBroadcaster {
- name = classOf[AkkaBroadcaster].getName
-
+object AkkaBroadcaster {
val broadcasterDispatcher = Dispatchers.fromConfig("akka.rest.comet-dispatcher")
+}
+
+class AkkaBroadcaster extends org.atmosphere.jersey.JerseyBroadcaster {
+ import AkkaBroadcaster._
+ name = classOf[AkkaBroadcaster].getName
//FIXME should be supervised
val caster = actorOf(new Actor {
diff --git a/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala b/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala
index 775c8b554d..7ea963872f 100644
--- a/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala
+++ b/akka-http/src/main/scala/AkkaClusterBroadcastFilter.scala
@@ -24,6 +24,11 @@ class AkkaClusterBroadcastFilter extends Actor with ClusterBroadcastFilter {
@BeanProperty var clusterName = ""
@BeanProperty var broadcaster : Broadcaster = null
+ def init() {
+ //Since this class is instantiated by Atmosphere, we need to make sure it's started
+ self.start
+ }
+
/**
* Stops the actor
*/
@@ -48,7 +53,4 @@ class AkkaClusterBroadcastFilter extends Actor with ClusterBroadcastFilter {
case b @ ClusterCometBroadcast(c, _) if (c == clusterName) && (broadcaster ne null) => broadcaster broadcast b
case _ =>
}
-
- //Since this class is instantiated by Atmosphere, we need to make sure it's started
- self.start
}
diff --git a/akka-http/src/main/scala/Initializer.scala b/akka-http/src/main/scala/Initializer.scala
index da95a39b77..c1cd8bfc87 100644
--- a/akka-http/src/main/scala/Initializer.scala
+++ b/akka-http/src/main/scala/Initializer.scala
@@ -13,7 +13,7 @@ import se.scalablesolutions.akka.util.{Logging, Bootable}
import javax.servlet.{ServletContextListener, ServletContextEvent}
/**
- * This class can be added to web.xml mappings as a listener to start and shutdown Akka.
+ * This class can be added to web.xml mappings as a listener to start and postStop Akka.
*
*
* ...
diff --git a/akka-jta/src/main/scala/AtomikosTransactionService.scala b/akka-jta/src/main/scala/AtomikosTransactionService.scala
index 305ddb6ace..4acbb1a013 100644
--- a/akka-jta/src/main/scala/AtomikosTransactionService.scala
+++ b/akka-jta/src/main/scala/AtomikosTransactionService.scala
@@ -36,6 +36,6 @@ class AtomikosTransactionService extends TransactionService with TransactionProt
"Could not create a new Atomikos J2EE Transaction Manager, due to: " + e.toString)
}
)))
- // TODO: gracefully shutdown of the TM
- //txService.shutdown(false)
+ // TODO: gracefully postStop of the TM
+ //txService.postStop(false)
}
diff --git a/akka-kernel/src/main/scala/EmbeddedAppServer.scala b/akka-kernel/src/main/scala/EmbeddedAppServer.scala
index b77a215157..9afcfbe572 100644
--- a/akka-kernel/src/main/scala/EmbeddedAppServer.scala
+++ b/akka-kernel/src/main/scala/EmbeddedAppServer.scala
@@ -6,12 +6,13 @@ package se.scalablesolutions.akka.kernel
import javax.ws.rs.core.UriBuilder
import javax.servlet.ServletConfig
+import java.io.File
import se.scalablesolutions.akka.actor.BootableActorLoaderService
import se.scalablesolutions.akka.util.{Bootable, Logging}
-import se.scalablesolutions.akka.comet.{ AkkaServlet }
+import se.scalablesolutions.akka.comet.AkkaServlet
+
import org.eclipse.jetty.xml.XmlConfiguration
-import java.io.File
import org.eclipse.jetty.server.{Handler, Server}
import org.eclipse.jetty.server.handler.{HandlerList, HandlerCollection, ContextHandler}
@@ -28,16 +29,17 @@ trait EmbeddedAppServer extends Bootable with Logging {
protected var server: Option[Server] = None
- abstract override def onLoad = {
+ abstract override def onLoad = {
super.onLoad
if (config.getBool("akka.rest.service", true)) {
log.info("Attempting to start Akka REST service (Jersey)")
System.setProperty("jetty.port",REST_PORT.toString)
System.setProperty("jetty.host",REST_HOSTNAME)
- System.setProperty("jetty.home",HOME.get + "/deploy/root")
+ System.setProperty("jetty.home",HOME.getOrElse(throwNoAkkaHomeException) + "/deploy/root")
- val configuration = new XmlConfiguration(new File(HOME.get + "/config/microkernel-server.xml").toURI.toURL)
+ val configuration = new XmlConfiguration(
+ new File(HOME.getOrElse(throwNoAkkaHomeException) + "/config/microkernel-server.xml").toURI.toURL)
server = Option(configuration.configure.asInstanceOf[Server]) map { s => //Set the correct classloader to our contexts
applicationLoader foreach { loader =>
diff --git a/akka-kernel/src/main/scala/Kernel.scala b/akka-kernel/src/main/scala/Kernel.scala
index 6489a60680..646ca34bcc 100644
--- a/akka-kernel/src/main/scala/Kernel.scala
+++ b/akka-kernel/src/main/scala/Kernel.scala
@@ -15,7 +15,7 @@ object Main {
}
/**
- * The Akka Kernel, is used to start And shutdown Akka in standalone/kernel mode.
+ * The Akka Kernel, is used to start And postStop Akka in standalone/kernel mode.
*
* @author Jonas Bonér
*/
diff --git a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala
index be5fc4f4c7..0c6f239ef7 100644
--- a/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala
+++ b/akka-persistence/akka-persistence-cassandra/src/main/scala/CassandraStorage.scala
@@ -29,7 +29,7 @@ object CassandraStorage extends Storage {
*
* @author Jonas Bonér
*/
-class CassandraPersistentMap(id: String) extends PersistentMap[Array[Byte], Array[Byte]] {
+class CassandraPersistentMap(id: String) extends PersistentMapBinary {
val uuid = id
val storage = CassandraStorageBackend
}
diff --git a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala
index ccaf7518f1..4d9ff48a60 100644
--- a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala
+++ b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala
@@ -7,9 +7,11 @@ package se.scalablesolutions.akka.persistence.common
import se.scalablesolutions.akka.stm._
import se.scalablesolutions.akka.stm.TransactionManagement.transaction
import se.scalablesolutions.akka.util.Logging
-import se.scalablesolutions.akka.AkkaException
-class StorageException(message: String) extends AkkaException(message)
+// FIXME move to 'stm' package + add message with more info
+class NoTransactionInScopeException extends RuntimeException
+
+class StorageException(message: String) extends RuntimeException(message)
/**
* Example Scala usage.
@@ -80,24 +82,90 @@ trait Storage {
*/
trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
with Transactional with Committable with Abortable with Logging {
- protected val newAndUpdatedEntries = TransactionalMap[K, V]()
- protected val removedEntries = TransactionalVector[K]()
protected val shouldClearOnCommit = Ref[Boolean]()
+ // operations on the Map
+ trait Op
+ case object GET extends Op
+ case object PUT extends Op
+ case object REM extends Op
+ case object UPD extends Op
+
+ // append only log: records all mutating operations
+ protected val appendOnlyTxLog = TransactionalVector[LogEntry]()
+
+ case class LogEntry(key: K, value: Option[V], op: Op)
+
+ // need to override in subclasses e.g. "sameElements" for Array[Byte]
+ def equal(k1: K, k2: K): Boolean = k1 == k2
+
+ // Seqable type that's required for maintaining the log of distinct keys affected in current transaction
+ type T <: Equals
+
+ // converts key K to the Seqable type Equals
+ def toEquals(k: K): T
+
+ // keys affected in the current transaction
+ protected val keysInCurrentTx = TransactionalMap[T, K]()
+
+ protected def addToListOfKeysInTx(key: K): Unit =
+ keysInCurrentTx += (toEquals(key), key)
+
+ protected def clearDistinctKeys = keysInCurrentTx.clear
+
+ protected def filterTxLogByKey(key: K): IndexedSeq[LogEntry] =
+ appendOnlyTxLog filter(e => equal(e.key, key))
+
+ // need to get current value considering the underlying storage as well as the transaction log
+ protected def getCurrentValue(key: K): Option[V] = {
+
+ // get all mutating entries for this key for this tx
+ val txEntries = filterTxLogByKey(key)
+
+ // get the snapshot from the underlying store for this key
+ val underlying = try {
+ storage.getMapStorageEntryFor(uuid, key)
+ } catch { case e: Exception => None }
+
+ if (txEntries.isEmpty) underlying
+ else replay(txEntries, key, underlying)
+ }
+
+ // replay all tx entries for key k with seed = initial
+ private def replay(txEntries: IndexedSeq[LogEntry], key: K, initial: Option[V]): Option[V] = {
+ import scala.collection.mutable._
+
+ val m = initial match {
+ case None => Map.empty[K, V]
+ case Some(v) => Map((key, v))
+ }
+ txEntries.foreach {case LogEntry(k, v, o) => o match {
+ case PUT => m.put(k, v.get)
+ case REM => m -= k
+ case UPD => m.update(k, v.get)
+ }}
+ m get key
+ }
+
// to be concretized in subclasses
val storage: MapStorageBackend[K, V]
def commit = {
- if (shouldClearOnCommit.isDefined && shouldClearOnCommit.get) storage.removeMapStorageFor(uuid)
- removedEntries.toList.foreach(key => storage.removeMapStorageFor(uuid, key))
- storage.insertMapStorageEntriesFor(uuid, newAndUpdatedEntries.toList)
- newAndUpdatedEntries.clear
- removedEntries.clear
+ // if (shouldClearOnCommit.isDefined && shouldClearOnCommit.get) storage.removeMapStorageFor(uuid)
+
+ appendOnlyTxLog.foreach { case LogEntry(k, v, o) => o match {
+ case PUT => storage.insertMapStorageEntryFor(uuid, k, v.get)
+ case UPD => storage.insertMapStorageEntryFor(uuid, k, v.get)
+ case REM => storage.removeMapStorageFor(uuid, k)
+ }}
+
+ appendOnlyTxLog.clear
+ clearDistinctKeys
}
def abort = {
- newAndUpdatedEntries.clear
- removedEntries.clear
+ appendOnlyTxLog.clear
+ clearDistinctKeys
shouldClearOnCommit.swap(false)
}
@@ -118,68 +186,84 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
override def put(key: K, value: V): Option[V] = {
register
- newAndUpdatedEntries.put(key, value)
+ val curr = getCurrentValue(key)
+ appendOnlyTxLog add LogEntry(key, Some(value), PUT)
+ addToListOfKeysInTx(key)
+ curr
}
override def update(key: K, value: V) = {
register
- newAndUpdatedEntries.update(key, value)
+ val curr = getCurrentValue(key)
+ appendOnlyTxLog add LogEntry(key, Some(value), UPD)
+ addToListOfKeysInTx(key)
+ curr
}
override def remove(key: K) = {
register
- removedEntries.add(key)
- newAndUpdatedEntries.get(key)
+ val curr = getCurrentValue(key)
+ appendOnlyTxLog add LogEntry(key, None, REM)
+ addToListOfKeysInTx(key)
+ curr
}
- def slice(start: Option[K], count: Int): List[Tuple2[K, V]] =
+ def slice(start: Option[K], count: Int): List[(K, V)] =
slice(start, None, count)
- def slice(start: Option[K], finish: Option[K], count: Int): List[Tuple2[K, V]] = try {
- storage.getMapStorageRangeFor(uuid, start, finish, count)
- } catch { case e: Exception => Nil }
+ def slice(start: Option[K], finish: Option[K], count: Int): List[(K, V)]
override def clear = {
register
+ appendOnlyTxLog.clear
+ clearDistinctKeys
shouldClearOnCommit.swap(true)
}
override def contains(key: K): Boolean = try {
- newAndUpdatedEntries.contains(key) ||
- storage.getMapStorageEntryFor(uuid, key).isDefined
+ filterTxLogByKey(key) match {
+ case Seq() => // current tx doesn't use this
+ storage.getMapStorageEntryFor(uuid, key).isDefined // check storage
+ case txs => // present in log
+ txs.last.op != REM // last entry cannot be a REM
+ }
} catch { case e: Exception => false }
+ protected def existsInStorage(key: K): Option[V] = try {
+ storage.getMapStorageEntryFor(uuid, key)
+ } catch {
+ case e: Exception => None
+ }
+
override def size: Int = try {
- storage.getMapStorageSizeFor(uuid)
- } catch { case e: Exception => 0 }
+ // partition key set affected in current tx into those which r added & which r deleted
+ val (keysAdded, keysRemoved) = keysInCurrentTx.map {
+ case (kseq, k) => ((kseq, k), getCurrentValue(k))
+ }.partition(_._2.isDefined)
- override def get(key: K): Option[V] = {
- if (newAndUpdatedEntries.contains(key)) {
- newAndUpdatedEntries.get(key)
- }
- else try {
- storage.getMapStorageEntryFor(uuid, key)
- } catch { case e: Exception => None }
+ // keys which existed in storage but removed in current tx
+ val inStorageRemovedInTx =
+ keysRemoved.keySet
+ .map(_._2)
+ .filter(k => existsInStorage(k).isDefined)
+ .size
+
+ // all keys in storage
+ val keysInStorage =
+ storage.getMapStorageFor(uuid)
+ .map { case (k, v) => toEquals(k) }
+ .toSet
+
+ // (keys that existed UNION keys added ) - (keys removed)
+ (keysInStorage union keysAdded.keySet.map(_._1)).size - inStorageRemovedInTx
+ } catch {
+ case e: Exception => 0
}
- def iterator = elements
+ // get must consider underlying storage & current uncommitted tx log
+ override def get(key: K): Option[V] = getCurrentValue(key)
- override def elements: Iterator[Tuple2[K, V]] = {
- new Iterator[Tuple2[K, V]] {
- private val originalList: List[Tuple2[K, V]] = try {
- storage.getMapStorageFor(uuid)
- } catch {
- case e: Throwable => Nil
- }
- private var elements = newAndUpdatedEntries.toList union originalList.reverse
- override def next: Tuple2[K, V]= synchronized {
- val element = elements.head
- elements = elements.tail
- element
- }
- override def hasNext: Boolean = synchronized { !elements.isEmpty }
- }
- }
+ def iterator: Iterator[Tuple2[K, V]]
private def register = {
if (transaction.get.isEmpty) throw new NoTransactionInScopeException
@@ -187,6 +271,95 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
}
}
+trait PersistentMapBinary extends PersistentMap[Array[Byte], Array[Byte]] {
+ import scala.collection.mutable.ArraySeq
+
+ type T = ArraySeq[Byte]
+ def toEquals(k: Array[Byte]) = ArraySeq(k: _*)
+ override def equal(k1: Array[Byte], k2: Array[Byte]): Boolean = k1 sameElements k2
+
+ object COrdering {
+ implicit object ArraySeqOrdering extends Ordering[ArraySeq[Byte]] {
+ def compare(o1: ArraySeq[Byte], o2: ArraySeq[Byte]) =
+ new String(o1.toArray) compare new String(o2.toArray)
+ }
+ }
+
+ import scala.collection.immutable.{TreeMap, SortedMap}
+ private def replayAllKeys: SortedMap[ArraySeq[Byte], Array[Byte]] = {
+ import COrdering._
+
+ // need ArraySeq for ordering
+ val fromStorage =
+ TreeMap(storage.getMapStorageFor(uuid).map { case (k, v) => (ArraySeq(k: _*), v) }: _*)
+
+ val (keysAdded, keysRemoved) = keysInCurrentTx.map {
+ case (_, k) => (k, getCurrentValue(k))
+ }.partition(_._2.isDefined)
+
+ val inStorageRemovedInTx =
+ keysRemoved.keySet
+ .filter(k => existsInStorage(k).isDefined)
+ .map(k => ArraySeq(k: _*))
+
+ (fromStorage -- inStorageRemovedInTx) ++ keysAdded.map { case (k, Some(v)) => (ArraySeq(k: _*), v) }
+ }
+
+ override def slice(start: Option[Array[Byte]], finish: Option[Array[Byte]], count: Int): List[(Array[Byte], Array[Byte])] = try {
+ val newMap = replayAllKeys
+
+ if (newMap isEmpty) List[(Array[Byte], Array[Byte])]()
+
+ val startKey =
+ start match {
+ case Some(bytes) => Some(ArraySeq(bytes: _*))
+ case None => None
+ }
+
+ val endKey =
+ finish match {
+ case Some(bytes) => Some(ArraySeq(bytes: _*))
+ case None => None
+ }
+
+ ((startKey, endKey, count): @unchecked) match {
+ case ((Some(s), Some(e), _)) =>
+ newMap.range(s, e)
+ .toList
+ .map(e => (e._1.toArray, e._2))
+ .toList
+ case ((Some(s), None, c)) if c > 0 =>
+ newMap.from(s)
+ .iterator
+ .take(count)
+ .map(e => (e._1.toArray, e._2))
+ .toList
+ case ((Some(s), None, _)) =>
+ newMap.from(s)
+ .toList
+ .map(e => (e._1.toArray, e._2))
+ .toList
+ case ((None, Some(e), _)) =>
+ newMap.until(e)
+ .toList
+ .map(e => (e._1.toArray, e._2))
+ .toList
+ }
+ } catch { case e: Exception => Nil }
+
+ override def iterator: Iterator[(Array[Byte], Array[Byte])] = {
+ new Iterator[(Array[Byte], Array[Byte])] {
+ private var elements = replayAllKeys
+ override def next: (Array[Byte], Array[Byte]) = synchronized {
+ val (k, v) = elements.head
+ elements = elements.tail
+ (k.toArray, v)
+ }
+ override def hasNext: Boolean = synchronized { !elements.isEmpty }
+ }
+ }
+}
+
/**
* Implements a template for a concrete persistent transactional vector based storage.
*
@@ -198,42 +371,83 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa
protected val removedElems = TransactionalVector[T]()
protected val shouldClearOnCommit = Ref[Boolean]()
+ // operations on the Vector
+ trait Op
+ case object ADD extends Op
+ case object UPD extends Op
+ case object POP extends Op
+
+ // append only log: records all mutating operations
+ protected val appendOnlyTxLog = TransactionalVector[LogEntry]()
+
+ case class LogEntry(index: Option[Int], value: Option[T], op: Op)
+
+ // need to override in subclasses e.g. "sameElements" for Array[Byte]
+ def equal(v1: T, v2: T): Boolean = v1 == v2
+
val storage: VectorStorageBackend[T]
def commit = {
- for (element <- newElems) storage.insertVectorStorageEntryFor(uuid, element)
- for (entry <- updatedElems) storage.updateVectorStorageEntryFor(uuid, entry._1, entry._2)
- newElems.clear
- updatedElems.clear
+ for(entry <- appendOnlyTxLog) {
+ entry match {
+ case LogEntry(_, Some(v), ADD) => storage.insertVectorStorageEntryFor(uuid, v)
+ case LogEntry(Some(i), Some(v), UPD) => storage.updateVectorStorageEntryFor(uuid, i, v)
+ case LogEntry(_, _, POP) => //..
+ }
+ }
+ appendOnlyTxLog.clear
}
def abort = {
- newElems.clear
- updatedElems.clear
- removedElems.clear
+ appendOnlyTxLog.clear
shouldClearOnCommit.swap(false)
}
+ private def replay: List[T] = {
+ import scala.collection.mutable.ArrayBuffer
+ var elemsStorage = ArrayBuffer(storage.getVectorStorageRangeFor(uuid, None, None, storage.getVectorStorageSizeFor(uuid)).reverse: _*)
+
+ for(entry <- appendOnlyTxLog) {
+ entry match {
+ case LogEntry(_, Some(v), ADD) => elemsStorage += v
+ case LogEntry(Some(i), Some(v), UPD) => elemsStorage.update(i, v)
+ case LogEntry(_, _, POP) => elemsStorage = elemsStorage.drop(1)
+ }
+ }
+ elemsStorage.toList.reverse
+ }
+
def +(elem: T) = add(elem)
def add(elem: T) = {
register
- newElems + elem
+ appendOnlyTxLog + LogEntry(None, Some(elem), ADD)
}
def apply(index: Int): T = get(index)
def get(index: Int): T = {
- if (newElems.size > index) newElems(index)
- else storage.getVectorStorageEntryFor(uuid, index)
+ if (appendOnlyTxLog.isEmpty) {
+ storage.getVectorStorageEntryFor(uuid, index)
+ } else {
+ val curr = replay
+ curr(index)
+ }
}
override def slice(start: Int, finish: Int): IndexedSeq[T] = slice(Some(start), Some(finish))
def slice(start: Option[Int], finish: Option[Int], count: Int = 0): IndexedSeq[T] = {
- val buffer = new scala.collection.mutable.ArrayBuffer[T]
- storage.getVectorStorageRangeFor(uuid, start, finish, count).foreach(buffer.append(_))
- buffer
+ val curr = replay
+ val s = if (start.isDefined) start.get else 0
+ val cnt =
+ if (finish.isDefined) {
+ val f = finish.get
+ if (f >= s) (f - s) else count
+ }
+ else count
+ if (s == 0 && cnt == 0) List().toIndexedSeq
+ else curr.slice(s, s + cnt).toIndexedSeq
}
/**
@@ -241,12 +455,13 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa
*/
def pop: T = {
register
+ appendOnlyTxLog + LogEntry(None, None, POP)
throw new UnsupportedOperationException("PersistentVector::pop is not implemented")
}
def update(index: Int, newElem: T) = {
register
- storage.updateVectorStorageEntryFor(uuid, index, newElem)
+ appendOnlyTxLog + LogEntry(Some(index), Some(newElem), UPD)
}
override def first: T = get(0)
@@ -260,7 +475,7 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa
}
}
- def length: Int = storage.getVectorStorageSizeFor(uuid) + newElems.length
+ def length: Int = replay.length
private def register = {
if (transaction.get.isEmpty) throw new NoTransactionInScopeException
diff --git a/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala b/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala
index df74040b68..7e6a95f9a1 100644
--- a/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala
+++ b/akka-persistence/akka-persistence-common/src/main/scala/StorageBackend.scala
@@ -38,7 +38,7 @@ trait RefStorageBackend[T] extends StorageBackend {
// for Queue
trait QueueStorageBackend[T] extends StorageBackend {
// add to the end of the queue
- def enqueue(name: String, item: T): Boolean
+ def enqueue(name: String, item: T): Option[Int]
// pop from the front of the queue
def dequeue(name: String): Option[T]
diff --git a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala
index 98776253a5..83e47e3ba5 100644
--- a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala
+++ b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorage.scala
@@ -9,7 +9,7 @@ import se.scalablesolutions.akka.persistence.common._
import se.scalablesolutions.akka.util.UUID
object MongoStorage extends Storage {
- type ElementType = AnyRef
+ type ElementType = Array[Byte]
def newMap: PersistentMap[ElementType, ElementType] = newMap(UUID.newUuid.toString)
def newVector: PersistentVector[ElementType] = newVector(UUID.newUuid.toString)
@@ -29,7 +29,7 @@ object MongoStorage extends Storage {
*
* @author Debasish Ghosh
*/
-class MongoPersistentMap(id: String) extends PersistentMap[AnyRef, AnyRef] {
+class MongoPersistentMap(id: String) extends PersistentMapBinary {
val uuid = id
val storage = MongoStorageBackend
}
@@ -40,12 +40,12 @@ class MongoPersistentMap(id: String) extends PersistentMap[AnyRef, AnyRef] {
*
* @author Debaissh Ghosh
*/
-class MongoPersistentVector(id: String) extends PersistentVector[AnyRef] {
+class MongoPersistentVector(id: String) extends PersistentVector[Array[Byte]] {
val uuid = id
val storage = MongoStorageBackend
}
-class MongoPersistentRef(id: String) extends PersistentRef[AnyRef] {
+class MongoPersistentRef(id: String) extends PersistentRef[Array[Byte]] {
val uuid = id
val storage = MongoStorageBackend
}
diff --git a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala
index 950165567d..01d8ababce 100644
--- a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala
+++ b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala
@@ -9,13 +9,8 @@ import se.scalablesolutions.akka.persistence.common._
import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.config.Config.config
-import sjson.json.Serializer._
-
import java.util.NoSuchElementException
-
-import com.mongodb._
-
-import java.util.{Map=>JMap, List=>JList, ArrayList=>JArrayList}
+import com.novus.casbah.mongodb.Imports._
/**
* A module for supporting MongoDB based persistence.
@@ -28,294 +23,208 @@ import java.util.{Map=>JMap, List=>JList, ArrayList=>JArrayList}
* @author Debasish Ghosh
*/
private[akka] object MongoStorageBackend extends
- MapStorageBackend[AnyRef, AnyRef] with
- VectorStorageBackend[AnyRef] with
- RefStorageBackend[AnyRef] with
+ MapStorageBackend[Array[Byte], Array[Byte]] with
+ VectorStorageBackend[Array[Byte]] with
+ RefStorageBackend[Array[Byte]] with
Logging {
- // enrich with null safe findOne
- class RichDBCollection(value: DBCollection) {
- def findOneNS(o: DBObject): Option[DBObject] = {
- value.findOne(o) match {
- case null => None
- case x => Some(x)
- }
- }
- }
-
- implicit def enrichDBCollection(c: DBCollection) = new RichDBCollection(c)
-
- val KEY = "key"
- val VALUE = "value"
+ val KEY = "__key"
+ val REF = "__ref"
val COLLECTION = "akka_coll"
- val MONGODB_SERVER_HOSTNAME = config.getString("akka.storage.mongodb.hostname", "127.0.0.1")
- val MONGODB_SERVER_DBNAME = config.getString("akka.storage.mongodb.dbname", "testdb")
- val MONGODB_SERVER_PORT = config.getInt("akka.storage.mongodb.port", 27017)
+ val HOSTNAME = config.getString("akka.storage.mongodb.hostname", "127.0.0.1")
+ val DBNAME = config.getString("akka.storage.mongodb.dbname", "testdb")
+ val PORT = config.getInt("akka.storage.mongodb.port", 27017)
- val db = new Mongo(MONGODB_SERVER_HOSTNAME, MONGODB_SERVER_PORT)
- val coll = db.getDB(MONGODB_SERVER_DBNAME).getCollection(COLLECTION)
+ val db: MongoDB = MongoConnection(HOSTNAME, PORT)(DBNAME)
+ val coll: MongoCollection = db(COLLECTION)
- private[this] val serializer = SJSON
+ def drop() { db.dropDatabase() }
- def insertMapStorageEntryFor(name: String, key: AnyRef, value: AnyRef) {
+ def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]) {
insertMapStorageEntriesFor(name, List((key, value)))
}
- def insertMapStorageEntriesFor(name: String, entries: List[Tuple2[AnyRef, AnyRef]]) {
- import java.util.{Map, HashMap}
-
- val m: Map[AnyRef, AnyRef] = new HashMap
- for ((k, v) <- entries) {
- m.put(k, serializer.out(v))
- }
-
- nullSafeFindOne(name) match {
- case None =>
- coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, m))
- case Some(dbo) => {
- // collate the maps
- val o = dbo.get(VALUE).asInstanceOf[Map[AnyRef, AnyRef]]
- o.putAll(m)
-
- val newdbo = new BasicDBObject().append(KEY, name).append(VALUE, o)
- coll.update(new BasicDBObject().append(KEY, name), newdbo, true, false)
+ def insertMapStorageEntriesFor(name: String, entries: List[(Array[Byte], Array[Byte])]) {
+ db.safely { db =>
+ val q: DBObject = MongoDBObject(KEY -> name)
+ coll.findOne(q) match {
+ case Some(dbo) =>
+ entries.foreach { case (k, v) => dbo += new String(k) -> v }
+ db.safely { db => coll.update(q, dbo, true, false) }
+ case None =>
+ val builder = MongoDBObject.newBuilder
+ builder += KEY -> name
+ entries.foreach { case (k, v) => builder += new String(k) -> v }
+ coll += builder.result.asDBObject
}
}
}
def removeMapStorageFor(name: String): Unit = {
- val q = new BasicDBObject
- q.put(KEY, name)
- coll.remove(q)
+ val q: DBObject = MongoDBObject(KEY -> name)
+ db.safely { db => coll.remove(q) }
}
- def removeMapStorageFor(name: String, key: AnyRef): Unit = {
- nullSafeFindOne(name) match {
- case None =>
- case Some(dbo) => {
- val orig = dbo.get(VALUE).asInstanceOf[DBObject].toMap
- 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)
- // and insert
- coll.insert(new BasicDBObject().append(KEY, name).append(VALUE, orig))
- }
+ private def queryFor[T](name: String)(body: (MongoDBObject, Option[DBObject]) => T): T = {
+ val q = MongoDBObject(KEY -> name)
+ body(q, coll.findOne(q))
+ }
+
+ def removeMapStorageFor(name: String, key: Array[Byte]): Unit = queryFor(name) { (q, dbo) =>
+ dbo.foreach { d =>
+ d -= new String(key)
+ db.safely { db => coll.update(q, d, true, false) }
}
}
- def getMapStorageEntryFor(name: String, key: AnyRef): Option[AnyRef] =
- getValueForKey(name, key.asInstanceOf[String])
-
- def getMapStorageSizeFor(name: String): Int = {
- nullSafeFindOne(name) match {
- case None => 0
- case Some(dbo) =>
- dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]].keySet.size
- }
+ def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = queryFor(name) { (q, dbo) =>
+ dbo.map { d =>
+ d.getAs[Array[Byte]](new String(key))
+ }.getOrElse(None)
}
- def getMapStorageFor(name: String): List[Tuple2[AnyRef, AnyRef]] = {
- val m =
- nullSafeFindOne(name) match {
- case None =>
- throw new NoSuchElementException(name + " not present")
- case Some(dbo) =>
- dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
- }
- val n =
- List(m.keySet.toArray: _*).asInstanceOf[List[String]]
- val vals =
- for(s <- n)
- yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]]))
- vals.asInstanceOf[List[Tuple2[String, AnyRef]]]
+ def getMapStorageSizeFor(name: String): Int = queryFor(name) { (q, dbo) =>
+ dbo.map { d =>
+ d.size - 2 // need to exclude object id and our KEY
+ }.getOrElse(0)
}
- def getMapStorageRangeFor(name: String, start: Option[AnyRef],
- finish: Option[AnyRef],
- count: Int): List[Tuple2[AnyRef, AnyRef]] = {
- val m =
- nullSafeFindOne(name) match {
- case None =>
- throw new NoSuchElementException(name + " not present")
- case Some(dbo) =>
- dbo.get(VALUE).asInstanceOf[JMap[String, AnyRef]]
- }
-
- /**
- * count is the max number of results to return. Start with
- * start or 0 (if start is not defined) and go until
- * you hit finish or count.
- */
- val s = if (start.isDefined) start.get.asInstanceOf[Int] else 0
- val cnt =
- if (finish.isDefined) {
- val f = finish.get.asInstanceOf[Int]
- if (f >= s) math.min(count, (f - s)) else count
- }
- else count
-
- val n =
- List(m.keySet.toArray: _*).asInstanceOf[List[String]].sortWith((e1, e2) => (e1 compareTo e2) < 0).slice(s, s + cnt)
- val vals =
- for(s <- n)
- yield (s, serializer.in[AnyRef](m.get(s).asInstanceOf[Array[Byte]]))
- vals.asInstanceOf[List[Tuple2[String, AnyRef]]]
+ def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = queryFor(name) { (q, dbo) =>
+ dbo.map { d =>
+ for {
+ (k, v) <- d.toList
+ if k != "_id" && k != KEY
+ } yield (k.getBytes, v.asInstanceOf[Array[Byte]])
+ }.getOrElse(List.empty[(Array[Byte], Array[Byte])])
}
- private def getValueForKey(name: String, key: String): Option[AnyRef] = {
- try {
- nullSafeFindOne(name) match {
- case None => None
- case Some(dbo) =>
- Some(serializer.in[AnyRef](
- dbo.get(VALUE)
- .asInstanceOf[JMap[String, AnyRef]]
- .get(key).asInstanceOf[Array[Byte]]))
- }
- } catch {
- case e =>
- throw new NoSuchElementException(e.getMessage)
- }
+ def getMapStorageRangeFor(name: String, start: Option[Array[Byte]],
+ finish: Option[Array[Byte]],
+ count: Int): List[(Array[Byte], Array[Byte])] = queryFor(name) { (q, dbo) =>
+ dbo.map { d =>
+ // get all keys except the special ones
+ val keys = d.keys
+ .filter(k => k != "_id" && k != KEY)
+ .toList
+ .sortWith(_ < _)
+
+ // if the supplied start is not defined, get the head of keys
+ val s = start.map(new String(_)).getOrElse(keys.head)
+
+ // if the supplied finish is not defined, get the last element of keys
+ val f = finish.map(new String(_)).getOrElse(keys.last)
+
+ // slice from keys: both ends inclusive
+ val ks = keys.slice(keys.indexOf(s), scala.math.min(count, keys.indexOf(f) + 1))
+ ks.map(k => (k.getBytes, d.getAs[Array[Byte]](k).get))
+ }.getOrElse(List.empty[(Array[Byte], Array[Byte])])
}
- def insertVectorStorageEntriesFor(name: String, elements: List[AnyRef]) = {
- val q = new BasicDBObject
- q.put(KEY, name)
-
- val currentList =
- coll.findOneNS(q) match {
- case None =>
- new JArrayList[AnyRef]
- case Some(dbo) =>
- dbo.get(VALUE).asInstanceOf[JArrayList[AnyRef]]
- }
- if (!currentList.isEmpty) {
- // record exists
- // remove before adding
- coll.remove(q)
- }
-
- // add to the current list
- elements.map(serializer.out(_)).foreach(currentList.add(_))
-
- coll.insert(
- new BasicDBObject()
- .append(KEY, name)
- .append(VALUE, currentList)
- )
- }
-
- def insertVectorStorageEntryFor(name: String, element: AnyRef) = {
+ def insertVectorStorageEntryFor(name: String, element: Array[Byte]) = {
insertVectorStorageEntriesFor(name, List(element))
}
- def getVectorStorageEntryFor(name: String, index: Int): AnyRef = {
- try {
- val o =
- nullSafeFindOne(name) match {
- case None =>
- throw new NoSuchElementException(name + " not present")
+ def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]) = {
+ // lookup with name
+ val q: DBObject = MongoDBObject(KEY -> name)
- case Some(dbo) =>
- dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
+ db.safely { db =>
+ coll.findOne(q) match {
+ // exists : need to update
+ case Some(dbo) =>
+ dbo -= KEY
+ dbo -= "_id"
+ val listBuilder = MongoDBList.newBuilder
+
+ // expensive!
+ listBuilder ++= (elements ++ dbo.toSeq.sortWith((e1, e2) => (e1._1.toInt < e2._1.toInt)).map(_._2))
+
+ val builder = MongoDBObject.newBuilder
+ builder += KEY -> name
+ builder ++= listBuilder.result
+ coll.update(q, builder.result.asDBObject, true, false)
+
+ // new : just add
+ case None =>
+ val listBuilder = MongoDBList.newBuilder
+ listBuilder ++= elements
+
+ val builder = MongoDBObject.newBuilder
+ builder += KEY -> name
+ builder ++= listBuilder.result
+ coll += builder.result.asDBObject
}
- serializer.in[AnyRef](
- o.get(index).asInstanceOf[Array[Byte]])
- } catch {
- case e =>
- throw new NoSuchElementException(e.getMessage)
}
}
- def getVectorStorageRangeFor(name: String,
- start: Option[Int], finish: Option[Int], count: Int): List[AnyRef] = {
- try {
- val o =
- nullSafeFindOne(name) match {
- case None =>
- throw new NoSuchElementException(name + " not present")
+ def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]) = queryFor(name) { (q, dbo) =>
+ dbo.foreach { d =>
+ d += ((index.toString, elem))
+ db.safely { db => coll.update(q, d, true, false) }
+ }
+ }
- case Some(dbo) =>
- dbo.get(VALUE).asInstanceOf[JList[AnyRef]]
- }
+ def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = queryFor(name) { (q, dbo) =>
+ dbo.map { d =>
+ d(index.toString).asInstanceOf[Array[Byte]]
+ }.getOrElse(Array.empty[Byte])
+ }
- val s = if (start.isDefined) start.get else 0
+ /**
+ * if start and finish both are defined, ignore count and
+ * report the range [start, finish)
+ * if start is not defined, assume start = 0
+ * if start == 0 and finish == 0, return an empty collection
+ */
+ def getVectorStorageRangeFor(name: String, start: Option[Int], finish: Option[Int], count: Int): List[Array[Byte]] = queryFor(name) { (q, dbo) =>
+ dbo.map { d =>
+ val ls = d.filter { case (k, v) => k != KEY && k != "_id" }
+ .toSeq
+ .sortWith((e1, e2) => (e1._1.toInt < e2._1.toInt))
+ .map(_._2)
+
+ val st = start.getOrElse(0)
val cnt =
if (finish.isDefined) {
val f = finish.get
- if (f >= s) (f - s) else count
+ if (f >= st) (f - st) else count
}
else count
-
- // pick the subrange and make a Scala list
- val l =
- List(o.subList(s, s + cnt).toArray: _*)
-
- for(e <- l)
- yield serializer.in[AnyRef](e.asInstanceOf[Array[Byte]])
- } catch {
- case e =>
- throw new NoSuchElementException(e.getMessage)
- }
+ if (st == 0 && cnt == 0) List()
+ ls.slice(st, st + cnt).asInstanceOf[List[Array[Byte]]]
+ }.getOrElse(List.empty[Array[Byte]])
}
- def updateVectorStorageEntryFor(name: String, index: Int, elem: AnyRef) = {
- val q = new BasicDBObject
- q.put(KEY, name)
-
- val dbobj =
- coll.findOneNS(q) match {
- case None =>
- throw new NoSuchElementException(name + " not present")
- case Some(dbo) => dbo
- }
- val currentList = dbobj.get(VALUE).asInstanceOf[JArrayList[AnyRef]]
- currentList.set(index, serializer.out(elem))
- coll.update(q,
- new BasicDBObject().append(KEY, name).append(VALUE, currentList))
+ def getVectorStorageSizeFor(name: String): Int = queryFor(name) { (q, dbo) =>
+ dbo.map { d => d.size - 2 }.getOrElse(0)
}
- def getVectorStorageSizeFor(name: String): Int = {
- nullSafeFindOne(name) match {
- case None => 0
- case Some(dbo) =>
- dbo.get(VALUE).asInstanceOf[JList[AnyRef]].size
- }
- }
+ def insertRefStorageFor(name: String, element: Array[Byte]) = {
+ // lookup with name
+ val q: DBObject = MongoDBObject(KEY -> name)
- private def nullSafeFindOne(name: String): Option[DBObject] = {
- val o = new BasicDBObject
- o.put(KEY, name)
- coll.findOneNS(o)
- }
+ db.safely { db =>
+ coll.findOne(q) match {
+ // exists : need to update
+ case Some(dbo) =>
+ dbo += ((REF, element))
+ coll.update(q, dbo, true, false)
- def insertRefStorageFor(name: String, element: AnyRef) = {
- nullSafeFindOne(name) match {
- case None =>
- case Some(dbo) => {
- val q = new BasicDBObject
- q.put(KEY, name)
- coll.remove(q)
+ // not found : make one
+ case None =>
+ val builder = MongoDBObject.newBuilder
+ builder += KEY -> name
+ builder += REF -> element
+ coll += builder.result.asDBObject
}
}
- coll.insert(
- new BasicDBObject()
- .append(KEY, name)
- .append(VALUE, serializer.out(element)))
}
- def getRefStorageFor(name: String): Option[AnyRef] = {
- nullSafeFindOne(name) match {
- case None => None
- case Some(dbo) =>
- Some(serializer.in[AnyRef](dbo.get(VALUE).asInstanceOf[Array[Byte]]))
- }
+ def getRefStorageFor(name: String): Option[Array[Byte]] = queryFor(name) { (q, dbo) =>
+ dbo.map { d =>
+ d.getAs[Array[Byte]](REF)
+ }.getOrElse(None)
}
}
diff --git a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala
index 1acc9ee97d..01f735b254 100644
--- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala
+++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoPersistentActorSpec.scala
@@ -1,32 +1,19 @@
package se.scalablesolutions.akka.persistence.mongo
-import org.junit.{Test, Before}
-import org.junit.Assert._
-import org.scalatest.junit.JUnitSuite
-
-import _root_.dispatch.json.{JsNumber, JsValue}
-import _root_.dispatch.json.Js._
+import org.scalatest.Spec
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
import se.scalablesolutions.akka.actor.{Transactor, Actor, ActorRef}
import Actor._
-/**
- * A persistent actor based on MongoDB storage.
- *
- * Demonstrates a bank account operation consisting of messages that:
- * checks balance Balance
- * debits amountDebit
- * debits multiple amountsMultiDebit
- * credits amountCredit
- *
- * Needs a running Mongo server.
- * @author Debasish Ghosh
- */
case class Balance(accountNo: String)
-case class Debit(accountNo: String, amount: BigInt, failer: ActorRef)
-case class MultiDebit(accountNo: String, amounts: List[BigInt], failer: ActorRef)
-case class Credit(accountNo: String, amount: BigInt)
+case class Debit(accountNo: String, amount: Int, failer: ActorRef)
+case class MultiDebit(accountNo: String, amounts: List[Int], failer: ActorRef)
+case class Credit(accountNo: String, amount: Int)
case class Log(start: Int, finish: Int)
case object LogSize
@@ -35,63 +22,65 @@ class BankAccountActor extends Transactor {
private lazy val accountState = MongoStorage.newMap
private lazy val txnLog = MongoStorage.newVector
+ import sjson.json.DefaultProtocol._
+ import sjson.json.JsonSerialization._
+
def receive: Receive = {
// check balance
case Balance(accountNo) =>
- txnLog.add("Balance:" + accountNo)
- self.reply(accountState.get(accountNo).get)
+ txnLog.add(("Balance:" + accountNo).getBytes)
+ self.reply(
+ accountState.get(accountNo.getBytes)
+ .map(frombinary[Int](_))
+ .getOrElse(0))
// debit amount: can fail
case Debit(accountNo, amount, failer) =>
- txnLog.add("Debit:" + accountNo + " " + amount)
+ txnLog.add(("Debit:" + accountNo + " " + amount).getBytes)
+ val m = accountState.get(accountNo.getBytes)
+ .map(frombinary[Int](_))
+ .getOrElse(0)
+
+ accountState.put(accountNo.getBytes, tobinary(m - amount))
+ if (amount > m) failer !! "Failure"
- val m: BigInt =
- accountState.get(accountNo) match {
- case Some(JsNumber(n)) =>
- BigInt(n.asInstanceOf[BigDecimal].intValue)
- case None => 0
- }
- accountState.put(accountNo, (m - amount))
- if (amount > m)
- failer !! "Failure"
self.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 sum = amounts.foldRight(0)(_ + _)
+ txnLog.add(("MultiDebit:" + accountNo + " " + sum).getBytes)
- val m: BigInt =
- accountState.get(accountNo) match {
- case Some(JsNumber(n)) => BigInt(n.toString)
- case None => 0
+ val m = accountState.get(accountNo.getBytes)
+ .map(frombinary[Int](_))
+ .getOrElse(0)
+
+ var cbal = m
+ amounts.foreach { amount =>
+ accountState.put(accountNo.getBytes, tobinary(m - amount))
+ cbal = cbal - amount
+ if (cbal < 0) failer !! "Failure"
}
- var bal: BigInt = 0
- amounts.foreach {amount =>
- bal = bal + amount
- accountState.put(accountNo, (m - bal))
- }
- if (bal > m) failer !! "Failure"
- self.reply(m - bal)
+
+ self.reply(m - sum)
// credit amount
case Credit(accountNo, amount) =>
- txnLog.add("Credit:" + accountNo + " " + amount)
+ txnLog.add(("Credit:" + accountNo + " " + amount).getBytes)
+ val m = accountState.get(accountNo.getBytes)
+ .map(frombinary[Int](_))
+ .getOrElse(0)
+
+ accountState.put(accountNo.getBytes, tobinary(m + amount))
- val m: BigInt =
- accountState.get(accountNo) match {
- case Some(JsNumber(n)) =>
- BigInt(n.asInstanceOf[BigDecimal].intValue)
- case None => 0
- }
- accountState.put(accountNo, (m + amount))
self.reply(m + amount)
case LogSize =>
- self.reply(txnLog.length.asInstanceOf[AnyRef])
+ self.reply(txnLog.length)
case Log(start, finish) =>
- self.reply(txnLog.slice(start, finish))
+ self.reply(txnLog.slice(start, finish).map(new String(_)))
}
}
@@ -102,82 +91,71 @@ class BankAccountActor extends Transactor {
}
}
-class MongoPersistentActorSpec extends JUnitSuite {
- @Test
- def testSuccessfulDebit = {
- val bactor = actorOf[BankAccountActor]
- bactor.start
- val failer = actorOf[PersistentFailerActor]
- failer.start
- bactor !! Credit("a-123", 5000)
- bactor !! Debit("a-123", 3000, failer)
+@RunWith(classOf[JUnitRunner])
+class MongoPersistentActorSpec extends
+ Spec with
+ ShouldMatchers with
+ BeforeAndAfterEach {
- val JsNumber(b) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- assertEquals(BigInt(2000), BigInt(b.intValue))
-
- bactor !! Credit("a-123", 7000)
-
- val JsNumber(b1) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- assertEquals(BigInt(9000), BigInt(b1.intValue))
-
- bactor !! Debit("a-123", 8000, failer)
-
- val JsNumber(b2) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- assertEquals(BigInt(1000), BigInt(b2.intValue))
-
- assert(7 == (bactor !! LogSize).get.asInstanceOf[Int])
-
- import scala.collection.mutable.ArrayBuffer
- assert((bactor !! Log(0, 7)).get.asInstanceOf[ArrayBuffer[String]].size == 7)
- assert((bactor !! Log(0, 0)).get.asInstanceOf[ArrayBuffer[String]].size == 0)
- assert((bactor !! Log(1, 2)).get.asInstanceOf[ArrayBuffer[String]].size == 1)
- assert((bactor !! Log(6, 7)).get.asInstanceOf[ArrayBuffer[String]].size == 1)
- assert((bactor !! Log(0, 1)).get.asInstanceOf[ArrayBuffer[String]].size == 1)
+ override def beforeEach {
+ MongoStorageBackend.drop
}
- @Test
- def testUnsuccessfulDebit = {
- val bactor = actorOf[BankAccountActor]
- bactor.start
- bactor !! Credit("a-123", 5000)
-
- val JsNumber(b) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- assertEquals(BigInt(5000), BigInt(b.intValue))
-
- val failer = actorOf[PersistentFailerActor]
- failer.start
- try {
- bactor !! Debit("a-123", 7000, failer)
- fail("should throw exception")
- } catch { case e: RuntimeException => {}}
-
- val JsNumber(b1) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- assertEquals(BigInt(5000), BigInt(b1.intValue))
-
- // should not count the failed one
- assert(3 == (bactor !! LogSize).get.asInstanceOf[Int])
+ override def afterEach {
+ MongoStorageBackend.drop
}
- @Test
- def testUnsuccessfulMultiDebit = {
- val bactor = actorOf[BankAccountActor]
- bactor.start
- bactor !! Credit("a-123", 5000)
+ describe("successful debit") {
+ it("should debit successfully") {
+ val bactor = actorOf[BankAccountActor]
+ bactor.start
+ val failer = actorOf[PersistentFailerActor]
+ failer.start
+ bactor !! Credit("a-123", 5000)
+ bactor !! Debit("a-123", 3000, failer)
- val JsNumber(b) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- assertEquals(BigInt(5000), BigInt(b.intValue))
+ (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(2000)
- val failer = actorOf[PersistentFailerActor]
- failer.start
- try {
- bactor !! MultiDebit("a-123", List(500, 2000, 1000, 3000), failer)
- fail("should throw exception")
- } catch { case e: RuntimeException => {}}
+ bactor !! Credit("a-123", 7000)
+ (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(9000)
- val JsNumber(b1) = (bactor !! Balance("a-123")).get.asInstanceOf[JsValue]
- assertEquals(BigInt(5000), BigInt(b1.intValue))
+ bactor !! Debit("a-123", 8000, failer)
+ (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(1000)
- // should not count the failed one
- assert(3 == (bactor !! LogSize).get.asInstanceOf[Int])
+ (bactor !! LogSize).get.asInstanceOf[Int] should equal(7)
+ (bactor !! Log(0, 7)).get.asInstanceOf[Iterable[String]].size should equal(7)
+ }
+ }
+
+ describe("unsuccessful debit") {
+ it("debit should fail") {
+ val bactor = actorOf[BankAccountActor]
+ bactor.start
+ val failer = actorOf[PersistentFailerActor]
+ failer.start
+ bactor !! Credit("a-123", 5000)
+ (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000)
+ evaluating {
+ bactor !! Debit("a-123", 7000, failer)
+ } should produce [Exception]
+ (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000)
+ (bactor !! LogSize).get.asInstanceOf[Int] should equal(3)
+ }
+ }
+
+ describe("unsuccessful multidebit") {
+ it("multidebit should fail") {
+ val bactor = actorOf[BankAccountActor]
+ bactor.start
+ val failer = actorOf[PersistentFailerActor]
+ failer.start
+ bactor !! Credit("a-123", 5000)
+ (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000)
+ evaluating {
+ bactor !! MultiDebit("a-123", List(1000, 2000, 4000), failer)
+ } should produce [Exception]
+ (bactor !! Balance("a-123")).get.asInstanceOf[Int] should equal(5000)
+ (bactor !! LogSize).get.asInstanceOf[Int] should equal(3)
+ }
}
}
diff --git a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala
index e518b28d66..e9576cc152 100644
--- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala
+++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoStorageSpec.scala
@@ -1,364 +1,158 @@
package se.scalablesolutions.akka.persistence.mongo
-import org.junit.{Test, Before}
-import org.junit.Assert._
-import org.scalatest.junit.JUnitSuite
-import _root_.dispatch.json._
-import _root_.dispatch.json.Js._
+import org.scalatest.Spec
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
import java.util.NoSuchElementException
-@scala.reflect.BeanInfo case class Foo(no: Int, name: String)
-class MongoStorageSpec extends JUnitSuite {
+@RunWith(classOf[JUnitRunner])
+class MongoStorageSpec extends
+ Spec with
+ ShouldMatchers with
+ BeforeAndAfterEach {
- val changeSetV = new scala.collection.mutable.ArrayBuffer[AnyRef]
- val changeSetM = new scala.collection.mutable.HashMap[AnyRef, AnyRef]
-
- @Before def initialize() = {
- MongoStorageBackend.coll.drop
+ override def beforeEach {
+ MongoStorageBackend.drop
}
- @Test
- def testVectorInsertForTransactionId = {
- changeSetV += "debasish" // string
- changeSetV += List(1, 2, 3) // Scala List
- changeSetV += List(100, 200)
- MongoStorageBackend.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
- assertEquals(
- 3,
- MongoStorageBackend.getVectorStorageSizeFor("U-A1"))
- changeSetV.clear
-
- // changeSetV should be reinitialized
- changeSetV += List(12, 23, 45)
- changeSetV += "maulindu"
- MongoStorageBackend.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
- assertEquals(
- 5,
- MongoStorageBackend.getVectorStorageSizeFor("U-A1"))
-
- // add more to the same changeSetV
- changeSetV += "ramanendu"
- changeSetV += Map(1 -> "dg", 2 -> "mc")
-
- // add for a diff transaction
- MongoStorageBackend.insertVectorStorageEntriesFor("U-A2", changeSetV.toList)
- assertEquals(
- 4,
- MongoStorageBackend.getVectorStorageSizeFor("U-A2"))
-
- // previous transaction change set should remain same
- assertEquals(
- 5,
- MongoStorageBackend.getVectorStorageSizeFor("U-A1"))
-
- // test single element entry
- MongoStorageBackend.insertVectorStorageEntryFor("U-A1", Map(1->1, 2->4, 3->9))
- assertEquals(
- 6,
- MongoStorageBackend.getVectorStorageSizeFor("U-A1"))
+ override def afterEach {
+ MongoStorageBackend.drop
}
- @Test
- def testVectorFetchForKeys = {
+ describe("persistent maps") {
+ it("should insert with single key and value") {
+ import MongoStorageBackend._
- // initially everything 0
- assertEquals(
- 0,
- MongoStorageBackend.getVectorStorageSizeFor("U-A2"))
-
- assertEquals(
- 0,
- MongoStorageBackend.getVectorStorageSizeFor("U-A1"))
-
- // get some stuff
- changeSetV += "debasish"
- changeSetV += List(BigDecimal(12), BigDecimal(13), BigDecimal(14))
- MongoStorageBackend.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
-
- assertEquals(
- 2,
- MongoStorageBackend.getVectorStorageSizeFor("U-A1"))
-
- val JsString(str) = MongoStorageBackend.getVectorStorageEntryFor("U-A1", 0).asInstanceOf[JsString]
- assertEquals("debasish", str)
-
- val l = MongoStorageBackend.getVectorStorageEntryFor("U-A1", 1).asInstanceOf[JsValue]
- val num_list = list ! num
- val num_list(l0) = l
- assertEquals(List(12, 13, 14), l0)
-
- changeSetV.clear
- changeSetV += Map(1->1, 2->4, 3->9)
- changeSetV += BigInt(2310)
- changeSetV += List(100, 200, 300)
- MongoStorageBackend.insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
-
- assertEquals(
- 5,
- MongoStorageBackend.getVectorStorageSizeFor("U-A1"))
-
- val r =
- MongoStorageBackend.getVectorStorageRangeFor("U-A1", Some(1), None, 3)
-
- assertEquals(3, r.size)
- val lr = r(0).asInstanceOf[JsValue]
- val num_list(l1) = lr
- assertEquals(List(12, 13, 14), l1)
- }
-
- @Test
- def testVectorFetchForNonExistentKeys = {
- try {
- MongoStorageBackend.getVectorStorageEntryFor("U-A1", 1)
- fail("should throw an exception")
- } catch {case e: NoSuchElementException => {}}
-
- try {
- MongoStorageBackend.getVectorStorageRangeFor("U-A1", Some(2), None, 12)
- fail("should throw an exception")
- } catch {case e: NoSuchElementException => {}}
- }
-
- @Test
- def testVectorUpdateForTransactionId = {
- import MongoStorageBackend._
-
- changeSetV += "debasish" // string
- changeSetV += List(1, 2, 3) // Scala List
- changeSetV += List(100, 200)
-
- insertVectorStorageEntriesFor("U-A1", changeSetV.toList)
- assertEquals(3, getVectorStorageSizeFor("U-A1"))
- updateVectorStorageEntryFor("U-A1", 0, "maulindu")
- val JsString(str) = getVectorStorageEntryFor("U-A1", 0).asInstanceOf[JsString]
- assertEquals("maulindu", str)
-
- updateVectorStorageEntryFor("U-A1", 1, Map("1"->"dg", "2"->"mc"))
- val JsObject(m) = getVectorStorageEntryFor("U-A1", 1).asInstanceOf[JsObject]
- assertEquals(m.keySet.size, 2)
- }
-
- @Test
- def testMapInsertForTransactionId = {
- fillMap
-
- // add some more to changeSet
- changeSetM += "5" -> Foo(12, "dg")
- changeSetM += "6" -> java.util.Calendar.getInstance.getTime
-
- // insert all into Mongo
- MongoStorageBackend.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
- assertEquals(
- 6,
- MongoStorageBackend.getMapStorageSizeFor("U-M1"))
-
- // individual insert api
- MongoStorageBackend.insertMapStorageEntryFor("U-M1", "7", "akka")
- MongoStorageBackend.insertMapStorageEntryFor("U-M1", "8", List(23, 25))
- assertEquals(
- 8,
- MongoStorageBackend.getMapStorageSizeFor("U-M1"))
-
- // add the same changeSet for another transaction
- MongoStorageBackend.insertMapStorageEntriesFor("U-M2", changeSetM.toList)
- assertEquals(
- 6,
- MongoStorageBackend.getMapStorageSizeFor("U-M2"))
-
- // the first transaction should remain the same
- assertEquals(
- 8,
- MongoStorageBackend.getMapStorageSizeFor("U-M1"))
- changeSetM.clear
- }
-
- @Test
- def testMapContents = {
- fillMap
- MongoStorageBackend.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
- MongoStorageBackend.getMapStorageEntryFor("U-M1", "2") match {
- case Some(x) => {
- val JsString(str) = x.asInstanceOf[JsValue]
- assertEquals("peter", str)
- }
- case None => fail("should fetch peter")
- }
- MongoStorageBackend.getMapStorageEntryFor("U-M1", "4") match {
- case Some(x) => {
- val num_list = list ! num
- val num_list(l0) = x.asInstanceOf[JsValue]
- assertEquals(3, l0.size)
- }
- case None => fail("should fetch list")
- }
- MongoStorageBackend.getMapStorageEntryFor("U-M1", "3") match {
- case Some(x) => {
- val num_list = list ! num
- val num_list(l0) = x.asInstanceOf[JsValue]
- assertEquals(2, l0.size)
- }
- case None => fail("should fetch list")
+ insertMapStorageEntryFor("t1", "odersky".getBytes, "scala".getBytes)
+ insertMapStorageEntryFor("t1", "gosling".getBytes, "java".getBytes)
+ insertMapStorageEntryFor("t1", "stroustrup".getBytes, "c++".getBytes)
+ getMapStorageSizeFor("t1") should equal(3)
+ new String(getMapStorageEntryFor("t1", "odersky".getBytes).get) should equal("scala")
+ new String(getMapStorageEntryFor("t1", "gosling".getBytes).get) should equal("java")
+ new String(getMapStorageEntryFor("t1", "stroustrup".getBytes).get) should equal("c++")
+ getMapStorageEntryFor("t1", "torvalds".getBytes) should equal(None)
}
- // get the entire map
- val l: List[Tuple2[AnyRef, AnyRef]] =
- MongoStorageBackend.getMapStorageFor("U-M1")
+ it("should insert with multiple keys and values") {
+ import MongoStorageBackend._
- assertEquals(4, l.size)
- assertTrue(l.map(_._1).contains("1"))
- assertTrue(l.map(_._1).contains("2"))
- assertTrue(l.map(_._1).contains("3"))
- assertTrue(l.map(_._1).contains("4"))
+ val l = List(("stroustrup", "c++"), ("odersky", "scala"), ("gosling", "java"))
+ insertMapStorageEntriesFor("t1", l.map { case (k, v) => (k.getBytes, v.getBytes) })
+ getMapStorageSizeFor("t1") should equal(3)
+ new String(getMapStorageEntryFor("t1", "stroustrup".getBytes).get) should equal("c++")
+ new String(getMapStorageEntryFor("t1", "gosling".getBytes).get) should equal("java")
+ new String(getMapStorageEntryFor("t1", "odersky".getBytes).get) should equal("scala")
+ getMapStorageEntryFor("t1", "torvalds".getBytes) should equal(None)
- val JsString(str) = l.filter(_._1 == "2").head._2
- assertEquals(str, "peter")
+ getMapStorageEntryFor("t2", "torvalds".getBytes) should equal(None)
- // trying to fetch for a non-existent transaction will throw
- try {
- MongoStorageBackend.getMapStorageFor("U-M2")
- fail("should throw an exception")
- } catch {case e: NoSuchElementException => {}}
+ getMapStorageFor("t1").map { case (k, v) => (new String(k), new String(v)) } should equal (l)
- changeSetM.clear
- }
+ removeMapStorageFor("t1", "gosling".getBytes)
+ getMapStorageSizeFor("t1") should equal(2)
- @Test
- def testMapContentsByRange = {
- fillMap
- changeSetM += "5" -> Map(1 -> "dg", 2 -> "mc")
- MongoStorageBackend.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
-
- // specify start and count
- val l: List[Tuple2[AnyRef, AnyRef]] =
- MongoStorageBackend.getMapStorageRangeFor(
- "U-M1", Some(Integer.valueOf(2)), None, 3)
-
- assertEquals(3, l.size)
- assertEquals("3", l(0)._1.asInstanceOf[String])
- val lst = l(0)._2.asInstanceOf[JsValue]
- val num_list = list ! num
- val num_list(l0) = lst
- assertEquals(List(100, 200), l0)
- assertEquals("4", l(1)._1.asInstanceOf[String])
- val ls = l(1)._2.asInstanceOf[JsValue]
- val num_list(l1) = ls
- assertEquals(List(10, 20, 30), l1)
-
- // specify start, finish and count where finish - start == count
- assertEquals(3,
- MongoStorageBackend.getMapStorageRangeFor(
- "U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(5)), 3).size)
-
- // specify start, finish and count where finish - start > count
- assertEquals(3,
- MongoStorageBackend.getMapStorageRangeFor(
- "U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(9)), 3).size)
-
- // do not specify start or finish
- assertEquals(3,
- MongoStorageBackend.getMapStorageRangeFor(
- "U-M1", None, None, 3).size)
-
- // specify finish and count
- assertEquals(3,
- MongoStorageBackend.getMapStorageRangeFor(
- "U-M1", None, Some(Integer.valueOf(3)), 3).size)
-
- // specify start, finish and count where finish < start
- assertEquals(3,
- MongoStorageBackend.getMapStorageRangeFor(
- "U-M1", Some(Integer.valueOf(2)), Some(Integer.valueOf(1)), 3).size)
-
- changeSetM.clear
- }
-
- @Test
- def testMapStorageRemove = {
- fillMap
- changeSetM += "5" -> Map(1 -> "dg", 2 -> "mc")
-
- MongoStorageBackend.insertMapStorageEntriesFor("U-M1", changeSetM.toList)
- assertEquals(5,
- MongoStorageBackend.getMapStorageSizeFor("U-M1"))
-
- // remove key "3"
- MongoStorageBackend.removeMapStorageFor("U-M1", "3")
- assertEquals(4,
- MongoStorageBackend.getMapStorageSizeFor("U-M1"))
-
- try {
- MongoStorageBackend.getMapStorageEntryFor("U-M1", "3")
- fail("should throw exception")
- } catch { case e => {}}
-
- // remove key "4"
- MongoStorageBackend.removeMapStorageFor("U-M1", "4")
- assertEquals(3,
- MongoStorageBackend.getMapStorageSizeFor("U-M1"))
-
- // remove key "2"
- MongoStorageBackend.removeMapStorageFor("U-M1", "2")
- assertEquals(2,
- MongoStorageBackend.getMapStorageSizeFor("U-M1"))
-
- // remove the whole stuff
- MongoStorageBackend.removeMapStorageFor("U-M1")
-
- try {
- MongoStorageBackend.getMapStorageFor("U-M1")
- fail("should throw exception")
- } catch { case e: NoSuchElementException => {}}
-
- changeSetM.clear
- }
-
- private def fillMap = {
- changeSetM += "1" -> "john"
- changeSetM += "2" -> "peter"
- changeSetM += "3" -> List(100, 200)
- changeSetM += "4" -> List(10, 20, 30)
- changeSetM
- }
-
- @Test
- def testRefStorage = {
- MongoStorageBackend.getRefStorageFor("U-R1") match {
- case None =>
- case Some(o) => fail("should be None")
+ removeMapStorageFor("t1")
+ getMapStorageSizeFor("t1") should equal(0)
}
- val m = Map("1"->1, "2"->4, "3"->9)
- MongoStorageBackend.insertRefStorageFor("U-R1", m)
- MongoStorageBackend.getRefStorageFor("U-R1") match {
- case None => fail("should not be empty")
- case Some(r) => {
- val a = r.asInstanceOf[JsValue]
- val m1 = Symbol("1") ? num
- val m2 = Symbol("2") ? num
- val m3 = Symbol("3") ? num
+ it("should do proper range queries") {
+ import MongoStorageBackend._
+ val l = List(
+ ("bjarne stroustrup", "c++"),
+ ("martin odersky", "scala"),
+ ("james gosling", "java"),
+ ("yukihiro matsumoto", "ruby"),
+ ("slava pestov", "factor"),
+ ("rich hickey", "clojure"),
+ ("ola bini", "ioke"),
+ ("dennis ritchie", "c"),
+ ("larry wall", "perl"),
+ ("guido van rossum", "python"),
+ ("james strachan", "groovy"))
+ insertMapStorageEntriesFor("t1", l.map { case (k, v) => (k.getBytes, v.getBytes) })
+ getMapStorageSizeFor("t1") should equal(l.size)
+ getMapStorageRangeFor("t1", None, None, 100).map { case (k, v) => (new String(k), new String(v)) } should equal(l.sortWith(_._1 < _._1))
+ getMapStorageRangeFor("t1", None, None, 5).map { case (k, v) => (new String(k), new String(v)) }.size should equal(5)
+ }
+ }
- val m1(n1) = a
- val m2(n2) = a
- val m3(n3) = a
+ describe("persistent vectors") {
+ it("should insert a single value") {
+ import MongoStorageBackend._
- assertEquals(n1, 1)
- assertEquals(n2, 4)
- assertEquals(n3, 9)
- }
+ insertVectorStorageEntryFor("t1", "martin odersky".getBytes)
+ insertVectorStorageEntryFor("t1", "james gosling".getBytes)
+ new String(getVectorStorageEntryFor("t1", 0)) should equal("james gosling")
+ new String(getVectorStorageEntryFor("t1", 1)) should equal("martin odersky")
}
- // insert another one
- // the previous one should be replaced
- val b = List("100", "jonas")
- MongoStorageBackend.insertRefStorageFor("U-R1", b)
- MongoStorageBackend.getRefStorageFor("U-R1") match {
- case None => fail("should not be empty")
- case Some(r) => {
- val a = r.asInstanceOf[JsValue]
- val str_lst = list ! str
- val str_lst(l) = a
- assertEquals(b, l)
- }
+ it("should insert multiple values") {
+ import MongoStorageBackend._
+
+ insertVectorStorageEntryFor("t1", "martin odersky".getBytes)
+ insertVectorStorageEntryFor("t1", "james gosling".getBytes)
+ insertVectorStorageEntriesFor("t1", List("ola bini".getBytes, "james strachan".getBytes, "dennis ritchie".getBytes))
+ new String(getVectorStorageEntryFor("t1", 0)) should equal("ola bini")
+ new String(getVectorStorageEntryFor("t1", 1)) should equal("james strachan")
+ new String(getVectorStorageEntryFor("t1", 2)) should equal("dennis ritchie")
+ new String(getVectorStorageEntryFor("t1", 3)) should equal("james gosling")
+ new String(getVectorStorageEntryFor("t1", 4)) should equal("martin odersky")
+ }
+
+ it("should fetch a range of values") {
+ import MongoStorageBackend._
+
+ insertVectorStorageEntryFor("t1", "martin odersky".getBytes)
+ insertVectorStorageEntryFor("t1", "james gosling".getBytes)
+ getVectorStorageSizeFor("t1") should equal(2)
+ insertVectorStorageEntriesFor("t1", List("ola bini".getBytes, "james strachan".getBytes, "dennis ritchie".getBytes))
+ getVectorStorageRangeFor("t1", None, None, 100).map(new String(_)) should equal(List("ola bini", "james strachan", "dennis ritchie", "james gosling", "martin odersky"))
+ getVectorStorageRangeFor("t1", Some(0), Some(5), 100).map(new String(_)) should equal(List("ola bini", "james strachan", "dennis ritchie", "james gosling", "martin odersky"))
+ getVectorStorageRangeFor("t1", Some(2), Some(5), 100).map(new String(_)) should equal(List("dennis ritchie", "james gosling", "martin odersky"))
+
+ getVectorStorageSizeFor("t1") should equal(5)
+ }
+
+ it("should insert and query complex structures") {
+ import MongoStorageBackend._
+ import sjson.json.DefaultProtocol._
+ import sjson.json.JsonSerialization._
+
+ // a list[AnyRef] should be added successfully
+ val l = List("ola bini".getBytes, tobinary(List(100, 200, 300)), tobinary(List(1, 2, 3)))
+
+ // for id = t1
+ insertVectorStorageEntriesFor("t1", l)
+ new String(getVectorStorageEntryFor("t1", 0)) should equal("ola bini")
+ frombinary[List[Int]](getVectorStorageEntryFor("t1", 1)) should equal(List(100, 200, 300))
+ frombinary[List[Int]](getVectorStorageEntryFor("t1", 2)) should equal(List(1, 2, 3))
+
+ getVectorStorageSizeFor("t1") should equal(3)
+
+ // some more for id = t1
+ val m = List(tobinary(Map(1 -> "dg", 2 -> "mc", 3 -> "nd")), tobinary(List("martin odersky", "james gosling")))
+ insertVectorStorageEntriesFor("t1", m)
+
+ // size should add up
+ getVectorStorageSizeFor("t1") should equal(5)
+
+ // now for a diff id
+ insertVectorStorageEntriesFor("t2", l)
+ getVectorStorageSizeFor("t2") should equal(3)
+ }
+ }
+
+ describe("persistent refs") {
+ it("should insert a ref") {
+ import MongoStorageBackend._
+
+ insertRefStorageFor("t1", "martin odersky".getBytes)
+ new String(getRefStorageFor("t1").get) should equal("martin odersky")
+ insertRefStorageFor("t1", "james gosling".getBytes)
+ new String(getRefStorageFor("t1").get) should equal("james gosling")
+ getRefStorageFor("t2") should equal(None)
}
}
}
diff --git a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala
new file mode 100644
index 0000000000..3b160c8c50
--- /dev/null
+++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala
@@ -0,0 +1,347 @@
+package se.scalablesolutions.akka.persistence.mongo
+
+import org.scalatest.Spec
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
+
+import se.scalablesolutions.akka.actor.{Actor, ActorRef}
+import se.scalablesolutions.akka.config.OneForOneStrategy
+import Actor._
+import se.scalablesolutions.akka.stm.global._
+import se.scalablesolutions.akka.config.ScalaConfig._
+import se.scalablesolutions.akka.util.Logging
+
+import MongoStorageBackend._
+
+case class GET(k: String)
+case class SET(k: String, v: String)
+case class REM(k: String)
+case class CONTAINS(k: String)
+case object MAP_SIZE
+case class MSET(kvs: List[(String, String)])
+case class REMOVE_AFTER_PUT(kvsToAdd: List[(String, String)], ksToRem: List[String])
+case class CLEAR_AFTER_PUT(kvsToAdd: List[(String, String)])
+case class PUT_WITH_SLICE(kvsToAdd: List[(String, String)], start: String, cnt: Int)
+case class PUT_REM_WITH_SLICE(kvsToAdd: List[(String, String)], ksToRem: List[String], start: String, cnt: Int)
+
+case class VADD(v: String)
+case class VUPD(i: Int, v: String)
+case class VUPD_AND_ABORT(i: Int, v: String)
+case class VGET(i: Int)
+case object VSIZE
+case class VGET_AFTER_VADD(vsToAdd: List[String], isToFetch: List[Int])
+case class VADD_WITH_SLICE(vsToAdd: List[String], start: Int, cnt: Int)
+
+object Storage {
+ class MongoSampleMapStorage extends Actor {
+ self.lifeCycle = Some(LifeCycle(Permanent))
+ val FOO_MAP = "akka.sample.map"
+
+ private var fooMap = atomic { MongoStorage.getMap(FOO_MAP) }
+
+ def receive = {
+ case SET(k, v) =>
+ atomic {
+ fooMap += (k.getBytes, v.getBytes)
+ }
+ self.reply((k, v))
+
+ case GET(k) =>
+ val v = atomic {
+ fooMap.get(k.getBytes).map(new String(_)).getOrElse(k + " Not found")
+ }
+ self.reply(v)
+
+ case REM(k) =>
+ val v = atomic {
+ fooMap -= k.getBytes
+ }
+ self.reply(k)
+
+ case CONTAINS(k) =>
+ val v = atomic {
+ fooMap contains k.getBytes
+ }
+ self.reply(v)
+
+ case MAP_SIZE =>
+ val v = atomic {
+ fooMap.size
+ }
+ self.reply(v)
+
+ case MSET(kvs) => atomic {
+ kvs.foreach {kv => fooMap += (kv._1.getBytes, kv._2.getBytes) }
+ }
+ self.reply(kvs.size)
+
+ case REMOVE_AFTER_PUT(kvs2add, ks2rem) => atomic {
+ kvs2add.foreach {kv =>
+ fooMap += (kv._1.getBytes, kv._2.getBytes)
+ }
+
+ ks2rem.foreach {k =>
+ fooMap -= k.getBytes
+ }}
+ self.reply(fooMap.size)
+
+ case CLEAR_AFTER_PUT(kvs2add) => atomic {
+ kvs2add.foreach {kv =>
+ fooMap += (kv._1.getBytes, kv._2.getBytes)
+ }
+ fooMap.clear
+ }
+ self.reply(true)
+
+ case PUT_WITH_SLICE(kvs2add, from, cnt) =>
+ val v = atomic {
+ kvs2add.foreach {kv =>
+ fooMap += (kv._1.getBytes, kv._2.getBytes)
+ }
+ fooMap.slice(Some(from.getBytes), cnt)
+ }
+ self.reply(v: List[(Array[Byte], Array[Byte])])
+
+ case PUT_REM_WITH_SLICE(kvs2add, ks2rem, from, cnt) =>
+ val v = atomic {
+ kvs2add.foreach {kv =>
+ fooMap += (kv._1.getBytes, kv._2.getBytes)
+ }
+ ks2rem.foreach {k =>
+ fooMap -= k.getBytes
+ }
+ fooMap.slice(Some(from.getBytes), cnt)
+ }
+ self.reply(v: List[(Array[Byte], Array[Byte])])
+ }
+ }
+
+ class MongoSampleVectorStorage extends Actor {
+ self.lifeCycle = Some(LifeCycle(Permanent))
+ val FOO_VECTOR = "akka.sample.vector"
+
+ private var fooVector = atomic { MongoStorage.getVector(FOO_VECTOR) }
+
+ def receive = {
+ case VADD(v) =>
+ val size =
+ atomic {
+ fooVector + v.getBytes
+ fooVector length
+ }
+ self.reply(size)
+
+ case VGET(index) =>
+ val ind =
+ atomic {
+ fooVector get index
+ }
+ self.reply(ind)
+
+ case VGET_AFTER_VADD(vs, is) =>
+ val els =
+ atomic {
+ vs.foreach(fooVector + _.getBytes)
+ (is.foldRight(List[Array[Byte]]())(fooVector.get(_) :: _)).map(new String(_))
+ }
+ self.reply(els)
+
+ case VUPD_AND_ABORT(index, value) =>
+ val l =
+ atomic {
+ fooVector.update(index, value.getBytes)
+ // force fail
+ fooVector get 100
+ }
+ self.reply(index)
+
+ case VADD_WITH_SLICE(vs, s, c) =>
+ val l =
+ atomic {
+ vs.foreach(fooVector + _.getBytes)
+ fooVector.slice(Some(s), None, c)
+ }
+ self.reply(l.map(new String(_)))
+ }
+ }
+}
+
+import Storage._
+
+@RunWith(classOf[JUnitRunner])
+class MongoTicket343Spec extends
+ Spec with
+ ShouldMatchers with
+ BeforeAndAfterAll with
+ BeforeAndAfterEach {
+
+
+ override def beforeAll {
+ MongoStorageBackend.drop
+ println("** destroyed database")
+ }
+
+ override def beforeEach {
+ MongoStorageBackend.drop
+ println("** destroyed database")
+ }
+
+ override def afterEach {
+ MongoStorageBackend.drop
+ println("** destroyed database")
+ }
+
+ describe("Ticket 343 Issue #1") {
+ it("remove after put should work within the same transaction") {
+ val proc = actorOf[MongoSampleMapStorage]
+ proc.start
+
+ (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft"))
+ (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft")
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+
+ (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3)
+
+ (proc !! GET("dg")).getOrElse("Get failed") should equal("1")
+ (proc !! GET("mc")).getOrElse("Get failed") should equal("2")
+ (proc !! GET("nd")).getOrElse("Get failed") should equal("3")
+
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4)
+
+ val add = List(("a", "1"), ("b", "2"), ("c", "3"))
+ val rem = List("a", "debasish")
+ (proc !! REMOVE_AFTER_PUT(add, rem)).getOrElse("REMOVE_AFTER_PUT failed") should equal(5)
+
+ (proc !! GET("debasish")).getOrElse("debasish not found") should equal("debasish Not found")
+ (proc !! GET("a")).getOrElse("a not found") should equal("a Not found")
+
+ (proc !! GET("b")).getOrElse("b not found") should equal("2")
+
+ (proc !! CONTAINS("b")).getOrElse("b not found") should equal(true)
+ (proc !! CONTAINS("debasish")).getOrElse("debasish not found") should equal(false)
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(5)
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #2") {
+ it("clear after put should work within the same transaction") {
+ val proc = actorOf[MongoSampleMapStorage]
+ proc.start
+
+ (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft"))
+ (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft")
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+
+ val add = List(("a", "1"), ("b", "2"), ("c", "3"))
+ (proc !! CLEAR_AFTER_PUT(add)).getOrElse("CLEAR_AFTER_PUT failed") should equal(true)
+
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #3") {
+ it("map size should change after the transaction") {
+ val proc = actorOf[MongoSampleMapStorage]
+ proc.start
+
+ (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft"))
+ (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft")
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+
+ (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3)
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4)
+
+ (proc !! GET("dg")).getOrElse("Get failed") should equal("1")
+ (proc !! GET("mc")).getOrElse("Get failed") should equal("2")
+ (proc !! GET("nd")).getOrElse("Get failed") should equal("3")
+ proc.stop
+ }
+ }
+
+ describe("slice test") {
+ it("should pass") {
+ val proc = actorOf[MongoSampleMapStorage]
+ proc.start
+
+ (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft"))
+ (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft")
+ // (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+
+ (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3)
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4)
+
+ (proc !! PUT_WITH_SLICE(List(("ec", "1"), ("tb", "2"), ("mc", "10")), "dg", 3)).get.asInstanceOf[List[(Array[Byte], Array[Byte])]].map { case (k, v) => (new String(k), new String(v)) } should equal(List(("dg", "1"), ("ec", "1"), ("mc", "10")))
+
+ (proc !! PUT_REM_WITH_SLICE(List(("fc", "1"), ("gb", "2"), ("xy", "10")), List("tb", "fc"), "dg", 5)).get.asInstanceOf[List[(Array[Byte], Array[Byte])]].map { case (k, v) => (new String(k), new String(v)) } should equal(List(("dg", "1"), ("ec", "1"), ("gb", "2"), ("mc", "10"), ("nd", "3")))
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #4") {
+ it("vector get should not ignore elements that were in vector before transaction") {
+
+ val proc = actorOf[MongoSampleVectorStorage]
+ proc.start
+
+ // add 4 elements in separate transactions
+ (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1)
+ (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2)
+ (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3)
+ (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4)
+
+ new String((proc !! VGET(0)).get.asInstanceOf[Array[Byte]] ) should equal("nilanjan")
+ new String((proc !! VGET(1)).get.asInstanceOf[Array[Byte]] ) should equal("ramanendu")
+ new String((proc !! VGET(2)).get.asInstanceOf[Array[Byte]] ) should equal("maulindu")
+ new String((proc !! VGET(3)).get.asInstanceOf[Array[Byte]] ) should equal("debasish")
+
+ // now add 3 more and do gets in the same transaction
+ (proc !! VGET_AFTER_VADD(List("a", "b", "c"), List(0, 2, 4))).get.asInstanceOf[List[String]] should equal(List("c", "a", "ramanendu"))
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #6") {
+ it("vector update should not ignore transaction") {
+ val proc = actorOf[MongoSampleVectorStorage]
+ proc.start
+
+ // add 4 elements in separate transactions
+ (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1)
+ (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2)
+ (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3)
+ (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4)
+
+ evaluating {
+ (proc !! VUPD_AND_ABORT(0, "virat")).getOrElse("VUPD_AND_ABORT failed")
+ } should produce [Exception]
+
+ // update aborts and hence values will remain unchanged
+ new String((proc !! VGET(0)).get.asInstanceOf[Array[Byte]] ) should equal("nilanjan")
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #5") {
+ it("vector slice() should not ignore elements added in current transaction") {
+ val proc = actorOf[MongoSampleVectorStorage]
+ proc.start
+
+ // add 4 elements in separate transactions
+ (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1)
+ (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2)
+ (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3)
+ (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4)
+
+ // slice with no new elements added in current transaction
+ (proc !! VADD_WITH_SLICE(List(), 2, 2)).getOrElse("VADD_WITH_SLICE failed") should equal(Vector("maulindu", "debasish"))
+
+ // slice with new elements added in current transaction
+ (proc !! VADD_WITH_SLICE(List("a", "b", "c", "d"), 2, 2)).getOrElse("VADD_WITH_SLICE failed") should equal(Vector("b", "a"))
+ proc.stop
+ }
+ }
+}
diff --git a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala
index c92761beea..1eca775567 100644
--- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala
+++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorage.scala
@@ -36,7 +36,7 @@ object RedisStorage extends Storage {
*
* @author Debasish Ghosh
*/
-class RedisPersistentMap(id: String) extends PersistentMap[Array[Byte], Array[Byte]] {
+class RedisPersistentMap(id: String) extends PersistentMapBinary {
val uuid = id
val storage = RedisStorageBackend
}
diff --git a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala
index eef60784a0..9fd3142019 100644
--- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala
+++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala
@@ -11,34 +11,17 @@ import se.scalablesolutions.akka.config.Config.config
import com.redis._
-trait Base64Encoder {
- def encode(bytes: Array[Byte]): Array[Byte]
- def decode(bytes: Array[Byte]): Array[Byte]
-}
-
trait Base64StringEncoder {
def byteArrayToString(bytes: Array[Byte]): String
def stringToByteArray(str: String): Array[Byte]
}
-trait NullBase64 {
- def encode(bytes: Array[Byte]): Array[Byte] = bytes
- def decode(bytes: Array[Byte]): Array[Byte] = bytes
-}
-
object CommonsCodec {
import org.apache.commons.codec.binary.Base64
import org.apache.commons.codec.binary.Base64._
val b64 = new Base64(true)
- trait CommonsCodecBase64 {
- def encode(bytes: Array[Byte]): Array[Byte] = encodeBase64(bytes)
- def decode(bytes: Array[Byte]): Array[Byte] = decodeBase64(bytes)
- }
-
- object Base64Encoder extends Base64Encoder with CommonsCodecBase64
-
trait CommonsCodecBase64StringEncoder {
def byteArrayToString(bytes: Array[Byte]) = encodeBase64URLSafeString(bytes)
def stringToByteArray(str: String) = b64.decode(str)
@@ -48,7 +31,6 @@ object CommonsCodec {
}
import CommonsCodec._
-import CommonsCodec.Base64Encoder._
import CommonsCodec.Base64StringEncoder._
/**
@@ -94,27 +76,7 @@ private [akka] object RedisStorageBackend extends
/**
* Map storage in Redis.
*
- * Maps are stored as key/value pairs in redis. Redis keys cannot contain spaces. But with
- * our use case, the keys will be specified by the user. Hence we need to encode the key
- * ourselves before sending to Redis. We use base64 encoding.
- *
- * Also since we are storing the key/value in the global namespace, we need to construct the
- * key suitably so as to avoid namespace clash. The following strategy is used:
- *
- * Unique identifier for the map = T1 (say)
- *
- * Map(
- * "debasish.address" -> "kolkata, India",
- * "debasish.company" -> "anshinsoft",
- * "debasish.programming_language" -> "scala",
- * )
- * will be stored as the following key-value pair in Redis:
- *
- *
- * base64(T1):base64("debasish.address") -> "kolkata, India"
- * base64(T1):base64("debasish.company") -> "anshinsoft"
- * base64(T1):base64("debasish.programming_language") -> "scala"
- *
+ * Maps are stored as key/value pairs in redis.
*/
def insertMapStorageEntryFor(name: String, key: Array[Byte], value: Array[Byte]): Unit = withErrorHandling {
insertMapStorageEntriesFor(name, List((key, value)))
@@ -134,12 +96,12 @@ private [akka] object RedisStorageBackend extends
* both parts of the key need to be based64 encoded since there can be spaces within each of them
*/
private [this] def makeRedisKey(name: String, key: Array[Byte]): String = withErrorHandling {
- "%s:%s".format(new String(encode(name.getBytes)), new String(encode(key)))
+ "%s:%s".format(name, new String(key))
}
private [this] def makeKeyFromRedisKey(redisKey: String) = withErrorHandling {
- val nk = redisKey.split(':').map{e: String => decode(e.getBytes)}
- (nk(0), nk(1))
+ val nk = redisKey.split(':')
+ (nk(0), nk(1).getBytes)
}
private [this] def mset(entries: List[(String, String)]): Unit = withErrorHandling {
@@ -149,11 +111,11 @@ private [akka] object RedisStorageBackend extends
}
def removeMapStorageFor(name: String): Unit = withErrorHandling {
- db.keys("%s:*".format(new String(encode(name.getBytes)))) match {
+ db.keys("%s:*".format(name)) match {
case None =>
throw new NoSuchElementException(name + " not present")
case Some(keys) =>
- keys.foreach(db.del(_))
+ keys.foreach(k => db.del(k.get))
}
}
@@ -162,28 +124,22 @@ private [akka] object RedisStorageBackend extends
}
def getMapStorageEntryFor(name: String, key: Array[Byte]): Option[Array[Byte]] = withErrorHandling {
- db.get(makeRedisKey(name, key)) match {
- case None =>
- throw new NoSuchElementException(new String(key) + " not present")
- case Some(s) => Some(stringToByteArray(s))
+ db.get(makeRedisKey(name, key))
+ .map(stringToByteArray(_))
+ .orElse(throw new NoSuchElementException(new String(key) + " not present"))
}
- }
def getMapStorageSizeFor(name: String): Int = withErrorHandling {
- db.keys("%s:*".format(new String(encode(name.getBytes)))) match {
- case None => 0
- case Some(keys) =>
- keys.length
- }
+ db.keys("%s:*".format(name)).map(_.length).getOrElse(0)
}
def getMapStorageFor(name: String): List[(Array[Byte], Array[Byte])] = withErrorHandling {
- db.keys("%s:*".format(new String(encode(name.getBytes)))) match {
- case None =>
+ db.keys("%s:*".format(name))
+ .map { keys =>
+ keys.map(key => (makeKeyFromRedisKey(key.get)._2, stringToByteArray(db.get(key.get).get))).toList
+ }.getOrElse {
throw new NoSuchElementException(name + " not present")
- case Some(keys) =>
- keys.map(key => (makeKeyFromRedisKey(key)._2, stringToByteArray(db.get(key).get))).toList
- }
+ }
}
def getMapStorageRangeFor(name: String, start: Option[Array[Byte]],
@@ -234,7 +190,7 @@ private [akka] object RedisStorageBackend extends
}
def insertVectorStorageEntryFor(name: String, element: Array[Byte]): Unit = withErrorHandling {
- db.lpush(new String(encode(name.getBytes)), byteArrayToString(element))
+ db.lpush(name, byteArrayToString(element))
}
def insertVectorStorageEntriesFor(name: String, elements: List[Array[Byte]]): Unit = withErrorHandling {
@@ -242,16 +198,15 @@ private [akka] object RedisStorageBackend extends
}
def updateVectorStorageEntryFor(name: String, index: Int, elem: Array[Byte]): Unit = withErrorHandling {
- db.lset(new String(encode(name.getBytes)), index, byteArrayToString(elem))
+ db.lset(name, index, byteArrayToString(elem))
}
def getVectorStorageEntryFor(name: String, index: Int): Array[Byte] = withErrorHandling {
- db.lindex(new String(encode(name.getBytes)), index) match {
- case None =>
+ db.lindex(name, index)
+ .map(stringToByteArray(_))
+ .getOrElse {
throw new NoSuchElementException(name + " does not have element at " + index)
- case Some(e) =>
- stringToByteArray(e)
- }
+ }
}
/**
@@ -270,60 +225,51 @@ private [akka] object RedisStorageBackend extends
else count
if (s == 0 && cnt == 0) List()
else
- db.lrange(new String(encode(name.getBytes)), s, s + cnt - 1) match {
+ db.lrange(name, s, s + cnt - 1) match {
case None =>
throw new NoSuchElementException(name + " does not have elements in the range specified")
case Some(l) =>
- l map ( e => stringToByteArray(e.get))
+ l map (e => stringToByteArray(e.get))
}
}
def getVectorStorageSizeFor(name: String): Int = withErrorHandling {
- db.llen(new String(encode(name.getBytes))) match {
- case None =>
- throw new NoSuchElementException(name + " not present")
- case Some(l) =>
- l
- }
+ db.llen(name).getOrElse { throw new NoSuchElementException(name + " not present") }
}
def insertRefStorageFor(name: String, element: Array[Byte]): Unit = withErrorHandling {
- db.set(new String(encode(name.getBytes)), byteArrayToString(element))
+ db.set(name, byteArrayToString(element))
}
def insertRefStorageFor(name: String, element: String): Unit = withErrorHandling {
- db.set(new String(encode(name.getBytes)), element)
+ db.set(name, element)
}
def getRefStorageFor(name: String): Option[Array[Byte]] = withErrorHandling {
- db.get(new String(encode(name.getBytes))) match {
- case None =>
+ db.get(name)
+ .map(stringToByteArray(_))
+ .orElse {
throw new NoSuchElementException(name + " not present")
- case Some(s) => Some(stringToByteArray(s))
- }
+ }
}
// add to the end of the queue
- def enqueue(name: String, item: Array[Byte]): Boolean = withErrorHandling {
- db.rpush(new String(encode(name.getBytes)), byteArrayToString(item))
+ def enqueue(name: String, item: Array[Byte]): Option[Int] = withErrorHandling {
+ db.rpush(name, byteArrayToString(item))
}
// pop from the front of the queue
def dequeue(name: String): Option[Array[Byte]] = withErrorHandling {
- db.lpop(new String(encode(name.getBytes))) match {
- case None =>
+ db.lpop(name)
+ .map(stringToByteArray(_))
+ .orElse {
throw new NoSuchElementException(name + " not present")
- case Some(s) => Some(stringToByteArray(s))
- }
+ }
}
// get the size of the queue
def size(name: String): Int = withErrorHandling {
- db.llen(new String(encode(name.getBytes))) match {
- case None =>
- throw new NoSuchElementException(name + " not present")
- case Some(l) => l
- }
+ db.llen(name).getOrElse { throw new NoSuchElementException(name + " not present") }
}
// return an array of items currently stored in the queue
@@ -331,14 +277,14 @@ private [akka] object RedisStorageBackend extends
def peek(name: String, start: Int, count: Int): List[Array[Byte]] = withErrorHandling {
count match {
case 1 =>
- db.lindex(new String(encode(name.getBytes)), start) match {
+ db.lindex(name, start) match {
case None =>
throw new NoSuchElementException("No element at " + start)
case Some(s) =>
List(stringToByteArray(s))
}
case n =>
- db.lrange(new String(encode(name.getBytes)), start, start + count - 1) match {
+ db.lrange(name, start, start + count - 1) match {
case None =>
throw new NoSuchElementException(
"No element found between " + start + " and " + (start + count - 1))
@@ -350,61 +296,54 @@ private [akka] object RedisStorageBackend extends
// completely delete the queue
def remove(name: String): Boolean = withErrorHandling {
- db.del(new String(encode(name.getBytes))) match {
- case Some(1) => true
- case _ => false
- }
+ db.del(name).map { case 1 => true }.getOrElse(false)
}
// add item to sorted set identified by name
def zadd(name: String, zscore: String, item: Array[Byte]): Boolean = withErrorHandling {
- db.zadd(new String(encode(name.getBytes)), zscore, byteArrayToString(item)) match {
- case Some(1) => true
- case _ => false
- }
+ db.zadd(name, zscore, byteArrayToString(item))
+ .map { e =>
+ e match {
+ case 1 => true
+ case _ => false
+ }
+ }.getOrElse(false)
}
// remove item from sorted set identified by name
def zrem(name: String, item: Array[Byte]): Boolean = withErrorHandling {
- db.zrem(new String(encode(name.getBytes)), byteArrayToString(item)) match {
- case Some(1) => true
- case _ => false
- }
+ db.zrem(name, byteArrayToString(item))
+ .map { e =>
+ e match {
+ case 1 => true
+ case _ => false
+ }
+ }.getOrElse(false)
}
// cardinality of the set identified by name
def zcard(name: String): Int = withErrorHandling {
- db.zcard(new String(encode(name.getBytes))) match {
- case None =>
- throw new NoSuchElementException(name + " not present")
- case Some(l) => l
- }
+ db.zcard(name).getOrElse { throw new NoSuchElementException(name + " not present") }
}
def zscore(name: String, item: Array[Byte]): Option[Float] = withErrorHandling {
- db.zscore(new String(encode(name.getBytes)), byteArrayToString(item)) match {
- case Some(s) => Some(s.toFloat)
- case None => None
- }
+ db.zscore(name, byteArrayToString(item)).map(_.toFloat)
}
def zrange(name: String, start: Int, end: Int): List[Array[Byte]] = withErrorHandling {
- db.zrange(new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC, false) match {
- case None =>
+ db.zrange(name, start.toString, end.toString, RedisClient.ASC, false)
+ .map(_.map(e => stringToByteArray(e.get)))
+ .getOrElse {
throw new NoSuchElementException(name + " not present")
- case Some(s) =>
- s.map(e => stringToByteArray(e.get))
- }
+ }
}
def zrangeWithScore(name: String, start: Int, end: Int): List[(Array[Byte], Float)] = withErrorHandling {
- db.zrangeWithScore(
- new String(encode(name.getBytes)), start.toString, end.toString, RedisClient.ASC) match {
- case None =>
- throw new NoSuchElementException(name + " not present")
- case Some(l) =>
- l.map{ case (elem, score) => (stringToByteArray(elem.get), score.get.toFloat) }
- }
+ db.zrangeWithScore(name, start.toString, end.toString, RedisClient.ASC)
+ .map(_.map { case (elem, score) => (stringToByteArray(elem.get), score.get.toFloat) })
+ .getOrElse {
+ throw new NoSuchElementException(name + " not present")
+ }
}
def flushDB = withErrorHandling(db.flushdb)
@@ -420,6 +359,7 @@ private [akka] object RedisStorageBackend extends
case e: java.lang.NullPointerException =>
throw new StorageException("Could not connect to Redis server")
case e =>
+ e.printStackTrace
throw new StorageException("Error in Redis: " + e.getMessage)
}
}
diff --git a/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala b/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala
new file mode 100644
index 0000000000..de236b9a5a
--- /dev/null
+++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala
@@ -0,0 +1,351 @@
+package se.scalablesolutions.akka.persistence.redis
+
+import org.scalatest.Spec
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach}
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
+
+import se.scalablesolutions.akka.actor.{Actor}
+import se.scalablesolutions.akka.config.OneForOneStrategy
+import Actor._
+import se.scalablesolutions.akka.persistence.common.PersistentVector
+import se.scalablesolutions.akka.stm.global._
+import se.scalablesolutions.akka.config.ScalaConfig._
+import se.scalablesolutions.akka.util.Logging
+
+import RedisStorageBackend._
+
+case class GET(k: String)
+case class SET(k: String, v: String)
+case class REM(k: String)
+case class CONTAINS(k: String)
+case object MAP_SIZE
+case class MSET(kvs: List[(String, String)])
+case class REMOVE_AFTER_PUT(kvsToAdd: List[(String, String)], ksToRem: List[String])
+case class CLEAR_AFTER_PUT(kvsToAdd: List[(String, String)])
+case class PUT_WITH_SLICE(kvsToAdd: List[(String, String)], start: String, cnt: Int)
+case class PUT_REM_WITH_SLICE(kvsToAdd: List[(String, String)], ksToRem: List[String], start: String, cnt: Int)
+
+case class VADD(v: String)
+case class VUPD(i: Int, v: String)
+case class VUPD_AND_ABORT(i: Int, v: String)
+case class VGET(i: Int)
+case object VSIZE
+case class VGET_AFTER_VADD(vsToAdd: List[String], isToFetch: List[Int])
+case class VADD_WITH_SLICE(vsToAdd: List[String], start: Int, cnt: Int)
+
+object Storage {
+ class RedisSampleMapStorage extends Actor {
+ self.lifeCycle = Some(LifeCycle(Permanent))
+ val FOO_MAP = "akka.sample.map"
+
+ private var fooMap = atomic { RedisStorage.getMap(FOO_MAP) }
+
+ def receive = {
+ case SET(k, v) =>
+ atomic {
+ fooMap += (k.getBytes, v.getBytes)
+ }
+ self.reply((k, v))
+
+ case GET(k) =>
+ val v = atomic {
+ fooMap.get(k.getBytes)
+ }
+ self.reply(v.collect {case byte => new String(byte)}.getOrElse(k + " Not found"))
+
+ case REM(k) =>
+ val v = atomic {
+ fooMap -= k.getBytes
+ }
+ self.reply(k)
+
+ case CONTAINS(k) =>
+ val v = atomic {
+ fooMap contains k.getBytes
+ }
+ self.reply(v)
+
+ case MAP_SIZE =>
+ val v = atomic {
+ fooMap.size
+ }
+ self.reply(v)
+
+ case MSET(kvs) =>
+ atomic {
+ kvs.foreach {kv =>
+ fooMap += (kv._1.getBytes, kv._2.getBytes)
+ }
+ }
+ self.reply(kvs.size)
+
+ case REMOVE_AFTER_PUT(kvs2add, ks2rem) =>
+ val v =
+ atomic {
+ kvs2add.foreach {kv =>
+ fooMap += (kv._1.getBytes, kv._2.getBytes)
+ }
+
+ ks2rem.foreach {k =>
+ fooMap -= k.getBytes
+ }
+ fooMap.size
+ }
+ self.reply(v)
+
+ case CLEAR_AFTER_PUT(kvs2add) =>
+ atomic {
+ kvs2add.foreach {kv =>
+ fooMap += (kv._1.getBytes, kv._2.getBytes)
+ }
+ fooMap.clear
+ }
+ self.reply(true)
+
+ case PUT_WITH_SLICE(kvs2add, from, cnt) =>
+ val v =
+ atomic {
+ kvs2add.foreach {kv =>
+ fooMap += (kv._1.getBytes, kv._2.getBytes)
+ }
+ fooMap.slice(Some(from.getBytes), cnt)
+ }
+ self.reply(v: List[(Array[Byte], Array[Byte])])
+
+ case PUT_REM_WITH_SLICE(kvs2add, ks2rem, from, cnt) =>
+ val v =
+ atomic {
+ kvs2add.foreach {kv =>
+ fooMap += (kv._1.getBytes, kv._2.getBytes)
+ }
+ ks2rem.foreach {k =>
+ fooMap -= k.getBytes
+ }
+ fooMap.slice(Some(from.getBytes), cnt)
+ }
+ self.reply(v: List[(Array[Byte], Array[Byte])])
+ }
+ }
+
+ class RedisSampleVectorStorage extends Actor {
+ self.lifeCycle = Some(LifeCycle(Permanent))
+ val FOO_VECTOR = "akka.sample.vector"
+
+ private var fooVector = atomic { RedisStorage.getVector(FOO_VECTOR) }
+
+ def receive = {
+ case VADD(v) =>
+ val size =
+ atomic {
+ fooVector + v.getBytes
+ fooVector length
+ }
+ self.reply(size)
+
+ case VGET(index) =>
+ val ind =
+ atomic {
+ fooVector get index
+ }
+ self.reply(ind)
+
+ case VGET_AFTER_VADD(vs, is) =>
+ val els =
+ atomic {
+ vs.foreach(fooVector + _.getBytes)
+ (is.foldRight(List[Array[Byte]]())(fooVector.get(_) :: _)).map(new String(_))
+ }
+ self.reply(els)
+
+ case VUPD_AND_ABORT(index, value) =>
+ val l =
+ atomic {
+ fooVector.update(index, value.getBytes)
+ // force fail
+ fooVector get 100
+ }
+ self.reply(index)
+
+ case VADD_WITH_SLICE(vs, s, c) =>
+ val l =
+ atomic {
+ vs.foreach(fooVector + _.getBytes)
+ fooVector.slice(Some(s), None, c)
+ }
+ self.reply(l.map(new String(_)))
+ }
+ }
+}
+
+import Storage._
+
+@RunWith(classOf[JUnitRunner])
+class RedisTicket343Spec extends
+ Spec with
+ ShouldMatchers with
+ BeforeAndAfterAll with
+ BeforeAndAfterEach {
+
+ override def beforeAll {
+ flushDB
+ println("** destroyed database")
+ }
+
+ override def afterEach {
+ flushDB
+ println("** destroyed database")
+ }
+
+ describe("Ticket 343 Issue #1") {
+ it("remove after put should work within the same transaction") {
+ val proc = actorOf[RedisSampleMapStorage]
+ proc.start
+
+ (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft"))
+ (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft")
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+
+ (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3)
+
+ (proc !! GET("dg")).getOrElse("Get failed") should equal("1")
+ (proc !! GET("mc")).getOrElse("Get failed") should equal("2")
+ (proc !! GET("nd")).getOrElse("Get failed") should equal("3")
+
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4)
+
+ val add = List(("a", "1"), ("b", "2"), ("c", "3"))
+ val rem = List("a", "debasish")
+ (proc !! REMOVE_AFTER_PUT(add, rem)).getOrElse("REMOVE_AFTER_PUT failed") should equal(5)
+
+ (proc !! GET("debasish")).getOrElse("debasish not found") should equal("debasish Not found")
+ (proc !! GET("a")).getOrElse("a not found") should equal("a Not found")
+
+ (proc !! GET("b")).getOrElse("b not found") should equal("2")
+
+ (proc !! CONTAINS("b")).getOrElse("b not found") should equal(true)
+ (proc !! CONTAINS("debasish")).getOrElse("debasish not found") should equal(false)
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(5)
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #2") {
+ it("clear after put should work within the same transaction") {
+ val proc = actorOf[RedisSampleMapStorage]
+ proc.start
+
+ (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft"))
+ (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft")
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+
+ val add = List(("a", "1"), ("b", "2"), ("c", "3"))
+ (proc !! CLEAR_AFTER_PUT(add)).getOrElse("CLEAR_AFTER_PUT failed") should equal(true)
+
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #3") {
+ it("map size should change after the transaction") {
+ val proc = actorOf[RedisSampleMapStorage]
+ proc.start
+
+ (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft"))
+ (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft")
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+
+ (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3)
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4)
+
+ (proc !! GET("dg")).getOrElse("Get failed") should equal("1")
+ (proc !! GET("mc")).getOrElse("Get failed") should equal("2")
+ (proc !! GET("nd")).getOrElse("Get failed") should equal("3")
+ proc.stop
+ }
+ }
+
+ describe("slice test") {
+ it("should pass") {
+ val proc = actorOf[RedisSampleMapStorage]
+ proc.start
+
+ (proc !! SET("debasish", "anshinsoft")).getOrElse("Set failed") should equal(("debasish", "anshinsoft"))
+ (proc !! GET("debasish")).getOrElse("Get failed") should equal("anshinsoft")
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(1)
+
+ (proc !! MSET(List(("dg", "1"), ("mc", "2"), ("nd", "3")))).getOrElse("Mset failed") should equal(3)
+ (proc !! MAP_SIZE).getOrElse("Size failed") should equal(4)
+
+ (proc !! PUT_WITH_SLICE(List(("ec", "1"), ("tb", "2"), ("mc", "10")), "dg", 3)).get.asInstanceOf[List[(Array[Byte], Array[Byte])]].map { case (k, v) => (new String(k), new String(v)) } should equal(List(("dg", "1"), ("ec", "1"), ("mc", "10")))
+
+ (proc !! PUT_REM_WITH_SLICE(List(("fc", "1"), ("gb", "2"), ("xy", "10")), List("tb", "fc"), "dg", 5)).get.asInstanceOf[List[(Array[Byte], Array[Byte])]].map { case (k, v) => (new String(k), new String(v)) } should equal(List(("dg", "1"), ("ec", "1"), ("gb", "2"), ("mc", "10"), ("nd", "3")))
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #4") {
+ it("vector get should not ignore elements that were in vector before transaction") {
+ val proc = actorOf[RedisSampleVectorStorage]
+ proc.start
+
+ // add 4 elements in separate transactions
+ (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1)
+ (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2)
+ (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3)
+ (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4)
+
+ new String((proc !! VGET(0)).get.asInstanceOf[Array[Byte]] ) should equal("nilanjan")
+ new String((proc !! VGET(1)).get.asInstanceOf[Array[Byte]] ) should equal("ramanendu")
+ new String((proc !! VGET(2)).get.asInstanceOf[Array[Byte]] ) should equal("maulindu")
+ new String((proc !! VGET(3)).get.asInstanceOf[Array[Byte]] ) should equal("debasish")
+
+ // now add 3 more and do gets in the same transaction
+ (proc !! VGET_AFTER_VADD(List("a", "b", "c"), List(0, 2, 4))).get.asInstanceOf[List[String]] should equal(List("c", "a", "ramanendu"))
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #6") {
+ it("vector update should not ignore transaction") {
+ val proc = actorOf[RedisSampleVectorStorage]
+ proc.start
+
+ // add 4 elements in separate transactions
+ (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1)
+ (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2)
+ (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3)
+ (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4)
+
+ evaluating {
+ (proc !! VUPD_AND_ABORT(0, "virat")).getOrElse("VUPD_AND_ABORT failed")
+ } should produce [Exception]
+
+ // update aborts and hence values will remain unchanged
+ new String((proc !! VGET(0)).get.asInstanceOf[Array[Byte]] ) should equal("nilanjan")
+ proc.stop
+ }
+ }
+
+ describe("Ticket 343 Issue #5") {
+ it("vector slice() should not ignore elements added in current transaction") {
+ val proc = actorOf[RedisSampleVectorStorage]
+ proc.start
+
+ // add 4 elements in separate transactions
+ (proc !! VADD("debasish")).getOrElse("VADD failed") should equal(1)
+ (proc !! VADD("maulindu")).getOrElse("VADD failed") should equal(2)
+ (proc !! VADD("ramanendu")).getOrElse("VADD failed") should equal(3)
+ (proc !! VADD("nilanjan")).getOrElse("VADD failed") should equal(4)
+
+ // slice with no new elements added in current transaction
+ (proc !! VADD_WITH_SLICE(List(), 2, 2)).getOrElse("VADD_WITH_SLICE failed") should equal(Vector("maulindu", "debasish"))
+
+ // slice with new elements added in current transaction
+ (proc !! VADD_WITH_SLICE(List("a", "b", "c", "d"), 2, 2)).getOrElse("VADD_WITH_SLICE failed") should equal(Vector("b", "a"))
+ proc.stop
+ }
+ }
+}
diff --git a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java b/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
index 0ab1a0aa10..ab05937f61 100644
--- a/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
+++ b/akka-remote/src/main/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
@@ -14,10 +14,10 @@ public final class RemoteProtocol {
JAVA_ACTOR(1, 2),
TYPED_ACTOR(2, 3),
;
-
-
+
+
public final int getNumber() { return value; }
-
+
public static ActorType valueOf(int value) {
switch (value) {
case 1: return SCALA_ACTOR;
@@ -26,7 +26,7 @@ public final class RemoteProtocol {
default: return null;
}
}
-
+
public static com.google.protobuf.Internal.EnumLiteMap
internalGetValueMap() {
return internalValueMap;
@@ -38,7 +38,7 @@ public final class RemoteProtocol {
return ActorType.valueOf(number)
; }
};
-
+
public final com.google.protobuf.Descriptors.EnumValueDescriptor
getValueDescriptor() {
return getDescriptor().getValues().get(index);
@@ -51,9 +51,9 @@ public final class RemoteProtocol {
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(0);
}
-
+
private static final ActorType[] VALUES = {
- SCALA_ACTOR, JAVA_ACTOR, TYPED_ACTOR,
+ SCALA_ACTOR, JAVA_ACTOR, TYPED_ACTOR,
};
public static ActorType valueOf(
com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
@@ -69,14 +69,14 @@ public final class RemoteProtocol {
this.index = index;
this.value = value;
}
-
+
static {
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor();
}
-
+
// @@protoc_insertion_point(enum_scope:ActorType)
}
-
+
public enum SerializationSchemeType
implements com.google.protobuf.ProtocolMessageEnum {
JAVA(0, 1),
@@ -85,10 +85,10 @@ public final class RemoteProtocol {
JAVA_JSON(3, 4),
PROTOBUF(4, 5),
;
-
-
+
+
public final int getNumber() { return value; }
-
+
public static SerializationSchemeType valueOf(int value) {
switch (value) {
case 1: return JAVA;
@@ -99,7 +99,7 @@ public final class RemoteProtocol {
default: return null;
}
}
-
+
public static com.google.protobuf.Internal.EnumLiteMap
internalGetValueMap() {
return internalValueMap;
@@ -111,7 +111,7 @@ public final class RemoteProtocol {
return SerializationSchemeType.valueOf(number)
; }
};
-
+
public final com.google.protobuf.Descriptors.EnumValueDescriptor
getValueDescriptor() {
return getDescriptor().getValues().get(index);
@@ -124,9 +124,9 @@ public final class RemoteProtocol {
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(1);
}
-
+
private static final SerializationSchemeType[] VALUES = {
- JAVA, SBINARY, SCALA_JSON, JAVA_JSON, PROTOBUF,
+ JAVA, SBINARY, SCALA_JSON, JAVA_JSON, PROTOBUF,
};
public static SerializationSchemeType valueOf(
com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
@@ -142,23 +142,23 @@ public final class RemoteProtocol {
this.index = index;
this.value = value;
}
-
+
static {
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor();
}
-
+
// @@protoc_insertion_point(enum_scope:SerializationSchemeType)
}
-
+
public enum LifeCycleType
implements com.google.protobuf.ProtocolMessageEnum {
PERMANENT(0, 1),
TEMPORARY(1, 2),
;
-
-
+
+
public final int getNumber() { return value; }
-
+
public static LifeCycleType valueOf(int value) {
switch (value) {
case 1: return PERMANENT;
@@ -166,7 +166,7 @@ public final class RemoteProtocol {
default: return null;
}
}
-
+
public static com.google.protobuf.Internal.EnumLiteMap
internalGetValueMap() {
return internalValueMap;
@@ -178,7 +178,7 @@ public final class RemoteProtocol {
return LifeCycleType.valueOf(number)
; }
};
-
+
public final com.google.protobuf.Descriptors.EnumValueDescriptor
getValueDescriptor() {
return getDescriptor().getValues().get(index);
@@ -191,9 +191,9 @@ public final class RemoteProtocol {
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(2);
}
-
+
private static final LifeCycleType[] VALUES = {
- PERMANENT, TEMPORARY,
+ PERMANENT, TEMPORARY,
};
public static LifeCycleType valueOf(
com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
@@ -209,14 +209,14 @@ public final class RemoteProtocol {
this.index = index;
this.value = value;
}
-
+
static {
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor();
}
-
+
// @@protoc_insertion_point(enum_scope:LifeCycleType)
}
-
+
public static final class RemoteActorRefProtocol extends
com.google.protobuf.GeneratedMessage {
// Use RemoteActorRefProtocol.newBuilder() to construct.
@@ -224,54 +224,54 @@ public final class RemoteProtocol {
initFields();
}
private RemoteActorRefProtocol(boolean noInit) {}
-
+
private static final RemoteActorRefProtocol defaultInstance;
public static RemoteActorRefProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public RemoteActorRefProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_fieldAccessorTable;
}
-
+
// required string uuid = 1;
public static final int UUID_FIELD_NUMBER = 1;
private boolean hasUuid;
private java.lang.String uuid_ = "";
public boolean hasUuid() { return hasUuid; }
public java.lang.String getUuid() { return uuid_; }
-
+
// required string actorClassname = 2;
public static final int ACTORCLASSNAME_FIELD_NUMBER = 2;
private boolean hasActorClassname;
private java.lang.String actorClassname_ = "";
public boolean hasActorClassname() { return hasActorClassname; }
public java.lang.String getActorClassname() { return actorClassname_; }
-
+
// required .AddressProtocol homeAddress = 3;
public static final int HOMEADDRESS_FIELD_NUMBER = 3;
private boolean hasHomeAddress;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol homeAddress_;
public boolean hasHomeAddress() { return hasHomeAddress; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { return homeAddress_; }
-
+
// optional uint64 timeout = 4;
public static final int TIMEOUT_FIELD_NUMBER = 4;
private boolean hasTimeout;
private long timeout_ = 0L;
public boolean hasTimeout() { return hasTimeout; }
public long getTimeout() { return timeout_; }
-
+
private void initFields() {
homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
}
@@ -282,7 +282,7 @@ public final class RemoteProtocol {
if (!getHomeAddress().isInitialized()) return false;
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -300,12 +300,12 @@ public final class RemoteProtocol {
}
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasUuid()) {
size += com.google.protobuf.CodedOutputStream
@@ -327,7 +327,7 @@ public final class RemoteProtocol {
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -394,31 +394,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -427,20 +427,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -450,7 +450,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -459,7 +459,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -469,7 +469,7 @@ public final class RemoteProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol)other);
@@ -478,7 +478,7 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) return this;
if (other.hasUuid()) {
@@ -496,7 +496,7 @@ public final class RemoteProtocol {
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -542,8 +542,8 @@ public final class RemoteProtocol {
}
}
}
-
-
+
+
// required string uuid = 1;
public boolean hasUuid() {
return result.hasUuid();
@@ -564,7 +564,7 @@ public final class RemoteProtocol {
result.uuid_ = getDefaultInstance().getUuid();
return this;
}
-
+
// required string actorClassname = 2;
public boolean hasActorClassname() {
return result.hasActorClassname();
@@ -585,7 +585,7 @@ public final class RemoteProtocol {
result.actorClassname_ = getDefaultInstance().getActorClassname();
return this;
}
-
+
// required .AddressProtocol homeAddress = 3;
public boolean hasHomeAddress() {
return result.hasHomeAddress();
@@ -622,7 +622,7 @@ public final class RemoteProtocol {
result.homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
return this;
}
-
+
// optional uint64 timeout = 4;
public boolean hasTimeout() {
return result.hasTimeout();
@@ -640,19 +640,19 @@ public final class RemoteProtocol {
result.timeout_ = 0L;
return this;
}
-
+
// @@protoc_insertion_point(builder_scope:RemoteActorRefProtocol)
}
-
+
static {
defaultInstance = new RemoteActorRefProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:RemoteActorRefProtocol)
}
-
+
public static final class SerializedActorRefProtocol extends
com.google.protobuf.GeneratedMessage {
// Use SerializedActorRefProtocol.newBuilder() to construct.
@@ -660,110 +660,110 @@ public final class RemoteProtocol {
initFields();
}
private SerializedActorRefProtocol(boolean noInit) {}
-
+
private static final SerializedActorRefProtocol defaultInstance;
public static SerializedActorRefProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public SerializedActorRefProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_fieldAccessorTable;
}
-
+
// required string uuid = 1;
public static final int UUID_FIELD_NUMBER = 1;
private boolean hasUuid;
private java.lang.String uuid_ = "";
public boolean hasUuid() { return hasUuid; }
public java.lang.String getUuid() { return uuid_; }
-
+
// required string id = 2;
public static final int ID_FIELD_NUMBER = 2;
private boolean hasId;
private java.lang.String id_ = "";
public boolean hasId() { return hasId; }
public java.lang.String getId() { return id_; }
-
+
// required string actorClassname = 3;
public static final int ACTORCLASSNAME_FIELD_NUMBER = 3;
private boolean hasActorClassname;
private java.lang.String actorClassname_ = "";
public boolean hasActorClassname() { return hasActorClassname; }
public java.lang.String getActorClassname() { return actorClassname_; }
-
+
// required .AddressProtocol originalAddress = 4;
public static final int ORIGINALADDRESS_FIELD_NUMBER = 4;
private boolean hasOriginalAddress;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol originalAddress_;
public boolean hasOriginalAddress() { return hasOriginalAddress; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { return originalAddress_; }
-
+
// optional bytes actorInstance = 5;
public static final int ACTORINSTANCE_FIELD_NUMBER = 5;
private boolean hasActorInstance;
private com.google.protobuf.ByteString actorInstance_ = com.google.protobuf.ByteString.EMPTY;
public boolean hasActorInstance() { return hasActorInstance; }
public com.google.protobuf.ByteString getActorInstance() { return actorInstance_; }
-
+
// optional string serializerClassname = 6;
public static final int SERIALIZERCLASSNAME_FIELD_NUMBER = 6;
private boolean hasSerializerClassname;
private java.lang.String serializerClassname_ = "";
public boolean hasSerializerClassname() { return hasSerializerClassname; }
public java.lang.String getSerializerClassname() { return serializerClassname_; }
-
+
// optional bool isTransactor = 7;
public static final int ISTRANSACTOR_FIELD_NUMBER = 7;
private boolean hasIsTransactor;
private boolean isTransactor_ = false;
public boolean hasIsTransactor() { return hasIsTransactor; }
public boolean getIsTransactor() { return isTransactor_; }
-
+
// optional 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 uint64 receiveTimeout = 9;
public static final int RECEIVETIMEOUT_FIELD_NUMBER = 9;
private boolean hasReceiveTimeout;
private long receiveTimeout_ = 0L;
public boolean hasReceiveTimeout() { return hasReceiveTimeout; }
public long getReceiveTimeout() { return receiveTimeout_; }
-
+
// optional .LifeCycleProtocol lifeCycle = 10;
public static final int LIFECYCLE_FIELD_NUMBER = 10;
private boolean hasLifeCycle;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol lifeCycle_;
public boolean hasLifeCycle() { return hasLifeCycle; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { return lifeCycle_; }
-
+
// optional .RemoteActorRefProtocol supervisor = 11;
public static final int SUPERVISOR_FIELD_NUMBER = 11;
private boolean hasSupervisor;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol supervisor_;
public boolean hasSupervisor() { return hasSupervisor; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { return supervisor_; }
-
+
// optional bytes hotswapStack = 12;
public static final int HOTSWAPSTACK_FIELD_NUMBER = 12;
private boolean hasHotswapStack;
private com.google.protobuf.ByteString hotswapStack_ = com.google.protobuf.ByteString.EMPTY;
public boolean hasHotswapStack() { return hasHotswapStack; }
public com.google.protobuf.ByteString getHotswapStack() { return hotswapStack_; }
-
+
// repeated .RemoteRequestProtocol messages = 13;
public static final int MESSAGES_FIELD_NUMBER = 13;
private java.util.List messages_ =
@@ -775,7 +775,7 @@ public final class RemoteProtocol {
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) {
return messages_.get(index);
}
-
+
private void initFields() {
originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
@@ -798,7 +798,7 @@ public final class RemoteProtocol {
}
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -843,12 +843,12 @@ public final class RemoteProtocol {
}
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasUuid()) {
size += com.google.protobuf.CodedOutputStream
@@ -906,7 +906,7 @@ public final class RemoteProtocol {
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -973,31 +973,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -1006,20 +1006,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -1029,7 +1029,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -1038,7 +1038,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -1052,7 +1052,7 @@ public final class RemoteProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol)other);
@@ -1061,7 +1061,7 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) return this;
if (other.hasUuid()) {
@@ -1109,7 +1109,7 @@ public final class RemoteProtocol {
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -1203,8 +1203,8 @@ public final class RemoteProtocol {
}
}
}
-
-
+
+
// required string uuid = 1;
public boolean hasUuid() {
return result.hasUuid();
@@ -1225,7 +1225,7 @@ public final class RemoteProtocol {
result.uuid_ = getDefaultInstance().getUuid();
return this;
}
-
+
// required string id = 2;
public boolean hasId() {
return result.hasId();
@@ -1246,7 +1246,7 @@ public final class RemoteProtocol {
result.id_ = getDefaultInstance().getId();
return this;
}
-
+
// required string actorClassname = 3;
public boolean hasActorClassname() {
return result.hasActorClassname();
@@ -1267,7 +1267,7 @@ public final class RemoteProtocol {
result.actorClassname_ = getDefaultInstance().getActorClassname();
return this;
}
-
+
// required .AddressProtocol originalAddress = 4;
public boolean hasOriginalAddress() {
return result.hasOriginalAddress();
@@ -1304,7 +1304,7 @@ public final class RemoteProtocol {
result.originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
return this;
}
-
+
// optional bytes actorInstance = 5;
public boolean hasActorInstance() {
return result.hasActorInstance();
@@ -1325,7 +1325,7 @@ public final class RemoteProtocol {
result.actorInstance_ = getDefaultInstance().getActorInstance();
return this;
}
-
+
// optional string serializerClassname = 6;
public boolean hasSerializerClassname() {
return result.hasSerializerClassname();
@@ -1346,7 +1346,7 @@ public final class RemoteProtocol {
result.serializerClassname_ = getDefaultInstance().getSerializerClassname();
return this;
}
-
+
// optional bool isTransactor = 7;
public boolean hasIsTransactor() {
return result.hasIsTransactor();
@@ -1364,7 +1364,7 @@ public final class RemoteProtocol {
result.isTransactor_ = false;
return this;
}
-
+
// optional uint64 timeout = 8;
public boolean hasTimeout() {
return result.hasTimeout();
@@ -1382,7 +1382,7 @@ public final class RemoteProtocol {
result.timeout_ = 0L;
return this;
}
-
+
// optional uint64 receiveTimeout = 9;
public boolean hasReceiveTimeout() {
return result.hasReceiveTimeout();
@@ -1400,7 +1400,7 @@ public final class RemoteProtocol {
result.receiveTimeout_ = 0L;
return this;
}
-
+
// optional .LifeCycleProtocol lifeCycle = 10;
public boolean hasLifeCycle() {
return result.hasLifeCycle();
@@ -1437,7 +1437,7 @@ public final class RemoteProtocol {
result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
return this;
}
-
+
// optional .RemoteActorRefProtocol supervisor = 11;
public boolean hasSupervisor() {
return result.hasSupervisor();
@@ -1474,7 +1474,7 @@ public final class RemoteProtocol {
result.supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
return this;
}
-
+
// optional bytes hotswapStack = 12;
public boolean hasHotswapStack() {
return result.hasHotswapStack();
@@ -1495,7 +1495,7 @@ public final class RemoteProtocol {
result.hotswapStack_ = getDefaultInstance().getHotswapStack();
return this;
}
-
+
// repeated .RemoteRequestProtocol messages = 13;
public java.util.List getMessagesList() {
return java.util.Collections.unmodifiableList(result.messages_);
@@ -1546,19 +1546,19 @@ public final class RemoteProtocol {
result.messages_ = java.util.Collections.emptyList();
return this;
}
-
+
// @@protoc_insertion_point(builder_scope:SerializedActorRefProtocol)
}
-
+
static {
defaultInstance = new SerializedActorRefProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:SerializedActorRefProtocol)
}
-
+
public static final class MessageProtocol extends
com.google.protobuf.GeneratedMessage {
// Use MessageProtocol.newBuilder() to construct.
@@ -1566,47 +1566,47 @@ public final class RemoteProtocol {
initFields();
}
private MessageProtocol(boolean noInit) {}
-
+
private static final MessageProtocol defaultInstance;
public static MessageProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public MessageProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_fieldAccessorTable;
}
-
+
// required .SerializationSchemeType serializationScheme = 1;
public static final int SERIALIZATIONSCHEME_FIELD_NUMBER = 1;
private boolean hasSerializationScheme;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType serializationScheme_;
public boolean hasSerializationScheme() { return hasSerializationScheme; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { return serializationScheme_; }
-
+
// required bytes message = 2;
public static final int MESSAGE_FIELD_NUMBER = 2;
private boolean hasMessage;
private com.google.protobuf.ByteString message_ = com.google.protobuf.ByteString.EMPTY;
public boolean hasMessage() { return hasMessage; }
public com.google.protobuf.ByteString getMessage() { return message_; }
-
+
// optional bytes messageManifest = 3;
public static final int MESSAGEMANIFEST_FIELD_NUMBER = 3;
private boolean hasMessageManifest;
private com.google.protobuf.ByteString messageManifest_ = com.google.protobuf.ByteString.EMPTY;
public boolean hasMessageManifest() { return hasMessageManifest; }
public com.google.protobuf.ByteString getMessageManifest() { return messageManifest_; }
-
+
private void initFields() {
serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA;
}
@@ -1615,7 +1615,7 @@ public final class RemoteProtocol {
if (!hasMessage) return false;
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -1630,12 +1630,12 @@ public final class RemoteProtocol {
}
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasSerializationScheme()) {
size += com.google.protobuf.CodedOutputStream
@@ -1653,7 +1653,7 @@ public final class RemoteProtocol {
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -1720,31 +1720,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -1753,20 +1753,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -1776,7 +1776,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -1785,7 +1785,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -1795,7 +1795,7 @@ public final class RemoteProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol)other);
@@ -1804,7 +1804,7 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) return this;
if (other.hasSerializationScheme()) {
@@ -1819,7 +1819,7 @@ public final class RemoteProtocol {
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -1862,8 +1862,8 @@ public final class RemoteProtocol {
}
}
}
-
-
+
+
// required .SerializationSchemeType serializationScheme = 1;
public boolean hasSerializationScheme() {
return result.hasSerializationScheme();
@@ -1884,7 +1884,7 @@ public final class RemoteProtocol {
result.serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA;
return this;
}
-
+
// required bytes message = 2;
public boolean hasMessage() {
return result.hasMessage();
@@ -1905,7 +1905,7 @@ public final class RemoteProtocol {
result.message_ = getDefaultInstance().getMessage();
return this;
}
-
+
// optional bytes messageManifest = 3;
public boolean hasMessageManifest() {
return result.hasMessageManifest();
@@ -1926,19 +1926,19 @@ public final class RemoteProtocol {
result.messageManifest_ = getDefaultInstance().getMessageManifest();
return this;
}
-
+
// @@protoc_insertion_point(builder_scope:MessageProtocol)
}
-
+
static {
defaultInstance = new MessageProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:MessageProtocol)
}
-
+
public static final class ActorInfoProtocol extends
com.google.protobuf.GeneratedMessage {
// Use ActorInfoProtocol.newBuilder() to construct.
@@ -1946,61 +1946,68 @@ public final class RemoteProtocol {
initFields();
}
private ActorInfoProtocol(boolean noInit) {}
-
+
private static final ActorInfoProtocol defaultInstance;
public static ActorInfoProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public ActorInfoProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_fieldAccessorTable;
}
-
+
// required string uuid = 1;
public static final int UUID_FIELD_NUMBER = 1;
private boolean hasUuid;
private java.lang.String uuid_ = "";
public boolean hasUuid() { return hasUuid; }
public java.lang.String getUuid() { return uuid_; }
-
+
// required string target = 2;
public static final int TARGET_FIELD_NUMBER = 2;
private boolean hasTarget;
private java.lang.String target_ = "";
public boolean hasTarget() { return hasTarget; }
public java.lang.String getTarget() { return target_; }
-
+
// required uint64 timeout = 3;
public static final int TIMEOUT_FIELD_NUMBER = 3;
private boolean hasTimeout;
private long timeout_ = 0L;
public boolean hasTimeout() { return hasTimeout; }
public long getTimeout() { return timeout_; }
-
+
// required .ActorType actorType = 4;
public static final int ACTORTYPE_FIELD_NUMBER = 4;
private boolean hasActorType;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType actorType_;
public boolean hasActorType() { return hasActorType; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() { return actorType_; }
-
+
// optional .TypedActorInfoProtocol typedActorInfo = 5;
public static final int TYPEDACTORINFO_FIELD_NUMBER = 5;
private boolean hasTypedActorInfo;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol typedActorInfo_;
public boolean hasTypedActorInfo() { return hasTypedActorInfo; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { return typedActorInfo_; }
-
+
+ // optional string id = 6;
+ public static final int ID_FIELD_NUMBER = 6;
+ private boolean hasId;
+ private java.lang.String id_ = "";
+ public boolean hasId() { return hasId; }
+ public java.lang.String getId() { return id_; }
+
private void initFields() {
actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR;
typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
@@ -2015,7 +2022,7 @@ public final class RemoteProtocol {
}
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -2034,14 +2041,17 @@ public final class RemoteProtocol {
if (hasTypedActorInfo()) {
output.writeMessage(5, getTypedActorInfo());
}
+ if (hasId()) {
+ output.writeString(6, getId());
+ }
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasUuid()) {
size += com.google.protobuf.CodedOutputStream
@@ -2063,11 +2073,15 @@ public final class RemoteProtocol {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(5, getTypedActorInfo());
}
+ if (hasId()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(6, getId());
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -2134,31 +2148,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -2167,20 +2181,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -2190,7 +2204,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -2199,7 +2213,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -2209,7 +2223,7 @@ public final class RemoteProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol)other);
@@ -2218,7 +2232,7 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) return this;
if (other.hasUuid()) {
@@ -2236,10 +2250,13 @@ public final class RemoteProtocol {
if (other.hasTypedActorInfo()) {
mergeTypedActorInfo(other.getTypedActorInfo());
}
+ if (other.hasId()) {
+ setId(other.getId());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -2292,11 +2309,15 @@ public final class RemoteProtocol {
setTypedActorInfo(subBuilder.buildPartial());
break;
}
+ case 50: {
+ setId(input.readString());
+ break;
+ }
}
}
}
-
-
+
+
// required string uuid = 1;
public boolean hasUuid() {
return result.hasUuid();
@@ -2317,7 +2338,7 @@ public final class RemoteProtocol {
result.uuid_ = getDefaultInstance().getUuid();
return this;
}
-
+
// required string target = 2;
public boolean hasTarget() {
return result.hasTarget();
@@ -2338,7 +2359,7 @@ public final class RemoteProtocol {
result.target_ = getDefaultInstance().getTarget();
return this;
}
-
+
// required uint64 timeout = 3;
public boolean hasTimeout() {
return result.hasTimeout();
@@ -2356,7 +2377,7 @@ public final class RemoteProtocol {
result.timeout_ = 0L;
return this;
}
-
+
// required .ActorType actorType = 4;
public boolean hasActorType() {
return result.hasActorType();
@@ -2377,7 +2398,7 @@ public final class RemoteProtocol {
result.actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR;
return this;
}
-
+
// optional .TypedActorInfoProtocol typedActorInfo = 5;
public boolean hasTypedActorInfo() {
return result.hasTypedActorInfo();
@@ -2414,19 +2435,40 @@ public final class RemoteProtocol {
result.typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
return this;
}
-
+
+ // optional string id = 6;
+ public boolean hasId() {
+ return result.hasId();
+ }
+ public java.lang.String getId() {
+ return result.getId();
+ }
+ public Builder setId(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasId = true;
+ result.id_ = value;
+ return this;
+ }
+ public Builder clearId() {
+ result.hasId = false;
+ result.id_ = getDefaultInstance().getId();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:ActorInfoProtocol)
}
-
+
static {
defaultInstance = new ActorInfoProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:ActorInfoProtocol)
}
-
+
public static final class TypedActorInfoProtocol extends
com.google.protobuf.GeneratedMessage {
// Use TypedActorInfoProtocol.newBuilder() to construct.
@@ -2434,40 +2476,40 @@ public final class RemoteProtocol {
initFields();
}
private TypedActorInfoProtocol(boolean noInit) {}
-
+
private static final TypedActorInfoProtocol defaultInstance;
public static TypedActorInfoProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public TypedActorInfoProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_fieldAccessorTable;
}
-
+
// required string interface = 1;
public static final int INTERFACE_FIELD_NUMBER = 1;
private boolean hasInterface;
private java.lang.String interface_ = "";
public boolean hasInterface() { return hasInterface; }
public java.lang.String getInterface() { return interface_; }
-
+
// required string method = 2;
public static final int METHOD_FIELD_NUMBER = 2;
private boolean hasMethod;
private java.lang.String method_ = "";
public boolean hasMethod() { return hasMethod; }
public java.lang.String getMethod() { return method_; }
-
+
private void initFields() {
}
public final boolean isInitialized() {
@@ -2475,7 +2517,7 @@ public final class RemoteProtocol {
if (!hasMethod) return false;
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -2487,12 +2529,12 @@ public final class RemoteProtocol {
}
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasInterface()) {
size += com.google.protobuf.CodedOutputStream
@@ -2506,7 +2548,7 @@ public final class RemoteProtocol {
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -2573,31 +2615,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -2606,20 +2648,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -2629,7 +2671,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -2638,7 +2680,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -2648,7 +2690,7 @@ public final class RemoteProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol)other);
@@ -2657,7 +2699,7 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) return this;
if (other.hasInterface()) {
@@ -2669,7 +2711,7 @@ public final class RemoteProtocol {
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -2702,8 +2744,8 @@ public final class RemoteProtocol {
}
}
}
-
-
+
+
// required string interface = 1;
public boolean hasInterface() {
return result.hasInterface();
@@ -2724,7 +2766,7 @@ public final class RemoteProtocol {
result.interface_ = getDefaultInstance().getInterface();
return this;
}
-
+
// required string method = 2;
public boolean hasMethod() {
return result.hasMethod();
@@ -2745,19 +2787,19 @@ public final class RemoteProtocol {
result.method_ = getDefaultInstance().getMethod();
return this;
}
-
+
// @@protoc_insertion_point(builder_scope:TypedActorInfoProtocol)
}
-
+
static {
defaultInstance = new TypedActorInfoProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:TypedActorInfoProtocol)
}
-
+
public static final class RemoteRequestProtocol extends
com.google.protobuf.GeneratedMessage {
// Use RemoteRequestProtocol.newBuilder() to construct.
@@ -2765,68 +2807,80 @@ public final class RemoteProtocol {
initFields();
}
private RemoteRequestProtocol(boolean noInit) {}
-
+
private static final RemoteRequestProtocol defaultInstance;
public static RemoteRequestProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public RemoteRequestProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_fieldAccessorTable;
}
-
+
// required uint64 id = 1;
public static final int ID_FIELD_NUMBER = 1;
private boolean hasId;
private long id_ = 0L;
public boolean hasId() { return hasId; }
public long getId() { return id_; }
-
+
// required .MessageProtocol message = 2;
public static final int MESSAGE_FIELD_NUMBER = 2;
private boolean hasMessage;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_;
public boolean hasMessage() { return hasMessage; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; }
-
+
// required .ActorInfoProtocol actorInfo = 3;
public static final int ACTORINFO_FIELD_NUMBER = 3;
private boolean hasActorInfo;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol actorInfo_;
public boolean hasActorInfo() { return hasActorInfo; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { return actorInfo_; }
-
+
// required bool isOneWay = 4;
public static final int ISONEWAY_FIELD_NUMBER = 4;
private boolean hasIsOneWay;
private boolean isOneWay_ = false;
public boolean hasIsOneWay() { return hasIsOneWay; }
public boolean getIsOneWay() { return isOneWay_; }
-
+
// optional string supervisorUuid = 5;
public static final int SUPERVISORUUID_FIELD_NUMBER = 5;
private boolean hasSupervisorUuid;
private java.lang.String supervisorUuid_ = "";
public boolean hasSupervisorUuid() { return hasSupervisorUuid; }
public java.lang.String getSupervisorUuid() { return supervisorUuid_; }
-
+
// optional .RemoteActorRefProtocol sender = 6;
public static final int SENDER_FIELD_NUMBER = 6;
private boolean hasSender;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol sender_;
public boolean hasSender() { return hasSender; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { return sender_; }
-
+
+ // repeated .MetadataEntryProtocol metadata = 7;
+ public static final int METADATA_FIELD_NUMBER = 7;
+ private java.util.List metadata_ =
+ java.util.Collections.emptyList();
+ public java.util.List getMetadataList() {
+ return metadata_;
+ }
+ public int getMetadataCount() { return metadata_.size(); }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) {
+ return metadata_.get(index);
+ }
+
private void initFields() {
message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
@@ -2842,9 +2896,12 @@ public final class RemoteProtocol {
if (hasSender()) {
if (!getSender().isInitialized()) return false;
}
+ for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) {
+ if (!element.isInitialized()) return false;
+ }
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -2866,14 +2923,17 @@ public final class RemoteProtocol {
if (hasSender()) {
output.writeMessage(6, getSender());
}
+ for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) {
+ output.writeMessage(7, element);
+ }
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasId()) {
size += com.google.protobuf.CodedOutputStream
@@ -2899,11 +2959,15 @@ public final class RemoteProtocol {
size += com.google.protobuf.CodedOutputStream
.computeMessageSize(6, getSender());
}
+ for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(7, element);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -2970,31 +3034,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -3003,20 +3067,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -3026,7 +3090,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -3035,17 +3099,21 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
"build() has already been called on this Builder.");
}
+ if (result.metadata_ != java.util.Collections.EMPTY_LIST) {
+ result.metadata_ =
+ java.util.Collections.unmodifiableList(result.metadata_);
+ }
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol returnMe = result;
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol)other);
@@ -3054,7 +3122,7 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance()) return this;
if (other.hasId()) {
@@ -3075,10 +3143,16 @@ public final class RemoteProtocol {
if (other.hasSender()) {
mergeSender(other.getSender());
}
+ if (!other.metadata_.isEmpty()) {
+ if (result.metadata_.isEmpty()) {
+ result.metadata_ = new java.util.ArrayList();
+ }
+ result.metadata_.addAll(other.metadata_);
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -3139,11 +3213,17 @@ public final class RemoteProtocol {
setSender(subBuilder.buildPartial());
break;
}
+ case 58: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder();
+ input.readMessage(subBuilder, extensionRegistry);
+ addMetadata(subBuilder.buildPartial());
+ break;
+ }
}
}
}
-
-
+
+
// required uint64 id = 1;
public boolean hasId() {
return result.hasId();
@@ -3161,7 +3241,7 @@ public final class RemoteProtocol {
result.id_ = 0L;
return this;
}
-
+
// required .MessageProtocol message = 2;
public boolean hasMessage() {
return result.hasMessage();
@@ -3198,7 +3278,7 @@ public final class RemoteProtocol {
result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
return this;
}
-
+
// required .ActorInfoProtocol actorInfo = 3;
public boolean hasActorInfo() {
return result.hasActorInfo();
@@ -3235,7 +3315,7 @@ public final class RemoteProtocol {
result.actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
return this;
}
-
+
// required bool isOneWay = 4;
public boolean hasIsOneWay() {
return result.hasIsOneWay();
@@ -3253,7 +3333,7 @@ public final class RemoteProtocol {
result.isOneWay_ = false;
return this;
}
-
+
// optional string supervisorUuid = 5;
public boolean hasSupervisorUuid() {
return result.hasSupervisorUuid();
@@ -3274,7 +3354,7 @@ public final class RemoteProtocol {
result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid();
return this;
}
-
+
// optional .RemoteActorRefProtocol sender = 6;
public boolean hasSender() {
return result.hasSender();
@@ -3311,19 +3391,70 @@ public final class RemoteProtocol {
result.sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
return this;
}
-
+
+ // repeated .MetadataEntryProtocol metadata = 7;
+ public java.util.List getMetadataList() {
+ return java.util.Collections.unmodifiableList(result.metadata_);
+ }
+ public int getMetadataCount() {
+ return result.getMetadataCount();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) {
+ return result.getMetadata(index);
+ }
+ public Builder setMetadata(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.metadata_.set(index, value);
+ return this;
+ }
+ public Builder setMetadata(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) {
+ result.metadata_.set(index, builderForValue.build());
+ return this;
+ }
+ public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ if (result.metadata_.isEmpty()) {
+ result.metadata_ = new java.util.ArrayList();
+ }
+ result.metadata_.add(value);
+ return this;
+ }
+ public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) {
+ if (result.metadata_.isEmpty()) {
+ result.metadata_ = new java.util.ArrayList();
+ }
+ result.metadata_.add(builderForValue.build());
+ return this;
+ }
+ public Builder addAllMetadata(
+ java.lang.Iterable extends se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol> values) {
+ if (result.metadata_.isEmpty()) {
+ result.metadata_ = new java.util.ArrayList();
+ }
+ super.addAll(values, result.metadata_);
+ return this;
+ }
+ public Builder clearMetadata() {
+ result.metadata_ = java.util.Collections.emptyList();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:RemoteRequestProtocol)
}
-
+
static {
defaultInstance = new RemoteRequestProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:RemoteRequestProtocol)
}
-
+
public static final class RemoteReplyProtocol extends
com.google.protobuf.GeneratedMessage {
// Use RemoteReplyProtocol.newBuilder() to construct.
@@ -3331,68 +3462,80 @@ public final class RemoteProtocol {
initFields();
}
private RemoteReplyProtocol(boolean noInit) {}
-
+
private static final RemoteReplyProtocol defaultInstance;
public static RemoteReplyProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public RemoteReplyProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_fieldAccessorTable;
}
-
+
// required uint64 id = 1;
public static final int ID_FIELD_NUMBER = 1;
private boolean hasId;
private long id_ = 0L;
public boolean hasId() { return hasId; }
public long getId() { return id_; }
-
+
// optional .MessageProtocol message = 2;
public static final int MESSAGE_FIELD_NUMBER = 2;
private boolean hasMessage;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_;
public boolean hasMessage() { return hasMessage; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; }
-
+
// optional .ExceptionProtocol exception = 3;
public static final int EXCEPTION_FIELD_NUMBER = 3;
private boolean hasException;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol exception_;
public boolean hasException() { return hasException; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return exception_; }
-
+
// optional string supervisorUuid = 4;
public static final int SUPERVISORUUID_FIELD_NUMBER = 4;
private boolean hasSupervisorUuid;
private java.lang.String supervisorUuid_ = "";
public boolean hasSupervisorUuid() { return hasSupervisorUuid; }
public java.lang.String getSupervisorUuid() { return supervisorUuid_; }
-
+
// required bool isActor = 5;
public static final int ISACTOR_FIELD_NUMBER = 5;
private boolean hasIsActor;
private boolean isActor_ = false;
public boolean hasIsActor() { return hasIsActor; }
public boolean getIsActor() { return isActor_; }
-
+
// required bool isSuccessful = 6;
public static final int ISSUCCESSFUL_FIELD_NUMBER = 6;
private boolean hasIsSuccessful;
private boolean isSuccessful_ = false;
public boolean hasIsSuccessful() { return hasIsSuccessful; }
public boolean getIsSuccessful() { return isSuccessful_; }
-
+
+ // repeated .MetadataEntryProtocol metadata = 7;
+ public static final int METADATA_FIELD_NUMBER = 7;
+ private java.util.List metadata_ =
+ java.util.Collections.emptyList();
+ public java.util.List getMetadataList() {
+ return metadata_;
+ }
+ public int getMetadataCount() { return metadata_.size(); }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) {
+ return metadata_.get(index);
+ }
+
private void initFields() {
message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
@@ -3407,9 +3550,12 @@ public final class RemoteProtocol {
if (hasException()) {
if (!getException().isInitialized()) return false;
}
+ for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) {
+ if (!element.isInitialized()) return false;
+ }
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -3431,14 +3577,17 @@ public final class RemoteProtocol {
if (hasIsSuccessful()) {
output.writeBool(6, getIsSuccessful());
}
+ for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) {
+ output.writeMessage(7, element);
+ }
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasId()) {
size += com.google.protobuf.CodedOutputStream
@@ -3464,11 +3613,15 @@ public final class RemoteProtocol {
size += com.google.protobuf.CodedOutputStream
.computeBoolSize(6, getIsSuccessful());
}
+ for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(7, element);
+ }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -3535,31 +3688,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -3568,20 +3721,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -3591,7 +3744,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -3600,17 +3753,21 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
"build() has already been called on this Builder.");
}
+ if (result.metadata_ != java.util.Collections.EMPTY_LIST) {
+ result.metadata_ =
+ java.util.Collections.unmodifiableList(result.metadata_);
+ }
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol returnMe = result;
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol)other);
@@ -3619,7 +3776,7 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance()) return this;
if (other.hasId()) {
@@ -3640,10 +3797,16 @@ public final class RemoteProtocol {
if (other.hasIsSuccessful()) {
setIsSuccessful(other.getIsSuccessful());
}
+ if (!other.metadata_.isEmpty()) {
+ if (result.metadata_.isEmpty()) {
+ result.metadata_ = new java.util.ArrayList();
+ }
+ result.metadata_.addAll(other.metadata_);
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -3699,11 +3862,17 @@ public final class RemoteProtocol {
setIsSuccessful(input.readBool());
break;
}
+ case 58: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder();
+ input.readMessage(subBuilder, extensionRegistry);
+ addMetadata(subBuilder.buildPartial());
+ break;
+ }
}
}
}
-
-
+
+
// required uint64 id = 1;
public boolean hasId() {
return result.hasId();
@@ -3721,7 +3890,7 @@ public final class RemoteProtocol {
result.id_ = 0L;
return this;
}
-
+
// optional .MessageProtocol message = 2;
public boolean hasMessage() {
return result.hasMessage();
@@ -3758,7 +3927,7 @@ public final class RemoteProtocol {
result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
return this;
}
-
+
// optional .ExceptionProtocol exception = 3;
public boolean hasException() {
return result.hasException();
@@ -3795,7 +3964,7 @@ public final class RemoteProtocol {
result.exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
return this;
}
-
+
// optional string supervisorUuid = 4;
public boolean hasSupervisorUuid() {
return result.hasSupervisorUuid();
@@ -3816,7 +3985,7 @@ public final class RemoteProtocol {
result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid();
return this;
}
-
+
// required bool isActor = 5;
public boolean hasIsActor() {
return result.hasIsActor();
@@ -3834,7 +4003,7 @@ public final class RemoteProtocol {
result.isActor_ = false;
return this;
}
-
+
// required bool isSuccessful = 6;
public boolean hasIsSuccessful() {
return result.hasIsSuccessful();
@@ -3852,19 +4021,726 @@ public final class RemoteProtocol {
result.isSuccessful_ = false;
return this;
}
-
+
+ // repeated .MetadataEntryProtocol metadata = 7;
+ public java.util.List getMetadataList() {
+ return java.util.Collections.unmodifiableList(result.metadata_);
+ }
+ public int getMetadataCount() {
+ return result.getMetadataCount();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) {
+ return result.getMetadata(index);
+ }
+ public Builder setMetadata(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.metadata_.set(index, value);
+ return this;
+ }
+ public Builder setMetadata(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) {
+ result.metadata_.set(index, builderForValue.build());
+ return this;
+ }
+ public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ if (result.metadata_.isEmpty()) {
+ result.metadata_ = new java.util.ArrayList();
+ }
+ result.metadata_.add(value);
+ return this;
+ }
+ public Builder addMetadata(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) {
+ if (result.metadata_.isEmpty()) {
+ result.metadata_ = new java.util.ArrayList();
+ }
+ result.metadata_.add(builderForValue.build());
+ return this;
+ }
+ public Builder addAllMetadata(
+ java.lang.Iterable extends se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol> values) {
+ if (result.metadata_.isEmpty()) {
+ result.metadata_ = new java.util.ArrayList();
+ }
+ super.addAll(values, result.metadata_);
+ return this;
+ }
+ public Builder clearMetadata() {
+ result.metadata_ = java.util.Collections.emptyList();
+ return this;
+ }
+
// @@protoc_insertion_point(builder_scope:RemoteReplyProtocol)
}
-
+
static {
defaultInstance = new RemoteReplyProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:RemoteReplyProtocol)
}
-
+
+ public static final class UuidProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use UuidProtocol.newBuilder() to construct.
+ private UuidProtocol() {
+ initFields();
+ }
+ private UuidProtocol(boolean noInit) {}
+
+ private static final UuidProtocol defaultInstance;
+ public static UuidProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public UuidProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_UuidProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_UuidProtocol_fieldAccessorTable;
+ }
+
+ // required uint64 high = 1;
+ public static final int HIGH_FIELD_NUMBER = 1;
+ private boolean hasHigh;
+ private long high_ = 0L;
+ public boolean hasHigh() { return hasHigh; }
+ public long getHigh() { return high_; }
+
+ // required uint64 low = 2;
+ public static final int LOW_FIELD_NUMBER = 2;
+ private boolean hasLow;
+ private long low_ = 0L;
+ public boolean hasLow() { return hasLow; }
+ public long getLow() { return low_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ if (!hasHigh) return false;
+ if (!hasLow) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasHigh()) {
+ output.writeUInt64(1, getHigh());
+ }
+ if (hasLow()) {
+ output.writeUInt64(2, getLow());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasHigh()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(1, getHigh());
+ }
+ if (hasLow()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeUInt64Size(2, getLow());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) return this;
+ if (other.hasHigh()) {
+ setHigh(other.getHigh());
+ }
+ if (other.hasLow()) {
+ setLow(other.getLow());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 8: {
+ setHigh(input.readUInt64());
+ break;
+ }
+ case 16: {
+ setLow(input.readUInt64());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required uint64 high = 1;
+ public boolean hasHigh() {
+ return result.hasHigh();
+ }
+ public long getHigh() {
+ return result.getHigh();
+ }
+ public Builder setHigh(long value) {
+ result.hasHigh = true;
+ result.high_ = value;
+ return this;
+ }
+ public Builder clearHigh() {
+ result.hasHigh = false;
+ result.high_ = 0L;
+ return this;
+ }
+
+ // required uint64 low = 2;
+ public boolean hasLow() {
+ return result.hasLow();
+ }
+ public long getLow() {
+ return result.getLow();
+ }
+ public Builder setLow(long value) {
+ result.hasLow = true;
+ result.low_ = value;
+ return this;
+ }
+ public Builder clearLow() {
+ result.hasLow = false;
+ result.low_ = 0L;
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:UuidProtocol)
+ }
+
+ static {
+ defaultInstance = new UuidProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:UuidProtocol)
+ }
+
+ public static final class MetadataEntryProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use MetadataEntryProtocol.newBuilder() to construct.
+ private MetadataEntryProtocol() {
+ initFields();
+ }
+ private MetadataEntryProtocol(boolean noInit) {}
+
+ private static final MetadataEntryProtocol defaultInstance;
+ public static MetadataEntryProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public MetadataEntryProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MetadataEntryProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MetadataEntryProtocol_fieldAccessorTable;
+ }
+
+ // required string key = 1;
+ public static final int KEY_FIELD_NUMBER = 1;
+ private boolean hasKey;
+ private java.lang.String key_ = "";
+ public boolean hasKey() { return hasKey; }
+ public java.lang.String getKey() { return key_; }
+
+ // required bytes value = 2;
+ public static final int VALUE_FIELD_NUMBER = 2;
+ private boolean hasValue;
+ private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY;
+ public boolean hasValue() { return hasValue; }
+ public com.google.protobuf.ByteString getValue() { return value_; }
+
+ private void initFields() {
+ }
+ public final boolean isInitialized() {
+ if (!hasKey) return false;
+ if (!hasValue) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasKey()) {
+ output.writeString(1, getKey());
+ }
+ if (hasValue()) {
+ output.writeBytes(2, getValue());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasKey()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(1, getKey());
+ }
+ if (hasValue()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeBytesSize(2, getValue());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(
+ com.google.protobuf.ByteString data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(
+ byte[] data,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseDelimitedFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseDelimitedFrom(
+ java.io.InputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ Builder builder = newBuilder();
+ if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
+ return builder.buildParsed();
+ } else {
+ return null;
+ }
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input, extensionRegistry)
+ .buildParsed();
+ }
+
+ public static Builder newBuilder() { return Builder.create(); }
+ public Builder newBuilderForType() { return newBuilder(); }
+ public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol prototype) {
+ return newBuilder().mergeFrom(prototype);
+ }
+ public Builder toBuilder() { return newBuilder(this); }
+
+ public static final class Builder extends
+ com.google.protobuf.GeneratedMessage.Builder {
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol internalGetResult() {
+ return result;
+ }
+
+ public Builder clear() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "Cannot call clear() after build().");
+ }
+ result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDefaultInstance()) return this;
+ if (other.hasKey()) {
+ setKey(other.getKey());
+ }
+ if (other.hasValue()) {
+ setValue(other.getValue());
+ }
+ this.mergeUnknownFields(other.getUnknownFields());
+ return this;
+ }
+
+ public Builder mergeFrom(
+ com.google.protobuf.CodedInputStream input,
+ com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+ throws java.io.IOException {
+ com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+ com.google.protobuf.UnknownFieldSet.newBuilder(
+ this.getUnknownFields());
+ while (true) {
+ int tag = input.readTag();
+ switch (tag) {
+ case 0:
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ default: {
+ if (!parseUnknownField(input, unknownFields,
+ extensionRegistry, tag)) {
+ this.setUnknownFields(unknownFields.build());
+ return this;
+ }
+ break;
+ }
+ case 10: {
+ setKey(input.readString());
+ break;
+ }
+ case 18: {
+ setValue(input.readBytes());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required string key = 1;
+ public boolean hasKey() {
+ return result.hasKey();
+ }
+ public java.lang.String getKey() {
+ return result.getKey();
+ }
+ public Builder setKey(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasKey = true;
+ result.key_ = value;
+ return this;
+ }
+ public Builder clearKey() {
+ result.hasKey = false;
+ result.key_ = getDefaultInstance().getKey();
+ return this;
+ }
+
+ // required bytes value = 2;
+ public boolean hasValue() {
+ return result.hasValue();
+ }
+ public com.google.protobuf.ByteString getValue() {
+ return result.getValue();
+ }
+ public Builder setValue(com.google.protobuf.ByteString value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasValue = true;
+ result.value_ = value;
+ return this;
+ }
+ public Builder clearValue() {
+ result.hasValue = false;
+ result.value_ = getDefaultInstance().getValue();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:MetadataEntryProtocol)
+ }
+
+ static {
+ defaultInstance = new MetadataEntryProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:MetadataEntryProtocol)
+ }
+
public static final class LifeCycleProtocol extends
com.google.protobuf.GeneratedMessage {
// Use LifeCycleProtocol.newBuilder() to construct.
@@ -3872,61 +4748,33 @@ public final class RemoteProtocol {
initFields();
}
private LifeCycleProtocol(boolean noInit) {}
-
+
private static final LifeCycleProtocol defaultInstance;
public static LifeCycleProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public LifeCycleProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_fieldAccessorTable;
}
-
+
// required .LifeCycleType lifeCycle = 1;
public static final int LIFECYCLE_FIELD_NUMBER = 1;
private boolean hasLifeCycle;
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType lifeCycle_;
public boolean hasLifeCycle() { return hasLifeCycle; }
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { return lifeCycle_; }
-
- // optional string preRestart = 2;
- public static final int PRERESTART_FIELD_NUMBER = 2;
- private boolean hasPreRestart;
- private java.lang.String preRestart_ = "";
- public boolean hasPreRestart() { return hasPreRestart; }
- public java.lang.String getPreRestart() { return preRestart_; }
-
- // optional string postRestart = 3;
- public static final int POSTRESTART_FIELD_NUMBER = 3;
- private boolean hasPostRestart;
- private java.lang.String postRestart_ = "";
- public boolean hasPostRestart() { return hasPostRestart; }
- public java.lang.String getPostRestart() { return postRestart_; }
-
- // optional string init = 4;
- public static final int INIT_FIELD_NUMBER = 4;
- private boolean hasInit;
- private java.lang.String init_ = "";
- public boolean hasInit() { return hasInit; }
- public java.lang.String getInit() { return init_; }
-
- // optional string shutdown = 5;
- public static final int SHUTDOWN_FIELD_NUMBER = 5;
- private boolean hasShutdown;
- private java.lang.String shutdown_ = "";
- public boolean hasShutdown() { return hasShutdown; }
- public java.lang.String getShutdown() { return shutdown_; }
-
+
private void initFields() {
lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT;
}
@@ -3934,59 +4782,31 @@ public final class RemoteProtocol {
if (!hasLifeCycle) return false;
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
if (hasLifeCycle()) {
output.writeEnum(1, getLifeCycle().getNumber());
}
- if (hasPreRestart()) {
- output.writeString(2, getPreRestart());
- }
- if (hasPostRestart()) {
- output.writeString(3, getPostRestart());
- }
- if (hasInit()) {
- output.writeString(4, getInit());
- }
- if (hasShutdown()) {
- output.writeString(5, getShutdown());
- }
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasLifeCycle()) {
size += com.google.protobuf.CodedOutputStream
.computeEnumSize(1, getLifeCycle().getNumber());
}
- if (hasPreRestart()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(2, getPreRestart());
- }
- if (hasPostRestart()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(3, getPostRestart());
- }
- if (hasInit()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(4, getInit());
- }
- if (hasShutdown()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(5, getShutdown());
- }
size += getUnknownFields().getSerializedSize();
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -4053,31 +4873,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -4086,20 +4906,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -4109,7 +4929,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -4118,7 +4938,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -4128,7 +4948,7 @@ public final class RemoteProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol)other);
@@ -4137,28 +4957,16 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) return this;
if (other.hasLifeCycle()) {
setLifeCycle(other.getLifeCycle());
}
- if (other.hasPreRestart()) {
- setPreRestart(other.getPreRestart());
- }
- if (other.hasPostRestart()) {
- setPostRestart(other.getPostRestart());
- }
- if (other.hasInit()) {
- setInit(other.getInit());
- }
- if (other.hasShutdown()) {
- setShutdown(other.getShutdown());
- }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -4190,27 +4998,11 @@ public final class RemoteProtocol {
}
break;
}
- case 18: {
- setPreRestart(input.readString());
- break;
- }
- case 26: {
- setPostRestart(input.readString());
- break;
- }
- case 34: {
- setInit(input.readString());
- break;
- }
- case 42: {
- setShutdown(input.readString());
- break;
- }
}
}
}
-
-
+
+
// required .LifeCycleType lifeCycle = 1;
public boolean hasLifeCycle() {
return result.hasLifeCycle();
@@ -4231,103 +5023,19 @@ public final class RemoteProtocol {
result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT;
return this;
}
-
- // optional string preRestart = 2;
- public boolean hasPreRestart() {
- return result.hasPreRestart();
- }
- public java.lang.String getPreRestart() {
- return result.getPreRestart();
- }
- public Builder setPreRestart(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasPreRestart = true;
- result.preRestart_ = value;
- return this;
- }
- public Builder clearPreRestart() {
- result.hasPreRestart = false;
- result.preRestart_ = getDefaultInstance().getPreRestart();
- return this;
- }
-
- // optional string postRestart = 3;
- public boolean hasPostRestart() {
- return result.hasPostRestart();
- }
- public java.lang.String getPostRestart() {
- return result.getPostRestart();
- }
- public Builder setPostRestart(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasPostRestart = true;
- result.postRestart_ = value;
- return this;
- }
- public Builder clearPostRestart() {
- result.hasPostRestart = false;
- result.postRestart_ = getDefaultInstance().getPostRestart();
- return this;
- }
-
- // optional string init = 4;
- public boolean hasInit() {
- return result.hasInit();
- }
- public java.lang.String getInit() {
- return result.getInit();
- }
- public Builder setInit(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasInit = true;
- result.init_ = value;
- return this;
- }
- public Builder clearInit() {
- result.hasInit = false;
- result.init_ = getDefaultInstance().getInit();
- return this;
- }
-
- // optional string shutdown = 5;
- public boolean hasShutdown() {
- return result.hasShutdown();
- }
- public java.lang.String getShutdown() {
- return result.getShutdown();
- }
- public Builder setShutdown(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasShutdown = true;
- result.shutdown_ = value;
- return this;
- }
- public Builder clearShutdown() {
- result.hasShutdown = false;
- result.shutdown_ = getDefaultInstance().getShutdown();
- return this;
- }
-
+
// @@protoc_insertion_point(builder_scope:LifeCycleProtocol)
}
-
+
static {
defaultInstance = new LifeCycleProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:LifeCycleProtocol)
}
-
+
public static final class AddressProtocol extends
com.google.protobuf.GeneratedMessage {
// Use AddressProtocol.newBuilder() to construct.
@@ -4335,40 +5043,40 @@ public final class RemoteProtocol {
initFields();
}
private AddressProtocol(boolean noInit) {}
-
+
private static final AddressProtocol defaultInstance;
public static AddressProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public AddressProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_fieldAccessorTable;
}
-
+
// required string hostname = 1;
public static final int HOSTNAME_FIELD_NUMBER = 1;
private boolean hasHostname;
private java.lang.String hostname_ = "";
public boolean hasHostname() { return hasHostname; }
public java.lang.String getHostname() { return hostname_; }
-
+
// required uint32 port = 2;
public static final int PORT_FIELD_NUMBER = 2;
private boolean hasPort;
private int port_ = 0;
public boolean hasPort() { return hasPort; }
public int getPort() { return port_; }
-
+
private void initFields() {
}
public final boolean isInitialized() {
@@ -4376,7 +5084,7 @@ public final class RemoteProtocol {
if (!hasPort) return false;
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -4388,12 +5096,12 @@ public final class RemoteProtocol {
}
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasHostname()) {
size += com.google.protobuf.CodedOutputStream
@@ -4407,7 +5115,7 @@ public final class RemoteProtocol {
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -4474,31 +5182,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -4507,20 +5215,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -4530,7 +5238,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -4539,7 +5247,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -4549,7 +5257,7 @@ public final class RemoteProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol)other);
@@ -4558,7 +5266,7 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) return this;
if (other.hasHostname()) {
@@ -4570,7 +5278,7 @@ public final class RemoteProtocol {
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -4603,8 +5311,8 @@ public final class RemoteProtocol {
}
}
}
-
-
+
+
// required string hostname = 1;
public boolean hasHostname() {
return result.hasHostname();
@@ -4625,7 +5333,7 @@ public final class RemoteProtocol {
result.hostname_ = getDefaultInstance().getHostname();
return this;
}
-
+
// required uint32 port = 2;
public boolean hasPort() {
return result.hasPort();
@@ -4643,19 +5351,19 @@ public final class RemoteProtocol {
result.port_ = 0;
return this;
}
-
+
// @@protoc_insertion_point(builder_scope:AddressProtocol)
}
-
+
static {
defaultInstance = new AddressProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:AddressProtocol)
}
-
+
public static final class ExceptionProtocol extends
com.google.protobuf.GeneratedMessage {
// Use ExceptionProtocol.newBuilder() to construct.
@@ -4663,40 +5371,40 @@ public final class RemoteProtocol {
initFields();
}
private ExceptionProtocol(boolean noInit) {}
-
+
private static final ExceptionProtocol defaultInstance;
public static ExceptionProtocol getDefaultInstance() {
return defaultInstance;
}
-
+
public ExceptionProtocol getDefaultInstanceForType() {
return defaultInstance;
}
-
+
public static final com.google.protobuf.Descriptors.Descriptor
getDescriptor() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_descriptor;
}
-
+
protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
internalGetFieldAccessorTable() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_fieldAccessorTable;
}
-
+
// required string classname = 1;
public static final int CLASSNAME_FIELD_NUMBER = 1;
private boolean hasClassname;
private java.lang.String classname_ = "";
public boolean hasClassname() { return hasClassname; }
public java.lang.String getClassname() { return classname_; }
-
+
// required string message = 2;
public static final int MESSAGE_FIELD_NUMBER = 2;
private boolean hasMessage;
private java.lang.String message_ = "";
public boolean hasMessage() { return hasMessage; }
public java.lang.String getMessage() { return message_; }
-
+
private void initFields() {
}
public final boolean isInitialized() {
@@ -4704,7 +5412,7 @@ public final class RemoteProtocol {
if (!hasMessage) return false;
return true;
}
-
+
public void writeTo(com.google.protobuf.CodedOutputStream output)
throws java.io.IOException {
getSerializedSize();
@@ -4716,12 +5424,12 @@ public final class RemoteProtocol {
}
getUnknownFields().writeTo(output);
}
-
+
private int memoizedSerializedSize = -1;
public int getSerializedSize() {
int size = memoizedSerializedSize;
if (size != -1) return size;
-
+
size = 0;
if (hasClassname()) {
size += com.google.protobuf.CodedOutputStream
@@ -4735,7 +5443,7 @@ public final class RemoteProtocol {
memoizedSerializedSize = size;
return size;
}
-
+
public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
com.google.protobuf.ByteString data)
throws com.google.protobuf.InvalidProtocolBufferException {
@@ -4802,31 +5510,31 @@ public final class RemoteProtocol {
return newBuilder().mergeFrom(input, extensionRegistry)
.buildParsed();
}
-
+
public static Builder newBuilder() { return Builder.create(); }
public Builder newBuilderForType() { return newBuilder(); }
public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol prototype) {
return newBuilder().mergeFrom(prototype);
}
public Builder toBuilder() { return newBuilder(this); }
-
+
public static final class Builder extends
com.google.protobuf.GeneratedMessage.Builder {
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol result;
-
+
// Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder()
private Builder() {}
-
+
private static Builder create() {
Builder builder = new Builder();
builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol();
return builder;
}
-
+
protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol internalGetResult() {
return result;
}
-
+
public Builder clear() {
if (result == null) {
throw new IllegalStateException(
@@ -4835,20 +5543,20 @@ public final class RemoteProtocol {
result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol();
return this;
}
-
+
public Builder clone() {
return create().mergeFrom(result);
}
-
+
public com.google.protobuf.Descriptors.Descriptor
getDescriptorForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDescriptor();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getDefaultInstanceForType() {
return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
}
-
+
public boolean isInitialized() {
return result.isInitialized();
}
@@ -4858,7 +5566,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildParsed()
throws com.google.protobuf.InvalidProtocolBufferException {
if (!isInitialized()) {
@@ -4867,7 +5575,7 @@ public final class RemoteProtocol {
}
return buildPartial();
}
-
+
public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildPartial() {
if (result == null) {
throw new IllegalStateException(
@@ -4877,7 +5585,7 @@ public final class RemoteProtocol {
result = null;
return returnMe;
}
-
+
public Builder mergeFrom(com.google.protobuf.Message other) {
if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol) {
return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol)other);
@@ -4886,7 +5594,7 @@ public final class RemoteProtocol {
return this;
}
}
-
+
public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol other) {
if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) return this;
if (other.hasClassname()) {
@@ -4898,7 +5606,7 @@ public final class RemoteProtocol {
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
-
+
public Builder mergeFrom(
com.google.protobuf.CodedInputStream input,
com.google.protobuf.ExtensionRegistryLite extensionRegistry)
@@ -4931,8 +5639,8 @@ public final class RemoteProtocol {
}
}
}
-
-
+
+
// required string classname = 1;
public boolean hasClassname() {
return result.hasClassname();
@@ -4953,7 +5661,7 @@ public final class RemoteProtocol {
result.classname_ = getDefaultInstance().getClassname();
return this;
}
-
+
// required string message = 2;
public boolean hasMessage() {
return result.hasMessage();
@@ -4974,19 +5682,19 @@ public final class RemoteProtocol {
result.message_ = getDefaultInstance().getMessage();
return this;
}
-
+
// @@protoc_insertion_point(builder_scope:ExceptionProtocol)
}
-
+
static {
defaultInstance = new ExceptionProtocol(true);
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
defaultInstance.initFields();
}
-
+
// @@protoc_insertion_point(class_scope:ExceptionProtocol)
}
-
+
private static com.google.protobuf.Descriptors.Descriptor
internal_static_RemoteActorRefProtocol_descriptor;
private static
@@ -5022,6 +5730,16 @@ public final class RemoteProtocol {
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_RemoteReplyProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_UuidProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_UuidProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_MetadataEntryProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_MetadataEntryProtocol_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
internal_static_LifeCycleProtocol_descriptor;
private static
@@ -5037,7 +5755,7 @@ public final class RemoteProtocol {
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_ExceptionProtocol_fieldAccessorTable;
-
+
public static com.google.protobuf.Descriptors.FileDescriptor
getDescriptor() {
return descriptor;
@@ -5062,33 +5780,36 @@ public final class RemoteProtocol {
"tProtocol\"r\n\017MessageProtocol\0225\n\023serializ" +
"ationScheme\030\001 \002(\0162\030.SerializationSchemeT" +
"ype\022\017\n\007message\030\002 \002(\014\022\027\n\017messageManifest\030" +
- "\003 \001(\014\"\222\001\n\021ActorInfoProtocol\022\014\n\004uuid\030\001 \002(" +
+ "\003 \001(\014\"\236\001\n\021ActorInfoProtocol\022\014\n\004uuid\030\001 \002(" +
"\t\022\016\n\006target\030\002 \002(\t\022\017\n\007timeout\030\003 \002(\004\022\035\n\tac" +
"torType\030\004 \002(\0162\n.ActorType\022/\n\016typedActorI" +
- "nfo\030\005 \001(\0132\027.TypedActorInfoProtocol\";\n\026Ty",
- "pedActorInfoProtocol\022\021\n\tinterface\030\001 \002(\t\022" +
- "\016\n\006method\030\002 \002(\t\"\300\001\n\025RemoteRequestProtoco" +
- "l\022\n\n\002id\030\001 \002(\004\022!\n\007message\030\002 \002(\0132\020.Message" +
- "Protocol\022%\n\tactorInfo\030\003 \002(\0132\022.ActorInfoP" +
- "rotocol\022\020\n\010isOneWay\030\004 \002(\010\022\026\n\016supervisorU" +
- "uid\030\005 \001(\t\022\'\n\006sender\030\006 \001(\0132\027.RemoteActorR" +
- "efProtocol\"\252\001\n\023RemoteReplyProtocol\022\n\n\002id" +
- "\030\001 \002(\004\022!\n\007message\030\002 \001(\0132\020.MessageProtoco" +
- "l\022%\n\texception\030\003 \001(\0132\022.ExceptionProtocol" +
- "\022\026\n\016supervisorUuid\030\004 \001(\t\022\017\n\007isActor\030\005 \002(",
- "\010\022\024\n\014isSuccessful\030\006 \002(\010\"\177\n\021LifeCycleProt" +
- "ocol\022!\n\tlifeCycle\030\001 \002(\0162\016.LifeCycleType\022" +
- "\022\n\npreRestart\030\002 \001(\t\022\023\n\013postRestart\030\003 \001(\t" +
- "\022\014\n\004init\030\004 \001(\t\022\020\n\010shutdown\030\005 \001(\t\"1\n\017Addr" +
- "essProtocol\022\020\n\010hostname\030\001 \002(\t\022\014\n\004port\030\002 " +
- "\002(\r\"7\n\021ExceptionProtocol\022\021\n\tclassname\030\001 " +
- "\002(\t\022\017\n\007message\030\002 \002(\t*=\n\tActorType\022\017\n\013SCA" +
- "LA_ACTOR\020\001\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTO" +
- "R\020\003*]\n\027SerializationSchemeType\022\010\n\004JAVA\020\001" +
- "\022\013\n\007SBINARY\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JS",
- "ON\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tP" +
- "ERMANENT\020\001\022\r\n\tTEMPORARY\020\002B-\n)se.scalable" +
- "solutions.akka.remote.protocolH\001"
+ "nfo\030\005 \001(\0132\027.TypedActorInfoProtocol\022\n\n\002id",
+ "\030\006 \001(\t\";\n\026TypedActorInfoProtocol\022\021\n\tinte" +
+ "rface\030\001 \002(\t\022\016\n\006method\030\002 \002(\t\"\352\001\n\025RemoteRe" +
+ "questProtocol\022\n\n\002id\030\001 \002(\004\022!\n\007message\030\002 \002" +
+ "(\0132\020.MessageProtocol\022%\n\tactorInfo\030\003 \002(\0132" +
+ "\022.ActorInfoProtocol\022\020\n\010isOneWay\030\004 \002(\010\022\026\n" +
+ "\016supervisorUuid\030\005 \001(\t\022\'\n\006sender\030\006 \001(\0132\027." +
+ "RemoteActorRefProtocol\022(\n\010metadata\030\007 \003(\013" +
+ "2\026.MetadataEntryProtocol\"\324\001\n\023RemoteReply" +
+ "Protocol\022\n\n\002id\030\001 \002(\004\022!\n\007message\030\002 \001(\0132\020." +
+ "MessageProtocol\022%\n\texception\030\003 \001(\0132\022.Exc",
+ "eptionProtocol\022\026\n\016supervisorUuid\030\004 \001(\t\022\017" +
+ "\n\007isActor\030\005 \002(\010\022\024\n\014isSuccessful\030\006 \002(\010\022(\n" +
+ "\010metadata\030\007 \003(\0132\026.MetadataEntryProtocol\"" +
+ ")\n\014UuidProtocol\022\014\n\004high\030\001 \002(\004\022\013\n\003low\030\002 \002" +
+ "(\004\"3\n\025MetadataEntryProtocol\022\013\n\003key\030\001 \002(\t" +
+ "\022\r\n\005value\030\002 \002(\014\"6\n\021LifeCycleProtocol\022!\n\t" +
+ "lifeCycle\030\001 \002(\0162\016.LifeCycleType\"1\n\017Addre" +
+ "ssProtocol\022\020\n\010hostname\030\001 \002(\t\022\014\n\004port\030\002 \002" +
+ "(\r\"7\n\021ExceptionProtocol\022\021\n\tclassname\030\001 \002" +
+ "(\t\022\017\n\007message\030\002 \002(\t*=\n\tActorType\022\017\n\013SCAL",
+ "A_ACTOR\020\001\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTOR" +
+ "\020\003*]\n\027SerializationSchemeType\022\010\n\004JAVA\020\001\022" +
+ "\013\n\007SBINARY\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JSO" +
+ "N\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tPE" +
+ "RMANENT\020\001\022\r\n\tTEMPORARY\020\002B-\n)se.scalables" +
+ "olutions.akka.remote.protocolH\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -5124,7 +5845,7 @@ public final class RemoteProtocol {
internal_static_ActorInfoProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_ActorInfoProtocol_descriptor,
- new java.lang.String[] { "Uuid", "Target", "Timeout", "ActorType", "TypedActorInfo", },
+ new java.lang.String[] { "Uuid", "Target", "Timeout", "ActorType", "TypedActorInfo", "Id", },
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder.class);
internal_static_TypedActorInfoProtocol_descriptor =
@@ -5140,7 +5861,7 @@ public final class RemoteProtocol {
internal_static_RemoteRequestProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RemoteRequestProtocol_descriptor,
- new java.lang.String[] { "Id", "Message", "ActorInfo", "IsOneWay", "SupervisorUuid", "Sender", },
+ new java.lang.String[] { "Id", "Message", "ActorInfo", "IsOneWay", "SupervisorUuid", "Sender", "Metadata", },
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder.class);
internal_static_RemoteReplyProtocol_descriptor =
@@ -5148,19 +5869,35 @@ public final class RemoteProtocol {
internal_static_RemoteReplyProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RemoteReplyProtocol_descriptor,
- new java.lang.String[] { "Id", "Message", "Exception", "SupervisorUuid", "IsActor", "IsSuccessful", },
+ new java.lang.String[] { "Id", "Message", "Exception", "SupervisorUuid", "IsActor", "IsSuccessful", "Metadata", },
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.Builder.class);
- internal_static_LifeCycleProtocol_descriptor =
+ internal_static_UuidProtocol_descriptor =
getDescriptor().getMessageTypes().get(7);
+ internal_static_UuidProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_UuidProtocol_descriptor,
+ new java.lang.String[] { "High", "Low", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder.class);
+ internal_static_MetadataEntryProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(8);
+ internal_static_MetadataEntryProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_MetadataEntryProtocol_descriptor,
+ new java.lang.String[] { "Key", "Value", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder.class);
+ internal_static_LifeCycleProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(9);
internal_static_LifeCycleProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_LifeCycleProtocol_descriptor,
- new java.lang.String[] { "LifeCycle", "PreRestart", "PostRestart", "Init", "Shutdown", },
+ new java.lang.String[] { "LifeCycle", },
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder.class);
internal_static_AddressProtocol_descriptor =
- getDescriptor().getMessageTypes().get(8);
+ getDescriptor().getMessageTypes().get(10);
internal_static_AddressProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_AddressProtocol_descriptor,
@@ -5168,7 +5905,7 @@ public final class RemoteProtocol {
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder.class);
internal_static_ExceptionProtocol_descriptor =
- getDescriptor().getMessageTypes().get(9);
+ getDescriptor().getMessageTypes().get(11);
internal_static_ExceptionProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_ExceptionProtocol_descriptor,
@@ -5183,8 +5920,8 @@ public final class RemoteProtocol {
new com.google.protobuf.Descriptors.FileDescriptor[] {
}, assigner);
}
-
+
public static void internalForceInit() {}
-
+
// @@protoc_insertion_point(outer_class_scope)
}
diff --git a/akka-remote/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto
index 567bf54eba..4f2fa5c2dd 100644
--- a/akka-remote/src/main/protocol/RemoteProtocol.proto
+++ b/akka-remote/src/main/protocol/RemoteProtocol.proto
@@ -16,7 +16,7 @@ option optimize_for = SPEED;
* on the original node.
*/
message RemoteActorRefProtocol {
- required string uuid = 1;
+ required UuidProtocol uuid = 1;
required string actorClassname = 2;
required AddressProtocol homeAddress = 3;
optional uint64 timeout = 4;
@@ -28,7 +28,7 @@ message RemoteActorRefProtocol {
* from its original host.
*/
message SerializedActorRefProtocol {
- required string uuid = 1;
+ required UuidProtocol uuid = 1;
required string id = 2;
required string actorClassname = 3;
required AddressProtocol originalAddress = 4;
@@ -56,11 +56,12 @@ message MessageProtocol {
* Defines the actor info.
*/
message ActorInfoProtocol {
- required string uuid = 1;
+ required UuidProtocol uuid = 1;
required string target = 2;
required uint64 timeout = 3;
required ActorType actorType = 4;
optional TypedActorInfoProtocol typedActorInfo = 5;
+ optional string id = 6;
}
/**
@@ -75,24 +76,42 @@ message TypedActorInfoProtocol {
* Defines a remote message request.
*/
message RemoteRequestProtocol {
- required uint64 id = 1;
+ required UuidProtocol uuid = 1;
required MessageProtocol message = 2;
required ActorInfoProtocol actorInfo = 3;
required bool isOneWay = 4;
optional string supervisorUuid = 5;
optional RemoteActorRefProtocol sender = 6;
+ repeated MetadataEntryProtocol metadata = 7;
}
/**
* Defines a remote message reply.
*/
message RemoteReplyProtocol {
- required uint64 id = 1;
+ required UuidProtocol uuid = 1;
optional MessageProtocol message = 2;
optional ExceptionProtocol exception = 3;
optional string supervisorUuid = 4;
required bool isActor = 5;
required bool isSuccessful = 6;
+ repeated MetadataEntryProtocol metadata = 7;
+}
+
+/**
+ * Defines a UUID.
+ */
+message UuidProtocol {
+ required uint64 high = 1;
+ required uint64 low = 2;
+}
+
+/**
+ * Defines a meta data entry.
+ */
+message MetadataEntryProtocol {
+ required string key = 1;
+ required bytes value = 2;
}
/**
@@ -154,4 +173,4 @@ message AddressProtocol {
message ExceptionProtocol {
required string classname = 1;
required string message = 2;
-}
+}
\ No newline at end of file
diff --git a/akka-remote/src/main/scala/remote/Cluster.scala b/akka-remote/src/main/scala/remote/Cluster.scala
index 8a5864a51b..6e1e99f0b2 100644
--- a/akka-remote/src/main/scala/remote/Cluster.scala
+++ b/akka-remote/src/main/scala/remote/Cluster.scala
@@ -35,6 +35,11 @@ trait Cluster {
* The order of application is undefined and may vary
*/
def foreach(f: (RemoteAddress) => Unit): Unit
+
+ /**
+ * Returns all the endpoints in the cluster.
+ */
+ def endpoints: Array[RemoteAddress]
}
/**
@@ -81,11 +86,11 @@ abstract class BasicClusterActor extends ClusterActor with Logging {
@volatile private var local: Node = Node(Nil)
@volatile private var remotes: Map[ADDR_T, Node] = Map()
- override def init = {
+ override def preStart = {
remotes = new HashMap[ADDR_T, Node]
}
- override def shutdown = {
+ override def postStop = {
remotes = Map()
}
@@ -196,6 +201,11 @@ abstract class BasicClusterActor extends ClusterActor with Logging {
* Applies the given function to all remote addresses known
*/
def foreach(f: (RemoteAddress) => Unit): Unit = remotes.valuesIterator.toList.flatMap(_.endpoints).foreach(f)
+
+ /**
+ * Returns all the endpoints in the cluster.
+ */
+ def endpoints: Array[RemoteAddress] = remotes.toArray.asInstanceOf[Array[RemoteAddress]]
}
/**
@@ -211,7 +221,7 @@ object Cluster extends Cluster with Logging {
lazy val DEFAULT_CLUSTER_ACTOR_CLASS_NAME = classOf[JGroupsClusterActor].getName
@volatile private[remote] var clusterActorRef: Option[ActorRef] = None
- @volatile private[akka] var classLoader : Option[ClassLoader] = Some(getClass.getClassLoader)
+ @volatile private[akka] var classLoader: Option[ClassLoader] = Some(getClass.getClassLoader)
private[remote] def createClusterActor(): Option[ActorRef] = {
val name = config.getString("akka.remote.cluster.actor", DEFAULT_CLUSTER_ACTOR_CLASS_NAME)
@@ -233,7 +243,7 @@ object Cluster extends Cluster with Logging {
RestartStrategy(OneForOne, 5, 1000, List(classOf[Exception])),
Supervise(actor, LifeCycle(Permanent)) :: Nil)))
- private[this] def clusterActor = if(clusterActorRef.isEmpty) None else Some(clusterActorRef.get.actor.asInstanceOf[ClusterActor])
+ private[this] def clusterActor = if (clusterActorRef.isEmpty) None else Some(clusterActorRef.get.actor.asInstanceOf[ClusterActor])
def name = clusterActor.map(_.name).getOrElse("No cluster")
@@ -257,6 +267,10 @@ object Cluster extends Cluster with Logging {
def foreach(f: (RemoteAddress) => Unit): Unit = clusterActor.foreach(_.foreach(f))
+ def endpoints: Array[RemoteAddress] = clusterActor
+ .getOrElse(throw new IllegalStateException("No cluster actor is defined"))
+ .endpoints
+
def start(): Unit = start(None)
def start(serializerClassLoader: Option[ClassLoader]): Unit = synchronized {
diff --git a/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala b/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala
index 54ef3807d4..07cbf4d65b 100644
--- a/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala
+++ b/akka-remote/src/main/scala/remote/JGroupsClusterActor.scala
@@ -54,8 +54,8 @@ class JGroupsClusterActor extends BasicClusterActor {
protected def toAllNodes(msg : Array[Byte]): Unit =
for (c <- channel) c.send(new JG_MSG(null, null, msg))
- override def shutdown = {
- super.shutdown
+ override def postStop = {
+ super.postStop
log info ("Shutting down %s", toString)
isActive = false
channel.foreach(Util shutdown _)
diff --git a/akka-remote/src/main/scala/remote/RemoteClient.scala b/akka-remote/src/main/scala/remote/RemoteClient.scala
index 459c260a62..26cc275956 100644
--- a/akka-remote/src/main/scala/remote/RemoteClient.scala
+++ b/akka-remote/src/main/scala/remote/RemoteClient.scala
@@ -4,14 +4,14 @@
package se.scalablesolutions.akka.remote
-import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
-import se.scalablesolutions.akka.actor.{Exit, Actor, ActorRef, ActorType, RemoteActorRef, RemoteActorSerialization, IllegalActorStateException}
+import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _}
+import se.scalablesolutions.akka.actor.{Exit, Actor, ActorRef, ActorType, RemoteActorRef, IllegalActorStateException}
import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture}
import se.scalablesolutions.akka.util.{ListenerManagement, UUID, Logging, Duration}
import se.scalablesolutions.akka.config.Config._
+import se.scalablesolutions.akka.serialization.RemoteActorSerialization._
import se.scalablesolutions.akka.AkkaException
import Actor._
-import RemoteActorSerialization._
import org.jboss.netty.channel._
import group.DefaultChannelGroup
@@ -30,6 +30,7 @@ import java.util.concurrent.atomic.AtomicLong
import scala.collection.mutable.{HashSet, HashMap}
import scala.reflect.BeanProperty
+import se.scalablesolutions.akka.actor._
/**
* Life-cycle events for RemoteClient.
@@ -64,8 +65,6 @@ object RemoteClient extends Logging {
private val remoteClients = new HashMap[String, RemoteClient]
private val remoteActors = new HashMap[RemoteServer.Address, HashSet[String]]
- // FIXME: simplify overloaded methods when we have Scala 2.8
-
def actorFor(classNameOrServiceId: String, hostname: String, port: Int): ActorRef =
actorFor(classNameOrServiceId, classNameOrServiceId, 5000L, hostname, port, None)
@@ -87,6 +86,27 @@ object RemoteClient extends Logging {
def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int): ActorRef =
RemoteActorRef(serviceId, className, hostname, port, timeout, None)
+ def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, hostname: String, port: Int) : T = {
+ typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, 5000L, hostname, port, None)
+ }
+
+ def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, timeout: Long, hostname: String, port: Int) : T = {
+ typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, timeout, hostname, port, None)
+ }
+
+ def typedActorFor[T](intfClass: Class[T], serviceIdOrClassName: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader) : T = {
+ typedActorFor(intfClass, serviceIdOrClassName, serviceIdOrClassName, timeout, hostname, port, Some(loader))
+ }
+
+ def typedActorFor[T](intfClass: Class[T], serviceId: String, implClassName: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader) : T = {
+ typedActorFor(intfClass, serviceId, implClassName, timeout, hostname, port, Some(loader))
+ }
+
+ private[akka] def typedActorFor[T](intfClass: Class[T], serviceId: String, implClassName: String, timeout: Long, hostname: String, port: Int, loader: Option[ClassLoader]) : T = {
+ val actorRef = RemoteActorRef(serviceId, implClassName, hostname, port, timeout, loader, ActorType.TypedActor)
+ TypedActor.createProxyForRemoteActorRef(intfClass, actorRef)
+ }
+
private[akka] def actorFor(serviceId: String, className: String, timeout: Long, hostname: String, port: Int, loader: ClassLoader): ActorRef =
RemoteActorRef(serviceId, className, hostname, port, timeout, Some(loader))
@@ -173,7 +193,7 @@ class RemoteClient private[akka] (
extends Logging with ListenerManagement {
val name = "RemoteClient@" + hostname + "::" + port
- //FIXME Should these be clear:ed on shutdown?
+ //FIXME Should these be clear:ed on postStop?
private val futures = new ConcurrentHashMap[Long, CompletableFuture[_]]
private val supervisors = new ConcurrentHashMap[String, ActorRef]
@@ -208,10 +228,10 @@ class RemoteClient private[akka] (
val channel = connection.awaitUninterruptibly.getChannel
openChannels.add(channel)
if (!connection.isSuccess) {
- foreachListener(_ ! RemoteClientError(connection.getCause, this))
+ notifyListeners(RemoteClientError(connection.getCause, this))
log.error(connection.getCause, "Remote client connection to [%s:%s] has failed", hostname, port)
}
- foreachListener(_ ! RemoteClientStarted(this))
+ notifyListeners(RemoteClientStarted(this))
isRunning = true
}
}
@@ -220,7 +240,7 @@ class RemoteClient private[akka] (
log.info("Shutting down %s", name)
if (isRunning) {
isRunning = false
- foreachListener(_ ! RemoteClientShutdown(this))
+ notifyListeners(RemoteClientShutdown(this))
timer.stop
timer = null
openChannels.close.awaitUninterruptibly
@@ -238,7 +258,7 @@ class RemoteClient private[akka] (
@deprecated("Use removeListener instead")
def deregisterListener(actorRef: ActorRef) = removeListener(actorRef)
- override def foreachListener(f: (ActorRef) => Unit): Unit = super.foreachListener(f)
+ override def notifyListeners(message: => Any): Unit = super.notifyListeners(message)
protected override def manageLifeCycleOfListeners = false
@@ -275,7 +295,7 @@ class RemoteClient private[akka] (
} else {
val exception = new RemoteClientException(
"Remote client is not running, make sure you have invoked 'RemoteClient.connect' before using it.", this)
- foreachListener(l => l ! RemoteClientError(exception, this))
+ notifyListeners(RemoteClientError(exception, this))
throw exception
}
@@ -391,12 +411,12 @@ class RemoteClientHandler(
futures.remove(reply.getId)
} else {
val exception = new RemoteClientException("Unknown message received in remote client handler: " + result, client)
- client.foreachListener(_ ! RemoteClientError(exception, client))
+ client.notifyListeners(RemoteClientError(exception, client))
throw exception
}
} catch {
case e: Exception =>
- client.foreachListener(_ ! RemoteClientError(e, client))
+ client.notifyListeners(RemoteClientError(e, client))
log.error("Unexpected exception in remote client handler: %s", e)
throw e
}
@@ -411,7 +431,7 @@ class RemoteClientHandler(
client.connection = bootstrap.connect(remoteAddress)
client.connection.awaitUninterruptibly // Wait until the connection attempt succeeds or fails.
if (!client.connection.isSuccess) {
- client.foreachListener(_ ! RemoteClientError(client.connection.getCause, client))
+ client.notifyListeners(RemoteClientError(client.connection.getCause, client))
log.error(client.connection.getCause, "Reconnection to [%s] has failed", remoteAddress)
}
}
@@ -421,7 +441,7 @@ class RemoteClientHandler(
override def channelConnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
def connect = {
- client.foreachListener(_ ! RemoteClientConnected(client))
+ client.notifyListeners(RemoteClientConnected(client))
log.debug("Remote client connected to [%s]", ctx.getChannel.getRemoteAddress)
client.resetReconnectionTimeWindow
}
@@ -438,12 +458,12 @@ class RemoteClientHandler(
}
override def channelDisconnected(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
- client.foreachListener(_ ! RemoteClientDisconnected(client))
+ client.notifyListeners(RemoteClientDisconnected(client))
log.debug("Remote client disconnected from [%s]", ctx.getChannel.getRemoteAddress)
}
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
- client.foreachListener(_ ! RemoteClientError(event.getCause, client))
+ client.notifyListeners(RemoteClientError(event.getCause, client))
log.error(event.getCause, "Unexpected exception from downstream in remote client")
event.getChannel.close
}
diff --git a/akka-remote/src/main/scala/remote/RemoteServer.scala b/akka-remote/src/main/scala/remote/RemoteServer.scala
index 2ce24b3fff..c1f25b6d4f 100644
--- a/akka-remote/src/main/scala/remote/RemoteServer.scala
+++ b/akka-remote/src/main/scala/remote/RemoteServer.scala
@@ -9,12 +9,15 @@ import java.net.InetSocketAddress
import java.util.concurrent.{ConcurrentHashMap, Executors}
import java.util.{Map => JMap}
-import se.scalablesolutions.akka.actor._
+import se.scalablesolutions.akka.actor.{
+ Actor, TypedActor, ActorRef, IllegalActorStateException, RemoteActorSystemMessage}
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.util._
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType._
import se.scalablesolutions.akka.config.Config._
+import se.scalablesolutions.akka.serialization.RemoteActorSerialization
+import se.scalablesolutions.akka.serialization.RemoteActorSerialization._
import org.jboss.netty.bootstrap.ServerBootstrap
import org.jboss.netty.channel._
@@ -27,6 +30,7 @@ import org.jboss.netty.handler.ssl.SslHandler
import scala.collection.mutable.Map
import scala.reflect.BeanProperty
+import se.scalablesolutions.akka.dispatch.{DefaultCompletableFuture, CompletableFuture}
/**
* Use this object if you need a single remote server on a specific node.
@@ -130,8 +134,8 @@ object RemoteServer {
actorsFor(RemoteServer.Address(address.getHostName, address.getPort)).actors.put(uuid, actor)
}
- private[akka] def registerTypedActor(address: InetSocketAddress, name: String, typedActor: AnyRef) = guard.withWriteGuard {
- actorsFor(RemoteServer.Address(address.getHostName, address.getPort)).typedActors.put(name, typedActor)
+ private[akka] def registerTypedActor(address: InetSocketAddress, uuid: String, typedActor: AnyRef) = guard.withWriteGuard {
+ actorsFor(RemoteServer.Address(address.getHostName, address.getPort)).typedActors.put(uuid, typedActor)
}
private[akka] def getOrCreateServer(address: InetSocketAddress): RemoteServer = guard.withWriteGuard {
@@ -190,8 +194,10 @@ case class RemoteServerClientDisconnected(@BeanProperty val server: RemoteServer
class RemoteServer extends Logging with ListenerManagement {
def name = "RemoteServer@" + hostname + ":" + port
- private[akka] var hostname = RemoteServer.HOSTNAME
- private[akka] var port = RemoteServer.PORT
+ private[akka] var address = RemoteServer.Address(RemoteServer.HOSTNAME,RemoteServer.PORT)
+
+ def hostname = address.hostname
+ def port = address.port
@volatile private var _isRunning = false
@@ -227,13 +233,11 @@ class RemoteServer extends Logging with ListenerManagement {
private def start(_hostname: String, _port: Int, loader: Option[ClassLoader]): RemoteServer = synchronized {
try {
if (!_isRunning) {
- hostname = _hostname
- port = _port
+ address = RemoteServer.Address(_hostname,_port)
log.info("Starting remote server at [%s:%s]", hostname, port)
RemoteServer.register(hostname, port, this)
- val remoteActorSet = RemoteServer.actorsFor(RemoteServer.Address(hostname, port))
val pipelineFactory = new RemoteServerPipelineFactory(
- name, openChannels, loader, remoteActorSet.actors, remoteActorSet.typedActors,this)
+ name, openChannels, loader, this)
bootstrap.setPipelineFactory(pipelineFactory)
bootstrap.setOption("child.tcpNoDelay", true)
bootstrap.setOption("child.keepAlive", true)
@@ -242,12 +246,12 @@ class RemoteServer extends Logging with ListenerManagement {
openChannels.add(bootstrap.bind(new InetSocketAddress(hostname, port)))
_isRunning = true
Cluster.registerLocalNode(hostname, port)
- foreachListener(_ ! RemoteServerStarted(this))
+ notifyListeners(RemoteServerStarted(this))
}
} catch {
case e =>
log.error(e, "Could not start up remote server")
- foreachListener(_ ! RemoteServerError(e, this))
+ notifyListeners(RemoteServerError(e, this))
}
this
}
@@ -260,7 +264,7 @@ class RemoteServer extends Logging with ListenerManagement {
openChannels.close.awaitUninterruptibly
bootstrap.releaseExternalResources
Cluster.deregisterLocalNode(hostname, port)
- foreachListener(_ ! RemoteServerShutdown(this))
+ notifyListeners(RemoteServerShutdown(this))
} catch {
case e: java.nio.channels.ClosedChannelException => {}
case e => log.warning("Could not close remote server channel in a graceful way")
@@ -268,12 +272,28 @@ class RemoteServer extends Logging with ListenerManagement {
}
}
- // TODO: register typed actor in RemoteServer as well
+ /**
+ * Register typed actor by interface name.
+ */
+ def registerTypedActor(intfClass: Class[_], typedActor: AnyRef) : Unit = registerTypedActor(intfClass.getName, typedActor)
+
+ /**
+ * Register remote typed actor by a specific id.
+ * @param id custom actor id
+ * @param typedActor typed actor to register
+ */
+ def registerTypedActor(id: String, typedActor: AnyRef): Unit = synchronized {
+ val typedActors = RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).typedActors
+ if (!typedActors.contains(id)) {
+ log.debug("Registering server side remote actor [%s] with id [%s] on [%s:%d]", typedActor.getClass.getName, id, hostname, port)
+ typedActors.put(id, typedActor)
+ }
+ }
/**
* Register Remote Actor by the Actor's 'id' field. It starts the Actor if it is not started already.
*/
- def register(actorRef: ActorRef): Unit = register(actorRef.id,actorRef)
+ def register(actorRef: ActorRef): Unit = register(actorRef.id, actorRef)
/**
* Register Remote Actor by a specific 'id' passed as argument.
@@ -282,11 +302,11 @@ class RemoteServer extends Logging with ListenerManagement {
*/
def register(id: String, actorRef: ActorRef): Unit = synchronized {
if (_isRunning) {
- val actors = RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors
- if (!actors.contains(id)) {
+ val actorMap = actors()
+ if (!actorMap.contains(id)) {
if (!actorRef.isRunning) actorRef.start
log.debug("Registering server side remote actor [%s] with id [%s]", actorRef.actorClass.getName, id)
- actors.put(id, actorRef)
+ actorMap.put(id, actorRef)
}
}
}
@@ -296,10 +316,10 @@ class RemoteServer extends Logging with ListenerManagement {
*/
def unregister(actorRef: ActorRef):Unit = synchronized {
if (_isRunning) {
- log.debug("Unregistering server side remote actor [%s] with id [%s]", actorRef.actorClass.getName, actorRef.id)
- val actors = RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors
- actors.remove(actorRef.id)
- if (actorRef.registeredInRemoteNodeDuringSerialization) actors.remove(actorRef.uuid)
+ log.debug("Unregistering server side remote actor [%s] with id [%s:%s]", actorRef.actorClass.getName, actorRef.id, actorRef.uuid)
+ val actorMap = actors()
+ actorMap remove actorRef.id
+ if (actorRef.registeredInRemoteNodeDuringSerialization) actorMap remove actorRef.uuid
}
}
@@ -311,16 +331,32 @@ class RemoteServer extends Logging with ListenerManagement {
def unregister(id: String):Unit = synchronized {
if (_isRunning) {
log.info("Unregistering server side remote actor with id [%s]", id)
- val actors = RemoteServer.actorsFor(RemoteServer.Address(hostname, port)).actors
- val actorRef = actors.get(id)
- actors.remove(id)
- if (actorRef.registeredInRemoteNodeDuringSerialization) actors.remove(actorRef.uuid)
+ val actorMap = actors()
+ val actorRef = actorMap get id
+ actorMap remove id
+ if (actorRef.registeredInRemoteNodeDuringSerialization) actorMap remove actorRef.uuid
+ }
+ }
+
+ /**
+ * Unregister Remote Typed Actor by specific 'id'.
+ *
+ * NOTE: You need to call this method if you have registered an actor by a custom ID.
+ */
+ def unregisterTypedActor(id: String):Unit = synchronized {
+ if (_isRunning) {
+ log.info("Unregistering server side remote typed actor with id [%s]", id)
+ val registeredTypedActors = typedActors()
+ registeredTypedActors.remove(id)
}
}
protected override def manageLifeCycleOfListeners = false
- protected[akka] override def foreachListener(f: (ActorRef) => Unit): Unit = super.foreachListener(f)
+ protected[akka] override def notifyListeners(message: => Any): Unit = super.notifyListeners(message)
+
+ private[akka] def actors() = RemoteServer.actorsFor(address).actors
+ private[akka] def typedActors() = RemoteServer.actorsFor(address).typedActors
}
object RemoteServerSslContext {
@@ -345,8 +381,6 @@ class RemoteServerPipelineFactory(
val name: String,
val openChannels: ChannelGroup,
val loader: Option[ClassLoader],
- val actors: JMap[String, ActorRef],
- val typedActors: JMap[String, AnyRef],
val server: RemoteServer) extends ChannelPipelineFactory {
import RemoteServer._
@@ -370,7 +404,7 @@ class RemoteServerPipelineFactory(
case _ => (join(), join())
}
- val remoteServer = new RemoteServerHandler(name, openChannels, loader, actors, typedActors,server)
+ val remoteServer = new RemoteServerHandler(name, openChannels, loader, server)
val stages = ssl ++ dec ++ join(lenDec, protobufDec) ++ enc ++ join(lenPrep, protobufEnc, remoteServer)
new StaticChannelPipeline(stages: _*)
}
@@ -384,15 +418,13 @@ class RemoteServerHandler(
val name: String,
val openChannels: ChannelGroup,
val applicationLoader: Option[ClassLoader],
- val actors: JMap[String, ActorRef],
- val typedActors: JMap[String, AnyRef],
val server: RemoteServer) extends SimpleChannelUpstreamHandler with Logging {
val AW_PROXY_PREFIX = "$$ProxiedByAW".intern
applicationLoader.foreach(MessageSerializer.setClassLoader(_))
/**
- * ChannelOpen overridden to store open channels for a clean shutdown of a RemoteServer.
+ * ChannelOpen overridden to store open channels for a clean postStop of a RemoteServer.
* If a channel is closed before, it is automatically removed from the open channels group.
*/
override def channelOpen(ctx: ChannelHandlerContext, event: ChannelStateEvent) = openChannels.add(ctx.getChannel)
@@ -407,18 +439,18 @@ class RemoteServerHandler(
def operationComplete(future: ChannelFuture): Unit = {
if (future.isSuccess) {
openChannels.add(future.getChannel)
- server.foreachListener(_ ! RemoteServerClientConnected(server))
+ server.notifyListeners(RemoteServerClientConnected(server))
} else future.getChannel.close
}
})
} else {
- server.foreachListener(_ ! RemoteServerClientConnected(server))
+ server.notifyListeners(RemoteServerClientConnected(server))
}
}
override def channelClosed(ctx: ChannelHandlerContext, event: ChannelStateEvent) = {
log.debug("Remote client disconnected from [%s]", server.name)
- server.foreachListener(_ ! RemoteServerClientDisconnected(server))
+ server.notifyListeners(RemoteServerClientDisconnected(server))
}
override def handleUpstream(ctx: ChannelHandlerContext, event: ChannelEvent) = {
@@ -440,7 +472,7 @@ class RemoteServerHandler(
override def exceptionCaught(ctx: ChannelHandlerContext, event: ExceptionEvent) = {
log.error(event.getCause, "Unexpected exception from remote downstream")
event.getChannel.close
- server.foreachListener(_ ! RemoteServerError(event.getCause, server))
+ server.notifyListeners(RemoteServerError(event.getCause, server))
}
private def handleRemoteRequestProtocol(request: RemoteRequestProtocol, channel: Channel) = {
@@ -467,27 +499,36 @@ class RemoteServerHandler(
case RemoteActorSystemMessage.Stop => actorRef.stop
case _ => // then match on user defined messages
if (request.getIsOneWay) actorRef.!(message)(sender)
- else {
- try {
- val resultOrNone = (actorRef.!!(message)(sender)).as[AnyRef]
- val result = if (resultOrNone.isDefined) resultOrNone.get else null
+ else actorRef.postMessageToMailboxAndCreateFutureResultWithTimeout(message,request.getActorInfo.getTimeout,None,Some(
+ new DefaultCompletableFuture[AnyRef](request.getActorInfo.getTimeout){
+ override def onComplete(result: AnyRef) {
+ log.debug("Returning result from actor invocation [%s]", result)
+ val replyBuilder = RemoteReplyProtocol.newBuilder
+ .setId(request.getId)
+ .setMessage(MessageSerializer.serialize(result))
+ .setIsSuccessful(true)
+ .setIsActor(true)
- log.debug("Returning result from actor invocation [%s]", result)
- val replyBuilder = RemoteReplyProtocol.newBuilder
- .setId(request.getId)
- .setMessage(MessageSerializer.serialize(result))
- .setIsSuccessful(true)
- .setIsActor(true)
+ if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
- if (request.hasSupervisorUuid) replyBuilder.setSupervisorUuid(request.getSupervisorUuid)
- channel.write(replyBuilder.build)
+ try {
+ channel.write(replyBuilder.build)
+ } catch {
+ case e: Throwable =>
+ server.notifyListeners(RemoteServerError(e, server))
+ }
+ }
- } catch {
- case e: Throwable =>
- channel.write(createErrorReplyMessage(e, request, true))
- server.foreachListener(_ ! RemoteServerError(e, server))
- }
+ override def onCompleteException(exception: Throwable) {
+ try {
+ channel.write(createErrorReplyMessage(exception, request, true))
+ } catch {
+ case e: Throwable =>
+ server.notifyListeners(RemoteServerError(e, server))
+ }
+ }
}
+ ))
}
}
@@ -517,13 +558,39 @@ class RemoteServerHandler(
} catch {
case e: InvocationTargetException =>
channel.write(createErrorReplyMessage(e.getCause, request, false))
- server.foreachListener(_ ! RemoteServerError(e, server))
+ server.notifyListeners(RemoteServerError(e, server))
case e: Throwable =>
channel.write(createErrorReplyMessage(e, request, false))
- server.foreachListener(_ ! RemoteServerError(e, server))
+ server.notifyListeners(RemoteServerError(e, server))
}
}
+ /**
+ * Find a registered actor by ID (default) or UUID.
+ * Actors are registered by id apart from registering during serialization see SerializationProtocol.
+ */
+ private def findActorByIdOrUuid(id: String, uuid: String) : ActorRef = {
+ val registeredActors = server.actors()
+ var actorRefOrNull = registeredActors get id
+ if (actorRefOrNull eq null) {
+ actorRefOrNull = registeredActors get uuid
+ }
+ actorRefOrNull
+ }
+
+ /**
+ * Find a registered typed actor by ID (default) or UUID.
+ * Actors are registered by id apart from registering during serialization see SerializationProtocol.
+ */
+ private def findTypedActorByIdOrUUid(id: String, uuid: String) : AnyRef = {
+ val registeredActors = server.typedActors()
+ var actorRefOrNull = registeredActors get id
+ if (actorRefOrNull eq null) {
+ actorRefOrNull = registeredActors get uuid
+ }
+ actorRefOrNull
+ }
+
/**
* Creates a new instance of the actor with name, uuid and timeout specified as arguments.
*
@@ -533,10 +600,12 @@ class RemoteServerHandler(
*/
private def createActor(actorInfo: ActorInfoProtocol): ActorRef = {
val uuid = actorInfo.getUuid
+ val id = actorInfo.getId
+
val name = actorInfo.getTarget
val timeout = actorInfo.getTimeout
- val actorRefOrNull = actors get uuid
+ val actorRefOrNull = findActorByIdOrUuid(id, uuid)
if (actorRefOrNull eq null) {
try {
@@ -545,14 +614,15 @@ class RemoteServerHandler(
else Class.forName(name)
val actorRef = Actor.actorOf(clazz.newInstance.asInstanceOf[Actor])
actorRef.uuid = uuid
+ actorRef.id = id
actorRef.timeout = timeout
actorRef.remoteAddress = None
- actors.put(uuid, actorRef)
+ server.actors.put(id, actorRef) // register by id
actorRef
} catch {
case e =>
log.error(e, "Could not create remote actor instance")
- server.foreachListener(_ ! RemoteServerError(e, server))
+ server.notifyListeners(RemoteServerError(e, server))
throw e
}
} else actorRefOrNull
@@ -560,7 +630,9 @@ class RemoteServerHandler(
private def createTypedActor(actorInfo: ActorInfoProtocol): AnyRef = {
val uuid = actorInfo.getUuid
- val typedActorOrNull = typedActors get uuid
+ val id = actorInfo.getId
+
+ val typedActorOrNull = findTypedActorByIdOrUUid(id, uuid)
if (typedActorOrNull eq null) {
val typedActorInfo = actorInfo.getTypedActorInfo
@@ -577,12 +649,12 @@ class RemoteServerHandler(
val newInstance = TypedActor.newInstance(
interfaceClass, targetClass.asInstanceOf[Class[_ <: TypedActor]], actorInfo.getTimeout).asInstanceOf[AnyRef]
- typedActors.put(uuid, newInstance)
+ server.typedActors.put(id, newInstance) // register by id
newInstance
} catch {
case e =>
log.error(e, "Could not create remote typed actor instance")
- server.foreachListener(_ ! RemoteServerError(e, server))
+ server.notifyListeners(RemoteServerError(e, server))
throw e
}
} else typedActorOrNull
diff --git a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
index 4050c2026f..bc1aa9052f 100644
--- a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
+++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
@@ -2,17 +2,18 @@
* Copyright (C) 2009-2010 Scalable Solutions AB
*/
-package se.scalablesolutions.akka.actor
+package se.scalablesolutions.akka.serialization
-import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
-import se.scalablesolutions.akka.config.ScalaConfig._
+import se.scalablesolutions.akka.actor.{Actor, ActorRef, LocalActorRef, RemoteActorRef, IllegalActorStateException, ActorType}
import se.scalablesolutions.akka.stm.global._
import se.scalablesolutions.akka.stm.TransactionManagement._
import se.scalablesolutions.akka.stm.TransactionManagement
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
import se.scalablesolutions.akka.remote.{RemoteServer, MessageSerializer}
-import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType._
-import se.scalablesolutions.akka.serialization.Serializer
+import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _}
+import ActorTypeProtocol._
+import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, FaultHandlingStrategy}
+import se.scalablesolutions.akka.config.ScalaConfig._
import com.google.protobuf.ByteString
import se.scalablesolutions.akka.util.UUID
@@ -68,7 +69,7 @@ trait SerializerBasedActorFormat[T <: Actor] extends Format[T] {
}
/**
- * Module for local actor serialization
+ * Module for local actor serialization.
*/
object ActorSerialization {
@@ -250,6 +251,7 @@ object RemoteActorSerialization {
val actorInfoBuilder = ActorInfoProtocol.newBuilder
.setUuid(uuid)
+ .setId(actorRef.id)
.setTarget(actorClassName)
.setTimeout(timeout)
diff --git a/akka-remote/src/test/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java b/akka-remote/src/test/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
deleted file mode 100644
index 0ab1a0aa10..0000000000
--- a/akka-remote/src/test/java/se/scalablesolutions/akka/remote/protocol/RemoteProtocol.java
+++ /dev/null
@@ -1,5190 +0,0 @@
-// Generated by the protocol buffer compiler. DO NOT EDIT!
-// source: RemoteProtocol.proto
-
-package se.scalablesolutions.akka.remote.protocol;
-
-public final class RemoteProtocol {
- private RemoteProtocol() {}
- public static void registerAllExtensions(
- com.google.protobuf.ExtensionRegistry registry) {
- }
- public enum ActorType
- implements com.google.protobuf.ProtocolMessageEnum {
- SCALA_ACTOR(0, 1),
- JAVA_ACTOR(1, 2),
- TYPED_ACTOR(2, 3),
- ;
-
-
- public final int getNumber() { return value; }
-
- public static ActorType valueOf(int value) {
- switch (value) {
- case 1: return SCALA_ACTOR;
- case 2: return JAVA_ACTOR;
- case 3: return TYPED_ACTOR;
- default: return null;
- }
- }
-
- public static com.google.protobuf.Internal.EnumLiteMap
- internalGetValueMap() {
- return internalValueMap;
- }
- private static com.google.protobuf.Internal.EnumLiteMap
- internalValueMap =
- new com.google.protobuf.Internal.EnumLiteMap() {
- public ActorType findValueByNumber(int number) {
- return ActorType.valueOf(number)
- ; }
- };
-
- public final com.google.protobuf.Descriptors.EnumValueDescriptor
- getValueDescriptor() {
- return getDescriptor().getValues().get(index);
- }
- public final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptorForType() {
- return getDescriptor();
- }
- public static final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(0);
- }
-
- private static final ActorType[] VALUES = {
- SCALA_ACTOR, JAVA_ACTOR, TYPED_ACTOR,
- };
- public static ActorType valueOf(
- com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
- if (desc.getType() != getDescriptor()) {
- throw new java.lang.IllegalArgumentException(
- "EnumValueDescriptor is not for this type.");
- }
- return VALUES[desc.getIndex()];
- }
- private final int index;
- private final int value;
- private ActorType(int index, int value) {
- this.index = index;
- this.value = value;
- }
-
- static {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor();
- }
-
- // @@protoc_insertion_point(enum_scope:ActorType)
- }
-
- public enum SerializationSchemeType
- implements com.google.protobuf.ProtocolMessageEnum {
- JAVA(0, 1),
- SBINARY(1, 2),
- SCALA_JSON(2, 3),
- JAVA_JSON(3, 4),
- PROTOBUF(4, 5),
- ;
-
-
- public final int getNumber() { return value; }
-
- public static SerializationSchemeType valueOf(int value) {
- switch (value) {
- case 1: return JAVA;
- case 2: return SBINARY;
- case 3: return SCALA_JSON;
- case 4: return JAVA_JSON;
- case 5: return PROTOBUF;
- default: return null;
- }
- }
-
- public static com.google.protobuf.Internal.EnumLiteMap
- internalGetValueMap() {
- return internalValueMap;
- }
- private static com.google.protobuf.Internal.EnumLiteMap
- internalValueMap =
- new com.google.protobuf.Internal.EnumLiteMap() {
- public SerializationSchemeType findValueByNumber(int number) {
- return SerializationSchemeType.valueOf(number)
- ; }
- };
-
- public final com.google.protobuf.Descriptors.EnumValueDescriptor
- getValueDescriptor() {
- return getDescriptor().getValues().get(index);
- }
- public final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptorForType() {
- return getDescriptor();
- }
- public static final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(1);
- }
-
- private static final SerializationSchemeType[] VALUES = {
- JAVA, SBINARY, SCALA_JSON, JAVA_JSON, PROTOBUF,
- };
- public static SerializationSchemeType valueOf(
- com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
- if (desc.getType() != getDescriptor()) {
- throw new java.lang.IllegalArgumentException(
- "EnumValueDescriptor is not for this type.");
- }
- return VALUES[desc.getIndex()];
- }
- private final int index;
- private final int value;
- private SerializationSchemeType(int index, int value) {
- this.index = index;
- this.value = value;
- }
-
- static {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor();
- }
-
- // @@protoc_insertion_point(enum_scope:SerializationSchemeType)
- }
-
- public enum LifeCycleType
- implements com.google.protobuf.ProtocolMessageEnum {
- PERMANENT(0, 1),
- TEMPORARY(1, 2),
- ;
-
-
- public final int getNumber() { return value; }
-
- public static LifeCycleType valueOf(int value) {
- switch (value) {
- case 1: return PERMANENT;
- case 2: return TEMPORARY;
- default: return null;
- }
- }
-
- public static com.google.protobuf.Internal.EnumLiteMap
- internalGetValueMap() {
- return internalValueMap;
- }
- private static com.google.protobuf.Internal.EnumLiteMap
- internalValueMap =
- new com.google.protobuf.Internal.EnumLiteMap() {
- public LifeCycleType findValueByNumber(int number) {
- return LifeCycleType.valueOf(number)
- ; }
- };
-
- public final com.google.protobuf.Descriptors.EnumValueDescriptor
- getValueDescriptor() {
- return getDescriptor().getValues().get(index);
- }
- public final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptorForType() {
- return getDescriptor();
- }
- public static final com.google.protobuf.Descriptors.EnumDescriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(2);
- }
-
- private static final LifeCycleType[] VALUES = {
- PERMANENT, TEMPORARY,
- };
- public static LifeCycleType valueOf(
- com.google.protobuf.Descriptors.EnumValueDescriptor desc) {
- if (desc.getType() != getDescriptor()) {
- throw new java.lang.IllegalArgumentException(
- "EnumValueDescriptor is not for this type.");
- }
- return VALUES[desc.getIndex()];
- }
- private final int index;
- private final int value;
- private LifeCycleType(int index, int value) {
- this.index = index;
- this.value = value;
- }
-
- static {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.getDescriptor();
- }
-
- // @@protoc_insertion_point(enum_scope:LifeCycleType)
- }
-
- public static final class RemoteActorRefProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use RemoteActorRefProtocol.newBuilder() to construct.
- private RemoteActorRefProtocol() {
- initFields();
- }
- private RemoteActorRefProtocol(boolean noInit) {}
-
- private static final RemoteActorRefProtocol defaultInstance;
- public static RemoteActorRefProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public RemoteActorRefProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_fieldAccessorTable;
- }
-
- // required string uuid = 1;
- public static final int UUID_FIELD_NUMBER = 1;
- private boolean hasUuid;
- private java.lang.String uuid_ = "";
- public boolean hasUuid() { return hasUuid; }
- public java.lang.String getUuid() { return uuid_; }
-
- // required string actorClassname = 2;
- public static final int ACTORCLASSNAME_FIELD_NUMBER = 2;
- private boolean hasActorClassname;
- private java.lang.String actorClassname_ = "";
- public boolean hasActorClassname() { return hasActorClassname; }
- public java.lang.String getActorClassname() { return actorClassname_; }
-
- // required .AddressProtocol homeAddress = 3;
- public static final int HOMEADDRESS_FIELD_NUMBER = 3;
- private boolean hasHomeAddress;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol homeAddress_;
- public boolean hasHomeAddress() { return hasHomeAddress; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() { return homeAddress_; }
-
- // optional uint64 timeout = 4;
- public static final int TIMEOUT_FIELD_NUMBER = 4;
- private boolean hasTimeout;
- private long timeout_ = 0L;
- public boolean hasTimeout() { return hasTimeout; }
- public long getTimeout() { return timeout_; }
-
- private void initFields() {
- homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
- }
- public final boolean isInitialized() {
- if (!hasUuid) return false;
- if (!hasActorClassname) return false;
- if (!hasHomeAddress) return false;
- if (!getHomeAddress().isInitialized()) return false;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasUuid()) {
- output.writeString(1, getUuid());
- }
- if (hasActorClassname()) {
- output.writeString(2, getActorClassname());
- }
- if (hasHomeAddress()) {
- output.writeMessage(3, getHomeAddress());
- }
- if (hasTimeout()) {
- output.writeUInt64(4, getTimeout());
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasUuid()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(1, getUuid());
- }
- if (hasActorClassname()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(2, getActorClassname());
- }
- if (hasHomeAddress()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(3, getHomeAddress());
- }
- if (hasTimeout()) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(4, getTimeout());
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) return this;
- if (other.hasUuid()) {
- setUuid(other.getUuid());
- }
- if (other.hasActorClassname()) {
- setActorClassname(other.getActorClassname());
- }
- if (other.hasHomeAddress()) {
- mergeHomeAddress(other.getHomeAddress());
- }
- if (other.hasTimeout()) {
- setTimeout(other.getTimeout());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 10: {
- setUuid(input.readString());
- break;
- }
- case 18: {
- setActorClassname(input.readString());
- break;
- }
- case 26: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder();
- if (hasHomeAddress()) {
- subBuilder.mergeFrom(getHomeAddress());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setHomeAddress(subBuilder.buildPartial());
- break;
- }
- case 32: {
- setTimeout(input.readUInt64());
- break;
- }
- }
- }
- }
-
-
- // required string uuid = 1;
- 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 string actorClassname = 2;
- public boolean hasActorClassname() {
- return result.hasActorClassname();
- }
- public java.lang.String getActorClassname() {
- return result.getActorClassname();
- }
- public Builder setActorClassname(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasActorClassname = true;
- result.actorClassname_ = value;
- return this;
- }
- public Builder clearActorClassname() {
- result.hasActorClassname = false;
- result.actorClassname_ = getDefaultInstance().getActorClassname();
- return this;
- }
-
- // required .AddressProtocol homeAddress = 3;
- public boolean hasHomeAddress() {
- return result.hasHomeAddress();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getHomeAddress() {
- return result.getHomeAddress();
- }
- public Builder setHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasHomeAddress = true;
- result.homeAddress_ = value;
- return this;
- }
- public Builder setHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder builderForValue) {
- result.hasHomeAddress = true;
- result.homeAddress_ = builderForValue.build();
- return this;
- }
- public Builder mergeHomeAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) {
- if (result.hasHomeAddress() &&
- result.homeAddress_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) {
- result.homeAddress_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.homeAddress_).mergeFrom(value).buildPartial();
- } else {
- result.homeAddress_ = value;
- }
- result.hasHomeAddress = true;
- return this;
- }
- public Builder clearHomeAddress() {
- result.hasHomeAddress = false;
- result.homeAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
- return this;
- }
-
- // optional uint64 timeout = 4;
- public boolean hasTimeout() {
- return result.hasTimeout();
- }
- public long getTimeout() {
- return result.getTimeout();
- }
- public Builder setTimeout(long value) {
- result.hasTimeout = true;
- result.timeout_ = value;
- return this;
- }
- public Builder clearTimeout() {
- result.hasTimeout = false;
- result.timeout_ = 0L;
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:RemoteActorRefProtocol)
- }
-
- static {
- defaultInstance = new RemoteActorRefProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:RemoteActorRefProtocol)
- }
-
- public static final class SerializedActorRefProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use SerializedActorRefProtocol.newBuilder() to construct.
- private SerializedActorRefProtocol() {
- initFields();
- }
- private SerializedActorRefProtocol(boolean noInit) {}
-
- private static final SerializedActorRefProtocol defaultInstance;
- public static SerializedActorRefProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public SerializedActorRefProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_fieldAccessorTable;
- }
-
- // required string uuid = 1;
- public static final int UUID_FIELD_NUMBER = 1;
- private boolean hasUuid;
- private java.lang.String uuid_ = "";
- public boolean hasUuid() { return hasUuid; }
- public java.lang.String getUuid() { return uuid_; }
-
- // required string id = 2;
- public static final int ID_FIELD_NUMBER = 2;
- private boolean hasId;
- private java.lang.String id_ = "";
- public boolean hasId() { return hasId; }
- public java.lang.String getId() { return id_; }
-
- // required string actorClassname = 3;
- public static final int ACTORCLASSNAME_FIELD_NUMBER = 3;
- private boolean hasActorClassname;
- private java.lang.String actorClassname_ = "";
- public boolean hasActorClassname() { return hasActorClassname; }
- public java.lang.String getActorClassname() { return actorClassname_; }
-
- // required .AddressProtocol originalAddress = 4;
- public static final int ORIGINALADDRESS_FIELD_NUMBER = 4;
- private boolean hasOriginalAddress;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol originalAddress_;
- public boolean hasOriginalAddress() { return hasOriginalAddress; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() { return originalAddress_; }
-
- // optional bytes actorInstance = 5;
- public static final int ACTORINSTANCE_FIELD_NUMBER = 5;
- private boolean hasActorInstance;
- private com.google.protobuf.ByteString actorInstance_ = com.google.protobuf.ByteString.EMPTY;
- public boolean hasActorInstance() { return hasActorInstance; }
- public com.google.protobuf.ByteString getActorInstance() { return actorInstance_; }
-
- // optional string serializerClassname = 6;
- public static final int SERIALIZERCLASSNAME_FIELD_NUMBER = 6;
- private boolean hasSerializerClassname;
- private java.lang.String serializerClassname_ = "";
- public boolean hasSerializerClassname() { return hasSerializerClassname; }
- public java.lang.String getSerializerClassname() { return serializerClassname_; }
-
- // optional bool isTransactor = 7;
- public static final int ISTRANSACTOR_FIELD_NUMBER = 7;
- private boolean hasIsTransactor;
- private boolean isTransactor_ = false;
- public boolean hasIsTransactor() { return hasIsTransactor; }
- public boolean getIsTransactor() { return isTransactor_; }
-
- // optional 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 uint64 receiveTimeout = 9;
- public static final int RECEIVETIMEOUT_FIELD_NUMBER = 9;
- private boolean hasReceiveTimeout;
- private long receiveTimeout_ = 0L;
- public boolean hasReceiveTimeout() { return hasReceiveTimeout; }
- public long getReceiveTimeout() { return receiveTimeout_; }
-
- // optional .LifeCycleProtocol lifeCycle = 10;
- public static final int LIFECYCLE_FIELD_NUMBER = 10;
- private boolean hasLifeCycle;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol lifeCycle_;
- public boolean hasLifeCycle() { return hasLifeCycle; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { return lifeCycle_; }
-
- // optional .RemoteActorRefProtocol supervisor = 11;
- public static final int SUPERVISOR_FIELD_NUMBER = 11;
- private boolean hasSupervisor;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol supervisor_;
- public boolean hasSupervisor() { return hasSupervisor; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { return supervisor_; }
-
- // optional bytes hotswapStack = 12;
- public static final int HOTSWAPSTACK_FIELD_NUMBER = 12;
- private boolean hasHotswapStack;
- private com.google.protobuf.ByteString hotswapStack_ = com.google.protobuf.ByteString.EMPTY;
- public boolean hasHotswapStack() { return hasHotswapStack; }
- public com.google.protobuf.ByteString getHotswapStack() { return hotswapStack_; }
-
- // repeated .RemoteRequestProtocol messages = 13;
- public static final int MESSAGES_FIELD_NUMBER = 13;
- private java.util.List messages_ =
- java.util.Collections.emptyList();
- public java.util.List getMessagesList() {
- return messages_;
- }
- public int getMessagesCount() { return messages_.size(); }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) {
- return messages_.get(index);
- }
-
- private void initFields() {
- originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
- lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
- supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
- }
- public final boolean isInitialized() {
- if (!hasUuid) return false;
- if (!hasId) return false;
- if (!hasActorClassname) return false;
- if (!hasOriginalAddress) return false;
- if (!getOriginalAddress().isInitialized()) return false;
- if (hasLifeCycle()) {
- if (!getLifeCycle().isInitialized()) return false;
- }
- if (hasSupervisor()) {
- if (!getSupervisor().isInitialized()) return false;
- }
- for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) {
- if (!element.isInitialized()) return false;
- }
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasUuid()) {
- output.writeString(1, getUuid());
- }
- if (hasId()) {
- output.writeString(2, getId());
- }
- if (hasActorClassname()) {
- output.writeString(3, getActorClassname());
- }
- if (hasOriginalAddress()) {
- output.writeMessage(4, getOriginalAddress());
- }
- if (hasActorInstance()) {
- output.writeBytes(5, getActorInstance());
- }
- if (hasSerializerClassname()) {
- output.writeString(6, getSerializerClassname());
- }
- if (hasIsTransactor()) {
- output.writeBool(7, getIsTransactor());
- }
- if (hasTimeout()) {
- output.writeUInt64(8, getTimeout());
- }
- if (hasReceiveTimeout()) {
- output.writeUInt64(9, getReceiveTimeout());
- }
- if (hasLifeCycle()) {
- output.writeMessage(10, getLifeCycle());
- }
- if (hasSupervisor()) {
- output.writeMessage(11, getSupervisor());
- }
- if (hasHotswapStack()) {
- output.writeBytes(12, getHotswapStack());
- }
- for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) {
- output.writeMessage(13, element);
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasUuid()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(1, getUuid());
- }
- if (hasId()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(2, getId());
- }
- if (hasActorClassname()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(3, getActorClassname());
- }
- if (hasOriginalAddress()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(4, getOriginalAddress());
- }
- if (hasActorInstance()) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(5, getActorInstance());
- }
- if (hasSerializerClassname()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(6, getSerializerClassname());
- }
- if (hasIsTransactor()) {
- size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(7, getIsTransactor());
- }
- if (hasTimeout()) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(8, getTimeout());
- }
- if (hasReceiveTimeout()) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(9, getReceiveTimeout());
- }
- if (hasLifeCycle()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(10, getLifeCycle());
- }
- if (hasSupervisor()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(11, getSupervisor());
- }
- if (hasHotswapStack()) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(12, getHotswapStack());
- }
- for (se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol element : getMessagesList()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(13, element);
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- if (result.messages_ != java.util.Collections.EMPTY_LIST) {
- result.messages_ =
- java.util.Collections.unmodifiableList(result.messages_);
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) return this;
- if (other.hasUuid()) {
- setUuid(other.getUuid());
- }
- if (other.hasId()) {
- setId(other.getId());
- }
- if (other.hasActorClassname()) {
- setActorClassname(other.getActorClassname());
- }
- if (other.hasOriginalAddress()) {
- mergeOriginalAddress(other.getOriginalAddress());
- }
- if (other.hasActorInstance()) {
- setActorInstance(other.getActorInstance());
- }
- if (other.hasSerializerClassname()) {
- setSerializerClassname(other.getSerializerClassname());
- }
- if (other.hasIsTransactor()) {
- setIsTransactor(other.getIsTransactor());
- }
- if (other.hasTimeout()) {
- setTimeout(other.getTimeout());
- }
- if (other.hasReceiveTimeout()) {
- setReceiveTimeout(other.getReceiveTimeout());
- }
- if (other.hasLifeCycle()) {
- mergeLifeCycle(other.getLifeCycle());
- }
- if (other.hasSupervisor()) {
- mergeSupervisor(other.getSupervisor());
- }
- if (other.hasHotswapStack()) {
- setHotswapStack(other.getHotswapStack());
- }
- if (!other.messages_.isEmpty()) {
- if (result.messages_.isEmpty()) {
- result.messages_ = new java.util.ArrayList();
- }
- result.messages_.addAll(other.messages_);
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 10: {
- setUuid(input.readString());
- break;
- }
- case 18: {
- setId(input.readString());
- break;
- }
- case 26: {
- setActorClassname(input.readString());
- break;
- }
- case 34: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder();
- if (hasOriginalAddress()) {
- subBuilder.mergeFrom(getOriginalAddress());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setOriginalAddress(subBuilder.buildPartial());
- break;
- }
- case 42: {
- setActorInstance(input.readBytes());
- break;
- }
- case 50: {
- setSerializerClassname(input.readString());
- break;
- }
- case 56: {
- setIsTransactor(input.readBool());
- break;
- }
- case 64: {
- setTimeout(input.readUInt64());
- break;
- }
- case 72: {
- setReceiveTimeout(input.readUInt64());
- break;
- }
- case 82: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder();
- if (hasLifeCycle()) {
- subBuilder.mergeFrom(getLifeCycle());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setLifeCycle(subBuilder.buildPartial());
- break;
- }
- case 90: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder();
- if (hasSupervisor()) {
- subBuilder.mergeFrom(getSupervisor());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setSupervisor(subBuilder.buildPartial());
- break;
- }
- case 98: {
- setHotswapStack(input.readBytes());
- break;
- }
- case 106: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder();
- input.readMessage(subBuilder, extensionRegistry);
- addMessages(subBuilder.buildPartial());
- break;
- }
- }
- }
- }
-
-
- // required string uuid = 1;
- 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 string id = 2;
- public boolean hasId() {
- return result.hasId();
- }
- public java.lang.String getId() {
- return result.getId();
- }
- public Builder setId(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasId = true;
- result.id_ = value;
- return this;
- }
- public Builder clearId() {
- result.hasId = false;
- result.id_ = getDefaultInstance().getId();
- return this;
- }
-
- // required string actorClassname = 3;
- public boolean hasActorClassname() {
- return result.hasActorClassname();
- }
- public java.lang.String getActorClassname() {
- return result.getActorClassname();
- }
- public Builder setActorClassname(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasActorClassname = true;
- result.actorClassname_ = value;
- return this;
- }
- public Builder clearActorClassname() {
- result.hasActorClassname = false;
- result.actorClassname_ = getDefaultInstance().getActorClassname();
- return this;
- }
-
- // required .AddressProtocol originalAddress = 4;
- public boolean hasOriginalAddress() {
- return result.hasOriginalAddress();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getOriginalAddress() {
- return result.getOriginalAddress();
- }
- public Builder setOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasOriginalAddress = true;
- result.originalAddress_ = value;
- return this;
- }
- public Builder setOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder builderForValue) {
- result.hasOriginalAddress = true;
- result.originalAddress_ = builderForValue.build();
- return this;
- }
- public Builder mergeOriginalAddress(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol value) {
- if (result.hasOriginalAddress() &&
- result.originalAddress_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) {
- result.originalAddress_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder(result.originalAddress_).mergeFrom(value).buildPartial();
- } else {
- result.originalAddress_ = value;
- }
- result.hasOriginalAddress = true;
- return this;
- }
- public Builder clearOriginalAddress() {
- result.hasOriginalAddress = false;
- result.originalAddress_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
- return this;
- }
-
- // optional bytes actorInstance = 5;
- public boolean hasActorInstance() {
- return result.hasActorInstance();
- }
- public com.google.protobuf.ByteString getActorInstance() {
- return result.getActorInstance();
- }
- public Builder setActorInstance(com.google.protobuf.ByteString value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasActorInstance = true;
- result.actorInstance_ = value;
- return this;
- }
- public Builder clearActorInstance() {
- result.hasActorInstance = false;
- result.actorInstance_ = getDefaultInstance().getActorInstance();
- return this;
- }
-
- // optional string serializerClassname = 6;
- public boolean hasSerializerClassname() {
- return result.hasSerializerClassname();
- }
- public java.lang.String getSerializerClassname() {
- return result.getSerializerClassname();
- }
- public Builder setSerializerClassname(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasSerializerClassname = true;
- result.serializerClassname_ = value;
- return this;
- }
- public Builder clearSerializerClassname() {
- result.hasSerializerClassname = false;
- result.serializerClassname_ = getDefaultInstance().getSerializerClassname();
- return this;
- }
-
- // optional bool isTransactor = 7;
- public boolean hasIsTransactor() {
- return result.hasIsTransactor();
- }
- public boolean getIsTransactor() {
- return result.getIsTransactor();
- }
- public Builder setIsTransactor(boolean value) {
- result.hasIsTransactor = true;
- result.isTransactor_ = value;
- return this;
- }
- public Builder clearIsTransactor() {
- result.hasIsTransactor = false;
- result.isTransactor_ = false;
- return this;
- }
-
- // optional uint64 timeout = 8;
- public boolean hasTimeout() {
- return result.hasTimeout();
- }
- public long getTimeout() {
- return result.getTimeout();
- }
- public Builder setTimeout(long value) {
- result.hasTimeout = true;
- result.timeout_ = value;
- return this;
- }
- public Builder clearTimeout() {
- result.hasTimeout = false;
- result.timeout_ = 0L;
- return this;
- }
-
- // optional uint64 receiveTimeout = 9;
- public boolean hasReceiveTimeout() {
- return result.hasReceiveTimeout();
- }
- public long getReceiveTimeout() {
- return result.getReceiveTimeout();
- }
- public Builder setReceiveTimeout(long value) {
- result.hasReceiveTimeout = true;
- result.receiveTimeout_ = value;
- return this;
- }
- public Builder clearReceiveTimeout() {
- result.hasReceiveTimeout = false;
- result.receiveTimeout_ = 0L;
- return this;
- }
-
- // optional .LifeCycleProtocol lifeCycle = 10;
- public boolean hasLifeCycle() {
- return result.hasLifeCycle();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() {
- return result.getLifeCycle();
- }
- public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasLifeCycle = true;
- result.lifeCycle_ = value;
- return this;
- }
- public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder builderForValue) {
- result.hasLifeCycle = true;
- result.lifeCycle_ = builderForValue.build();
- return this;
- }
- public Builder mergeLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) {
- if (result.hasLifeCycle() &&
- result.lifeCycle_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) {
- result.lifeCycle_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(result.lifeCycle_).mergeFrom(value).buildPartial();
- } else {
- result.lifeCycle_ = value;
- }
- result.hasLifeCycle = true;
- return this;
- }
- public Builder clearLifeCycle() {
- result.hasLifeCycle = false;
- result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
- return this;
- }
-
- // optional .RemoteActorRefProtocol supervisor = 11;
- public boolean hasSupervisor() {
- return result.hasSupervisor();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() {
- return result.getSupervisor();
- }
- public Builder setSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasSupervisor = true;
- result.supervisor_ = value;
- return this;
- }
- public Builder setSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) {
- result.hasSupervisor = true;
- result.supervisor_ = builderForValue.build();
- return this;
- }
- public Builder mergeSupervisor(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
- if (result.hasSupervisor() &&
- result.supervisor_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) {
- result.supervisor_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.supervisor_).mergeFrom(value).buildPartial();
- } else {
- result.supervisor_ = value;
- }
- result.hasSupervisor = true;
- return this;
- }
- public Builder clearSupervisor() {
- result.hasSupervisor = false;
- result.supervisor_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
- return this;
- }
-
- // optional bytes hotswapStack = 12;
- public boolean hasHotswapStack() {
- return result.hasHotswapStack();
- }
- public com.google.protobuf.ByteString getHotswapStack() {
- return result.getHotswapStack();
- }
- public Builder setHotswapStack(com.google.protobuf.ByteString value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasHotswapStack = true;
- result.hotswapStack_ = value;
- return this;
- }
- public Builder clearHotswapStack() {
- result.hasHotswapStack = false;
- result.hotswapStack_ = getDefaultInstance().getHotswapStack();
- return this;
- }
-
- // repeated .RemoteRequestProtocol messages = 13;
- public java.util.List getMessagesList() {
- return java.util.Collections.unmodifiableList(result.messages_);
- }
- public int getMessagesCount() {
- return result.getMessagesCount();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getMessages(int index) {
- return result.getMessages(index);
- }
- public Builder setMessages(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.messages_.set(index, value);
- return this;
- }
- public Builder setMessages(int index, se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder builderForValue) {
- result.messages_.set(index, builderForValue.build());
- return this;
- }
- public Builder addMessages(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- if (result.messages_.isEmpty()) {
- result.messages_ = new java.util.ArrayList();
- }
- result.messages_.add(value);
- return this;
- }
- public Builder addMessages(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder builderForValue) {
- if (result.messages_.isEmpty()) {
- result.messages_ = new java.util.ArrayList();
- }
- result.messages_.add(builderForValue.build());
- return this;
- }
- public Builder addAllMessages(
- java.lang.Iterable extends se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol> values) {
- if (result.messages_.isEmpty()) {
- result.messages_ = new java.util.ArrayList();
- }
- super.addAll(values, result.messages_);
- return this;
- }
- public Builder clearMessages() {
- result.messages_ = java.util.Collections.emptyList();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:SerializedActorRefProtocol)
- }
-
- static {
- defaultInstance = new SerializedActorRefProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:SerializedActorRefProtocol)
- }
-
- public static final class MessageProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use MessageProtocol.newBuilder() to construct.
- private MessageProtocol() {
- initFields();
- }
- private MessageProtocol(boolean noInit) {}
-
- private static final MessageProtocol defaultInstance;
- public static MessageProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public MessageProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_fieldAccessorTable;
- }
-
- // required .SerializationSchemeType serializationScheme = 1;
- public static final int SERIALIZATIONSCHEME_FIELD_NUMBER = 1;
- private boolean hasSerializationScheme;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType serializationScheme_;
- public boolean hasSerializationScheme() { return hasSerializationScheme; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { return serializationScheme_; }
-
- // required bytes message = 2;
- public static final int MESSAGE_FIELD_NUMBER = 2;
- private boolean hasMessage;
- private com.google.protobuf.ByteString message_ = com.google.protobuf.ByteString.EMPTY;
- public boolean hasMessage() { return hasMessage; }
- public com.google.protobuf.ByteString getMessage() { return message_; }
-
- // optional bytes messageManifest = 3;
- public static final int MESSAGEMANIFEST_FIELD_NUMBER = 3;
- private boolean hasMessageManifest;
- private com.google.protobuf.ByteString messageManifest_ = com.google.protobuf.ByteString.EMPTY;
- public boolean hasMessageManifest() { return hasMessageManifest; }
- public com.google.protobuf.ByteString getMessageManifest() { return messageManifest_; }
-
- private void initFields() {
- serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA;
- }
- public final boolean isInitialized() {
- if (!hasSerializationScheme) return false;
- if (!hasMessage) return false;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasSerializationScheme()) {
- output.writeEnum(1, getSerializationScheme().getNumber());
- }
- if (hasMessage()) {
- output.writeBytes(2, getMessage());
- }
- if (hasMessageManifest()) {
- output.writeBytes(3, getMessageManifest());
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasSerializationScheme()) {
- size += com.google.protobuf.CodedOutputStream
- .computeEnumSize(1, getSerializationScheme().getNumber());
- }
- if (hasMessage()) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(2, getMessage());
- }
- if (hasMessageManifest()) {
- size += com.google.protobuf.CodedOutputStream
- .computeBytesSize(3, getMessageManifest());
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) return this;
- if (other.hasSerializationScheme()) {
- setSerializationScheme(other.getSerializationScheme());
- }
- if (other.hasMessage()) {
- setMessage(other.getMessage());
- }
- if (other.hasMessageManifest()) {
- setMessageManifest(other.getMessageManifest());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 8: {
- int rawValue = input.readEnum();
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.valueOf(rawValue);
- if (value == null) {
- unknownFields.mergeVarintField(1, rawValue);
- } else {
- setSerializationScheme(value);
- }
- break;
- }
- case 18: {
- setMessage(input.readBytes());
- break;
- }
- case 26: {
- setMessageManifest(input.readBytes());
- break;
- }
- }
- }
- }
-
-
- // required .SerializationSchemeType serializationScheme = 1;
- public boolean hasSerializationScheme() {
- return result.hasSerializationScheme();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() {
- return result.getSerializationScheme();
- }
- public Builder setSerializationScheme(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasSerializationScheme = true;
- result.serializationScheme_ = value;
- return this;
- }
- public Builder clearSerializationScheme() {
- result.hasSerializationScheme = false;
- result.serializationScheme_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA;
- return this;
- }
-
- // required bytes message = 2;
- public boolean hasMessage() {
- return result.hasMessage();
- }
- public com.google.protobuf.ByteString getMessage() {
- return result.getMessage();
- }
- public Builder setMessage(com.google.protobuf.ByteString value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasMessage = true;
- result.message_ = value;
- return this;
- }
- public Builder clearMessage() {
- result.hasMessage = false;
- result.message_ = getDefaultInstance().getMessage();
- return this;
- }
-
- // optional bytes messageManifest = 3;
- public boolean hasMessageManifest() {
- return result.hasMessageManifest();
- }
- public com.google.protobuf.ByteString getMessageManifest() {
- return result.getMessageManifest();
- }
- public Builder setMessageManifest(com.google.protobuf.ByteString value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasMessageManifest = true;
- result.messageManifest_ = value;
- return this;
- }
- public Builder clearMessageManifest() {
- result.hasMessageManifest = false;
- result.messageManifest_ = getDefaultInstance().getMessageManifest();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:MessageProtocol)
- }
-
- static {
- defaultInstance = new MessageProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:MessageProtocol)
- }
-
- public static final class ActorInfoProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use ActorInfoProtocol.newBuilder() to construct.
- private ActorInfoProtocol() {
- initFields();
- }
- private ActorInfoProtocol(boolean noInit) {}
-
- private static final ActorInfoProtocol defaultInstance;
- public static ActorInfoProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public ActorInfoProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_fieldAccessorTable;
- }
-
- // required string uuid = 1;
- public static final int UUID_FIELD_NUMBER = 1;
- private boolean hasUuid;
- private java.lang.String uuid_ = "";
- public boolean hasUuid() { return hasUuid; }
- public java.lang.String getUuid() { return uuid_; }
-
- // required string target = 2;
- public static final int TARGET_FIELD_NUMBER = 2;
- private boolean hasTarget;
- private java.lang.String target_ = "";
- public boolean hasTarget() { return hasTarget; }
- public java.lang.String getTarget() { return target_; }
-
- // required uint64 timeout = 3;
- public static final int TIMEOUT_FIELD_NUMBER = 3;
- private boolean hasTimeout;
- private long timeout_ = 0L;
- public boolean hasTimeout() { return hasTimeout; }
- public long getTimeout() { return timeout_; }
-
- // required .ActorType actorType = 4;
- public static final int ACTORTYPE_FIELD_NUMBER = 4;
- private boolean hasActorType;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType actorType_;
- public boolean hasActorType() { return hasActorType; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() { return actorType_; }
-
- // optional .TypedActorInfoProtocol typedActorInfo = 5;
- public static final int TYPEDACTORINFO_FIELD_NUMBER = 5;
- private boolean hasTypedActorInfo;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol typedActorInfo_;
- public boolean hasTypedActorInfo() { return hasTypedActorInfo; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() { return typedActorInfo_; }
-
- private void initFields() {
- actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR;
- typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
- }
- public final boolean isInitialized() {
- if (!hasUuid) return false;
- if (!hasTarget) return false;
- if (!hasTimeout) return false;
- if (!hasActorType) return false;
- if (hasTypedActorInfo()) {
- if (!getTypedActorInfo().isInitialized()) return false;
- }
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasUuid()) {
- output.writeString(1, getUuid());
- }
- if (hasTarget()) {
- output.writeString(2, getTarget());
- }
- if (hasTimeout()) {
- output.writeUInt64(3, getTimeout());
- }
- if (hasActorType()) {
- output.writeEnum(4, getActorType().getNumber());
- }
- if (hasTypedActorInfo()) {
- output.writeMessage(5, getTypedActorInfo());
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasUuid()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(1, getUuid());
- }
- if (hasTarget()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(2, getTarget());
- }
- if (hasTimeout()) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(3, getTimeout());
- }
- if (hasActorType()) {
- size += com.google.protobuf.CodedOutputStream
- .computeEnumSize(4, getActorType().getNumber());
- }
- if (hasTypedActorInfo()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(5, getTypedActorInfo());
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) return this;
- if (other.hasUuid()) {
- setUuid(other.getUuid());
- }
- if (other.hasTarget()) {
- setTarget(other.getTarget());
- }
- if (other.hasTimeout()) {
- setTimeout(other.getTimeout());
- }
- if (other.hasActorType()) {
- setActorType(other.getActorType());
- }
- if (other.hasTypedActorInfo()) {
- mergeTypedActorInfo(other.getTypedActorInfo());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 10: {
- setUuid(input.readString());
- break;
- }
- case 18: {
- setTarget(input.readString());
- break;
- }
- case 24: {
- setTimeout(input.readUInt64());
- break;
- }
- case 32: {
- int rawValue = input.readEnum();
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.valueOf(rawValue);
- if (value == null) {
- unknownFields.mergeVarintField(4, rawValue);
- } else {
- setActorType(value);
- }
- break;
- }
- case 42: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder();
- if (hasTypedActorInfo()) {
- subBuilder.mergeFrom(getTypedActorInfo());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setTypedActorInfo(subBuilder.buildPartial());
- break;
- }
- }
- }
- }
-
-
- // required string uuid = 1;
- 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 string target = 2;
- public boolean hasTarget() {
- return result.hasTarget();
- }
- public java.lang.String getTarget() {
- return result.getTarget();
- }
- public Builder setTarget(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasTarget = true;
- result.target_ = value;
- return this;
- }
- public Builder clearTarget() {
- result.hasTarget = false;
- result.target_ = getDefaultInstance().getTarget();
- return this;
- }
-
- // required uint64 timeout = 3;
- public boolean hasTimeout() {
- return result.hasTimeout();
- }
- public long getTimeout() {
- return result.getTimeout();
- }
- public Builder setTimeout(long value) {
- result.hasTimeout = true;
- result.timeout_ = value;
- return this;
- }
- public Builder clearTimeout() {
- result.hasTimeout = false;
- result.timeout_ = 0L;
- return this;
- }
-
- // required .ActorType actorType = 4;
- public boolean hasActorType() {
- return result.hasActorType();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType getActorType() {
- return result.getActorType();
- }
- public Builder setActorType(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasActorType = true;
- result.actorType_ = value;
- return this;
- }
- public Builder clearActorType() {
- result.hasActorType = false;
- result.actorType_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorType.SCALA_ACTOR;
- return this;
- }
-
- // optional .TypedActorInfoProtocol typedActorInfo = 5;
- public boolean hasTypedActorInfo() {
- return result.hasTypedActorInfo();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getTypedActorInfo() {
- return result.getTypedActorInfo();
- }
- public Builder setTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasTypedActorInfo = true;
- result.typedActorInfo_ = value;
- return this;
- }
- public Builder setTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder builderForValue) {
- result.hasTypedActorInfo = true;
- result.typedActorInfo_ = builderForValue.build();
- return this;
- }
- public Builder mergeTypedActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol value) {
- if (result.hasTypedActorInfo() &&
- result.typedActorInfo_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) {
- result.typedActorInfo_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder(result.typedActorInfo_).mergeFrom(value).buildPartial();
- } else {
- result.typedActorInfo_ = value;
- }
- result.hasTypedActorInfo = true;
- return this;
- }
- public Builder clearTypedActorInfo() {
- result.hasTypedActorInfo = false;
- result.typedActorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:ActorInfoProtocol)
- }
-
- static {
- defaultInstance = new ActorInfoProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:ActorInfoProtocol)
- }
-
- public static final class TypedActorInfoProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use TypedActorInfoProtocol.newBuilder() to construct.
- private TypedActorInfoProtocol() {
- initFields();
- }
- private TypedActorInfoProtocol(boolean noInit) {}
-
- private static final TypedActorInfoProtocol defaultInstance;
- public static TypedActorInfoProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public TypedActorInfoProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_TypedActorInfoProtocol_fieldAccessorTable;
- }
-
- // required string interface = 1;
- public static final int INTERFACE_FIELD_NUMBER = 1;
- private boolean hasInterface;
- private java.lang.String interface_ = "";
- public boolean hasInterface() { return hasInterface; }
- public java.lang.String getInterface() { return interface_; }
-
- // required string method = 2;
- public static final int METHOD_FIELD_NUMBER = 2;
- private boolean hasMethod;
- private java.lang.String method_ = "";
- public boolean hasMethod() { return hasMethod; }
- public java.lang.String getMethod() { return method_; }
-
- private void initFields() {
- }
- public final boolean isInitialized() {
- if (!hasInterface) return false;
- if (!hasMethod) return false;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasInterface()) {
- output.writeString(1, getInterface());
- }
- if (hasMethod()) {
- output.writeString(2, getMethod());
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasInterface()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(1, getInterface());
- }
- if (hasMethod()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(2, getMethod());
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.getDefaultInstance()) return this;
- if (other.hasInterface()) {
- setInterface(other.getInterface());
- }
- if (other.hasMethod()) {
- setMethod(other.getMethod());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 10: {
- setInterface(input.readString());
- break;
- }
- case 18: {
- setMethod(input.readString());
- break;
- }
- }
- }
- }
-
-
- // required string interface = 1;
- public boolean hasInterface() {
- return result.hasInterface();
- }
- public java.lang.String getInterface() {
- return result.getInterface();
- }
- public Builder setInterface(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasInterface = true;
- result.interface_ = value;
- return this;
- }
- public Builder clearInterface() {
- result.hasInterface = false;
- result.interface_ = getDefaultInstance().getInterface();
- return this;
- }
-
- // required string method = 2;
- public boolean hasMethod() {
- return result.hasMethod();
- }
- public java.lang.String getMethod() {
- return result.getMethod();
- }
- public Builder setMethod(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasMethod = true;
- result.method_ = value;
- return this;
- }
- public Builder clearMethod() {
- result.hasMethod = false;
- result.method_ = getDefaultInstance().getMethod();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:TypedActorInfoProtocol)
- }
-
- static {
- defaultInstance = new TypedActorInfoProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:TypedActorInfoProtocol)
- }
-
- public static final class RemoteRequestProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use RemoteRequestProtocol.newBuilder() to construct.
- private RemoteRequestProtocol() {
- initFields();
- }
- private RemoteRequestProtocol(boolean noInit) {}
-
- private static final RemoteRequestProtocol defaultInstance;
- public static RemoteRequestProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public RemoteRequestProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteRequestProtocol_fieldAccessorTable;
- }
-
- // required uint64 id = 1;
- public static final int ID_FIELD_NUMBER = 1;
- private boolean hasId;
- private long id_ = 0L;
- public boolean hasId() { return hasId; }
- public long getId() { return id_; }
-
- // required .MessageProtocol message = 2;
- public static final int MESSAGE_FIELD_NUMBER = 2;
- private boolean hasMessage;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_;
- public boolean hasMessage() { return hasMessage; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; }
-
- // required .ActorInfoProtocol actorInfo = 3;
- public static final int ACTORINFO_FIELD_NUMBER = 3;
- private boolean hasActorInfo;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol actorInfo_;
- public boolean hasActorInfo() { return hasActorInfo; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { return actorInfo_; }
-
- // required bool isOneWay = 4;
- public static final int ISONEWAY_FIELD_NUMBER = 4;
- private boolean hasIsOneWay;
- private boolean isOneWay_ = false;
- public boolean hasIsOneWay() { return hasIsOneWay; }
- public boolean getIsOneWay() { return isOneWay_; }
-
- // optional string supervisorUuid = 5;
- public static final int SUPERVISORUUID_FIELD_NUMBER = 5;
- private boolean hasSupervisorUuid;
- private java.lang.String supervisorUuid_ = "";
- public boolean hasSupervisorUuid() { return hasSupervisorUuid; }
- public java.lang.String getSupervisorUuid() { return supervisorUuid_; }
-
- // optional .RemoteActorRefProtocol sender = 6;
- public static final int SENDER_FIELD_NUMBER = 6;
- private boolean hasSender;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol sender_;
- public boolean hasSender() { return hasSender; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { return sender_; }
-
- private void initFields() {
- message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
- actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
- sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
- }
- public final boolean isInitialized() {
- if (!hasId) return false;
- if (!hasMessage) return false;
- if (!hasActorInfo) return false;
- if (!hasIsOneWay) return false;
- if (!getMessage().isInitialized()) return false;
- if (!getActorInfo().isInitialized()) return false;
- if (hasSender()) {
- if (!getSender().isInitialized()) return false;
- }
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasId()) {
- output.writeUInt64(1, getId());
- }
- if (hasMessage()) {
- output.writeMessage(2, getMessage());
- }
- if (hasActorInfo()) {
- output.writeMessage(3, getActorInfo());
- }
- if (hasIsOneWay()) {
- output.writeBool(4, getIsOneWay());
- }
- if (hasSupervisorUuid()) {
- output.writeString(5, getSupervisorUuid());
- }
- if (hasSender()) {
- output.writeMessage(6, getSender());
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasId()) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(1, getId());
- }
- if (hasMessage()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(2, getMessage());
- }
- if (hasActorInfo()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(3, getActorInfo());
- }
- if (hasIsOneWay()) {
- size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(4, getIsOneWay());
- }
- if (hasSupervisorUuid()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(5, getSupervisorUuid());
- }
- if (hasSender()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(6, getSender());
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.getDefaultInstance()) return this;
- if (other.hasId()) {
- setId(other.getId());
- }
- if (other.hasMessage()) {
- mergeMessage(other.getMessage());
- }
- if (other.hasActorInfo()) {
- mergeActorInfo(other.getActorInfo());
- }
- if (other.hasIsOneWay()) {
- setIsOneWay(other.getIsOneWay());
- }
- if (other.hasSupervisorUuid()) {
- setSupervisorUuid(other.getSupervisorUuid());
- }
- if (other.hasSender()) {
- mergeSender(other.getSender());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 8: {
- setId(input.readUInt64());
- break;
- }
- case 18: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder();
- if (hasMessage()) {
- subBuilder.mergeFrom(getMessage());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setMessage(subBuilder.buildPartial());
- break;
- }
- case 26: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder();
- if (hasActorInfo()) {
- subBuilder.mergeFrom(getActorInfo());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setActorInfo(subBuilder.buildPartial());
- break;
- }
- case 32: {
- setIsOneWay(input.readBool());
- break;
- }
- case 42: {
- setSupervisorUuid(input.readString());
- break;
- }
- case 50: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder();
- if (hasSender()) {
- subBuilder.mergeFrom(getSender());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setSender(subBuilder.buildPartial());
- break;
- }
- }
- }
- }
-
-
- // required uint64 id = 1;
- public boolean hasId() {
- return result.hasId();
- }
- public long getId() {
- return result.getId();
- }
- public Builder setId(long value) {
- result.hasId = true;
- result.id_ = value;
- return this;
- }
- public Builder clearId() {
- result.hasId = false;
- result.id_ = 0L;
- return this;
- }
-
- // required .MessageProtocol message = 2;
- public boolean hasMessage() {
- return result.hasMessage();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() {
- return result.getMessage();
- }
- public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasMessage = true;
- result.message_ = value;
- return this;
- }
- public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) {
- result.hasMessage = true;
- result.message_ = builderForValue.build();
- return this;
- }
- public Builder mergeMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) {
- if (result.hasMessage() &&
- result.message_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) {
- result.message_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial();
- } else {
- result.message_ = value;
- }
- result.hasMessage = true;
- return this;
- }
- public Builder clearMessage() {
- result.hasMessage = false;
- result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
- return this;
- }
-
- // required .ActorInfoProtocol actorInfo = 3;
- public boolean hasActorInfo() {
- return result.hasActorInfo();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() {
- return result.getActorInfo();
- }
- public Builder setActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasActorInfo = true;
- result.actorInfo_ = value;
- return this;
- }
- public Builder setActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder builderForValue) {
- result.hasActorInfo = true;
- result.actorInfo_ = builderForValue.build();
- return this;
- }
- public Builder mergeActorInfo(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) {
- if (result.hasActorInfo() &&
- result.actorInfo_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) {
- result.actorInfo_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(result.actorInfo_).mergeFrom(value).buildPartial();
- } else {
- result.actorInfo_ = value;
- }
- result.hasActorInfo = true;
- return this;
- }
- public Builder clearActorInfo() {
- result.hasActorInfo = false;
- result.actorInfo_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance();
- return this;
- }
-
- // required bool isOneWay = 4;
- public boolean hasIsOneWay() {
- return result.hasIsOneWay();
- }
- public boolean getIsOneWay() {
- return result.getIsOneWay();
- }
- public Builder setIsOneWay(boolean value) {
- result.hasIsOneWay = true;
- result.isOneWay_ = value;
- return this;
- }
- public Builder clearIsOneWay() {
- result.hasIsOneWay = false;
- result.isOneWay_ = false;
- return this;
- }
-
- // optional string supervisorUuid = 5;
- public boolean hasSupervisorUuid() {
- return result.hasSupervisorUuid();
- }
- public java.lang.String getSupervisorUuid() {
- return result.getSupervisorUuid();
- }
- public Builder setSupervisorUuid(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasSupervisorUuid = true;
- result.supervisorUuid_ = value;
- return this;
- }
- public Builder clearSupervisorUuid() {
- result.hasSupervisorUuid = false;
- result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid();
- return this;
- }
-
- // optional .RemoteActorRefProtocol sender = 6;
- public boolean hasSender() {
- return result.hasSender();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() {
- return result.getSender();
- }
- public Builder setSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasSender = true;
- result.sender_ = value;
- return this;
- }
- public Builder setSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) {
- result.hasSender = true;
- result.sender_ = builderForValue.build();
- return this;
- }
- public Builder mergeSender(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
- if (result.hasSender() &&
- result.sender_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) {
- result.sender_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.sender_).mergeFrom(value).buildPartial();
- } else {
- result.sender_ = value;
- }
- result.hasSender = true;
- return this;
- }
- public Builder clearSender() {
- result.hasSender = false;
- result.sender_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:RemoteRequestProtocol)
- }
-
- static {
- defaultInstance = new RemoteRequestProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:RemoteRequestProtocol)
- }
-
- public static final class RemoteReplyProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use RemoteReplyProtocol.newBuilder() to construct.
- private RemoteReplyProtocol() {
- initFields();
- }
- private RemoteReplyProtocol(boolean noInit) {}
-
- private static final RemoteReplyProtocol defaultInstance;
- public static RemoteReplyProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public RemoteReplyProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteReplyProtocol_fieldAccessorTable;
- }
-
- // required uint64 id = 1;
- public static final int ID_FIELD_NUMBER = 1;
- private boolean hasId;
- private long id_ = 0L;
- public boolean hasId() { return hasId; }
- public long getId() { return id_; }
-
- // optional .MessageProtocol message = 2;
- public static final int MESSAGE_FIELD_NUMBER = 2;
- private boolean hasMessage;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol message_;
- public boolean hasMessage() { return hasMessage; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; }
-
- // optional .ExceptionProtocol exception = 3;
- public static final int EXCEPTION_FIELD_NUMBER = 3;
- private boolean hasException;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol exception_;
- public boolean hasException() { return hasException; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return exception_; }
-
- // optional string supervisorUuid = 4;
- public static final int SUPERVISORUUID_FIELD_NUMBER = 4;
- private boolean hasSupervisorUuid;
- private java.lang.String supervisorUuid_ = "";
- public boolean hasSupervisorUuid() { return hasSupervisorUuid; }
- public java.lang.String getSupervisorUuid() { return supervisorUuid_; }
-
- // required bool isActor = 5;
- public static final int ISACTOR_FIELD_NUMBER = 5;
- private boolean hasIsActor;
- private boolean isActor_ = false;
- public boolean hasIsActor() { return hasIsActor; }
- public boolean getIsActor() { return isActor_; }
-
- // required bool isSuccessful = 6;
- public static final int ISSUCCESSFUL_FIELD_NUMBER = 6;
- private boolean hasIsSuccessful;
- private boolean isSuccessful_ = false;
- public boolean hasIsSuccessful() { return hasIsSuccessful; }
- public boolean getIsSuccessful() { return isSuccessful_; }
-
- private void initFields() {
- message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
- exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
- }
- public final boolean isInitialized() {
- if (!hasId) return false;
- if (!hasIsActor) return false;
- if (!hasIsSuccessful) return false;
- if (hasMessage()) {
- if (!getMessage().isInitialized()) return false;
- }
- if (hasException()) {
- if (!getException().isInitialized()) return false;
- }
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasId()) {
- output.writeUInt64(1, getId());
- }
- if (hasMessage()) {
- output.writeMessage(2, getMessage());
- }
- if (hasException()) {
- output.writeMessage(3, getException());
- }
- if (hasSupervisorUuid()) {
- output.writeString(4, getSupervisorUuid());
- }
- if (hasIsActor()) {
- output.writeBool(5, getIsActor());
- }
- if (hasIsSuccessful()) {
- output.writeBool(6, getIsSuccessful());
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasId()) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt64Size(1, getId());
- }
- if (hasMessage()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(2, getMessage());
- }
- if (hasException()) {
- size += com.google.protobuf.CodedOutputStream
- .computeMessageSize(3, getException());
- }
- if (hasSupervisorUuid()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(4, getSupervisorUuid());
- }
- if (hasIsActor()) {
- size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(5, getIsActor());
- }
- if (hasIsSuccessful()) {
- size += com.google.protobuf.CodedOutputStream
- .computeBoolSize(6, getIsSuccessful());
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.getDefaultInstance()) return this;
- if (other.hasId()) {
- setId(other.getId());
- }
- if (other.hasMessage()) {
- mergeMessage(other.getMessage());
- }
- if (other.hasException()) {
- mergeException(other.getException());
- }
- if (other.hasSupervisorUuid()) {
- setSupervisorUuid(other.getSupervisorUuid());
- }
- if (other.hasIsActor()) {
- setIsActor(other.getIsActor());
- }
- if (other.hasIsSuccessful()) {
- setIsSuccessful(other.getIsSuccessful());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 8: {
- setId(input.readUInt64());
- break;
- }
- case 18: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder();
- if (hasMessage()) {
- subBuilder.mergeFrom(getMessage());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setMessage(subBuilder.buildPartial());
- break;
- }
- case 26: {
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder();
- if (hasException()) {
- subBuilder.mergeFrom(getException());
- }
- input.readMessage(subBuilder, extensionRegistry);
- setException(subBuilder.buildPartial());
- break;
- }
- case 34: {
- setSupervisorUuid(input.readString());
- break;
- }
- case 40: {
- setIsActor(input.readBool());
- break;
- }
- case 48: {
- setIsSuccessful(input.readBool());
- break;
- }
- }
- }
- }
-
-
- // required uint64 id = 1;
- public boolean hasId() {
- return result.hasId();
- }
- public long getId() {
- return result.getId();
- }
- public Builder setId(long value) {
- result.hasId = true;
- result.id_ = value;
- return this;
- }
- public Builder clearId() {
- result.hasId = false;
- result.id_ = 0L;
- return this;
- }
-
- // optional .MessageProtocol message = 2;
- public boolean hasMessage() {
- return result.hasMessage();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() {
- return result.getMessage();
- }
- public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasMessage = true;
- result.message_ = value;
- return this;
- }
- public Builder setMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) {
- result.hasMessage = true;
- result.message_ = builderForValue.build();
- return this;
- }
- public Builder mergeMessage(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol value) {
- if (result.hasMessage() &&
- result.message_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) {
- result.message_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial();
- } else {
- result.message_ = value;
- }
- result.hasMessage = true;
- return this;
- }
- public Builder clearMessage() {
- result.hasMessage = false;
- result.message_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance();
- return this;
- }
-
- // optional .ExceptionProtocol exception = 3;
- public boolean hasException() {
- return result.hasException();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() {
- return result.getException();
- }
- public Builder setException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasException = true;
- result.exception_ = value;
- return this;
- }
- public Builder setException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder builderForValue) {
- result.hasException = true;
- result.exception_ = builderForValue.build();
- return this;
- }
- public Builder mergeException(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) {
- if (result.hasException() &&
- result.exception_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) {
- result.exception_ =
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(result.exception_).mergeFrom(value).buildPartial();
- } else {
- result.exception_ = value;
- }
- result.hasException = true;
- return this;
- }
- public Builder clearException() {
- result.hasException = false;
- result.exception_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
- return this;
- }
-
- // optional string supervisorUuid = 4;
- public boolean hasSupervisorUuid() {
- return result.hasSupervisorUuid();
- }
- public java.lang.String getSupervisorUuid() {
- return result.getSupervisorUuid();
- }
- public Builder setSupervisorUuid(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasSupervisorUuid = true;
- result.supervisorUuid_ = value;
- return this;
- }
- public Builder clearSupervisorUuid() {
- result.hasSupervisorUuid = false;
- result.supervisorUuid_ = getDefaultInstance().getSupervisorUuid();
- return this;
- }
-
- // required bool isActor = 5;
- public boolean hasIsActor() {
- return result.hasIsActor();
- }
- public boolean getIsActor() {
- return result.getIsActor();
- }
- public Builder setIsActor(boolean value) {
- result.hasIsActor = true;
- result.isActor_ = value;
- return this;
- }
- public Builder clearIsActor() {
- result.hasIsActor = false;
- result.isActor_ = false;
- return this;
- }
-
- // required bool isSuccessful = 6;
- public boolean hasIsSuccessful() {
- return result.hasIsSuccessful();
- }
- public boolean getIsSuccessful() {
- return result.getIsSuccessful();
- }
- public Builder setIsSuccessful(boolean value) {
- result.hasIsSuccessful = true;
- result.isSuccessful_ = value;
- return this;
- }
- public Builder clearIsSuccessful() {
- result.hasIsSuccessful = false;
- result.isSuccessful_ = false;
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:RemoteReplyProtocol)
- }
-
- static {
- defaultInstance = new RemoteReplyProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:RemoteReplyProtocol)
- }
-
- public static final class LifeCycleProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use LifeCycleProtocol.newBuilder() to construct.
- private LifeCycleProtocol() {
- initFields();
- }
- private LifeCycleProtocol(boolean noInit) {}
-
- private static final LifeCycleProtocol defaultInstance;
- public static LifeCycleProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public LifeCycleProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_fieldAccessorTable;
- }
-
- // required .LifeCycleType lifeCycle = 1;
- public static final int LIFECYCLE_FIELD_NUMBER = 1;
- private boolean hasLifeCycle;
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType lifeCycle_;
- public boolean hasLifeCycle() { return hasLifeCycle; }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { return lifeCycle_; }
-
- // optional string preRestart = 2;
- public static final int PRERESTART_FIELD_NUMBER = 2;
- private boolean hasPreRestart;
- private java.lang.String preRestart_ = "";
- public boolean hasPreRestart() { return hasPreRestart; }
- public java.lang.String getPreRestart() { return preRestart_; }
-
- // optional string postRestart = 3;
- public static final int POSTRESTART_FIELD_NUMBER = 3;
- private boolean hasPostRestart;
- private java.lang.String postRestart_ = "";
- public boolean hasPostRestart() { return hasPostRestart; }
- public java.lang.String getPostRestart() { return postRestart_; }
-
- // optional string init = 4;
- public static final int INIT_FIELD_NUMBER = 4;
- private boolean hasInit;
- private java.lang.String init_ = "";
- public boolean hasInit() { return hasInit; }
- public java.lang.String getInit() { return init_; }
-
- // optional string shutdown = 5;
- public static final int SHUTDOWN_FIELD_NUMBER = 5;
- private boolean hasShutdown;
- private java.lang.String shutdown_ = "";
- public boolean hasShutdown() { return hasShutdown; }
- public java.lang.String getShutdown() { return shutdown_; }
-
- private void initFields() {
- lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT;
- }
- public final boolean isInitialized() {
- if (!hasLifeCycle) return false;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasLifeCycle()) {
- output.writeEnum(1, getLifeCycle().getNumber());
- }
- if (hasPreRestart()) {
- output.writeString(2, getPreRestart());
- }
- if (hasPostRestart()) {
- output.writeString(3, getPostRestart());
- }
- if (hasInit()) {
- output.writeString(4, getInit());
- }
- if (hasShutdown()) {
- output.writeString(5, getShutdown());
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasLifeCycle()) {
- size += com.google.protobuf.CodedOutputStream
- .computeEnumSize(1, getLifeCycle().getNumber());
- }
- if (hasPreRestart()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(2, getPreRestart());
- }
- if (hasPostRestart()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(3, getPostRestart());
- }
- if (hasInit()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(4, getInit());
- }
- if (hasShutdown()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(5, getShutdown());
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) return this;
- if (other.hasLifeCycle()) {
- setLifeCycle(other.getLifeCycle());
- }
- if (other.hasPreRestart()) {
- setPreRestart(other.getPreRestart());
- }
- if (other.hasPostRestart()) {
- setPostRestart(other.getPostRestart());
- }
- if (other.hasInit()) {
- setInit(other.getInit());
- }
- if (other.hasShutdown()) {
- setShutdown(other.getShutdown());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 8: {
- int rawValue = input.readEnum();
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType value = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.valueOf(rawValue);
- if (value == null) {
- unknownFields.mergeVarintField(1, rawValue);
- } else {
- setLifeCycle(value);
- }
- break;
- }
- case 18: {
- setPreRestart(input.readString());
- break;
- }
- case 26: {
- setPostRestart(input.readString());
- break;
- }
- case 34: {
- setInit(input.readString());
- break;
- }
- case 42: {
- setShutdown(input.readString());
- break;
- }
- }
- }
- }
-
-
- // required .LifeCycleType lifeCycle = 1;
- public boolean hasLifeCycle() {
- return result.hasLifeCycle();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() {
- return result.getLifeCycle();
- }
- public Builder setLifeCycle(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasLifeCycle = true;
- result.lifeCycle_ = value;
- return this;
- }
- public Builder clearLifeCycle() {
- result.hasLifeCycle = false;
- result.lifeCycle_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT;
- return this;
- }
-
- // optional string preRestart = 2;
- public boolean hasPreRestart() {
- return result.hasPreRestart();
- }
- public java.lang.String getPreRestart() {
- return result.getPreRestart();
- }
- public Builder setPreRestart(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasPreRestart = true;
- result.preRestart_ = value;
- return this;
- }
- public Builder clearPreRestart() {
- result.hasPreRestart = false;
- result.preRestart_ = getDefaultInstance().getPreRestart();
- return this;
- }
-
- // optional string postRestart = 3;
- public boolean hasPostRestart() {
- return result.hasPostRestart();
- }
- public java.lang.String getPostRestart() {
- return result.getPostRestart();
- }
- public Builder setPostRestart(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasPostRestart = true;
- result.postRestart_ = value;
- return this;
- }
- public Builder clearPostRestart() {
- result.hasPostRestart = false;
- result.postRestart_ = getDefaultInstance().getPostRestart();
- return this;
- }
-
- // optional string init = 4;
- public boolean hasInit() {
- return result.hasInit();
- }
- public java.lang.String getInit() {
- return result.getInit();
- }
- public Builder setInit(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasInit = true;
- result.init_ = value;
- return this;
- }
- public Builder clearInit() {
- result.hasInit = false;
- result.init_ = getDefaultInstance().getInit();
- return this;
- }
-
- // optional string shutdown = 5;
- public boolean hasShutdown() {
- return result.hasShutdown();
- }
- public java.lang.String getShutdown() {
- return result.getShutdown();
- }
- public Builder setShutdown(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasShutdown = true;
- result.shutdown_ = value;
- return this;
- }
- public Builder clearShutdown() {
- result.hasShutdown = false;
- result.shutdown_ = getDefaultInstance().getShutdown();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:LifeCycleProtocol)
- }
-
- static {
- defaultInstance = new LifeCycleProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:LifeCycleProtocol)
- }
-
- public static final class AddressProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use AddressProtocol.newBuilder() to construct.
- private AddressProtocol() {
- initFields();
- }
- private AddressProtocol(boolean noInit) {}
-
- private static final AddressProtocol defaultInstance;
- public static AddressProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public AddressProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_fieldAccessorTable;
- }
-
- // required string hostname = 1;
- public static final int HOSTNAME_FIELD_NUMBER = 1;
- private boolean hasHostname;
- private java.lang.String hostname_ = "";
- public boolean hasHostname() { return hasHostname; }
- public java.lang.String getHostname() { return hostname_; }
-
- // required uint32 port = 2;
- public static final int PORT_FIELD_NUMBER = 2;
- private boolean hasPort;
- private int port_ = 0;
- public boolean hasPort() { return hasPort; }
- public int getPort() { return port_; }
-
- private void initFields() {
- }
- public final boolean isInitialized() {
- if (!hasHostname) return false;
- if (!hasPort) return false;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasHostname()) {
- output.writeString(1, getHostname());
- }
- if (hasPort()) {
- output.writeUInt32(2, getPort());
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasHostname()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(1, getHostname());
- }
- if (hasPort()) {
- size += com.google.protobuf.CodedOutputStream
- .computeUInt32Size(2, getPort());
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance()) return this;
- if (other.hasHostname()) {
- setHostname(other.getHostname());
- }
- if (other.hasPort()) {
- setPort(other.getPort());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 10: {
- setHostname(input.readString());
- break;
- }
- case 16: {
- setPort(input.readUInt32());
- break;
- }
- }
- }
- }
-
-
- // required string hostname = 1;
- public boolean hasHostname() {
- return result.hasHostname();
- }
- public java.lang.String getHostname() {
- return result.getHostname();
- }
- public Builder setHostname(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasHostname = true;
- result.hostname_ = value;
- return this;
- }
- public Builder clearHostname() {
- result.hasHostname = false;
- result.hostname_ = getDefaultInstance().getHostname();
- return this;
- }
-
- // required uint32 port = 2;
- public boolean hasPort() {
- return result.hasPort();
- }
- public int getPort() {
- return result.getPort();
- }
- public Builder setPort(int value) {
- result.hasPort = true;
- result.port_ = value;
- return this;
- }
- public Builder clearPort() {
- result.hasPort = false;
- result.port_ = 0;
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:AddressProtocol)
- }
-
- static {
- defaultInstance = new AddressProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:AddressProtocol)
- }
-
- public static final class ExceptionProtocol extends
- com.google.protobuf.GeneratedMessage {
- // Use ExceptionProtocol.newBuilder() to construct.
- private ExceptionProtocol() {
- initFields();
- }
- private ExceptionProtocol(boolean noInit) {}
-
- private static final ExceptionProtocol defaultInstance;
- public static ExceptionProtocol getDefaultInstance() {
- return defaultInstance;
- }
-
- public ExceptionProtocol getDefaultInstanceForType() {
- return defaultInstance;
- }
-
- public static final com.google.protobuf.Descriptors.Descriptor
- getDescriptor() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_descriptor;
- }
-
- protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internalGetFieldAccessorTable() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_fieldAccessorTable;
- }
-
- // required string classname = 1;
- public static final int CLASSNAME_FIELD_NUMBER = 1;
- private boolean hasClassname;
- private java.lang.String classname_ = "";
- public boolean hasClassname() { return hasClassname; }
- public java.lang.String getClassname() { return classname_; }
-
- // required string message = 2;
- public static final int MESSAGE_FIELD_NUMBER = 2;
- private boolean hasMessage;
- private java.lang.String message_ = "";
- public boolean hasMessage() { return hasMessage; }
- public java.lang.String getMessage() { return message_; }
-
- private void initFields() {
- }
- public final boolean isInitialized() {
- if (!hasClassname) return false;
- if (!hasMessage) return false;
- return true;
- }
-
- public void writeTo(com.google.protobuf.CodedOutputStream output)
- throws java.io.IOException {
- getSerializedSize();
- if (hasClassname()) {
- output.writeString(1, getClassname());
- }
- if (hasMessage()) {
- output.writeString(2, getMessage());
- }
- getUnknownFields().writeTo(output);
- }
-
- private int memoizedSerializedSize = -1;
- public int getSerializedSize() {
- int size = memoizedSerializedSize;
- if (size != -1) return size;
-
- size = 0;
- if (hasClassname()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(1, getClassname());
- }
- if (hasMessage()) {
- size += com.google.protobuf.CodedOutputStream
- .computeStringSize(2, getMessage());
- }
- size += getUnknownFields().getSerializedSize();
- memoizedSerializedSize = size;
- return size;
- }
-
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
- com.google.protobuf.ByteString data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
- com.google.protobuf.ByteString data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(byte[] data)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
- byte[] data,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws com.google.protobuf.InvalidProtocolBufferException {
- return newBuilder().mergeFrom(data, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(java.io.InputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseDelimitedFrom(java.io.InputStream input)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseDelimitedFrom(
- java.io.InputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- Builder builder = newBuilder();
- if (builder.mergeDelimitedFrom(input, extensionRegistry)) {
- return builder.buildParsed();
- } else {
- return null;
- }
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
- com.google.protobuf.CodedInputStream input)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input).buildParsed();
- }
- public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- return newBuilder().mergeFrom(input, extensionRegistry)
- .buildParsed();
- }
-
- public static Builder newBuilder() { return Builder.create(); }
- public Builder newBuilderForType() { return newBuilder(); }
- public static Builder newBuilder(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol prototype) {
- return newBuilder().mergeFrom(prototype);
- }
- public Builder toBuilder() { return newBuilder(this); }
-
- public static final class Builder extends
- com.google.protobuf.GeneratedMessage.Builder {
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol result;
-
- // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder()
- private Builder() {}
-
- private static Builder create() {
- Builder builder = new Builder();
- builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol();
- return builder;
- }
-
- protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol internalGetResult() {
- return result;
- }
-
- public Builder clear() {
- if (result == null) {
- throw new IllegalStateException(
- "Cannot call clear() after build().");
- }
- result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol();
- return this;
- }
-
- public Builder clone() {
- return create().mergeFrom(result);
- }
-
- public com.google.protobuf.Descriptors.Descriptor
- getDescriptorForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDescriptor();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol getDefaultInstanceForType() {
- return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance();
- }
-
- public boolean isInitialized() {
- return result.isInitialized();
- }
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol build() {
- if (result != null && !isInitialized()) {
- throw newUninitializedMessageException(result);
- }
- return buildPartial();
- }
-
- private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildParsed()
- throws com.google.protobuf.InvalidProtocolBufferException {
- if (!isInitialized()) {
- throw newUninitializedMessageException(
- result).asInvalidProtocolBufferException();
- }
- return buildPartial();
- }
-
- public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildPartial() {
- if (result == null) {
- throw new IllegalStateException(
- "build() has already been called on this Builder.");
- }
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol returnMe = result;
- result = null;
- return returnMe;
- }
-
- public Builder mergeFrom(com.google.protobuf.Message other) {
- if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol) {
- return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol)other);
- } else {
- super.mergeFrom(other);
- return this;
- }
- }
-
- public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol other) {
- if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) return this;
- if (other.hasClassname()) {
- setClassname(other.getClassname());
- }
- if (other.hasMessage()) {
- setMessage(other.getMessage());
- }
- this.mergeUnknownFields(other.getUnknownFields());
- return this;
- }
-
- public Builder mergeFrom(
- com.google.protobuf.CodedInputStream input,
- com.google.protobuf.ExtensionRegistryLite extensionRegistry)
- throws java.io.IOException {
- com.google.protobuf.UnknownFieldSet.Builder unknownFields =
- com.google.protobuf.UnknownFieldSet.newBuilder(
- this.getUnknownFields());
- while (true) {
- int tag = input.readTag();
- switch (tag) {
- case 0:
- this.setUnknownFields(unknownFields.build());
- return this;
- default: {
- if (!parseUnknownField(input, unknownFields,
- extensionRegistry, tag)) {
- this.setUnknownFields(unknownFields.build());
- return this;
- }
- break;
- }
- case 10: {
- setClassname(input.readString());
- break;
- }
- case 18: {
- setMessage(input.readString());
- break;
- }
- }
- }
- }
-
-
- // required string classname = 1;
- public boolean hasClassname() {
- return result.hasClassname();
- }
- public java.lang.String getClassname() {
- return result.getClassname();
- }
- public Builder setClassname(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasClassname = true;
- result.classname_ = value;
- return this;
- }
- public Builder clearClassname() {
- result.hasClassname = false;
- result.classname_ = getDefaultInstance().getClassname();
- return this;
- }
-
- // required string message = 2;
- public boolean hasMessage() {
- return result.hasMessage();
- }
- public java.lang.String getMessage() {
- return result.getMessage();
- }
- public Builder setMessage(java.lang.String value) {
- if (value == null) {
- throw new NullPointerException();
- }
- result.hasMessage = true;
- result.message_ = value;
- return this;
- }
- public Builder clearMessage() {
- result.hasMessage = false;
- result.message_ = getDefaultInstance().getMessage();
- return this;
- }
-
- // @@protoc_insertion_point(builder_scope:ExceptionProtocol)
- }
-
- static {
- defaultInstance = new ExceptionProtocol(true);
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
- defaultInstance.initFields();
- }
-
- // @@protoc_insertion_point(class_scope:ExceptionProtocol)
- }
-
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_RemoteActorRefProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_RemoteActorRefProtocol_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_SerializedActorRefProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_SerializedActorRefProtocol_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_MessageProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_MessageProtocol_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_ActorInfoProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_ActorInfoProtocol_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_TypedActorInfoProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_TypedActorInfoProtocol_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_RemoteRequestProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_RemoteRequestProtocol_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_RemoteReplyProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_RemoteReplyProtocol_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_LifeCycleProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_LifeCycleProtocol_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_AddressProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_AddressProtocol_fieldAccessorTable;
- private static com.google.protobuf.Descriptors.Descriptor
- internal_static_ExceptionProtocol_descriptor;
- private static
- com.google.protobuf.GeneratedMessage.FieldAccessorTable
- internal_static_ExceptionProtocol_fieldAccessorTable;
-
- public static com.google.protobuf.Descriptors.FileDescriptor
- getDescriptor() {
- return descriptor;
- }
- private static com.google.protobuf.Descriptors.FileDescriptor
- descriptor;
- static {
- java.lang.String[] descriptorData = {
- "\n\024RemoteProtocol.proto\"v\n\026RemoteActorRef" +
- "Protocol\022\014\n\004uuid\030\001 \002(\t\022\026\n\016actorClassname" +
- "\030\002 \002(\t\022%\n\013homeAddress\030\003 \002(\0132\020.AddressPro" +
- "tocol\022\017\n\007timeout\030\004 \001(\004\"\200\003\n\032SerializedAct" +
- "orRefProtocol\022\014\n\004uuid\030\001 \002(\t\022\n\n\002id\030\002 \002(\t\022" +
- "\026\n\016actorClassname\030\003 \002(\t\022)\n\017originalAddre" +
- "ss\030\004 \002(\0132\020.AddressProtocol\022\025\n\ractorInsta" +
- "nce\030\005 \001(\014\022\033\n\023serializerClassname\030\006 \001(\t\022\024" +
- "\n\014isTransactor\030\007 \001(\010\022\017\n\007timeout\030\010 \001(\004\022\026\n" +
- "\016receiveTimeout\030\t \001(\004\022%\n\tlifeCycle\030\n \001(\013",
- "2\022.LifeCycleProtocol\022+\n\nsupervisor\030\013 \001(\013" +
- "2\027.RemoteActorRefProtocol\022\024\n\014hotswapStac" +
- "k\030\014 \001(\014\022(\n\010messages\030\r \003(\0132\026.RemoteReques" +
- "tProtocol\"r\n\017MessageProtocol\0225\n\023serializ" +
- "ationScheme\030\001 \002(\0162\030.SerializationSchemeT" +
- "ype\022\017\n\007message\030\002 \002(\014\022\027\n\017messageManifest\030" +
- "\003 \001(\014\"\222\001\n\021ActorInfoProtocol\022\014\n\004uuid\030\001 \002(" +
- "\t\022\016\n\006target\030\002 \002(\t\022\017\n\007timeout\030\003 \002(\004\022\035\n\tac" +
- "torType\030\004 \002(\0162\n.ActorType\022/\n\016typedActorI" +
- "nfo\030\005 \001(\0132\027.TypedActorInfoProtocol\";\n\026Ty",
- "pedActorInfoProtocol\022\021\n\tinterface\030\001 \002(\t\022" +
- "\016\n\006method\030\002 \002(\t\"\300\001\n\025RemoteRequestProtoco" +
- "l\022\n\n\002id\030\001 \002(\004\022!\n\007message\030\002 \002(\0132\020.Message" +
- "Protocol\022%\n\tactorInfo\030\003 \002(\0132\022.ActorInfoP" +
- "rotocol\022\020\n\010isOneWay\030\004 \002(\010\022\026\n\016supervisorU" +
- "uid\030\005 \001(\t\022\'\n\006sender\030\006 \001(\0132\027.RemoteActorR" +
- "efProtocol\"\252\001\n\023RemoteReplyProtocol\022\n\n\002id" +
- "\030\001 \002(\004\022!\n\007message\030\002 \001(\0132\020.MessageProtoco" +
- "l\022%\n\texception\030\003 \001(\0132\022.ExceptionProtocol" +
- "\022\026\n\016supervisorUuid\030\004 \001(\t\022\017\n\007isActor\030\005 \002(",
- "\010\022\024\n\014isSuccessful\030\006 \002(\010\"\177\n\021LifeCycleProt" +
- "ocol\022!\n\tlifeCycle\030\001 \002(\0162\016.LifeCycleType\022" +
- "\022\n\npreRestart\030\002 \001(\t\022\023\n\013postRestart\030\003 \001(\t" +
- "\022\014\n\004init\030\004 \001(\t\022\020\n\010shutdown\030\005 \001(\t\"1\n\017Addr" +
- "essProtocol\022\020\n\010hostname\030\001 \002(\t\022\014\n\004port\030\002 " +
- "\002(\r\"7\n\021ExceptionProtocol\022\021\n\tclassname\030\001 " +
- "\002(\t\022\017\n\007message\030\002 \002(\t*=\n\tActorType\022\017\n\013SCA" +
- "LA_ACTOR\020\001\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTO" +
- "R\020\003*]\n\027SerializationSchemeType\022\010\n\004JAVA\020\001" +
- "\022\013\n\007SBINARY\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JS",
- "ON\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tP" +
- "ERMANENT\020\001\022\r\n\tTEMPORARY\020\002B-\n)se.scalable" +
- "solutions.akka.remote.protocolH\001"
- };
- com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
- new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
- public com.google.protobuf.ExtensionRegistry assignDescriptors(
- com.google.protobuf.Descriptors.FileDescriptor root) {
- descriptor = root;
- internal_static_RemoteActorRefProtocol_descriptor =
- getDescriptor().getMessageTypes().get(0);
- internal_static_RemoteActorRefProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_RemoteActorRefProtocol_descriptor,
- new java.lang.String[] { "Uuid", "ActorClassname", "HomeAddress", "Timeout", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder.class);
- internal_static_SerializedActorRefProtocol_descriptor =
- getDescriptor().getMessageTypes().get(1);
- internal_static_SerializedActorRefProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_SerializedActorRefProtocol_descriptor,
- new java.lang.String[] { "Uuid", "Id", "ActorClassname", "OriginalAddress", "ActorInstance", "SerializerClassname", "IsTransactor", "Timeout", "ReceiveTimeout", "LifeCycle", "Supervisor", "HotswapStack", "Messages", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder.class);
- internal_static_MessageProtocol_descriptor =
- getDescriptor().getMessageTypes().get(2);
- internal_static_MessageProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_MessageProtocol_descriptor,
- new java.lang.String[] { "SerializationScheme", "Message", "MessageManifest", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder.class);
- internal_static_ActorInfoProtocol_descriptor =
- getDescriptor().getMessageTypes().get(3);
- internal_static_ActorInfoProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_ActorInfoProtocol_descriptor,
- new java.lang.String[] { "Uuid", "Target", "Timeout", "ActorType", "TypedActorInfo", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder.class);
- internal_static_TypedActorInfoProtocol_descriptor =
- getDescriptor().getMessageTypes().get(4);
- internal_static_TypedActorInfoProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_TypedActorInfoProtocol_descriptor,
- new java.lang.String[] { "Interface", "Method", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.TypedActorInfoProtocol.Builder.class);
- internal_static_RemoteRequestProtocol_descriptor =
- getDescriptor().getMessageTypes().get(5);
- internal_static_RemoteRequestProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_RemoteRequestProtocol_descriptor,
- new java.lang.String[] { "Id", "Message", "ActorInfo", "IsOneWay", "SupervisorUuid", "Sender", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteRequestProtocol.Builder.class);
- internal_static_RemoteReplyProtocol_descriptor =
- getDescriptor().getMessageTypes().get(6);
- internal_static_RemoteReplyProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_RemoteReplyProtocol_descriptor,
- new java.lang.String[] { "Id", "Message", "Exception", "SupervisorUuid", "IsActor", "IsSuccessful", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.Builder.class);
- internal_static_LifeCycleProtocol_descriptor =
- getDescriptor().getMessageTypes().get(7);
- internal_static_LifeCycleProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_LifeCycleProtocol_descriptor,
- new java.lang.String[] { "LifeCycle", "PreRestart", "PostRestart", "Init", "Shutdown", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder.class);
- internal_static_AddressProtocol_descriptor =
- getDescriptor().getMessageTypes().get(8);
- internal_static_AddressProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_AddressProtocol_descriptor,
- new java.lang.String[] { "Hostname", "Port", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.AddressProtocol.Builder.class);
- internal_static_ExceptionProtocol_descriptor =
- getDescriptor().getMessageTypes().get(9);
- internal_static_ExceptionProtocol_fieldAccessorTable = new
- com.google.protobuf.GeneratedMessage.FieldAccessorTable(
- internal_static_ExceptionProtocol_descriptor,
- new java.lang.String[] { "Classname", "Message", },
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.class,
- se.scalablesolutions.akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder.class);
- return null;
- }
- };
- com.google.protobuf.Descriptors.FileDescriptor
- .internalBuildGeneratedFileFrom(descriptorData,
- new com.google.protobuf.Descriptors.FileDescriptor[] {
- }, assigner);
- }
-
- public static void internalForceInit() {}
-
- // @@protoc_insertion_point(outer_class_scope)
-}
diff --git a/akka-remote/src/test/resources/META-INF/aop.xml b/akka-remote/src/test/resources/META-INF/aop.xml
index bdc167ca54..be133a51b8 100644
--- a/akka-remote/src/test/resources/META-INF/aop.xml
+++ b/akka-remote/src/test/resources/META-INF/aop.xml
@@ -2,6 +2,7 @@
+
diff --git a/akka-remote/src/test/resources/logback-test.xml b/akka-remote/src/test/resources/logback-test.xml
deleted file mode 100644
index 78eae40ec4..0000000000
--- a/akka-remote/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,21 +0,0 @@
-
-
-
-
-
-
-
-
-
-
-
-
-
- [%4p] [%d{ISO8601}] [%t] %c{1}: %m%n
-
-
-
-
-
-
-
diff --git a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
index 7ff46ab910..6670722b02 100644
--- a/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ClientInitiatedRemoteActorSpec.scala
@@ -93,6 +93,7 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
actor.stop
}
+
@Test
def shouldSendOneWayAndReceiveReply = {
val actor = actorOf[SendOneWayAndReplyReceiverActor]
@@ -103,7 +104,7 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].sendTo = actor
sender.start
sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].sendOff
- assert(SendOneWayAndReplySenderActor.latch.await(1, TimeUnit.SECONDS))
+ assert(SendOneWayAndReplySenderActor.latch.await(3, TimeUnit.SECONDS))
assert(sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].state.isDefined === true)
assert("World" === sender.actor.asInstanceOf[SendOneWayAndReplySenderActor].state.get.asInstanceOf[String])
actor.stop
@@ -134,6 +135,6 @@ class ClientInitiatedRemoteActorSpec extends JUnitSuite {
assert("Expected exception; to test fault-tolerance" === e.getMessage())
}
actor.stop
- }
+ }
}
diff --git a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala
index 780828c310..8b28b35f57 100644
--- a/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/RemoteTypedActorSpec.scala
@@ -4,10 +4,7 @@
package se.scalablesolutions.akka.actor.remote
-import org.scalatest.Spec
-import org.scalatest.Assertions
import org.scalatest.matchers.ShouldMatchers
-import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
@@ -19,6 +16,7 @@ import se.scalablesolutions.akka.actor._
import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient}
import java.util.concurrent.{LinkedBlockingQueue, TimeUnit, BlockingQueue}
+import org.scalatest.{BeforeAndAfterEach, Spec, Assertions, BeforeAndAfterAll}
object RemoteTypedActorSpec {
val HOSTNAME = "localhost"
@@ -40,7 +38,7 @@ object RemoteTypedActorLog {
class RemoteTypedActorSpec extends
Spec with
ShouldMatchers with
- BeforeAndAfterAll {
+ BeforeAndAfterEach with BeforeAndAfterAll {
import RemoteTypedActorLog._
import RemoteTypedActorSpec._
@@ -82,6 +80,10 @@ class RemoteTypedActorSpec extends
ActorRegistry.shutdownAll
}
+ override def afterEach() {
+ server.typedActors.clear
+ }
+
describe("Remote Typed Actor ") {
it("should receive one-way message") {
diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
index 59cfe3778d..59f122c656 100644
--- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
@@ -5,8 +5,8 @@ import org.scalatest.junit.JUnitSuite
import org.junit.{Test, Before, After}
import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient}
-import se.scalablesolutions.akka.actor.{ActorRef, Actor}
-import Actor._
+import se.scalablesolutions.akka.actor.Actor._
+import se.scalablesolutions.akka.actor.{ActorRegistry, ActorRef, Actor}
object ServerInitiatedRemoteActorSpec {
val HOSTNAME = "localhost"
@@ -67,7 +67,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite {
Thread.sleep(1000)
}
- // make sure the servers shutdown cleanly after the test has finished
+ // make sure the servers postStop cleanly after the test has finished
@After
def finished {
try {
@@ -79,6 +79,7 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite {
}
}
+
@Test
def shouldSendWithBang {
val actor = RemoteClient.actorFor(
@@ -132,5 +133,50 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite {
}
actor.stop
}
+
+ @Test
+ def reflectiveAccessShouldNotCreateNewRemoteServerObject {
+ val server1 = new RemoteServer()
+ server1.start("localhost", 9990)
+
+ var found = RemoteServer.serverFor("localhost", 9990)
+ assert(found.isDefined, "sever not found")
+
+ val a = actor { case _ => }
+
+ found = RemoteServer.serverFor("localhost", 9990)
+ assert(found.isDefined, "sever not found after creating an actor")
+ }
+
+
+ @Test
+ def shouldNotRecreateRegisteredActor {
+ server.register(actorOf[RemoteActorSpecActorUnidirectional])
+ val actor = RemoteClient.actorFor("se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", HOSTNAME, PORT)
+ val numberOfActorsInRegistry = ActorRegistry.actors.length
+ actor ! "OneWay"
+ assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS))
+ assert(numberOfActorsInRegistry === ActorRegistry.actors.length)
+ actor.stop
+ }
+
+ @Test
+ def shouldUseServiceNameAsIdForRemoteActorRef {
+ server.register(actorOf[RemoteActorSpecActorUnidirectional])
+ server.register("my-service", actorOf[RemoteActorSpecActorUnidirectional])
+ val actor1 = RemoteClient.actorFor("se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec$RemoteActorSpecActorUnidirectional", HOSTNAME, PORT)
+ val actor2 = RemoteClient.actorFor("my-service", HOSTNAME, PORT)
+ val actor3 = RemoteClient.actorFor("my-service", HOSTNAME, PORT)
+
+ actor1 ! "OneWay"
+ actor2 ! "OneWay"
+ actor3 ! "OneWay"
+
+ assert(actor1.uuid != actor2.uuid)
+ assert(actor1.uuid != actor3.uuid)
+ assert(actor1.id != actor2.id)
+ assert(actor2.id == actor3.id)
+ }
+
}
diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala
new file mode 100644
index 0000000000..b800fbf2c3
--- /dev/null
+++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala
@@ -0,0 +1,112 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+
+package se.scalablesolutions.akka.actor.remote
+
+import org.scalatest.Spec
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
+
+import java.util.concurrent.TimeUnit
+
+import se.scalablesolutions.akka.remote.{RemoteServer, RemoteClient}
+import se.scalablesolutions.akka.actor._
+import RemoteTypedActorLog._
+
+object ServerInitiatedRemoteTypedActorSpec {
+ val HOSTNAME = "localhost"
+ val PORT = 9990
+ var server: RemoteServer = null
+}
+
+@RunWith(classOf[JUnitRunner])
+class ServerInitiatedRemoteTypedActorSpec extends
+ Spec with
+ ShouldMatchers with
+ BeforeAndAfterAll {
+ import ServerInitiatedRemoteTypedActorSpec._
+
+ private val unit = TimeUnit.MILLISECONDS
+
+
+ override def beforeAll = {
+ server = new RemoteServer()
+ server.start(HOSTNAME, PORT)
+
+ val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
+ server.registerTypedActor("typed-actor-service", typedActor)
+
+ Thread.sleep(1000)
+ }
+
+ // make sure the servers shutdown cleanly after the test has finished
+ override def afterAll = {
+ try {
+ server.shutdown
+ RemoteClient.shutdownAll
+ Thread.sleep(1000)
+ } catch {
+ case e => ()
+ }
+ }
+
+ describe("Server managed remote typed Actor ") {
+
+ it("should receive one-way message") {
+ clearMessageLogs
+ val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT)
+ expect("oneway") {
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ it("should respond to request-reply message") {
+ clearMessageLogs
+ val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT)
+ expect("pong") {
+ actor.requestReply("ping")
+ }
+ }
+
+ it("should not recreate registered actors") {
+ val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT)
+ val numberOfActorsInRegistry = ActorRegistry.actors.length
+ expect("oneway") {
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS)
+ }
+ assert(numberOfActorsInRegistry === ActorRegistry.actors.length)
+ }
+
+ it("should support multiple variants to get the actor from client side") {
+ var actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT)
+ expect("oneway") {
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS)
+ }
+ actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", HOSTNAME, PORT)
+ expect("oneway") {
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS)
+ }
+ actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], "typed-actor-service", 5000L, HOSTNAME, PORT, this.getClass().getClassLoader)
+ expect("oneway") {
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS)
+ }
+ }
+
+ it("should register and unregister typed actors") {
+ val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
+ server.registerTypedActor("my-test-service", typedActor)
+ assert(server.typedActors().get("my-test-service") != null)
+ server.unregisterTypedActor("my-test-service")
+ assert(server.typedActors().get("my-test-service") == null)
+ }
+ }
+}
+
diff --git a/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala b/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala
index 011c656f8d..e05e4b0394 100644
--- a/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala
+++ b/akka-remote/src/test/scala/serialization/ProtobufActorMessageSerializationSpec.scala
@@ -47,7 +47,7 @@ class ProtobufActorMessageSerializationSpec extends JUnitSuite {
Thread.sleep(1000)
}
- // make sure the servers shutdown cleanly after the test has finished
+ // make sure the servers postStop cleanly after the test has finished
@After
def finished() {
server.shutdown
diff --git a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala
index a9bcc35790..7e8babe168 100644
--- a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala
+++ b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala
@@ -7,6 +7,7 @@ import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
+import se.scalablesolutions.akka.serialization._
import se.scalablesolutions.akka.actor._
import ActorSerialization._
import Actor._
@@ -17,7 +18,6 @@ class SerializableTypeClassActorSpec extends
ShouldMatchers with
BeforeAndAfterAll {
- import se.scalablesolutions.akka.serialization.Serializer
object BinaryFormatMyActor {
implicit object MyActorFormat extends Format[MyActor] {
diff --git a/akka-remote/src/test/scala/serialization/Ticket436Spec.scala b/akka-remote/src/test/scala/serialization/Ticket436Spec.scala
new file mode 100644
index 0000000000..042f3f07be
--- /dev/null
+++ b/akka-remote/src/test/scala/serialization/Ticket436Spec.scala
@@ -0,0 +1,49 @@
+package se.scalablesolutions.akka.actor.serialization
+
+
+import org.scalatest.Spec
+import org.scalatest.matchers.ShouldMatchers
+import org.scalatest.BeforeAndAfterAll
+import org.scalatest.junit.JUnitRunner
+import org.junit.runner.RunWith
+
+import se.scalablesolutions.akka.serialization.Serializer
+import se.scalablesolutions.akka.serialization.Serializable.ScalaJSON
+import scala.reflect._
+import scala.annotation.target._
+import sjson.json.JSONTypeHint
+
+@BeanInfo class MyJsonObject(val key: String,
+ @(JSONTypeHint @field)(value = classOf[Int])
+ val map: Map[String, Int],
+ val standAloneInt: Int) extends ScalaJSON {
+ private def this() = this(null, null, -1)
+ override def toString(): String = try {
+ val mapValue: Int = map.getOrElse(key, -1)
+ println("Map value: %s".format(mapValue.asInstanceOf[AnyRef].getClass))
+ "Key: %s, Map value: %d, Stand Alone Int: %d".format(key, mapValue, standAloneInt)
+ } catch {
+ case e: ClassCastException => e.getMessage
+ case _ => "Unknown error"
+ }
+}
+
+@RunWith(classOf[JUnitRunner])
+class Ticket436Spec extends
+ Spec with
+ ShouldMatchers with
+ BeforeAndAfterAll {
+
+ describe("Serialization of Maps containing Int") {
+ it("should be able to serialize and de-serialize preserving the data types of the Map") {
+ val key: String = "myKey"
+ val value: Int = 123
+ val standAloneInt: Int = 35
+ val message = new MyJsonObject(key, Map(key -> value), standAloneInt)
+
+ val json = message.toJSON
+ val copy = Serializer.ScalaJSON.fromJSON[MyJsonObject](json)
+ copy.asInstanceOf[MyJsonObject].map.get("myKey").get.isInstanceOf[Int] should equal(true)
+ }
+ }
+}
diff --git a/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala b/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala
index 6588c7cdc4..d230bbaffe 100644
--- a/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala
+++ b/akka-remote/src/test/scala/serialization/UntypedActorSerializationSpec.scala
@@ -7,6 +7,7 @@ import org.scalatest.BeforeAndAfterAll
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
+import se.scalablesolutions.akka.serialization._
import se.scalablesolutions.akka.actor._
import ActorSerialization._
import Actor._
@@ -17,8 +18,6 @@ class UntypedActorSerializationSpec extends
ShouldMatchers with
BeforeAndAfterAll {
- import se.scalablesolutions.akka.serialization.Serializer
-
class MyUntypedActorFormat extends Format[MyUntypedActor] {
def fromBinary(bytes: Array[Byte], act: MyUntypedActor) = {
val p = Serializer.Protobuf.fromBinary(bytes, Some(classOf[ProtobufProtocol.Counter])).asInstanceOf[ProtobufProtocol.Counter]
diff --git a/akka-remote/src/test/scala/ticket/Ticket434Spec.scala b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala
new file mode 100644
index 0000000000..b27c17cfe5
--- /dev/null
+++ b/akka-remote/src/test/scala/ticket/Ticket434Spec.scala
@@ -0,0 +1,46 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package se.scalablesolutions.akka.actor.ticket
+
+import org.scalatest.Spec
+import org.scalatest.matchers.ShouldMatchers
+import se.scalablesolutions.akka.actor.Actor._
+import se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional
+import java.util.concurrent.TimeUnit
+import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer}
+import se.scalablesolutions.akka.remote.protocol.RemoteProtocol._
+
+
+class Ticket434Spec extends Spec with ShouldMatchers {
+
+ describe("A server managed remote actor") {
+ it("should possible be use a custom service name containing ':'") {
+ val server = new RemoteServer().start("localhost", 9999)
+ server.register("my:service", actorOf[RemoteActorSpecActorUnidirectional])
+
+ val actor = RemoteClient.actorFor("my:service", 5000L, "localhost", 9999)
+ actor ! "OneWay"
+
+ assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS))
+ actor.stop
+
+ server.shutdown
+ RemoteClient.shutdownAll
+ }
+ }
+
+ describe("The ActorInfoProtocol") {
+ it("should be possible to set the acor id and uuuid") {
+ val actorInfoBuilder = ActorInfoProtocol.newBuilder
+ .setUuid("unique-id")
+ .setId("some-id")
+ .setTarget("actorClassName")
+ .setTimeout(5000L)
+ .setActorType(ActorType.SCALA_ACTOR)
+ val actorInfo = actorInfoBuilder.build
+ assert(actorInfo.getUuid === "unique-id")
+ assert(actorInfo.getId === "some-id")
+ }
+ }
+}
diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1Impl.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1Impl.java
index 71eda6c270..0e05d5036c 100644
--- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1Impl.java
+++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer1Impl.java
@@ -1,8 +1,5 @@
package sample.camel;
-import org.apache.camel.Body;
-import org.apache.camel.Header;
-
import se.scalablesolutions.akka.actor.TypedActor;
/**
@@ -10,7 +7,7 @@ import se.scalablesolutions.akka.actor.TypedActor;
*/
public class RemoteTypedConsumer1Impl extends TypedActor implements RemoteTypedConsumer1 {
- public String foo(@Body String body, @Header("name") String header) {
+ public String foo(String body, String header) {
return String.format("remote1: body=%s header=%s", body, header);
}
}
diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2.java
new file mode 100644
index 0000000000..3b2494ee4e
--- /dev/null
+++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2.java
@@ -0,0 +1,15 @@
+package sample.camel;
+
+import org.apache.camel.Body;
+import org.apache.camel.Header;
+import se.scalablesolutions.akka.camel.consume;
+
+/**
+ * @author Martin Krasser
+ */
+public interface RemoteTypedConsumer2 {
+
+ @consume("jetty:http://localhost:6644/camel/remote-typed-actor-2")
+ public String foo(@Body String body, @Header("name") String header);
+
+}
diff --git a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2Impl.java b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2Impl.java
index 68e208d834..5c76485aa7 100644
--- a/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2Impl.java
+++ b/akka-samples/akka-sample-camel/src/main/java/sample/camel/RemoteTypedConsumer2Impl.java
@@ -1,16 +1,13 @@
package sample.camel;
-import org.apache.camel.Body;
-import org.apache.camel.Header;
-import se.scalablesolutions.akka.camel.consume;
+import se.scalablesolutions.akka.actor.TypedActor;
/**
* @author Martin Krasser
*/
-public class RemoteTypedConsumer2Impl {
+public class RemoteTypedConsumer2Impl extends TypedActor implements RemoteTypedConsumer2 {
- @consume("jetty:http://localhost:6644/camel/remote-typed-actor-2")
- public String foo(@Body String body, @Header("name") String header) {
+ public String foo(String body, String header) {
return String.format("remote2: body=%s header=%s", body, header);
}
diff --git a/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala b/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala
index 7f671b0e45..9fab1b9649 100644
--- a/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala
+++ b/akka-samples/akka-sample-camel/src/main/scala/ClientApplication.scala
@@ -1,7 +1,7 @@
package sample.camel
import se.scalablesolutions.akka.actor.Actor._
-import se.scalablesolutions.akka.actor.{TypedActor, Actor, ActorRef}
+import se.scalablesolutions.akka.actor.TypedActor
import se.scalablesolutions.akka.camel.Message
import se.scalablesolutions.akka.remote.RemoteClient
@@ -10,22 +10,20 @@ import se.scalablesolutions.akka.remote.RemoteClient
*/
object ClientApplication extends Application {
- //
- // TODO: completion of example
- //
-
- val actor1 = actorOf[RemoteActor1]
+ val actor1 = actorOf[RemoteActor1].start
val actor2 = RemoteClient.actorFor("remote2", "localhost", 7777)
- val actobj1 = TypedActor.newRemoteInstance(
- classOf[RemoteTypedConsumer1], classOf[RemoteTypedConsumer1Impl], "localhost", 7777)
- //val actobj2 = TODO: create reference to server-managed typed actor (RemoteTypedConsumer2Impl)
+ val typedActor1 = TypedActor.newRemoteInstance(
+ classOf[RemoteTypedConsumer1],
+ classOf[RemoteTypedConsumer1Impl], "localhost", 7777)
- actor1.start
+ val typedActor2 = RemoteClient.typedActorFor(
+ classOf[RemoteTypedConsumer2], "remote3", "localhost", 7777)
println(actor1 !! Message("actor1")) // activates and publishes actor remotely
println(actor2 !! Message("actor2")) // actor already activated and published remotely
- println(actobj1.foo("x", "y")) // activates and publishes typed actor methods remotely
- // ...
+ println(typedActor1.foo("x1", "y1")) // activates and publishes typed actor methods remotely
+ println(typedActor2.foo("x2", "y2")) // typed actor methods already activated and published remotely
+
}
diff --git a/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala b/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala
index 3eb40aaca9..b3ef404841 100644
--- a/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala
+++ b/akka-samples/akka-sample-camel/src/main/scala/ServerApplication.scala
@@ -3,6 +3,7 @@ package sample.camel
import se.scalablesolutions.akka.actor.Actor._
import se.scalablesolutions.akka.remote.RemoteNode
import se.scalablesolutions.akka.camel.CamelServiceManager
+import se.scalablesolutions.akka.actor.TypedActor
/**
* @author Martin Krasser
@@ -10,11 +11,14 @@ import se.scalablesolutions.akka.camel.CamelServiceManager
object ServerApplication extends Application {
import CamelServiceManager._
- //
- // TODO: completion of example
- //
-
startCamelService
+
+ val ua = actorOf[RemoteActor2].start
+ val ta = TypedActor.newInstance(
+ classOf[RemoteTypedConsumer2],
+ classOf[RemoteTypedConsumer2Impl], 2000)
+
RemoteNode.start("localhost", 7777)
- RemoteNode.register("remote2", actorOf[RemoteActor2].start)
+ RemoteNode.register("remote2", ua)
+ RemoteNode.registerTypedActor("remote3", ta)
}
diff --git a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala
index e3fd76a344..6f70d8071a 100644
--- a/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala
+++ b/akka-samples/akka-sample-chat/src/main/scala/ChatServer.scala
@@ -185,7 +185,7 @@ trait ChatServer extends Actor {
protected def sessionManagement: Receive
protected def shutdownSessions(): Unit
- override def shutdown = {
+ override def postStop = {
log.info("Chat server is shutting down...")
shutdownSessions
self.unlink(storage)
@@ -205,7 +205,7 @@ class ChatService extends
SessionManagement with
ChatManagement with
RedisChatStorageFactory {
- override def init = {
+ override def preStart = {
RemoteNode.start("localhost", 9999)
RemoteNode.register("chat:service", self)
}
diff --git a/akka-samples/akka-sample-lift/src/main/scala/akka/SimpleService.scala b/akka-samples/akka-sample-lift/src/main/scala/akka/SimpleService.scala
deleted file mode 100644
index d5358a7d89..0000000000
--- a/akka-samples/akka-sample-lift/src/main/scala/akka/SimpleService.scala
+++ /dev/null
@@ -1,94 +0,0 @@
-package sample.lift
-
-import se.scalablesolutions.akka.actor._
-import se.scalablesolutions.akka.actor.Actor._
-import se.scalablesolutions.akka.config.ScalaConfig._
-import se.scalablesolutions.akka.stm.TransactionalMap
-import se.scalablesolutions.akka.persistence.cassandra.CassandraStorage
-import scala.xml.Node
-import java.lang.Integer
-import javax.ws.rs.{GET, Path, Produces}
-import java.nio.ByteBuffer
-import net.liftweb.http._
-import net.liftweb.http.rest._
-
-class SimpleServiceActor extends Transactor {
- private val KEY = "COUNTER"
- private var hasStartedTicking = false
- private lazy val storage = TransactionalMap[String, Integer]()
-
- def receive = {
- case "Tick" => if (hasStartedTicking) {
- val counter = storage.get(KEY).get.asInstanceOf[Integer].intValue
- storage.put(KEY, new Integer(counter + 1))
- self.reply(Tick: {counter + 1}
)
- } else {
- storage.put(KEY, new Integer(0))
- hasStartedTicking = true
- self.reply(Tick: 0
)
- }
- }
-}
-
-class PersistentServiceActor extends Transactor {
-
- private val KEY = "COUNTER"
- private var hasStartedTicking = false
- private lazy val storage = CassandraStorage.newMap
-
- def receive = {
- case "Tick" => if (hasStartedTicking) {
- val bytes = storage.get(KEY.getBytes).get
- val counter = ByteBuffer.wrap(bytes).getInt
- storage.put(KEY.getBytes, ByteBuffer.allocate(4).putInt(counter + 1).array)
- self.reply(Tick:{counter + 1})
- } else {
- storage.put(KEY.getBytes, ByteBuffer.allocate(4).putInt(0).array)
- hasStartedTicking = true
- self.reply(Tick: 0)
- }
- }
-}
-
-
-/**
- * Try service out by invoking (multiple times):
- *
- * curl http://localhost:8080/liftcount
- *
- * Or browse to the URL from a web browser.
- */
-
-object SimpleRestService extends RestHelper {
- serve {
- case Get("liftcount" :: _, req) =>
- //Fetch the first actor of type SimpleServiceActor
- //Send it the "Tick" message and expect a Node back
- val result = for( a <- ActorRegistry.actorFor[SimpleServiceActor];
- r <- (a !! "Tick").as[Node] ) yield r
-
- //Return either the resulting NodeSeq or a default one
- (result getOrElse Error in counter
).asInstanceOf[Node]
- }
-}
-
-
-/**
- * Try service out by invoking (multiple times):
- *
- * curl http://localhost:8080/persistentliftcount
- *
- * Or browse to the URL from a web browser.
- */
- object PersistentRestService extends RestHelper {
- serve {
- case Get("persistentliftcount" :: _, req) =>
- //Fetch the first actor of type SimpleServiceActor
- //Send it the "Tick" message and expect a Node back
- val result = for( a <- ActorRegistry.actorFor[PersistentServiceActor];
- r <- (a !! "Tick").as[Node] ) yield r
-
- //Return either the resulting NodeSeq or a default one
- (result getOrElse Error in counter
).asInstanceOf[Node]
- }
- }
diff --git a/akka-samples/akka-sample-lift/src/main/scala/bootstrap/liftweb/Boot.scala b/akka-samples/akka-sample-lift/src/main/scala/bootstrap/liftweb/Boot.scala
deleted file mode 100644
index 2e56a5857a..0000000000
--- a/akka-samples/akka-sample-lift/src/main/scala/bootstrap/liftweb/Boot.scala
+++ /dev/null
@@ -1,60 +0,0 @@
-package bootstrap.liftweb
-
-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.http.auth._
-import _root_.net.liftweb.common._
-import Helpers._
-
-import se.scalablesolutions.akka.actor.{SupervisorFactory, Actor}
-import se.scalablesolutions.akka.actor.Actor._
-import se.scalablesolutions.akka.config.ScalaConfig._
-import se.scalablesolutions.akka.util.Logging
-
-import sample.lift._
-
-/**
- * A class that's instantiated early and run. It allows the application
- * to modify lift's environment
- */
-class Boot extends Logging {
- def boot {
- // where to search snippet
- LiftRules.addToPackages("sample.lift")
-
- LiftRules.httpAuthProtectedResource.prepend {
- case (Req("liftcount" :: Nil, _, _)) => Full(AuthRole("admin"))
- }
-
- LiftRules.authentication = HttpBasicAuthentication("lift") {
- case ("someuser", "1234", req) => {
- log.info("You are now authenticated !")
- userRoles(AuthRole("admin"))
- true
- }
- }
- LiftRules.statelessDispatchTable.append(SimpleRestService)
- LiftRules.statelessDispatchTable.append(PersistentRestService)
-
- LiftRules.passNotFoundToChain = true
-
- val factory = SupervisorFactory(
- SupervisorConfig(
- RestartStrategy(OneForOne, 3, 100, List(classOf[Exception])),
- Supervise(
- actorOf[SimpleServiceActor],
- LifeCycle(Permanent)) ::
- Supervise(
- actorOf[PersistentServiceActor],
- LifeCycle(Permanent)) ::
- Nil))
- factory.newInstance.start
-
- // Build SiteMap
- // val entries = Menu(Loc("Home", List("index"), "Home")) :: Nil
- // LiftRules.setSiteMap(SiteMap(entries:_*))
- }
-}
-
diff --git a/akka-samples/akka-sample-lift/src/main/scala/comet/.keep b/akka-samples/akka-sample-lift/src/main/scala/comet/.keep
deleted file mode 100644
index e69de29bb2..0000000000
diff --git a/akka-samples/akka-sample-lift/src/main/scala/model/.keep b/akka-samples/akka-sample-lift/src/main/scala/model/.keep
deleted file mode 100644
index e69de29bb2..0000000000
diff --git a/akka-samples/akka-sample-lift/src/main/scala/snippet/.keep b/akka-samples/akka-sample-lift/src/main/scala/snippet/.keep
deleted file mode 100644
index e69de29bb2..0000000000
diff --git a/akka-samples/akka-sample-lift/src/main/scala/snippet/HelloWorld.scala b/akka-samples/akka-sample-lift/src/main/scala/snippet/HelloWorld.scala
deleted file mode 100644
index aed272c0b2..0000000000
--- a/akka-samples/akka-sample-lift/src/main/scala/snippet/HelloWorld.scala
+++ /dev/null
@@ -1,6 +0,0 @@
-package sample.lift.snippet
-
-class HelloWorld {
- def howdy = Welcome to lift-akka at {new _root_.java.util.Date}
-}
-
diff --git a/akka-samples/akka-sample-lift/src/main/scala/view/.keep b/akka-samples/akka-sample-lift/src/main/scala/view/.keep
deleted file mode 100644
index e69de29bb2..0000000000
diff --git a/akka-samples/akka-sample-lift/src/main/webapp/WEB-INF/web.xml b/akka-samples/akka-sample-lift/src/main/webapp/WEB-INF/web.xml
deleted file mode 100644
index 3a1b672cec..0000000000
--- a/akka-samples/akka-sample-lift/src/main/webapp/WEB-INF/web.xml
+++ /dev/null
@@ -1,22 +0,0 @@
-
-
-
-
- LiftFilter
- Lift Filter
- The Filter that intercepts lift calls
- net.liftweb.http.LiftFilter
-
-
- LiftFilter
- /*
-
-
- AkkaServlet
- se.scalablesolutions.akka.comet.AkkaServlet
-
-
- AkkaServlet
- /*
-
-
diff --git a/akka-samples/akka-sample-lift/src/main/webapp/index.html b/akka-samples/akka-sample-lift/src/main/webapp/index.html
deleted file mode 100644
index aa25a1d91d..0000000000
--- a/akka-samples/akka-sample-lift/src/main/webapp/index.html
+++ /dev/null
@@ -1,15 +0,0 @@
-
- Welcome to the Akka + Lift Sample
- This page is served by Lift, and Lift alone. In order to demonstrate how AkkaServlet and
- Lift can work in harmony we have supplied a sample JAX-RS service that is secured using
- Lift's HTTP Basic Authentication.
-
- To access the Akka service, visit this url and enter the
- following access credentials:
-
- user: someuser
- password: 1234
-
-
-
-
diff --git a/akka-samples/akka-sample-lift/src/main/webapp/templates-hidden/default.html b/akka-samples/akka-sample-lift/src/main/webapp/templates-hidden/default.html
deleted file mode 100644
index 4a18c18f8a..0000000000
--- a/akka-samples/akka-sample-lift/src/main/webapp/templates-hidden/default.html
+++ /dev/null
@@ -1,17 +0,0 @@
-
-
-
-
-
-
- Akka with Lift Example
-
-
-
-
-
-
-
-
-
-
diff --git a/akka-samples/akka-sample-lift/src/test/scala/LiftConsole.scala b/akka-samples/akka-sample-lift/src/test/scala/LiftConsole.scala
deleted file mode 100644
index 43296bc1f4..0000000000
--- a/akka-samples/akka-sample-lift/src/test/scala/LiftConsole.scala
+++ /dev/null
@@ -1,16 +0,0 @@
-/*import _root_.bootstrap.liftweb.Boot
-import _root_.scala.tools.nsc.MainGenericRunner
-
-object LiftConsole {
- def main(args : Array[String]) {
- // Instantiate your project's Boot file
- val b = new Boot()
- // Boot your project
- b.boot
- // Now run the MainGenericRunner to get your repl
- MainGenericRunner.main(args)
- // After the repl exits, then exit the scala script
- exit(0)
- }
-}
-*/
diff --git a/akka-samples/akka-sample-lift/src/test/scala/RunWebApp.scala b/akka-samples/akka-sample-lift/src/test/scala/RunWebApp.scala
deleted file mode 100644
index fd8ea053c3..0000000000
--- a/akka-samples/akka-sample-lift/src/test/scala/RunWebApp.scala
+++ /dev/null
@@ -1,27 +0,0 @@
-import org.eclipse.jetty.webapp.WebAppContext
-import org.eclipse.jetty.server.Server
-
-object RunWebApp extends Application {
- val server = new Server(8080)
- val context = new WebAppContext()
- context.setServer(server)
- context.setContextPath("/")
- context.setWar("src/main/webapp")
-
- server.setHandler(context)
-
- try {
- println(">>> STARTING EMBEDDED JETTY SERVER, PRESS ANY KEY TO STOP")
- server.start()
- while (System.in.available() == 0) {
- Thread.sleep(5000)
- }
- server.stop()
- server.join()
- } catch {
- case exc : Exception => {
- exc.printStackTrace()
- System.exit(100)
- }
- }
-}
diff --git a/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd b/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd
index 2a42ec0900..e66090fe16 100644
--- a/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd
+++ b/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-0.10.xsd
@@ -39,8 +39,6 @@
-
-
diff --git a/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-1.0-SNAPSHOT.xsd b/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-1.0-SNAPSHOT.xsd
new file mode 100644
index 0000000000..84a382a78e
--- /dev/null
+++ b/akka-spring/src/main/resources/se/scalablesolutions/akka/spring/akka-1.0-SNAPSHOT.xsd
@@ -0,0 +1,355 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ Name of the remote host.
+
+
+
+
+
+
+ Port of the remote host.
+
+
+
+
+
+
+ Management type for remote actors: client managed or server managed.
+
+
+
+
+
+
+ Custom service name for server managed actor.
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ Name of the interface implemented by implementation class.
+
+
+
+
+
+
+ Name of the implementation class.
+
+
+
+
+
+
+ The default timeout for '!!' invocations.
+
+
+
+
+
+
+ Set this to true if messages should have REQUIRES_NEW semantics.
+
+
+
+
+
+
+ Defines the lifecycle, can be either 'permanent' or 'temporary'.
+
+
+
+
+
+
+ Supported scopes are 'singleton' and 'prototype'.
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ Name of the implementation class.
+
+
+
+
+
+
+ The default timeout for '!!' invocations.
+
+
+
+
+
+
+ Set this to true if messages should have REQUIRES_NEW semantics.
+
+
+
+
+
+
+ Defines the lifecycle, can be either 'permanent' or 'temporary'.
+
+
+
+
+
+
+ Supported scopes are 'singleton' and 'prototype'.
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ Name of the remote host.
+
+
+
+
+
+
+ Port of the remote host.
+
+
+
+
+
+
+ Custom service name or class name for the server managed actor.
+
+
+
+
+
+
+ Name of the interface the typed actor implements.
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+ Failover scheme, can be one of 'AllForOne' or 'OneForOne'.
+
+
+
+
+
+
+ Maximal number of restarts.
+
+
+
+
+
+
+ Time range for maximal number of restart.
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala b/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala
new file mode 100644
index 0000000000..55aa82b8e4
--- /dev/null
+++ b/akka-spring/src/main/scala/ActorBeanDefinitionParser.scala
@@ -0,0 +1,72 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+package se.scalablesolutions.akka.spring
+
+import org.springframework.beans.factory.support.BeanDefinitionBuilder
+import org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser
+import org.springframework.beans.factory.xml.ParserContext
+import AkkaSpringConfigurationTags._
+import org.w3c.dom.Element
+
+
+/**
+ * Parser for custom namespace configuration.
+ * @author michaelkober
+ */
+class TypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser {
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ val typedActorConf = parseActor(element)
+ typedActorConf.typed = TYPED_ACTOR_TAG
+ typedActorConf.setAsProperties(builder)
+ }
+
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean]
+}
+
+
+/**
+ * Parser for custom namespace configuration.
+ * @author michaelkober
+ */
+class UntypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser {
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ val untypedActorConf = parseActor(element)
+ untypedActorConf.typed = UNTYPED_ACTOR_TAG
+ untypedActorConf.setAsProperties(builder)
+ }
+
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean]
+}
+
+
+/**
+ * Parser for custom namespace configuration.
+ * @author michaelkober
+ */
+class ActorForBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorForParser {
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
+ */
+ override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
+ val actorForConf = parseActorFor(element)
+ actorForConf.setAsProperties(builder)
+ }
+
+ /*
+ * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
+ */
+ override def getBeanClass(element: Element): Class[_] = classOf[ActorForFactoryBean]
+}
diff --git a/akka-spring/src/main/scala/ActorFactoryBean.scala b/akka-spring/src/main/scala/ActorFactoryBean.scala
index 11d5274a70..caa344825a 100644
--- a/akka-spring/src/main/scala/ActorFactoryBean.scala
+++ b/akka-spring/src/main/scala/ActorFactoryBean.scala
@@ -4,22 +4,19 @@
package se.scalablesolutions.akka.spring
-import java.beans.PropertyDescriptor
-import java.lang.reflect.Method
-import javax.annotation.PreDestroy
-import javax.annotation.PostConstruct
-
import org.springframework.beans.{BeanUtils,BeansException,BeanWrapper,BeanWrapperImpl}
-import org.springframework.beans.factory.BeanFactory
+import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer}
+//import org.springframework.beans.factory.BeanFactory
import org.springframework.beans.factory.config.AbstractFactoryBean
import org.springframework.context.{ApplicationContext,ApplicationContextAware}
-import org.springframework.util.ReflectionUtils
+//import org.springframework.util.ReflectionUtils
import org.springframework.util.StringUtils
import se.scalablesolutions.akka.actor.{ActorRef, AspectInitRegistry, TypedActorConfiguration, TypedActor,Actor}
import se.scalablesolutions.akka.dispatch.MessageDispatcher
import se.scalablesolutions.akka.util.{Logging, Duration}
import scala.reflect.BeanProperty
+import java.net.InetSocketAddress
/**
* Exception to use when something goes wrong during bean creation.
@@ -49,13 +46,15 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App
@BeanProperty var transactional: Boolean = false
@BeanProperty var host: String = ""
@BeanProperty var port: Int = _
+ @BeanProperty var serverManaged: Boolean = false
+ @BeanProperty var serviceName: String = ""
@BeanProperty var lifecycle: String = ""
@BeanProperty var dispatcher: DispatcherProperties = _
@BeanProperty var scope: String = VAL_SCOPE_SINGLETON
@BeanProperty var property: PropertyEntries = _
@BeanProperty var applicationContext: ApplicationContext = _
- // Holds info about if deps has been set or not. Depends on
+ // Holds info about if deps have been set or not. Depends on
// if interface is specified or not. We must set deps on
// target instance if interface is specified
var hasSetDependecies = false
@@ -94,7 +93,16 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App
if (implementation == null || implementation == "") throw new AkkaBeansException(
"The 'implementation' part of the 'akka:typed-actor' element in the Spring config file can't be null or empty string")
- TypedActor.newInstance(interface.toClass, implementation.toClass, createConfig)
+ val typedActor: AnyRef = TypedActor.newInstance(interface.toClass, implementation.toClass, createConfig)
+ if (isRemote && serverManaged) {
+ val server = RemoteServer.getOrCreateServer(new InetSocketAddress(host, port))
+ if (serviceName.isEmpty) {
+ server.registerTypedActor(interface, typedActor)
+ } else {
+ server.registerTypedActor(serviceName, typedActor)
+ }
+ }
+ typedActor
}
/**
@@ -111,7 +119,16 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App
actorRef.makeTransactionRequired
}
if (isRemote) {
- actorRef.makeRemote(host, port)
+ if (serverManaged) {
+ val server = RemoteServer.getOrCreateServer(new InetSocketAddress(host, port))
+ if (serviceName.isEmpty) {
+ server.register(actorRef)
+ } else {
+ server.register(serviceName, actorRef)
+ }
+ } else {
+ actorRef.makeRemote(host, port)
+ }
}
if (hasDispatcher) {
if (dispatcher.dispatcherType != THREAD_BASED){
@@ -159,7 +176,7 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App
private[akka] def createConfig: TypedActorConfiguration = {
val config = new TypedActorConfiguration().timeout(Duration(timeout, "millis"))
if (transactional) config.makeTransactionRequired
- if (isRemote) config.makeRemote(host, port)
+ if (isRemote && !serverManaged) config.makeRemote(host, port)
if (hasDispatcher) {
if (dispatcher.dispatcherType != THREAD_BASED) {
config.dispatcher(dispatcherInstance())
@@ -191,3 +208,39 @@ class ActorFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with App
}
}
}
+
+/**
+ * Factory bean for remote client actor-for.
+ *
+ * @author michaelkober
+ */
+class ActorForFactoryBean extends AbstractFactoryBean[AnyRef] with Logging with ApplicationContextAware {
+ import StringReflect._
+ import AkkaSpringConfigurationTags._
+
+ @BeanProperty var interface: String = ""
+ @BeanProperty var host: String = ""
+ @BeanProperty var port: Int = _
+ @BeanProperty var serviceName: String = ""
+ //@BeanProperty var scope: String = VAL_SCOPE_SINGLETON
+ @BeanProperty var applicationContext: ApplicationContext = _
+
+ override def isSingleton = false
+
+ /*
+ * @see org.springframework.beans.factory.FactoryBean#getObjectType()
+ */
+ def getObjectType: Class[AnyRef] = classOf[AnyRef]
+
+ /*
+ * @see org.springframework.beans.factory.config.AbstractFactoryBean#createInstance()
+ */
+ def createInstance: AnyRef = {
+ if (interface.isEmpty) {
+ RemoteClient.actorFor(serviceName, host, port)
+ } else {
+ RemoteClient.typedActorFor(interface.toClass, serviceName, host, port)
+ }
+ }
+}
+
diff --git a/akka-spring/src/main/scala/ActorParser.scala b/akka-spring/src/main/scala/ActorParser.scala
index 69073bd52f..9858e9ad7e 100644
--- a/akka-spring/src/main/scala/ActorParser.scala
+++ b/akka-spring/src/main/scala/ActorParser.scala
@@ -6,6 +6,7 @@ package se.scalablesolutions.akka.spring
import org.springframework.util.xml.DomUtils
import org.w3c.dom.Element
import scala.collection.JavaConversions._
+import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.actor.IllegalActorStateException
@@ -27,11 +28,17 @@ trait ActorParser extends BeanParser with DispatcherParser {
val objectProperties = new ActorProperties()
val remoteElement = DomUtils.getChildElementByTagName(element, REMOTE_TAG);
val dispatcherElement = DomUtils.getChildElementByTagName(element, DISPATCHER_TAG)
- val propertyEntries = DomUtils.getChildElementsByTagName(element,PROPERTYENTRY_TAG)
+ val propertyEntries = DomUtils.getChildElementsByTagName(element, PROPERTYENTRY_TAG)
if (remoteElement != null) {
objectProperties.host = mandatory(remoteElement, HOST)
objectProperties.port = mandatory(remoteElement, PORT).toInt
+ objectProperties.serverManaged = (remoteElement.getAttribute(MANAGED_BY) != null) && (remoteElement.getAttribute(MANAGED_BY).equals(SERVER_MANAGED))
+ val serviceName = remoteElement.getAttribute(SERVICE_NAME)
+ if ((serviceName != null) && (!serviceName.isEmpty)) {
+ objectProperties.serviceName = serviceName
+ objectProperties.serverManaged = true
+ }
}
if (dispatcherElement != null) {
@@ -43,7 +50,7 @@ trait ActorParser extends BeanParser with DispatcherParser {
val entry = new PropertyEntry
entry.name = element.getAttribute("name");
entry.value = element.getAttribute("value")
- entry.ref = element.getAttribute("ref")
+ entry.ref = element.getAttribute("ref")
objectProperties.propertyEntries.add(entry)
}
@@ -59,15 +66,13 @@ trait ActorParser extends BeanParser with DispatcherParser {
objectProperties.target = mandatory(element, IMPLEMENTATION)
objectProperties.transactional = if (element.getAttribute(TRANSACTIONAL).isEmpty) false else element.getAttribute(TRANSACTIONAL).toBoolean
- if (!element.getAttribute(INTERFACE).isEmpty) {
+ if (element.hasAttribute(INTERFACE)) {
objectProperties.interface = element.getAttribute(INTERFACE)
}
-
- if (!element.getAttribute(LIFECYCLE).isEmpty) {
+ if (element.hasAttribute(LIFECYCLE)) {
objectProperties.lifecycle = element.getAttribute(LIFECYCLE)
}
-
- if (!element.getAttribute(SCOPE).isEmpty) {
+ if (element.hasAttribute(SCOPE)) {
objectProperties.scope = element.getAttribute(SCOPE)
}
@@ -75,3 +80,158 @@ trait ActorParser extends BeanParser with DispatcherParser {
}
}
+
+/**
+ * Parser trait for custom namespace configuration for RemoteClient actor-for.
+ * @author michaelkober
+ */
+trait ActorForParser extends BeanParser {
+ import AkkaSpringConfigurationTags._
+
+ /**
+ * Parses the given element and returns a ActorForProperties.
+ * @param element dom element to parse
+ * @return configuration for the typed actor
+ */
+ def parseActorFor(element: Element): ActorForProperties = {
+ val objectProperties = new ActorForProperties()
+
+ objectProperties.host = mandatory(element, HOST)
+ objectProperties.port = mandatory(element, PORT).toInt
+ objectProperties.serviceName = mandatory(element, SERVICE_NAME)
+ if (element.hasAttribute(INTERFACE)) {
+ objectProperties.interface = element.getAttribute(INTERFACE)
+ }
+ objectProperties
+ }
+
+}
+
+/**
+ * Base trait with utility methods for bean parsing.
+ */
+trait BeanParser extends Logging {
+
+ /**
+ * Get a mandatory element attribute.
+ * @param element the element with the mandatory attribute
+ * @param attribute name of the mandatory attribute
+ */
+ def mandatory(element: Element, attribute: String): String = {
+ if ((element.getAttribute(attribute) == null) || (element.getAttribute(attribute).isEmpty)) {
+ throw new IllegalArgumentException("Mandatory attribute missing: " + attribute)
+ } else {
+ element.getAttribute(attribute)
+ }
+ }
+
+ /**
+ * Get a mandatory child element.
+ * @param element the parent element
+ * @param childName name of the mandatory child element
+ */
+ def mandatoryElement(element: Element, childName: String): Element = {
+ val childElement = DomUtils.getChildElementByTagName(element, childName);
+ if (childElement == null) {
+ throw new IllegalArgumentException("Mandatory element missing: ''")
+ } else {
+ childElement
+ }
+ }
+
+}
+
+
+/**
+ * Parser trait for custom namespace for Akka dispatcher configuration.
+ * @author michaelkober
+ */
+trait DispatcherParser extends BeanParser {
+ import AkkaSpringConfigurationTags._
+
+ /**
+ * Parses the given element and returns a DispatcherProperties.
+ * @param element dom element to parse
+ * @return configuration for the dispatcher
+ */
+ def parseDispatcher(element: Element): DispatcherProperties = {
+ val properties = new DispatcherProperties()
+ var dispatcherElement = element
+ if (hasRef(element)) {
+ val ref = element.getAttribute(REF)
+ dispatcherElement = element.getOwnerDocument.getElementById(ref)
+ if (dispatcherElement == null) {
+ throw new IllegalArgumentException("Referenced dispatcher not found: '" + ref + "'")
+ }
+ }
+
+ properties.dispatcherType = mandatory(dispatcherElement, TYPE)
+ if (properties.dispatcherType == THREAD_BASED) {
+ val allowedParentNodes = "akka:typed-actor" :: "akka:untyped-actor" :: "typed-actor" :: "untyped-actor" :: Nil
+ if (!allowedParentNodes.contains(dispatcherElement.getParentNode.getNodeName)) {
+ throw new IllegalArgumentException("Thread based dispatcher must be nested in 'typed-actor' or 'untyped-actor' element!")
+ }
+ }
+
+ if (properties.dispatcherType == HAWT) { // no name for HawtDispatcher
+ properties.name = dispatcherElement.getAttribute(NAME)
+ if (dispatcherElement.hasAttribute(AGGREGATE)) {
+ properties.aggregate = dispatcherElement.getAttribute(AGGREGATE).toBoolean
+ }
+ } else {
+ properties.name = mandatory(dispatcherElement, NAME)
+ }
+
+ val threadPoolElement = DomUtils.getChildElementByTagName(dispatcherElement, THREAD_POOL_TAG);
+ if (threadPoolElement != null) {
+ if (properties.dispatcherType == THREAD_BASED) {
+ throw new IllegalArgumentException("Element 'thread-pool' not allowed for this dispatcher type.")
+ }
+ val threadPoolProperties = parseThreadPool(threadPoolElement)
+ properties.threadPool = threadPoolProperties
+ }
+ properties
+ }
+
+ /**
+ * Parses the given element and returns a ThreadPoolProperties.
+ * @param element dom element to parse
+ * @return configuration for the thread pool
+ */
+ def parseThreadPool(element: Element): ThreadPoolProperties = {
+ val properties = new ThreadPoolProperties()
+ properties.queue = element.getAttribute(QUEUE)
+ if (element.hasAttribute(CAPACITY)) {
+ properties.capacity = element.getAttribute(CAPACITY).toInt
+ }
+ if (element.hasAttribute(BOUND)) {
+ properties.bound = element.getAttribute(BOUND).toInt
+ }
+ if (element.hasAttribute(FAIRNESS)) {
+ properties.fairness = element.getAttribute(FAIRNESS).toBoolean
+ }
+ if (element.hasAttribute(CORE_POOL_SIZE)) {
+ properties.corePoolSize = element.getAttribute(CORE_POOL_SIZE).toInt
+ }
+ if (element.hasAttribute(MAX_POOL_SIZE)) {
+ properties.maxPoolSize = element.getAttribute(MAX_POOL_SIZE).toInt
+ }
+ if (element.hasAttribute(KEEP_ALIVE)) {
+ properties.keepAlive = element.getAttribute(KEEP_ALIVE).toLong
+ }
+ if (element.hasAttribute(REJECTION_POLICY)) {
+ properties.rejectionPolicy = element.getAttribute(REJECTION_POLICY)
+ }
+ if (element.hasAttribute(MAILBOX_CAPACITY)) {
+ properties.mailboxCapacity = element.getAttribute(MAILBOX_CAPACITY).toInt
+ }
+ properties
+ }
+
+ def hasRef(element: Element): Boolean = {
+ val ref = element.getAttribute(REF)
+ (ref != null) && !ref.isEmpty
+ }
+
+}
+
diff --git a/akka-spring/src/main/scala/ActorProperties.scala b/akka-spring/src/main/scala/ActorProperties.scala
index 15c7e61fe0..0f86942935 100644
--- a/akka-spring/src/main/scala/ActorProperties.scala
+++ b/akka-spring/src/main/scala/ActorProperties.scala
@@ -8,7 +8,7 @@ import org.springframework.beans.factory.support.BeanDefinitionBuilder
import AkkaSpringConfigurationTags._
/**
- * Data container for typed actor configuration data.
+ * Data container for actor configuration data.
* @author michaelkober
* @author Martin Krasser
*/
@@ -20,6 +20,8 @@ class ActorProperties {
var transactional: Boolean = false
var host: String = ""
var port: Int = _
+ var serverManaged: Boolean = false
+ var serviceName: String = ""
var lifecycle: String = ""
var scope:String = VAL_SCOPE_SINGLETON
var dispatcher: DispatcherProperties = _
@@ -34,6 +36,8 @@ class ActorProperties {
builder.addPropertyValue("typed", typed)
builder.addPropertyValue(HOST, host)
builder.addPropertyValue(PORT, port)
+ builder.addPropertyValue("serverManaged", serverManaged)
+ builder.addPropertyValue("serviceName", serviceName)
builder.addPropertyValue(TIMEOUT, timeout)
builder.addPropertyValue(IMPLEMENTATION, target)
builder.addPropertyValue(INTERFACE, interface)
@@ -45,3 +49,26 @@ class ActorProperties {
}
}
+
+/**
+ * Data container for actor configuration data.
+ * @author michaelkober
+ */
+class ActorForProperties {
+ var interface: String = ""
+ var host: String = ""
+ var port: Int = _
+ var serviceName: String = ""
+
+ /**
+ * Sets the properties to the given builder.
+ * @param builder bean definition builder
+ */
+ def setAsProperties(builder: BeanDefinitionBuilder) {
+ builder.addPropertyValue(HOST, host)
+ builder.addPropertyValue(PORT, port)
+ builder.addPropertyValue("serviceName", serviceName)
+ builder.addPropertyValue(INTERFACE, interface)
+ }
+
+}
diff --git a/akka-spring/src/main/scala/AkkaNamespaceHandler.scala b/akka-spring/src/main/scala/AkkaNamespaceHandler.scala
index a478b7b262..b1c58baa20 100644
--- a/akka-spring/src/main/scala/AkkaNamespaceHandler.scala
+++ b/akka-spring/src/main/scala/AkkaNamespaceHandler.scala
@@ -12,10 +12,11 @@ import AkkaSpringConfigurationTags._
*/
class AkkaNamespaceHandler extends NamespaceHandlerSupport {
def init = {
- registerBeanDefinitionParser(TYPED_ACTOR_TAG, new TypedActorBeanDefinitionParser());
- registerBeanDefinitionParser(UNTYPED_ACTOR_TAG, new UntypedActorBeanDefinitionParser());
- registerBeanDefinitionParser(SUPERVISION_TAG, new SupervisionBeanDefinitionParser());
- registerBeanDefinitionParser(DISPATCHER_TAG, new DispatcherBeanDefinitionParser());
- registerBeanDefinitionParser(CAMEL_SERVICE_TAG, new CamelServiceBeanDefinitionParser);
+ registerBeanDefinitionParser(TYPED_ACTOR_TAG, new TypedActorBeanDefinitionParser())
+ registerBeanDefinitionParser(UNTYPED_ACTOR_TAG, new UntypedActorBeanDefinitionParser())
+ registerBeanDefinitionParser(SUPERVISION_TAG, new SupervisionBeanDefinitionParser())
+ registerBeanDefinitionParser(DISPATCHER_TAG, new DispatcherBeanDefinitionParser())
+ registerBeanDefinitionParser(CAMEL_SERVICE_TAG, new CamelServiceBeanDefinitionParser)
+ registerBeanDefinitionParser(ACTOR_FOR_TAG, new ActorForBeanDefinitionParser());
}
}
diff --git a/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala b/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala
index 518727bd4c..0e4de3576f 100644
--- a/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala
+++ b/akka-spring/src/main/scala/AkkaSpringConfigurationTags.scala
@@ -19,6 +19,7 @@ object AkkaSpringConfigurationTags {
val DISPATCHER_TAG = "dispatcher"
val PROPERTYENTRY_TAG = "property"
val CAMEL_SERVICE_TAG = "camel-service"
+ val ACTOR_FOR_TAG = "actor-for"
// actor sub tags
val REMOTE_TAG = "remote"
@@ -45,6 +46,8 @@ object AkkaSpringConfigurationTags {
val TRANSACTIONAL = "transactional"
val HOST = "host"
val PORT = "port"
+ val MANAGED_BY = "managed-by"
+ val SERVICE_NAME = "service-name"
val LIFECYCLE = "lifecycle"
val SCOPE = "scope"
@@ -68,6 +71,7 @@ object AkkaSpringConfigurationTags {
val KEEP_ALIVE = "keep-alive"
val BOUND ="bound"
val REJECTION_POLICY ="rejection-policy"
+ val MAILBOX_CAPACITY ="mailbox-capacity"
// --- VALUES
//
@@ -97,9 +101,11 @@ object AkkaSpringConfigurationTags {
// dispatcher types
val EXECUTOR_BASED_EVENT_DRIVEN = "executor-based-event-driven"
val EXECUTOR_BASED_EVENT_DRIVEN_WORK_STEALING = "executor-based-event-driven-work-stealing"
- val REACTOR_BASED_THREAD_POOL_EVENT_DRIVEN = "reactor-based-thread-pool-event-driven"
- val REACTOR_BASED_SINGLE_THREAD_EVENT_DRIVEN = "reactor-based-single-thread-event-driven"
val THREAD_BASED = "thread-based"
val HAWT = "hawt"
+ // managed by types
+ val SERVER_MANAGED = "server"
+ val CLIENT_MANAGED = "client"
+
}
diff --git a/akka-spring/src/main/scala/BeanParser.scala b/akka-spring/src/main/scala/BeanParser.scala
deleted file mode 100644
index 1bbba9f09f..0000000000
--- a/akka-spring/src/main/scala/BeanParser.scala
+++ /dev/null
@@ -1,42 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-package se.scalablesolutions.akka.spring
-
-import se.scalablesolutions.akka.util.Logging
-import org.w3c.dom.Element
-import org.springframework.util.xml.DomUtils
-
-/**
- * Base trait with utility methods for bean parsing.
- */
-trait BeanParser extends Logging {
-
- /**
- * Get a mandatory element attribute.
- * @param element the element with the mandatory attribute
- * @param attribute name of the mandatory attribute
- */
- def mandatory(element: Element, attribute: String): String = {
- if ((element.getAttribute(attribute) == null) || (element.getAttribute(attribute).isEmpty)) {
- throw new IllegalArgumentException("Mandatory attribute missing: " + attribute)
- } else {
- element.getAttribute(attribute)
- }
- }
-
- /**
- * Get a mandatory child element.
- * @param element the parent element
- * @param childName name of the mandatory child element
- */
- def mandatoryElement(element: Element, childName: String): Element = {
- val childElement = DomUtils.getChildElementByTagName(element, childName);
- if (childElement == null) {
- throw new IllegalArgumentException("Mandatory element missing: ''")
- } else {
- childElement
- }
- }
-
-}
diff --git a/akka-spring/src/main/scala/DispatcherFactoryBean.scala b/akka-spring/src/main/scala/DispatcherFactoryBean.scala
index 06c9994c7f..4d13fa6814 100644
--- a/akka-spring/src/main/scala/DispatcherFactoryBean.scala
+++ b/akka-spring/src/main/scala/DispatcherFactoryBean.scala
@@ -26,8 +26,6 @@ object DispatcherFactoryBean {
var dispatcher = properties.dispatcherType match {
case EXECUTOR_BASED_EVENT_DRIVEN => Dispatchers.newExecutorBasedEventDrivenDispatcher(properties.name)
case EXECUTOR_BASED_EVENT_DRIVEN_WORK_STEALING => Dispatchers.newExecutorBasedEventDrivenWorkStealingDispatcher(properties.name)
- case REACTOR_BASED_THREAD_POOL_EVENT_DRIVEN => Dispatchers.newReactorBasedThreadPoolEventDrivenDispatcher(properties.name)
- case REACTOR_BASED_SINGLE_THREAD_EVENT_DRIVEN => Dispatchers.newReactorBasedSingleThreadEventDrivenDispatcher(properties.name)
case THREAD_BASED => if (!actorRef.isDefined) {
throw new IllegalArgumentException("Need an ActorRef to create a thread based dispatcher.")
} else {
@@ -58,6 +56,9 @@ object DispatcherFactoryBean {
if (properties.threadPool.keepAlive > -1) {
threadPoolBuilder.setKeepAliveTimeInMillis(properties.threadPool.keepAlive)
}
+ if (properties.threadPool.mailboxCapacity > -1) {
+ threadPoolBuilder.setMailboxCapacity(properties.threadPool.mailboxCapacity)
+ }
if ((properties.threadPool.rejectionPolicy != null) && (!properties.threadPool.rejectionPolicy.isEmpty)) {
val policy: RejectedExecutionHandler = properties.threadPool.rejectionPolicy match {
case "abort-policy" => new AbortPolicy()
diff --git a/akka-spring/src/main/scala/DispatcherParser.scala b/akka-spring/src/main/scala/DispatcherParser.scala
deleted file mode 100644
index c4257230f7..0000000000
--- a/akka-spring/src/main/scala/DispatcherParser.scala
+++ /dev/null
@@ -1,98 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-package se.scalablesolutions.akka.spring
-
-import org.w3c.dom.Element
-import org.springframework.util.xml.DomUtils
-
-/**
- * Parser trait for custom namespace for Akka dispatcher configuration.
- * @author michaelkober
- */
-trait DispatcherParser extends BeanParser {
- import AkkaSpringConfigurationTags._
-
- /**
- * Parses the given element and returns a DispatcherProperties.
- * @param element dom element to parse
- * @return configuration for the dispatcher
- */
- def parseDispatcher(element: Element): DispatcherProperties = {
- val properties = new DispatcherProperties()
- var dispatcherElement = element
- if (hasRef(element)) {
- val ref = element.getAttribute(REF)
- dispatcherElement = element.getOwnerDocument.getElementById(ref)
- if (dispatcherElement == null) {
- throw new IllegalArgumentException("Referenced dispatcher not found: '" + ref + "'")
- }
- }
-
- properties.dispatcherType = mandatory(dispatcherElement, TYPE)
- if (properties.dispatcherType == THREAD_BASED) {
- val allowedParentNodes = "akka:typed-actor" :: "akka:untyped-actor" :: "typed-actor" :: "untyped-actor" :: Nil
- if (!allowedParentNodes.contains(dispatcherElement.getParentNode.getNodeName)) {
- throw new IllegalArgumentException("Thread based dispatcher must be nested in 'typed-actor' or 'untyped-actor' element!")
- }
- }
-
- if (properties.dispatcherType == HAWT) { // no name for HawtDispatcher
- properties.name = dispatcherElement.getAttribute(NAME)
- if (dispatcherElement.hasAttribute(AGGREGATE)) {
- properties.aggregate = dispatcherElement.getAttribute(AGGREGATE).toBoolean
- }
- } else {
- properties.name = mandatory(dispatcherElement, NAME)
- }
-
- val threadPoolElement = DomUtils.getChildElementByTagName(dispatcherElement, THREAD_POOL_TAG);
- if (threadPoolElement != null) {
- if (properties.dispatcherType == REACTOR_BASED_SINGLE_THREAD_EVENT_DRIVEN ||
- properties.dispatcherType == THREAD_BASED) {
- throw new IllegalArgumentException("Element 'thread-pool' not allowed for this dispatcher type.")
- }
- val threadPoolProperties = parseThreadPool(threadPoolElement)
- properties.threadPool = threadPoolProperties
- }
- properties
- }
-
- /**
- * Parses the given element and returns a ThreadPoolProperties.
- * @param element dom element to parse
- * @return configuration for the thread pool
- */
- def parseThreadPool(element: Element): ThreadPoolProperties = {
- val properties = new ThreadPoolProperties()
- properties.queue = element.getAttribute(QUEUE)
- if (element.hasAttribute(CAPACITY)) {
- properties.capacity = element.getAttribute(CAPACITY).toInt
- }
- if (element.hasAttribute(BOUND)) {
- properties.bound = element.getAttribute(BOUND).toInt
- }
- if (element.hasAttribute(FAIRNESS)) {
- properties.fairness = element.getAttribute(FAIRNESS).toBoolean
- }
- if (element.hasAttribute(CORE_POOL_SIZE)) {
- properties.corePoolSize = element.getAttribute(CORE_POOL_SIZE).toInt
- }
- if (element.hasAttribute(MAX_POOL_SIZE)) {
- properties.maxPoolSize = element.getAttribute(MAX_POOL_SIZE).toInt
- }
- if (element.hasAttribute(KEEP_ALIVE)) {
- properties.keepAlive = element.getAttribute(KEEP_ALIVE).toLong
- }
- if (element.hasAttribute(REJECTION_POLICY)) {
- properties.rejectionPolicy = element.getAttribute(REJECTION_POLICY)
- }
- properties
- }
-
- def hasRef(element: Element): Boolean = {
- val ref = element.getAttribute(REF)
- (ref != null) && !ref.isEmpty
- }
-
-}
diff --git a/akka-spring/src/main/scala/DispatcherProperties.scala b/akka-spring/src/main/scala/DispatcherProperties.scala
index 183b3825bb..89d97670ca 100644
--- a/akka-spring/src/main/scala/DispatcherProperties.scala
+++ b/akka-spring/src/main/scala/DispatcherProperties.scala
@@ -45,6 +45,7 @@ class ThreadPoolProperties {
var maxPoolSize = -1
var keepAlive = -1L
var rejectionPolicy = ""
+ var mailboxCapacity = -1
override def toString : String = {
"ThreadPoolProperties[queue=" + queue +
@@ -54,6 +55,7 @@ class ThreadPoolProperties {
", corePoolSize=" + corePoolSize +
", maxPoolSize=" + maxPoolSize +
", keepAlive=" + keepAlive +
- ", policy=" + rejectionPolicy + "]"
+ ", policy=" + rejectionPolicy +
+ ", mailboxCapacity=" + mailboxCapacity + "]"
}
}
diff --git a/akka-spring/src/main/scala/PropertyEntries.scala b/akka-spring/src/main/scala/PropertyEntries.scala
index bf1898a805..9a7dc098de 100644
--- a/akka-spring/src/main/scala/PropertyEntries.scala
+++ b/akka-spring/src/main/scala/PropertyEntries.scala
@@ -18,3 +18,19 @@ class PropertyEntries {
entryList.append(entry)
}
}
+
+/**
+ * Represents a property element
+ * @author Johan Rask
+ */
+class PropertyEntry {
+ var name: String = _
+ var value: String = null
+ var ref: String = null
+
+
+ override def toString(): String = {
+ format("name = %s,value = %s, ref = %s", name, value, ref)
+ }
+}
+
diff --git a/akka-spring/src/main/scala/PropertyEntry.scala b/akka-spring/src/main/scala/PropertyEntry.scala
deleted file mode 100644
index 9fe6357fc0..0000000000
--- a/akka-spring/src/main/scala/PropertyEntry.scala
+++ /dev/null
@@ -1,19 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-package se.scalablesolutions.akka.spring
-
-/**
- * Represents a property element
- * @author Johan Rask
- */
-class PropertyEntry {
- var name: String = _
- var value: String = null
- var ref: String = null
-
-
- override def toString(): String = {
- format("name = %s,value = %s, ref = %s", name, value, ref)
- }
-}
diff --git a/akka-spring/src/main/scala/TypedActorBeanDefinitionParser.scala b/akka-spring/src/main/scala/TypedActorBeanDefinitionParser.scala
deleted file mode 100644
index e8e0cef7d4..0000000000
--- a/akka-spring/src/main/scala/TypedActorBeanDefinitionParser.scala
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-package se.scalablesolutions.akka.spring
-
-import org.springframework.beans.factory.support.BeanDefinitionBuilder
-import org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser
-import org.springframework.beans.factory.xml.ParserContext
-import AkkaSpringConfigurationTags._
-import org.w3c.dom.Element
-
-
-/**
- * Parser for custom namespace configuration.
- * @author michaelkober
- */
-class TypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser {
- /*
- * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
- */
- override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
- val typedActorConf = parseActor(element)
- typedActorConf.typed = TYPED_ACTOR_TAG
- typedActorConf.setAsProperties(builder)
- }
-
- /*
- * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
- */
- override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean]
-}
diff --git a/akka-spring/src/main/scala/UntypedActorBeanDefinitionParser.scala b/akka-spring/src/main/scala/UntypedActorBeanDefinitionParser.scala
deleted file mode 100644
index 752e18559f..0000000000
--- a/akka-spring/src/main/scala/UntypedActorBeanDefinitionParser.scala
+++ /dev/null
@@ -1,31 +0,0 @@
-/**
- * Copyright (C) 2009-2010 Scalable Solutions AB
- */
-package se.scalablesolutions.akka.spring
-
-import org.springframework.beans.factory.support.BeanDefinitionBuilder
-import org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser
-import org.springframework.beans.factory.xml.ParserContext
-import AkkaSpringConfigurationTags._
-import org.w3c.dom.Element
-
-
-/**
- * Parser for custom namespace configuration.
- * @author michaelkober
- */
-class UntypedActorBeanDefinitionParser extends AbstractSingleBeanDefinitionParser with ActorParser {
- /*
- * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#doParse(org.w3c.dom.Element, org.springframework.beans.factory.xml.ParserContext, org.springframework.beans.factory.support.BeanDefinitionBuilder)
- */
- override def doParse(element: Element, parserContext: ParserContext, builder: BeanDefinitionBuilder) {
- val untypedActorConf = parseActor(element)
- untypedActorConf.typed = UNTYPED_ACTOR_TAG
- untypedActorConf.setAsProperties(builder)
- }
-
- /*
- * @see org.springframework.beans.factory.xml.AbstractSingleBeanDefinitionParser#getBeanClass(org.w3c.dom.Element)
- */
- override def getBeanClass(element: Element): Class[_] = classOf[ActorFactoryBean]
-}
diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java
index 24c0fea352..6046f2bb5d 100644
--- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java
+++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/Pojo.java
@@ -14,7 +14,7 @@ public class Pojo extends TypedActor implements PojoInf, ApplicationContextAware
private String stringFromRef;
private boolean gotApplicationContext = false;
- private boolean initInvoked = false;
+ private boolean preStartInvoked = false;
public boolean gotApplicationContext() {
return gotApplicationContext;
@@ -41,11 +41,11 @@ public class Pojo extends TypedActor implements PojoInf, ApplicationContextAware
}
@Override
- public void init() {
- initInvoked = true;
+ public void preStart() {
+ preStartInvoked = true;
}
- public boolean isInitInvoked() {
- return initInvoked;
+ public boolean isPreStartInvoked() {
+ return preStartInvoked;
}
}
diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java
index 9ebf80e89b..0a313ceb18 100644
--- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java
+++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/PojoInf.java
@@ -8,6 +8,6 @@ public interface PojoInf {
public String getStringFromVal();
public String getStringFromRef();
public boolean gotApplicationContext();
- public boolean isInitInvoked();
+ public boolean isPreStartInvoked();
}
diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java
index f9d3381436..29e80d1c65 100644
--- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java
+++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/SampleBean.java
@@ -19,7 +19,7 @@ public class SampleBean extends TypedActor implements SampleBeanIntf {
}
@Override
- public void shutdown() {
+ public void postStop() {
down = true;
}
}
diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java
index f2c5e24884..5a2a272e6c 100644
--- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java
+++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/IMyPojo.java
@@ -8,14 +8,12 @@ package se.scalablesolutions.akka.spring.foo;
* To change this template use File | Settings | File Templates.
*/
public interface IMyPojo {
+ public void oneWay(String message);
+
public String getFoo();
- public String getBar();
-
- public void preRestart();
-
- public void postRestart();
-
public String longRunning();
+
+
}
diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java
index fe3e9ba767..8f610eef63 100644
--- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java
+++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/MyPojo.java
@@ -1,42 +1,34 @@
package se.scalablesolutions.akka.spring.foo;
-import se.scalablesolutions.akka.actor.*;
+import se.scalablesolutions.akka.actor.TypedActor;
-public class MyPojo extends TypedActor implements IMyPojo{
+import java.util.concurrent.CountDownLatch;
- private String foo;
- private String bar;
+public class MyPojo extends TypedActor implements IMyPojo {
+
+ public static CountDownLatch latch = new CountDownLatch(1);
+ public static String lastOneWayMessage = null;
+ private String foo = "foo";
- public MyPojo() {
- this.foo = "foo";
- this.bar = "bar";
- }
+ public MyPojo() {
+ }
+ public String getFoo() {
+ return foo;
+ }
- public String getFoo() {
- return foo;
- }
+ public void oneWay(String message) {
+ lastOneWayMessage = message;
+ latch.countDown();
+ }
-
- public String getBar() {
- return bar;
- }
-
- public void preRestart() {
- System.out.println("pre restart");
- }
-
- public void postRestart() {
- System.out.println("post restart");
- }
-
- public String longRunning() {
- try {
- Thread.sleep(6000);
- } catch (InterruptedException e) {
- }
- return "this took long";
+ public String longRunning() {
+ try {
+ Thread.sleep(6000);
+ } catch (InterruptedException e) {
}
+ return "this took long";
+ }
}
diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java
index e447b26a28..3063a1b529 100644
--- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java
+++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/PingActor.java
@@ -6,6 +6,8 @@ import se.scalablesolutions.akka.actor.ActorRef;
import org.springframework.context.ApplicationContext;
import org.springframework.context.ApplicationContextAware;
+import java.util.concurrent.CountDownLatch;
+
/**
* test class
@@ -14,6 +16,9 @@ public class PingActor extends UntypedActor implements ApplicationContextAware {
private String stringFromVal;
private String stringFromRef;
+ public static String lastMessage = null;
+ public static CountDownLatch latch = new CountDownLatch(1);
+
private boolean gotApplicationContext = false;
@@ -42,7 +47,6 @@ public class PingActor extends UntypedActor implements ApplicationContextAware {
stringFromRef = s;
}
-
private String longRunning() {
try {
Thread.sleep(6000);
@@ -53,12 +57,12 @@ public class PingActor extends UntypedActor implements ApplicationContextAware {
public void onReceive(Object message) throws Exception {
if (message instanceof String) {
- System.out.println("Ping received String message: " + message);
+ lastMessage = (String) message;
if (message.equals("longRunning")) {
- System.out.println("### starting pong");
ActorRef pongActor = UntypedActor.actorOf(PongActor.class).start();
pongActor.sendRequestReply("longRunning", getContext());
}
+ latch.countDown();
} else {
throw new IllegalArgumentException("Unknown message: " + message);
}
diff --git a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java
index 3b4e05453b..ce85267edc 100644
--- a/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java
+++ b/akka-spring/src/test/java/se/scalablesolutions/akka/spring/foo/StatefulPojo.java
@@ -5,6 +5,7 @@ import se.scalablesolutions.akka.stm.TransactionalMap;
import se.scalablesolutions.akka.stm.TransactionalVector;
import se.scalablesolutions.akka.stm.Ref;
import se.scalablesolutions.akka.actor.*;
+import se.scalablesolutions.akka.stm.local.Atomic;
public class StatefulPojo extends TypedActor {
private TransactionalMap mapState;
@@ -13,12 +14,16 @@ public class StatefulPojo extends TypedActor {
private boolean isInitialized = false;
@Override
- public void initTransactionalState() {
- if (!isInitialized) {
- mapState = new TransactionalMap();
- vectorState = new TransactionalVector();
- refState = new Ref();
- isInitialized = true;
+ public void preStart() {
+ if(!isInitialized) {
+ isInitialized = new Atomic() {
+ public Boolean atomically() {
+ mapState = new TransactionalMap();
+ vectorState = new TransactionalVector();
+ refState = new Ref();
+ return true;
+ }
+ }.execute();
}
}
diff --git a/akka-spring/src/test/resources/dispatcher-config.xml b/akka-spring/src/test/resources/dispatcher-config.xml
index 9f0dfa3802..728917c6c8 100644
--- a/akka-spring/src/test/resources/dispatcher-config.xml
+++ b/akka-spring/src/test/resources/dispatcher-config.xml
@@ -42,6 +42,13 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd">
bound="10" />
+
+
+
+
+
-
-
-
-
-
-
-
diff --git a/akka-spring/src/test/resources/server-managed-config.xml b/akka-spring/src/test/resources/server-managed-config.xml
new file mode 100644
index 0000000000..128b16c8b6
--- /dev/null
+++ b/akka-spring/src/test/resources/server-managed-config.xml
@@ -0,0 +1,57 @@
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/akka-spring/src/test/resources/typed-actor-config.xml b/akka-spring/src/test/resources/typed-actor-config.xml
index faca749469..989884e4fa 100644
--- a/akka-spring/src/test/resources/typed-actor-config.xml
+++ b/akka-spring/src/test/resources/typed-actor-config.xml
@@ -37,7 +37,7 @@ http://scalablesolutions.se/akka/akka-1.0-SNAPSHOT.xsd">
implementation="se.scalablesolutions.akka.spring.foo.MyPojo"
timeout="2000"
transactional="true">
-
+
-
+
val props = parser.parseDispatcher(dom(xml).getDocumentElement);
assert(props != null)
- assert(props.dispatcherType == "reactor-based-thread-pool-event-driven")
+ assert(props.dispatcherType == "executor-based-event-driven")
assert(props.name == "myDispatcher")
assert(props.threadPool.corePoolSize == 2)
assert(props.threadPool.maxPoolSize == 10)
@@ -86,16 +86,6 @@ class DispatcherBeanDefinitionParserTest extends Spec with ShouldMatchers {
evaluating {parser.parseDispatcher(dom(xml).getDocumentElement)} should produce[IllegalArgumentException]
}
- it("should throw IllegalArgumentException when configuring a single thread dispatcher with a thread pool") {
- val xml =
-
-
- evaluating {parser.parseDispatcher(dom(xml).getDocumentElement)} should produce[IllegalArgumentException]
- }
-
-
it("should throw IllegalArgumentException when configuring a thread based dispatcher without TypedActor or UntypedActor") {
val xml =
evaluating {parser.parseDispatcher(dom(xml).getDocumentElement)} should produce[IllegalArgumentException]
diff --git a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala
index 7b2c740c73..db62acde3f 100644
--- a/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala
+++ b/akka-spring/src/test/scala/DispatcherSpringFeatureTest.scala
@@ -6,8 +6,6 @@ package se.scalablesolutions.akka.spring
import foo.{IMyPojo, MyPojo, PingActor}
import se.scalablesolutions.akka.dispatch._
-import se.scalablesolutions.akka.actor.ActorRef
-
import org.scalatest.FeatureSpec
import org.scalatest.matchers.ShouldMatchers
import org.scalatest.junit.JUnitRunner
@@ -18,6 +16,10 @@ import org.springframework.context.ApplicationContext
import org.springframework.context.support.ClassPathXmlApplicationContext
import org.springframework.core.io.{ClassPathResource, Resource}
import java.util.concurrent._
+import se.scalablesolutions.akka.actor.{UntypedActor, Actor, ActorRef}
+
+
+
/**
* Tests for spring configuration of typed actors.
@@ -41,27 +43,39 @@ class DispatcherSpringFeatureTest extends FeatureSpec with ShouldMatchers {
assert(executor.getQueue().remainingCapacity() === 100)
}
+
scenario("get a dispatcher via ref from context") {
val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml")
val pojo = context.getBean("typed-actor-with-dispatcher-ref").asInstanceOf[IMyPojo]
assert(pojo != null)
}
- scenario("get a executor-event-driven-dispatcher with bounded-linked-blocking-queue with unbounded capacity from context") {
+ scenario("get a executor-event-driven-dispatcher with blocking-queue with unbounded capacity from context") {
val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml")
val dispatcher = context.getBean("executor-event-driven-dispatcher-2").asInstanceOf[ExecutorBasedEventDrivenDispatcher]
val executor = getThreadPoolExecutorAndAssert(dispatcher)
- assert(executor.getQueue().isInstanceOf[LinkedBlockingQueue[Runnable]])
+ assert(executor.getQueue().isInstanceOf[BlockingQueue[Runnable]])
assert(executor.getQueue().remainingCapacity() === Integer.MAX_VALUE)
assert(dispatcher.name === EVENT_DRIVEN_PREFIX + "dispatcher-2")
}
+ scenario("get a executor-event-driven-dispatcher with bounded-blocking-queue and with bounded mailbox capacity") {
+ val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml")
+ val dispatcher = context.getBean("executor-event-driven-dispatcher-mc").asInstanceOf[ExecutorBasedEventDrivenDispatcher]
+ assert(dispatcher.name === EVENT_DRIVEN_PREFIX + "dispatcher-mc")
+ val actorRef = UntypedActor.actorOf(classOf[PingActor])
+ actorRef.dispatcher = dispatcher
+ actorRef.start
+ assert(actorRef.mailbox.isInstanceOf[BlockingQueue[MessageInvocation]])
+ assert((actorRef.mailbox.asInstanceOf[BlockingQueue[MessageInvocation]]).remainingCapacity === 1000)
+ }
+
scenario("get a executor-event-driven-dispatcher with unbounded-linked-blocking-queue with bounded capacity from context") {
val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml")
val dispatcher = context.getBean("executor-event-driven-dispatcher-4").asInstanceOf[ExecutorBasedEventDrivenDispatcher]
assert(dispatcher.name === EVENT_DRIVEN_PREFIX + "dispatcher-4")
val executor = getThreadPoolExecutorAndAssert(dispatcher)
- assert(executor.getQueue().isInstanceOf[LinkedBlockingQueue[Runnable]])
+ assert(executor.getQueue().isInstanceOf[BlockingQueue[Runnable]])
assert(executor.getQueue().remainingCapacity() === 55)
}
@@ -70,7 +84,7 @@ class DispatcherSpringFeatureTest extends FeatureSpec with ShouldMatchers {
val dispatcher = context.getBean("executor-event-driven-dispatcher-5").asInstanceOf[ExecutorBasedEventDrivenDispatcher]
assert(dispatcher.name === EVENT_DRIVEN_PREFIX + "dispatcher-5")
val executor = getThreadPoolExecutorAndAssert(dispatcher)
- assert(executor.getQueue().isInstanceOf[LinkedBlockingQueue[Runnable]])
+ assert(executor.getQueue().isInstanceOf[BlockingQueue[Runnable]])
assert(executor.getQueue().remainingCapacity() === Integer.MAX_VALUE)
}
@@ -82,19 +96,6 @@ class DispatcherSpringFeatureTest extends FeatureSpec with ShouldMatchers {
assert(executor.getQueue().isInstanceOf[SynchronousQueue[Runnable]])
}
- scenario("get a reactor-based-thread-pool-event-driven-dispatcher with synchronous-queue from context") {
- val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml")
- val dispatcher = context.getBean("reactor-based-thread-pool-event-driven-dispatcher").asInstanceOf[ReactorBasedThreadPoolEventDrivenDispatcher]
- val executor = getThreadPoolExecutorAndAssert(dispatcher)
- assert(executor.getQueue().isInstanceOf[SynchronousQueue[Runnable]])
- }
-
- scenario("get a reactor-based-single-thread-event-driven-dispatcher with synchronous-queue from context") {
- val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml")
- val dispatcher = context.getBean("reactor-based-single-thread-event-driven-dispatcher").asInstanceOf[ReactorBasedSingleThreadEventDrivenDispatcher]
- assert(dispatcher != null)
- }
-
scenario("get a executor-based-event-driven-work-stealing-dispatcher from context") {
val context = new ClassPathXmlApplicationContext("/dispatcher-config.xml")
val dispatcher = context.getBean("executor-based-event-driven-work-stealing-dispatcher").asInstanceOf[ExecutorBasedEventDrivenWorkStealingDispatcher]
diff --git a/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala b/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala
index 27a42f3d6c..bd0b018e75 100644
--- a/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala
+++ b/akka-spring/src/test/scala/TypedActorBeanDefinitionParserTest.scala
@@ -19,7 +19,7 @@ import org.w3c.dom.Element
class TypedActorBeanDefinitionParserTest extends Spec with ShouldMatchers {
private class Parser extends ActorParser
- describe("An TypedActorParser") {
+ describe("A TypedActorParser") {
val parser = new Parser()
it("should parse the typed actor configuration") {
val xml =
+
+
+ val props = parser.parseActor(dom(xml).getDocumentElement);
+ assert(props != null)
+ assert(props.host === "com.some.host")
+ assert(props.port === 9999)
+ assert(props.serviceName === "my-service")
+ assert(props.serverManaged)
}
}
}
diff --git a/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala b/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala
index 8767b2e75a..3cdcd17cb0 100644
--- a/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala
+++ b/akka-spring/src/test/scala/TypedActorSpringFeatureTest.scala
@@ -4,10 +4,8 @@
package se.scalablesolutions.akka.spring
-import foo.{IMyPojo, MyPojo}
+import foo.{PingActor, IMyPojo, MyPojo}
import se.scalablesolutions.akka.dispatch.FutureTimeoutException
-import se.scalablesolutions.akka.remote.RemoteNode
-import org.scalatest.FeatureSpec
import org.scalatest.matchers.ShouldMatchers
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
@@ -16,13 +14,52 @@ import org.springframework.beans.factory.xml.XmlBeanDefinitionReader
import org.springframework.context.ApplicationContext
import org.springframework.context.support.ClassPathXmlApplicationContext
import org.springframework.core.io.{ClassPathResource, Resource}
+import org.scalatest.{BeforeAndAfterAll, FeatureSpec}
+import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer, RemoteNode}
+import java.util.concurrent.CountDownLatch
+import se.scalablesolutions.akka.actor.{TypedActor, RemoteTypedActorOne, Actor}
+import se.scalablesolutions.akka.actor.remote.RemoteTypedActorOneImpl
/**
* Tests for spring configuration of typed actors.
* @author michaelkober
*/
@RunWith(classOf[JUnitRunner])
-class TypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers {
+class TypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers with BeforeAndAfterAll {
+
+ var server1: RemoteServer = null
+ var server2: RemoteServer = null
+
+ override def beforeAll = {
+ val actor = Actor.actorOf[PingActor] // FIXME: remove this line when ticket 425 is fixed
+ server1 = new RemoteServer()
+ server1.start("localhost", 9990)
+ server2 = new RemoteServer()
+ server2.start("localhost", 9992)
+
+ val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
+ server1.registerTypedActor("typed-actor-service", typedActor)
+ }
+
+ // make sure the servers shutdown cleanly after the test has finished
+ override def afterAll = {
+ try {
+ server1.shutdown
+ server2.shutdown
+ RemoteClient.shutdownAll
+ Thread.sleep(1000)
+ } catch {
+ case e => ()
+ }
+ }
+
+ def getTypedActorFromContext(config: String, id: String) : IMyPojo = {
+ MyPojo.latch = new CountDownLatch(1)
+ val context = new ClassPathXmlApplicationContext(config)
+ val myPojo: IMyPojo = context.getBean(id).asInstanceOf[IMyPojo]
+ myPojo
+ }
+
feature("parse Spring application context") {
scenario("akka:typed-actor and akka:supervision and akka:dispatcher can be used as top level elements") {
@@ -37,41 +74,79 @@ class TypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers {
}
scenario("get a typed actor") {
- val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml")
- val myPojo = context.getBean("simple-typed-actor").asInstanceOf[IMyPojo]
- var msg = myPojo.getFoo()
- msg += myPojo.getBar()
- assert(msg === "foobar")
+ val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "simple-typed-actor")
+ assert(myPojo.getFoo() === "foo")
+ myPojo.oneWay("hello 1")
+ MyPojo.latch.await
+ assert(MyPojo.lastOneWayMessage === "hello 1")
}
scenario("FutureTimeoutException when timed out") {
- val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml")
- val myPojo = context.getBean("simple-typed-actor").asInstanceOf[IMyPojo]
+ val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "simple-typed-actor")
evaluating {myPojo.longRunning()} should produce[FutureTimeoutException]
-
}
scenario("typed-actor with timeout") {
- val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml")
- val myPojo = context.getBean("simple-typed-actor-long-timeout").asInstanceOf[IMyPojo]
+ val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "simple-typed-actor-long-timeout")
assert(myPojo.longRunning() === "this took long");
}
scenario("transactional typed-actor") {
- val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml")
- val myPojo = context.getBean("transactional-typed-actor").asInstanceOf[IMyPojo]
- var msg = myPojo.getFoo()
- msg += myPojo.getBar()
- assert(msg === "foobar")
+ val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "transactional-typed-actor")
+ assert(myPojo.getFoo() === "foo")
+ myPojo.oneWay("hello 2")
+ MyPojo.latch.await
+ assert(MyPojo.lastOneWayMessage === "hello 2")
}
scenario("get a remote typed-actor") {
- RemoteNode.start
- Thread.sleep(1000)
- val context = new ClassPathXmlApplicationContext("/typed-actor-config.xml")
- val myPojo = context.getBean("remote-typed-actor").asInstanceOf[IMyPojo]
- assert(myPojo.getFoo === "foo")
+ val myPojo = getTypedActorFromContext("/typed-actor-config.xml", "remote-typed-actor")
+ assert(myPojo.getFoo() === "foo")
+ myPojo.oneWay("hello 3")
+ MyPojo.latch.await
+ assert(MyPojo.lastOneWayMessage === "hello 3")
}
+
+ scenario("get a client-managed-remote-typed-actor") {
+ val myPojo = getTypedActorFromContext("/server-managed-config.xml", "client-managed-remote-typed-actor")
+ assert(myPojo.getFoo() === "foo")
+ myPojo.oneWay("hello client-managed-remote-typed-actor")
+ MyPojo.latch.await
+ assert(MyPojo.lastOneWayMessage === "hello client-managed-remote-typed-actor")
+ }
+
+ scenario("get a server-managed-remote-typed-actor") {
+ val serverPojo = getTypedActorFromContext("/server-managed-config.xml", "server-managed-remote-typed-actor")
+ //
+ val myPojoProxy = RemoteClient.typedActorFor(classOf[IMyPojo], classOf[IMyPojo].getName, 5000L, "localhost", 9990)
+ assert(myPojoProxy.getFoo() === "foo")
+ myPojoProxy.oneWay("hello server-managed-remote-typed-actor")
+ MyPojo.latch.await
+ assert(MyPojo.lastOneWayMessage === "hello server-managed-remote-typed-actor")
+ }
+
+ scenario("get a server-managed-remote-typed-actor-custom-id") {
+ val serverPojo = getTypedActorFromContext("/server-managed-config.xml", "server-managed-remote-typed-actor-custom-id")
+ //
+ val myPojoProxy = RemoteClient.typedActorFor(classOf[IMyPojo], "mypojo-service", 5000L, "localhost", 9990)
+ assert(myPojoProxy.getFoo() === "foo")
+ myPojoProxy.oneWay("hello server-managed-remote-typed-actor 2")
+ MyPojo.latch.await
+ assert(MyPojo.lastOneWayMessage === "hello server-managed-remote-typed-actor 2")
+ }
+
+ scenario("get a client proxy for server-managed-remote-typed-actor") {
+ MyPojo.latch = new CountDownLatch(1)
+ val context = new ClassPathXmlApplicationContext("/server-managed-config.xml")
+ val myPojo: IMyPojo = context.getBean("server-managed-remote-typed-actor-custom-id").asInstanceOf[IMyPojo]
+ // get client proxy from spring context
+ val myPojoProxy = context.getBean("typed-client-1").asInstanceOf[IMyPojo]
+ assert(myPojoProxy.getFoo() === "foo")
+ myPojoProxy.oneWay("hello")
+ MyPojo.latch.await
+ }
+
+
}
}
diff --git a/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala b/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala
index cf7d8d9805..0397d30bf0 100644
--- a/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala
+++ b/akka-spring/src/test/scala/UntypedActorSpringFeatureTest.scala
@@ -6,74 +6,146 @@ package se.scalablesolutions.akka.spring
import foo.PingActor
import se.scalablesolutions.akka.dispatch.ExecutorBasedEventDrivenWorkStealingDispatcher
-import se.scalablesolutions.akka.remote.RemoteNode
-import se.scalablesolutions.akka.actor.ActorRef
-import org.scalatest.FeatureSpec
import org.scalatest.matchers.ShouldMatchers
import org.scalatest.junit.JUnitRunner
import org.junit.runner.RunWith
-import org.springframework.context.ApplicationContext
import org.springframework.context.support.ClassPathXmlApplicationContext
+import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer}
+import org.scalatest.{BeforeAndAfterAll, FeatureSpec}
+import java.util.concurrent.CountDownLatch
+import se.scalablesolutions.akka.actor.{RemoteActorRef, ActorRegistry, Actor, ActorRef}
/**
* Tests for spring configuration of typed actors.
* @author michaelkober
*/
@RunWith(classOf[JUnitRunner])
-class UntypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers {
+class UntypedActorSpringFeatureTest extends FeatureSpec with ShouldMatchers with BeforeAndAfterAll {
+
+ var server1: RemoteServer = null
+ var server2: RemoteServer = null
+
+
+ override def beforeAll = {
+ val actor = Actor.actorOf[PingActor] // FIXME: remove this line when ticket 425 is fixed
+ server1 = new RemoteServer()
+ server1.start("localhost", 9990)
+ server2 = new RemoteServer()
+ server2.start("localhost", 9992)
+ }
+
+ // make sure the servers shutdown cleanly after the test has finished
+ override def afterAll = {
+ try {
+ server1.shutdown
+ server2.shutdown
+ RemoteClient.shutdownAll
+ Thread.sleep(1000)
+ } catch {
+ case e => ()
+ }
+ }
+
+
+ def getPingActorFromContext(config: String, id: String) : ActorRef = {
+ PingActor.latch = new CountDownLatch(1)
+ val context = new ClassPathXmlApplicationContext(config)
+ val pingActor = context.getBean(id).asInstanceOf[ActorRef]
+ assert(pingActor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor")
+ pingActor.start()
+ }
+
+
feature("parse Spring application context") {
scenario("get a untyped actor") {
- val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml")
- val myactor = context.getBean("simple-untyped-actor").asInstanceOf[ActorRef]
- assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor")
- myactor.start()
+ val myactor = getPingActorFromContext("/untyped-actor-config.xml", "simple-untyped-actor")
myactor.sendOneWay("Hello")
+ PingActor.latch.await
+ assert(PingActor.lastMessage === "Hello")
assert(myactor.isDefinedAt("some string message"))
}
scenario("untyped-actor with timeout") {
- val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml")
- val myactor = context.getBean("simple-untyped-actor-long-timeout").asInstanceOf[ActorRef]
- assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor")
- myactor.start()
- myactor.sendOneWay("Hello")
+ val myactor = getPingActorFromContext("/untyped-actor-config.xml", "simple-untyped-actor-long-timeout")
assert(myactor.getTimeout() === 10000)
+ myactor.sendOneWay("Hello 2")
+ PingActor.latch.await
+ assert(PingActor.lastMessage === "Hello 2")
}
scenario("transactional untyped-actor") {
- val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml")
- val myactor = context.getBean("transactional-untyped-actor").asInstanceOf[ActorRef]
- assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor")
- myactor.start()
- myactor.sendOneWay("Hello")
- assert(myactor.isDefinedAt("some string message"))
+ val myactor = getPingActorFromContext("/untyped-actor-config.xml", "transactional-untyped-actor")
+ myactor.sendOneWay("Hello 3")
+ PingActor.latch.await
+ assert(PingActor.lastMessage === "Hello 3")
}
scenario("get a remote typed-actor") {
- RemoteNode.start
- Thread.sleep(1000)
- val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml")
- val myactor = context.getBean("remote-untyped-actor").asInstanceOf[ActorRef]
- assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor")
- myactor.start()
- myactor.sendOneWay("Hello")
- assert(myactor.isDefinedAt("some string message"))
+ val myactor = getPingActorFromContext("/untyped-actor-config.xml", "remote-untyped-actor")
+ myactor.sendOneWay("Hello 4")
assert(myactor.getRemoteAddress().isDefined)
assert(myactor.getRemoteAddress().get.getHostName() === "localhost")
- assert(myactor.getRemoteAddress().get.getPort() === 9999)
+ assert(myactor.getRemoteAddress().get.getPort() === 9992)
+ PingActor.latch.await
+ assert(PingActor.lastMessage === "Hello 4")
}
scenario("untyped-actor with custom dispatcher") {
- val context = new ClassPathXmlApplicationContext("/untyped-actor-config.xml")
- val myactor = context.getBean("untyped-actor-with-dispatcher").asInstanceOf[ActorRef]
- assert(myactor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor")
- myactor.start()
- myactor.sendOneWay("Hello")
+ val myactor = getPingActorFromContext("/untyped-actor-config.xml", "untyped-actor-with-dispatcher")
assert(myactor.getTimeout() === 1000)
assert(myactor.getDispatcher.isInstanceOf[ExecutorBasedEventDrivenWorkStealingDispatcher])
+ myactor.sendOneWay("Hello 5")
+ PingActor.latch.await
+ assert(PingActor.lastMessage === "Hello 5")
}
+
+ scenario("create client managed remote untyped-actor") {
+ val myactor = getPingActorFromContext("/server-managed-config.xml", "client-managed-remote-untyped-actor")
+ myactor.sendOneWay("Hello client managed remote untyped-actor")
+ PingActor.latch.await
+ assert(PingActor.lastMessage === "Hello client managed remote untyped-actor")
+ assert(myactor.getRemoteAddress().isDefined)
+ assert(myactor.getRemoteAddress().get.getHostName() === "localhost")
+ assert(myactor.getRemoteAddress().get.getPort() === 9990)
+ }
+
+ scenario("create server managed remote untyped-actor") {
+ val myactor = getPingActorFromContext("/server-managed-config.xml", "server-managed-remote-untyped-actor")
+ val nrOfActors = ActorRegistry.actors.length
+ val actorRef = RemoteClient.actorFor("se.scalablesolutions.akka.spring.foo.PingActor", "localhost", 9990)
+ actorRef.sendOneWay("Hello server managed remote untyped-actor")
+ PingActor.latch.await
+ assert(PingActor.lastMessage === "Hello server managed remote untyped-actor")
+ assert(ActorRegistry.actors.length === nrOfActors)
+ }
+
+ scenario("create server managed remote untyped-actor with custom service id") {
+ val myactor = getPingActorFromContext("/server-managed-config.xml", "server-managed-remote-untyped-actor-custom-id")
+ val nrOfActors = ActorRegistry.actors.length
+ val actorRef = RemoteClient.actorFor("ping-service", "localhost", 9990)
+ actorRef.sendOneWay("Hello server managed remote untyped-actor")
+ PingActor.latch.await
+ assert(PingActor.lastMessage === "Hello server managed remote untyped-actor")
+ assert(ActorRegistry.actors.length === nrOfActors)
+ }
+
+ scenario("get client actor for server managed remote untyped-actor") {
+ PingActor.latch = new CountDownLatch(1)
+ val context = new ClassPathXmlApplicationContext("/server-managed-config.xml")
+ val pingActor = context.getBean("server-managed-remote-untyped-actor-custom-id").asInstanceOf[ActorRef]
+ assert(pingActor.getActorClassName() === "se.scalablesolutions.akka.spring.foo.PingActor")
+ pingActor.start()
+ val nrOfActors = ActorRegistry.actors.length
+ // get client actor ref from spring context
+ val actorRef = context.getBean("client-1").asInstanceOf[ActorRef]
+ assert(actorRef.isInstanceOf[RemoteActorRef])
+ actorRef.sendOneWay("Hello")
+ PingActor.latch.await
+ assert(ActorRegistry.actors.length === nrOfActors)
+ }
+
}
}
diff --git a/akka-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
index b27f5b4b4d..c3457cb43b 100644
--- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala
+++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
@@ -16,9 +16,8 @@ import org.codehaus.aspectwerkz.proxy.Proxy
import org.codehaus.aspectwerkz.annotation.{Aspect, Around}
import java.net.InetSocketAddress
-import java.lang.reflect.{InvocationTargetException, Method, Field}
-
import scala.reflect.BeanProperty
+import java.lang.reflect.{Method, Field, InvocationHandler, Proxy => JProxy}
/**
* TypedActor is a type-safe actor made out of a POJO with interface.
@@ -41,12 +40,12 @@ import scala.reflect.BeanProperty
* }
*
* @Override
- * public void init() {
+ * public void preStart() {
* ... // optional initialization on start
* }
*
* @Override
- * public void shutdown() {
+ * public void postStop() {
* ... // optional cleanup on stop
* }
*
@@ -79,11 +78,11 @@ import scala.reflect.BeanProperty
*
* def square(x: Int): Future[Integer] = future(x * x)
*
- * override def init = {
+ * override def preStart = {
* ... // optional initialization on start
* }
*
- * override def shutdown = {
+ * override def postStop = {
* ... // optional cleanup on stop
* }
*
@@ -390,7 +389,8 @@ object TypedActor extends Logging {
typedActor.initialize(proxy)
if (config._messageDispatcher.isDefined) actorRef.dispatcher = config._messageDispatcher.get
if (config._threadBasedDispatcher.isDefined) actorRef.dispatcher = Dispatchers.newThreadBasedDispatcher(actorRef)
- AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, None, config.timeout))
+ if (config._host.isDefined) actorRef.makeRemote(config._host.get)
+ AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, config._host, config.timeout))
actorRef.start
proxy.asInstanceOf[T]
}
@@ -408,24 +408,47 @@ object TypedActor extends Logging {
proxy.asInstanceOf[T]
}
-/*
- // NOTE: currently not used - but keep it around
- private[akka] def newInstance[T <: TypedActor](targetClass: Class[T],
- remoteAddress: Option[InetSocketAddress], timeout: Long): T = {
- val proxy = {
- val instance = Proxy.newInstance(targetClass, true, false)
- if (instance.isInstanceOf[TypedActor]) instance.asInstanceOf[TypedActor]
- else throw new IllegalActorStateException("Actor [" + targetClass.getName + "] is not a sub class of 'TypedActor'")
+ /**
+ * Create a proxy for a RemoteActorRef representing a server managed remote typed actor.
+ *
+ */
+ private[akka] def createProxyForRemoteActorRef[T](intfClass: Class[T], actorRef: ActorRef): T = {
+
+ class MyInvocationHandler extends InvocationHandler {
+ def invoke(proxy: AnyRef, method: Method, args: Array[AnyRef]): AnyRef = {
+ // do nothing, this is just a dummy
+ null
+ }
}
- val context = injectTypedActorContext(proxy)
- actorRef.actor.asInstanceOf[Dispatcher].initialize(targetClass, proxy, proxy, context)
- actorRef.timeout = timeout
- if (remoteAddress.isDefined) actorRef.makeRemote(remoteAddress.get)
- AspectInitRegistry.register(proxy, AspectInit(targetClass, proxy, actorRef, remoteAddress, timeout))
- actorRef.start
- proxy.asInstanceOf[T]
+ val handler = new MyInvocationHandler()
+
+ val interfaces = Array(intfClass, classOf[ServerManagedTypedActor]).asInstanceOf[Array[java.lang.Class[_]]]
+ val jProxy = JProxy.newProxyInstance(intfClass.getClassLoader(), interfaces, handler)
+ val awProxy = Proxy.newInstance(interfaces, Array(jProxy, jProxy), true, false)
+
+ AspectInitRegistry.register(awProxy, AspectInit(intfClass, null, actorRef, None, 5000L))
+ awProxy.asInstanceOf[T]
}
-*/
+
+
+ /*
+ // NOTE: currently not used - but keep it around
+ private[akka] def newInstance[T <: TypedActor](targetClass: Class[T],
+ remoteAddress: Option[InetSocketAddress], timeout: Long): T = {
+ val proxy = {
+ val instance = Proxy.newInstance(targetClass, true, false)
+ if (instance.isInstanceOf[TypedActor]) instance.asInstanceOf[TypedActor]
+ else throw new IllegalActorStateException("Actor [" + targetClass.getName + "] is not a sub class of 'TypedActor'")
+ }
+ val context = injectTypedActorContext(proxy)
+ actorRef.actor.asInstanceOf[Dispatcher].initialize(targetClass, proxy, proxy, context)
+ actorRef.timeout = timeout
+ if (remoteAddress.isDefined) actorRef.makeRemote(remoteAddress.get)
+ AspectInitRegistry.register(proxy, AspectInit(targetClass, proxy, actorRef, remoteAddress, timeout))
+ actorRef.start
+ proxy.asInstanceOf[T]
+ }
+ */
/**
* Stops the current Typed Actor.
@@ -519,11 +542,7 @@ object TypedActor extends Logging {
val typedActor =
if (instance.isInstanceOf[TypedActor]) instance.asInstanceOf[TypedActor]
else throw new IllegalArgumentException("Actor [" + targetClass.getName + "] is not a sub class of 'TypedActor'")
- typedActor.init
- import se.scalablesolutions.akka.stm.local.atomic
- atomic {
- typedActor.initTransactionalState
- }
+ typedActor.preStart
typedActor
}
@@ -546,6 +565,30 @@ object TypedActor extends Logging {
private[akka] def isJoinPoint(message: Any): Boolean = message.isInstanceOf[JoinPoint]
}
+
+/**
+ * AspectWerkz Aspect that is turning POJO into proxy to a server managed remote TypedActor.
+ *
+ * Is deployed on a 'perInstance' basis with the pointcut 'execution(* *.*(..))',
+ * e.g. all methods on the instance.
+ *
+ * @author Jonas Bonér
+ */
+@Aspect("perInstance")
+private[akka] sealed class ServerManagedTypedActorAspect extends ActorAspect {
+
+ @Around("execution(* *.*(..)) && this(se.scalablesolutions.akka.actor.ServerManagedTypedActor)")
+ def invoke(joinPoint: JoinPoint): AnyRef = {
+ if (!isInitialized) initialize(joinPoint)
+ remoteDispatch(joinPoint)
+ }
+
+ override def initialize(joinPoint: JoinPoint): Unit = {
+ super.initialize(joinPoint)
+ remoteAddress = actorRef.remoteAddress
+ }
+}
+
/**
* AspectWerkz Aspect that is turning POJO into TypedActor.
*
@@ -555,18 +598,9 @@ object TypedActor extends Logging {
* @author Jonas Bonér
*/
@Aspect("perInstance")
-private[akka] sealed class TypedActorAspect {
- @volatile private var isInitialized = false
- @volatile private var isStopped = false
- private var interfaceClass: Class[_] = _
- private var typedActor: TypedActor = _
- private var actorRef: ActorRef = _
- private var remoteAddress: Option[InetSocketAddress] = _
- private var timeout: Long = _
- private var uuid: String = _
- @volatile private var instance: TypedActor = _
+private[akka] sealed class TypedActorAspect extends ActorAspect {
- @Around("execution(* *.*(..))")
+ @Around("execution(* *.*(..)) && !this(se.scalablesolutions.akka.actor.ServerManagedTypedActor)")
def invoke(joinPoint: JoinPoint): AnyRef = {
if (!isInitialized) initialize(joinPoint)
dispatch(joinPoint)
@@ -576,12 +610,26 @@ private[akka] sealed class TypedActorAspect {
if (remoteAddress.isDefined) remoteDispatch(joinPoint)
else localDispatch(joinPoint)
}
+}
- private def localDispatch(joinPoint: JoinPoint): AnyRef = {
- val methodRtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
- val isOneWay = TypedActor.isOneWay(methodRtti)
+/**
+ * Base class for TypedActorAspect and ServerManagedTypedActorAspect to reduce code duplication.
+ */
+private[akka] abstract class ActorAspect {
+ @volatile protected var isInitialized = false
+ @volatile protected var isStopped = false
+ protected var interfaceClass: Class[_] = _
+ protected var typedActor: TypedActor = _
+ protected var actorRef: ActorRef = _
+ protected var timeout: Long = _
+ protected var uuid: String = _
+ protected var remoteAddress: Option[InetSocketAddress] = _
+
+ protected def localDispatch(joinPoint: JoinPoint): AnyRef = {
+ val methodRtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
+ val isOneWay = TypedActor.isOneWay(methodRtti)
val senderActorRef = Some(SenderContextInfo.senderActorRef.value)
- val senderProxy = Some(SenderContextInfo.senderProxy.value)
+ val senderProxy = Some(SenderContextInfo.senderProxy.value)
typedActor.context._sender = senderProxy
if (!actorRef.isRunning && !isStopped) {
@@ -602,7 +650,7 @@ private[akka] sealed class TypedActorAspect {
}
}
- private def remoteDispatch(joinPoint: JoinPoint): AnyRef = {
+ protected def remoteDispatch(joinPoint: JoinPoint): AnyRef = {
val methodRtti = joinPoint.getRtti.asInstanceOf[MethodRtti]
val isOneWay = TypedActor.isOneWay(methodRtti)
@@ -641,7 +689,7 @@ private[akka] sealed class TypedActorAspect {
(escapedArgs, isEscaped)
}
- private def initialize(joinPoint: JoinPoint): Unit = {
+ protected def initialize(joinPoint: JoinPoint): Unit = {
val init = AspectInitRegistry.initFor(joinPoint.getThis)
interfaceClass = init.interfaceClass
typedActor = init.targetInstance
@@ -653,6 +701,7 @@ private[akka] sealed class TypedActorAspect {
}
}
+
/**
* Internal helper class to help pass the contextual information between threads.
*
@@ -674,7 +723,7 @@ private[akka] object AspectInitRegistry extends ListenerManagement {
def register(proxy: AnyRef, init: AspectInit) = {
val res = initializations.put(proxy, init)
- foreachListener(_ ! AspectInitRegistered(proxy, init))
+ notifyListeners(AspectInitRegistered(proxy, init))
res
}
@@ -683,7 +732,7 @@ private[akka] object AspectInitRegistry extends ListenerManagement {
*/
def unregister(proxy: AnyRef): AspectInit = {
val init = initializations.remove(proxy)
- foreachListener(_ ! AspectInitUnregistered(proxy, init))
+ notifyListeners(AspectInitUnregistered(proxy, init))
init.actorRef.stop
init
}
@@ -704,5 +753,11 @@ private[akka] sealed case class AspectInit(
val timeout: Long) {
def this(interfaceClass: Class[_], targetInstance: TypedActor, actorRef: ActorRef, timeout: Long) =
this(interfaceClass, targetInstance, actorRef, None, timeout)
+
}
+
+/**
+ * Marker interface for server manager typed actors.
+ */
+private[akka] sealed trait ServerManagedTypedActor extends TypedActor
diff --git a/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala
index 339c4d297d..5ca249a3ec 100644
--- a/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala
+++ b/akka-typed-actor/src/main/scala/config/TypedActorGuiceConfigurator.scala
@@ -122,7 +122,6 @@ private[akka] class TypedActorGuiceConfigurator extends TypedActorConfiguratorBa
remoteAddress.foreach { address =>
actorRef.makeRemote(remoteAddress.get)
- RemoteServerModule.registerTypedActor(address, implementationClass.getName, proxy)
}
AspectInitRegistry.register(
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java
index 1b95517c22..cb002b0a9e 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/NestedTransactionalTypedActorImpl.java
@@ -10,7 +10,7 @@ public class NestedTransactionalTypedActorImpl extends TypedTransactor implement
private boolean isInitialized = false;
@Override
- public void init() {
+ public void preStart() {
if (!isInitialized) {
mapState = new TransactionalMap();
vectorState = new TransactionalVector();
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java
index 12985c72ce..1e567014d9 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/SamplePojoImpl.java
@@ -38,7 +38,7 @@ public class SamplePojoImpl extends TypedActor implements SamplePojo {
}
@Override
- public void shutdown() {
+ public void postStop() {
_down = true;
latch.countDown();
}
diff --git a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java
index 9b32f5d329..45bda4a675 100644
--- a/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java
+++ b/akka-typed-actor/src/test/java/se/scalablesolutions/akka/actor/TransactionalTypedActorImpl.java
@@ -2,6 +2,8 @@ package se.scalablesolutions.akka.actor;
import se.scalablesolutions.akka.actor.*;
import se.scalablesolutions.akka.stm.*;
+import se.scalablesolutions.akka.stm.local.*;
+import se.scalablesolutions.akka.stm.local.Atomic;
public class TransactionalTypedActorImpl extends TypedTransactor implements TransactionalTypedActor {
private TransactionalMap mapState;
@@ -10,12 +12,16 @@ public class TransactionalTypedActorImpl extends TypedTransactor implements Tran
private boolean isInitialized = false;
@Override
- public void initTransactionalState() {
+ public void preStart() {
if (!isInitialized) {
- mapState = new TransactionalMap();
- vectorState = new TransactionalVector();
- refState = new Ref();
- isInitialized = true;
+ isInitialized = new Atomic() {
+ public Boolean atomically() {
+ mapState = new TransactionalMap();
+ vectorState = new TransactionalVector();
+ refState = new Ref();
+ return true;
+ }
+ }.execute();
}
}
diff --git a/akka-typed-actor/src/test/resources/META-INF/aop.xml b/akka-typed-actor/src/test/resources/META-INF/aop.xml
index bdc167ca54..be133a51b8 100644
--- a/akka-typed-actor/src/test/resources/META-INF/aop.xml
+++ b/akka-typed-actor/src/test/resources/META-INF/aop.xml
@@ -2,6 +2,7 @@
+
diff --git a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
index 10fc40493b..052f4cc7de 100644
--- a/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
+++ b/akka-typed-actor/src/test/scala/actor/typed-actor/TypedActorLifecycleSpec.scala
@@ -87,7 +87,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft
SamplePojoImpl.reset
val pojo = TypedActor.newInstance(classOf[SimpleJavaPojo], classOf[SimpleJavaPojoImpl])
val supervisor = TypedActor.newInstance(classOf[SimpleJavaPojo], classOf[SimpleJavaPojoImpl])
- link(supervisor, pojo, new OneForOneStrategy(3, 2000), Array(classOf[Throwable]))
+ link(supervisor, pojo, OneForOneStrategy(3, 2000), Array(classOf[Throwable]))
pojo.throwException
Thread.sleep(500)
SimpleJavaPojoImpl._pre should be(true)
@@ -95,7 +95,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft
}
/*
- it("should shutdown non-supervised, annotated typed actor on TypedActor.stop") {
+ it("should postStop non-supervised, annotated typed actor on TypedActor.stop") {
val obj = TypedActor.newInstance(classOf[SamplePojoAnnotated])
assert(AspectInitRegistry.initFor(obj) ne null)
assert("hello akka" === obj.greet("akka"))
@@ -112,7 +112,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft
}
}
- it("should shutdown non-supervised, annotated typed actor on ActorRegistry.shutdownAll") {
+ it("should postStop non-supervised, annotated typed actor on ActorRegistry.shutdownAll") {
val obj = TypedActor.newInstance(classOf[SamplePojoAnnotated])
assert(AspectInitRegistry.initFor(obj) ne null)
assert("hello akka" === obj.greet("akka"))
@@ -147,7 +147,7 @@ class TypedActorLifecycleSpec extends Spec with ShouldMatchers with BeforeAndAft
}
}
- it("should shutdown supervised, annotated typed actor on failure") {
+ it("should postStop supervised, annotated typed actor on failure") {
val obj = conf2.getInstance[SamplePojoAnnotated](classOf[SamplePojoAnnotated])
val cdl = obj.newCountdownLatch(1)
assert(AspectInitRegistry.initFor(obj) ne null)
diff --git a/config/akka-reference.conf b/config/akka-reference.conf
index 9f8ec799f0..10a9c84118 100644
--- a/config/akka-reference.conf
+++ b/config/akka-reference.conf
@@ -25,6 +25,7 @@ akka {
# - TypedActor: methods with non-void return type
serialize-messages = off # Does a deep clone of (non-primitive) messages to ensure immutability
throughput = 5 # Default throughput for all ExecutorBasedEventDrivenDispatcher, set to 1 for complete fairness
+ throughput-deadline-ms = -1 # Default throughput deadline for all ExecutorBasedEventDrivenDispatcher, set to 0 or negative for no deadline
default-dispatcher {
type = "GlobalExecutorBasedEventDriven" # Must be one of the following, all "Global*" are non-configurable
@@ -44,20 +45,34 @@ akka {
allow-core-timeout = on # Allow core threads to time out
rejection-policy = "caller-runs" # abort, caller-runs, discard-oldest, discard
throughput = 5 # Throughput for ExecutorBasedEventDrivenDispatcher, set to 1 for complete fairness
+ throughput-deadline-ms = -1 # Throughput deadline for ExecutorBasedEventDrivenDispatcher, set to 0 or negative for no deadline
aggregate = off # Aggregate on/off for HawtDispatchers
mailbox-capacity = -1 # If negative (or zero) then an unbounded mailbox is used (default)
# If positive then a bounded mailbox is used and the capacity is set using the property
# NOTE: setting a mailbox to 'blocking' can be a bit dangerous,
# could lead to deadlock, use with care
+ #
+ # The following are only used for ExecutorBasedEventDriven
+ # and only if mailbox-capacity > 0
+ mailbox-push-timeout-ms = 10000 # Specifies the timeout (in milliseconds) to add a new message to a mailbox that is full
}
}
stm {
- fair = on # Should global transactions be fair or non-fair (non fair yield better performance)
- jta-aware = off # Option 'on' means that if there JTA Transaction Manager available then the STM will
- # begin (or join), commit or rollback the JTA transaction. Default is 'off'.
- timeout = 5 # Default timeout for blocking transactions and transaction set (in unit defined by
- # the time-unit property)
+ fair = on # Should global transactions be fair or non-fair (non fair yield better performance)
+ max-retries = 1000
+ timeout = 5 # Default timeout for blocking transactions and transaction set (in unit defined by
+ # the time-unit property)
+ write-skew = true
+ blocking-allowed = false
+ interruptible = false
+ speculative = true
+ quick-release = true
+ propagation = "requires"
+ trace-level = "none"
+ hooks = true
+ jta-aware = off # Option 'on' means that if there JTA Transaction Manager available then the STM will
+ # begin (or join), commit or rollback the JTA transaction. Default is 'off'.
}
jta {
diff --git a/config/microkernel-server.xml b/config/microkernel-server.xml
index ecb4bee120..d7b8087428 100644
--- a/config/microkernel-server.xml
+++ b/config/microkernel-server.xml
@@ -21,10 +21,7 @@
-
-
- 10
- 200
+
diff --git a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar b/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar
deleted file mode 100644
index 5d2a6a3632..0000000000
Binary files a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.jar and /dev/null differ
diff --git a/embedded-repo/com/redis/redisclient/1.2/redisclient-1.2.jar b/embedded-repo/com/redis/redisclient/1.2/redisclient-1.2.jar
deleted file mode 100644
index 91ff84b97c..0000000000
Binary files a/embedded-repo/com/redis/redisclient/1.2/redisclient-1.2.jar and /dev/null differ
diff --git a/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.jar b/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.jar
new file mode 100644
index 0000000000..66c18b6fbf
Binary files /dev/null and b/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.jar differ
diff --git a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom b/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.pom
old mode 100755
new mode 100644
similarity index 90%
rename from embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom
rename to embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.pom
index 16dd81402a..12558da1c4
--- a/embedded-repo/com/redis/redisclient/1.1/redisclient-1.1.pom
+++ b/embedded-repo/com/redis/redisclient/2.8.0-2.0/redisclient-2.8.0-2.0.pom
@@ -3,6 +3,6 @@
4.0.0
com.redis
redisclient
- 1.1
+ 2.8.0-2.0
jar
-
+
\ No newline at end of file
diff --git a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.jar b/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.jar
deleted file mode 100644
index 3f1593380b..0000000000
Binary files a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.jar and /dev/null differ
diff --git a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.pom b/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.pom
deleted file mode 100755
index 68f3763187..0000000000
--- a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.2/redisclient-2.8.0.Beta1-1.2.pom
+++ /dev/null
@@ -1,8 +0,0 @@
-
-
- 4.0.0
- com.redis
- redisclient
- 2.8.0.Beta1-1.2
- jar
-
diff --git a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.3-SNAPSHOT/redisclient-2.8.0.Beta1-1.3-SNAPSHOT.jar b/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.3-SNAPSHOT/redisclient-2.8.0.Beta1-1.3-SNAPSHOT.jar
deleted file mode 100644
index 0daede37f0..0000000000
Binary files a/embedded-repo/com/redis/redisclient/2.8.0.Beta1-1.3-SNAPSHOT/redisclient-2.8.0.Beta1-1.3-SNAPSHOT.jar and /dev/null differ
diff --git a/embedded-repo/com/redis/redisclient/2.8.0.RC2-1.4-SNAPSHOT/redisclient-2.8.0.RC2-1.4-SNAPSHOT.jar b/embedded-repo/com/redis/redisclient/2.8.0.RC2-1.4-SNAPSHOT/redisclient-2.8.0.RC2-1.4-SNAPSHOT.jar
deleted file mode 100644
index 261b5cc1be..0000000000
Binary files a/embedded-repo/com/redis/redisclient/2.8.0.RC2-1.4-SNAPSHOT/redisclient-2.8.0.RC2-1.4-SNAPSHOT.jar and /dev/null differ
diff --git a/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4-SNAPSHOT/redisclient-2.8.0.RC3-1.4-SNAPSHOT.jar b/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4-SNAPSHOT/redisclient-2.8.0.RC3-1.4-SNAPSHOT.jar
deleted file mode 100644
index d939a49d7c..0000000000
Binary files a/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4-SNAPSHOT/redisclient-2.8.0.RC3-1.4-SNAPSHOT.jar and /dev/null differ
diff --git a/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4/redisclient-2.8.0.RC3-1.4.jar b/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4/redisclient-2.8.0.RC3-1.4.jar
deleted file mode 100644
index 351ff49c9d..0000000000
Binary files a/embedded-repo/com/redis/redisclient/2.8.0.RC3-1.4/redisclient-2.8.0.RC3-1.4.jar and /dev/null differ
diff --git a/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.jar b/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.jar
deleted file mode 100644
index dcab322c81..0000000000
Binary files a/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.jar and /dev/null differ
diff --git a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.jar b/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.jar
deleted file mode 100644
index 4543815742..0000000000
Binary files a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.jar and /dev/null differ
diff --git a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.pom b/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.pom
deleted file mode 100755
index 9c6db30374..0000000000
--- a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.Beta1/sjson-0.5-SNAPSHOT-2.8.Beta1.pom
+++ /dev/null
@@ -1,8 +0,0 @@
-
-
- 4.0.0
- sjson.json
- sjson
- 0.5-SNAPSHOT-2.8.Beta1
- jar
-
diff --git a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.jar b/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.jar
deleted file mode 100644
index 11d149aeee..0000000000
Binary files a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.jar and /dev/null differ
diff --git a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.pom b/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.pom
deleted file mode 100644
index fa80e90e01..0000000000
--- a/embedded-repo/sjson/json/sjson/0.5-SNAPSHOT-2.8.RC2/sjson-0.5-SNAPSHOT-2.8.RC2.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-
-
- 4.0.0
- sjson.json
- sjson
- 0.5-SNAPSHOT-2.8.RC2
- POM was created from install:install-file
-
diff --git a/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.jar b/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.jar
deleted file mode 100644
index 5cf971df15..0000000000
Binary files a/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.jar and /dev/null differ
diff --git a/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.pom b/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.pom
deleted file mode 100644
index 708571fc10..0000000000
--- a/embedded-repo/sjson/json/sjson/0.6-SNAPSHOT-2.8.RC3/sjson-0.6-SNAPSHOT-2.8.RC3.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-
-
- 4.0.0
- sjson.json
- sjson
- 0.6-SNAPSHOT-2.8.RC3
- POM was created from install:install-file
-
diff --git a/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.pom b/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.pom
deleted file mode 100644
index dfc9017dcf..0000000000
--- a/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-
-
- 4.0.0
- sjson.json
- sjson
- 0.7-2.8.0
- POM was created from install:install-file
-
diff --git a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.jar b/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.jar
deleted file mode 100644
index f0d321a9e3..0000000000
Binary files a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.jar and /dev/null differ
diff --git a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.pom b/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.pom
deleted file mode 100644
index e8081164fc..0000000000
--- a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.0/sjson-0.7-SNAPSHOT-2.8.0.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-
-
- 4.0.0
- sjson.json
- sjson
- 0.7-SNAPSHOT-2.8.0
- POM was created from install:install-file
-
diff --git a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.jar b/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.jar
deleted file mode 100644
index 6c77d359b4..0000000000
Binary files a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.jar and /dev/null differ
diff --git a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.pom b/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.pom
deleted file mode 100644
index 7c02578f5f..0000000000
--- a/embedded-repo/sjson/json/sjson/0.7-SNAPSHOT-2.8.RC7/sjson-0.7-SNAPSHOT-2.8.RC7.pom
+++ /dev/null
@@ -1,9 +0,0 @@
-
-
- 4.0.0
- sjson.json
- sjson
- 0.7-SNAPSHOT-2.8.RC7
- POM was created from install:install-file
-
diff --git a/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.jar b/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.jar
new file mode 100644
index 0000000000..7dc4a9f66d
Binary files /dev/null and b/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.jar differ
diff --git a/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.pom b/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.pom
old mode 100755
new mode 100644
similarity index 92%
rename from embedded-repo/sjson/json/sjson/0.4/sjson-0.4.pom
rename to embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.pom
index 784adff55e..51167926f1
--- a/embedded-repo/sjson/json/sjson/0.4/sjson-0.4.pom
+++ b/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.pom
@@ -3,6 +3,6 @@
4.0.0
sjson.json
sjson
- 0.4
+ 0.8-2.8.0
jar
diff --git a/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.jar b/embedded-repo/sjson/json/sjson/0.8-SNAPSHOT-2.8.0/sjson-0.8-SNAPSHOT-2.8.0.jar
similarity index 67%
rename from embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.jar
rename to embedded-repo/sjson/json/sjson/0.8-SNAPSHOT-2.8.0/sjson-0.8-SNAPSHOT-2.8.0.jar
index e8700f11d3..1542632a82 100644
Binary files a/embedded-repo/sjson/json/sjson/0.7-2.8.0/sjson-0.7-2.8.0.jar and b/embedded-repo/sjson/json/sjson/0.8-SNAPSHOT-2.8.0/sjson-0.8-SNAPSHOT-2.8.0.jar differ
diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala
index cf7378b0d2..ad1339da56 100644
--- a/project/build/AkkaProject.scala
+++ b/project/build/AkkaProject.scala
@@ -41,6 +41,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
object Repositories {
lazy val AkkaRepo = MavenRepository("Akka Repository", "http://scalablesolutions.se/akka/repository")
+ lazy val CasbahRepo = MavenRepository("Casbah Repo", "http://repo.bumnetworks.com/releases")
+ lazy val CasbahSnapshotRepo = MavenRepository("Casbah Snapshots", "http://repo.bumnetworks.com/snapshots")
lazy val CodehausRepo = MavenRepository("Codehaus Repo", "http://repository.codehaus.org")
lazy val EmbeddedRepo = MavenRepository("Embedded Repo", (info.projectPath / "embedded-repo").asURL.toString)
lazy val FusesourceSnapshotRepo = MavenRepository("Fusesource Snapshots", "http://repo.fusesource.com/nexus/content/repositories/snapshots")
@@ -70,12 +72,13 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val jerseyContrModuleConfig = ModuleConfiguration("com.sun.jersey.contribs", JavaNetRepo)
lazy val jerseyModuleConfig = ModuleConfiguration("com.sun.jersey", JavaNetRepo)
lazy val jgroupsModuleConfig = ModuleConfiguration("jgroups", JBossRepo)
- lazy val liftModuleConfig = ModuleConfiguration("net.liftweb", ScalaToolsReleases)
lazy val multiverseModuleConfig = ModuleConfiguration("org.multiverse", CodehausRepo)
lazy val nettyModuleConfig = ModuleConfiguration("org.jboss.netty", JBossRepo)
lazy val scalaTestModuleConfig = ModuleConfiguration("org.scalatest", ScalaToolsSnapshots)
lazy val logbackModuleConfig = ModuleConfiguration("ch.qos.logback",sbt.DefaultMavenRepository)
lazy val atomikosModuleConfig = ModuleConfiguration("com.atomikos",sbt.DefaultMavenRepository)
+ lazy val casbahModuleConfig = ModuleConfiguration("com.novus", CasbahRepo)
+ lazy val timeModuleConfig = ModuleConfiguration("org.scala-tools", "time", CasbahSnapshotRepo)
lazy val embeddedRepo = EmbeddedRepo // This is the only exception, because the embedded repo is fast!
// -------------------------------------------------------------------------------------------------------------------
@@ -89,14 +92,13 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val HAWT_DISPATCH_VERSION = "1.0"
lazy val JACKSON_VERSION = "1.2.1"
lazy val JERSEY_VERSION = "1.2"
- lazy val LIFT_VERSION = "2.1-M1"
- lazy val MULTIVERSE_VERSION = "0.6"
+ lazy val MULTIVERSE_VERSION = "0.6.1"
lazy val SCALATEST_VERSION = "1.2-for-scala-2.8.0.final-SNAPSHOT"
lazy val LOGBACK_VERSION = "0.9.24"
lazy val SLF4J_VERSION = "1.6.0"
lazy val SPRING_VERSION = "3.0.3.RELEASE"
lazy val ASPECTWERKZ_VERSION = "2.2.1"
- lazy val JETTY_VERSION = "7.1.6.v20100715"
+ lazy val JETTY_VERSION = "7.1.4.v20100610"
// -------------------------------------------------------------------------------------------------------------------
// Dependencies
@@ -136,9 +138,10 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val dispatch_http = "net.databinder" % "dispatch-http_2.8.0" % DISPATCH_VERSION % "compile"
lazy val dispatch_json = "net.databinder" % "dispatch-json_2.8.0" % DISPATCH_VERSION % "compile"
- lazy val jetty = "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION % "compile"
- lazy val jetty_util = "org.eclipse.jetty" % "jetty-util" % JETTY_VERSION % "compile"
- lazy val jetty_xml = "org.eclipse.jetty" % "jetty-xml" % JETTY_VERSION % "compile"
+ lazy val jetty = "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION % "compile"
+ lazy val jetty_util = "org.eclipse.jetty" % "jetty-util" % JETTY_VERSION % "compile"
+ lazy val jetty_xml = "org.eclipse.jetty" % "jetty-xml" % JETTY_VERSION % "compile"
+ lazy val jetty_servlet = "org.eclipse.jetty" % "jetty-servlet" % JETTY_VERSION % "compile"
lazy val uuid = "com.eaio" % "uuid" % "3.2" % "compile"
@@ -167,11 +170,10 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val jta_1_1 = "org.apache.geronimo.specs" % "geronimo-jta_1.1_spec" % "1.1.1" % "compile" intransitive
- lazy val lift_util = "net.liftweb" % "lift-util_2.8.0" % LIFT_VERSION % "compile"
- lazy val lift_webkit = "net.liftweb" % "lift-webkit_2.8.0" % LIFT_VERSION % "compile"
-
lazy val mongo = "org.mongodb" % "mongo-java-driver" % "2.0" % "compile"
+ lazy val casbah = "com.novus" % "casbah_2.8.0" % "1.0.8.5" % "compile"
+
lazy val multiverse = "org.multiverse" % "multiverse-alpha" % MULTIVERSE_VERSION % "compile" intransitive
lazy val netty = "org.jboss.netty" % "netty" % "3.2.2.Final" % "compile"
@@ -182,13 +184,11 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val rabbit = "com.rabbitmq" % "amqp-client" % "1.8.1" % "compile"
- lazy val redis = "com.redis" % "redisclient" % "2.8.0-1.4" % "compile"
+ lazy val redis = "com.redis" % "redisclient" % "2.8.0-2.0" % "compile"
lazy val sbinary = "sbinary" % "sbinary" % "2.8.0-0.3.1" % "compile"
- lazy val servlet = "javax.servlet" % "servlet-api" % "2.5" % "compile"
-
- lazy val sjson = "sjson.json" % "sjson" % "0.7-2.8.0" % "compile"
+ lazy val sjson = "sjson.json" % "sjson" % "0.8-2.8.0" % "compile"
lazy val slf4j = "org.slf4j" % "slf4j-api" % SLF4J_VERSION % "compile"
@@ -428,13 +428,13 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
val jetty = Dependencies.jetty
val jetty_util = Dependencies.jetty_util
val jetty_xml = Dependencies.jetty_xml
+ val jetty_servlet = Dependencies.jetty_servlet
val jackson_core_asl = Dependencies.jackson_core_asl
val jersey = Dependencies.jersey
val jersey_contrib = Dependencies.jersey_contrib
val jersey_json = Dependencies.jersey_json
val jersey_server = Dependencies.jersey_server
val jsr311 = Dependencies.jsr311
- val servlet = Dependencies.servlet
val stax_api = Dependencies.stax_api
// testing
@@ -492,6 +492,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
class AkkaMongoProject(info: ProjectInfo) extends AkkaDefaultProject(info, distPath) {
val mongo = Dependencies.mongo
+ val casbah = Dependencies.casbah
override def testOptions = TestFilter((name: String) => name.endsWith("Test")) :: Nil
}
@@ -567,7 +568,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
// Provided by other bundles
"!se.scalablesolutions.akka.*",
- "!net.liftweb.*",
"!com.google.inject.*",
"!javax.transaction.*",
"!javax.ws.rs.*",
@@ -589,12 +589,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
// Scala bundle
val scala_bundle = "com.weiglewilczek.scala-lang-osgi" % "scala-library" % buildScalaVersion % "compile" intransitive
- // Lift bundles
-// val lift_util = Dependencies.lift_util.intransitive
-// val lift_actor = "net.liftweb" % "lift-actor" % LIFT_VERSION % "compile" intransitive
-// val lift_common = "net.liftweb" % "lift-common" % LIFT_VERSION % "compile" intransitive
-// val lift_json = "net.liftweb" % "lift-json" % LIFT_VERSION % "compile" intransitive
-
// Camel bundles
val camel_core = Dependencies.camel_core.intransitive
val fusesource_commonman = "org.fusesource.commonman" % "commons-management" % "1.0" intransitive
@@ -659,21 +653,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
class AkkaSamplePubSubProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath)
class AkkaSampleFSMProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath)
- class AkkaSampleLiftProject(info: ProjectInfo) extends DefaultWebProject(info) with DeployProject {
- //val commons_logging = Dependencies.commons_logging
- val lift_util = Dependencies.lift_util
- val lift_webkit = Dependencies.lift_webkit
- val servlet = Dependencies.servlet
-
- // testing
- val testJetty = Dependencies.testJetty
- val testJettyWebApp = Dependencies.testJettyWebApp
- val junit = Dependencies.junit
-
- def deployPath = AkkaParentProject.this.deployPath
- override def jarPath = warPath
- }
-
class AkkaSampleRestJavaProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath)
class AkkaSampleRemoteProject(info: ProjectInfo) extends AkkaDefaultProject(info, deployPath)
@@ -721,8 +700,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
new AkkaSamplePubSubProject(_), akka_kernel)
lazy val akka_sample_fsm = project("akka-sample-fsm", "akka-sample-fsm",
new AkkaSampleFSMProject(_), akka_kernel)
- lazy val akka_sample_lift = project("akka-sample-lift", "akka-sample-lift",
- new AkkaSampleLiftProject(_), akka_kernel)
lazy val akka_sample_rest_java = project("akka-sample-rest-java", "akka-sample-rest-java",
new AkkaSampleRestJavaProject(_), akka_kernel)
lazy val akka_sample_rest_scala = project("akka-sample-rest-scala", "akka-sample-rest-scala",