diff --git a/akka-actor/src/main/scala/actor/Actor.scala b/akka-actor/src/main/scala/actor/Actor.scala
index 872997c760..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
diff --git a/akka-actor/src/main/scala/actor/ActorRef.scala b/akka-actor/src/main/scala/actor/ActorRef.scala
index 3472ec4696..cddbb92fc3 100644
--- a/akka-actor/src/main/scala/actor/ActorRef.scala
+++ b/akka-actor/src/main/scala/actor/ActorRef.scala
@@ -28,6 +28,17 @@ import java.lang.reflect.Field
import scala.reflect.BeanProperty
+
+object ActorRefStatus {
+ /** LifeCycles for ActorRefs
+ */
+ private[akka] sealed trait StatusType
+ object UNSTARTED extends StatusType
+ object RUNNING extends StatusType
+ object BEING_RESTARTED extends StatusType
+ object SHUTDOWN extends StatusType
+}
+
/**
* ActorRef is an immutable and serializable handle to an Actor.
*
@@ -68,9 +79,7 @@ trait ActorRef extends
// Only mutable for RemoteServer in order to maintain identity across nodes
@volatile protected[akka] var _uuid = UUID.newUuid.toString
- @volatile protected[this] var _isRunning = false
- @volatile protected[this] var _isShutDown = false
- @volatile protected[akka] var _isBeingRestarted = false
+ @volatile protected[this] var _status: ActorRefStatus.StatusType = ActorRefStatus.UNSTARTED
@volatile protected[akka] var _homeAddress = new InetSocketAddress(RemoteServerModule.HOSTNAME, RemoteServerModule.PORT)
@volatile protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None
@volatile protected[akka] var registeredInRemoteNodeDuringSerialization = false
@@ -229,17 +238,25 @@ trait ActorRef extends
/**
* Is the actor being restarted?
*/
- def isBeingRestarted: Boolean = _isBeingRestarted
+ def isBeingRestarted: Boolean = _status == ActorRefStatus.BEING_RESTARTED
/**
* Is the actor running?
*/
- def isRunning: Boolean = _isRunning
+ def isRunning: Boolean = _status match {
+ case ActorRefStatus.BEING_RESTARTED | ActorRefStatus.RUNNING => true
+ case _ => false
+ }
/**
* Is the actor shut down?
*/
- def isShutdown: Boolean = _isShutDown
+ def isShutdown: Boolean = _status == ActorRefStatus.SHUTDOWN
+
+ /**
+ * Is the actor ever started?
+ */
+ def isUnstarted: Boolean = _status == ActorRefStatus.UNSTARTED
/**
* Is the actor able to handle the message passed in as arguments?
@@ -601,9 +618,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]
@@ -800,7 +817,7 @@ class LocalActorRef private[akka](
if (isTransactor) {
_transactionFactory = Some(TransactionFactory(_transactionConfig, id))
}
- _isRunning = true
+ _status = ActorRefStatus.RUNNING
if (!isInInitialization) initializeActorInstance
else runActorInitialization = true
}
@@ -815,8 +832,7 @@ class LocalActorRef private[akka](
cancelReceiveTimeout
dispatcher.unregister(this)
_transactionFactory = None
- _isRunning = false
- _isShutDown = true
+ _status = ActorRefStatus.SHUTDOWN
actor.postStop
ActorRegistry.unregister(this)
if (isRemotingEnabled) {
@@ -864,11 +880,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
}
}
@@ -877,13 +893,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
}
}
@@ -893,9 +909,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
}
/**
@@ -919,9 +933,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
}
@@ -936,10 +950,11 @@ class LocalActorRef private[akka](
val actor = spawnButDoNotStart(clazz)
try {
actor.makeRemote(hostname, port)
- actor.start
- } finally {
link(actor)
+ } finally {
+ actor.start
}
+ actor
}
/**
@@ -1001,7 +1016,7 @@ class LocalActorRef private[akka](
}
/**
- * Callback for the dispatcher. This is the ingle entry point to the user Actor implementation.
+ * Callback for the dispatcher. This is the single entry point to the user Actor implementation.
*/
protected[akka] def invoke(messageHandle: MessageInvocation): Unit = guard.withGuard {
if (isShutdown)
@@ -1038,12 +1053,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(
@@ -1062,7 +1083,7 @@ class LocalActorRef private[akka](
stop
} else {
- _isBeingRestarted = true
+ _status = ActorRefStatus.BEING_RESTARTED
val failedActor = actorInstance.get
guard.withGuard {
lifeCycle match {
@@ -1072,16 +1093,18 @@ class LocalActorRef private[akka](
Actor.log.info("Restarting actor [%s] configured as PERMANENT.", id)
Actor.log.debug("Restarting linked actors for actor [%s].", id)
restartLinkedActors(reason, maxNrOfRetries, withinTimeRange)
+
Actor.log.debug("Invoking 'preRestart' for failed actor instance [%s].", id)
if (isProxyableDispatcher(failedActor)) restartProxyableDispatcher(failedActor, reason)
- else restartActor(failedActor, reason)
- _isBeingRestarted = false
+ else restartActor(failedActor, reason)
+
+ _status = ActorRefStatus.RUNNING
}
}
}
}
- 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
@@ -1131,11 +1154,7 @@ 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)){
@@ -1235,7 +1254,7 @@ class LocalActorRef private[akka](
private def handleExceptionInDispatch(reason: Throwable, message: Any, topLevelTransaction: Boolean) = {
Actor.log.error(reason, "Exception when invoking \n\tactor [%s] \n\twith message [%s]", this, message)
- _isBeingRestarted = true
+ _status = ActorRefStatus.BEING_RESTARTED
// abort transaction set
if (isTransactionSetInScope) {
val txSet = getTransactionSetInScope
@@ -1375,13 +1394,12 @@ private[akka] case class RemoteActorRef private[akka] (
}
def start: ActorRef = {
- _isRunning = true
+ _status = ActorRefStatus.RUNNING
this
}
def stop: Unit = {
- _isRunning = false
- _isShutDown = true
+ _status = ActorRefStatus.SHUTDOWN
postMessageToMailbox(RemoteActorSystemMessage.Stop, None)
}
@@ -1417,8 +1435,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
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..329682de52 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
*/
@@ -121,4 +150,46 @@ object DataFlow {
def shutdown = in ! Exit
}
-}
\ No newline at end of file
+
+ /**
+ * @author Jonas Bonér
+ */
+ class DataFlowStream[T <: Any] extends Seq[T] {
+ private[this] val queue = new LinkedBlockingQueue[DataFlowVariable[T]]
+
+ def <<<(ref: DataFlowVariable[T]) = queue.offer(ref)
+
+ def <<<(value: T) = {
+ val ref = new DataFlowVariable[T]
+ ref << value
+ queue.offer(ref)
+ }
+
+ def apply(): T = {
+ val ref = queue.take
+ val result = ref()
+ ref.shutdown
+ result
+ }
+
+ def take: DataFlowVariable[T] = queue.take
+
+ //==== For Seq ====
+
+ def length: Int = queue.size
+
+ def apply(i: Int): T = {
+ if (i == 0) apply()
+ else throw new UnsupportedOperationException(
+ "Access by index other than '0' is not supported by DataFlowStream")
+ }
+
+ def iterator: Iterator[T] = new Iterator[T] {
+ private val iter = queue.iterator
+ def hasNext: Boolean = iter.hasNext
+ def next: T = { val ref = iter.next; ref() }
+ }
+
+ override def toList: List[T] = queue.toArray.toList.asInstanceOf[List[T]]
+ }
+}
diff --git a/akka-actor/src/main/scala/dispatch/Dispatchers.scala b/akka-actor/src/main/scala/dispatch/Dispatchers.scala
index 2ebba03928..7e7904ec29 100644
--- a/akka-actor/src/main/scala/dispatch/Dispatchers.scala
+++ b/akka-actor/src/main/scala/dispatch/Dispatchers.scala
@@ -44,9 +44,10 @@ import se.scalablesolutions.akka.util.{Duration, Logging, UUID}
* @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", -1)
- val MAILBOX_CONFIG = MailboxConfig(
+ 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
@@ -58,7 +59,7 @@ object Dispatchers extends Logging {
object globalHawtDispatcher extends HawtDispatcher
- object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher("global",THROUGHPUT,MAILBOX_CONFIG) {
+ object globalExecutorBasedEventDrivenDispatcher extends ExecutorBasedEventDrivenDispatcher("global",THROUGHPUT,THROUGHPUT_DEADLINE_MS,MAILBOX_CONFIG) {
override def register(actor: ActorRef) = {
if (isShutdown) init
super.register(actor)
@@ -116,14 +117,14 @@ 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, mailboxCapacity: Int, pushTimeOut: Duration) = new ExecutorBasedEventDrivenDispatcher(name, throughput, MailboxConfig(mailboxCapacity,Some(pushTimeOut),false))
+ def newExecutorBasedEventDrivenDispatcher(name: String, throughput: Int, throughputDeadlineMs: Int, mailboxCapacity: Int, pushTimeOut: Duration) = new ExecutorBasedEventDrivenDispatcher(name, throughput, throughputDeadlineMs, MailboxConfig(mailboxCapacity,Some(pushTimeOut),false))
/**
@@ -198,13 +199,28 @@ object Dispatchers extends Logging {
}
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),mailboxBounds,threadPoolConfig)
- case "Hawt" => new HawtDispatcher(cfg.getBool("aggregate").getOrElse(true))
- case "GlobalExecutorBasedEventDriven" => globalExecutorBasedEventDrivenDispatcher
- case "GlobalHawt" => globalHawtDispatcher
+ 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 unknown => throw new IllegalArgumentException("Unknown dispatcher type [%s]" format unknown)
+ 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 6cabdec5e5..19e9cd38e7 100644
--- a/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
+++ b/akka-actor/src/main/scala/dispatch/ExecutorBasedEventDrivenDispatcher.scala
@@ -65,12 +65,13 @@ import java.util.concurrent.{RejectedExecutionException, ConcurrentLinkedQueue,
class ExecutorBasedEventDrivenDispatcher(
_name: String,
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, capacity: Int) = this(_name,throughput,MailboxConfig(capacity,None,false))
- 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,Dispatchers.MAILBOX_CAPACITY) // 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
//FIXME remove this from ThreadPoolBuilder
mailboxCapacity = mailboxConfig.capacity
@@ -102,24 +103,28 @@ class ExecutorBasedEventDrivenDispatcher(
* @return true if the processing finished before the mailbox was empty, due to the throughput constraint
*/
final def processMailbox(): Boolean = {
- val throttle = throughput > 0
- var processedMessages = 0
- var nextMessage = self.dequeue
- if (nextMessage ne null) {
- do {
- nextMessage.invoke
+ 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
- if(throttle) { //Will be elided when false
- processedMessages += 1
- if (processedMessages >= throughput) //If we're throttled, break out
- return !self.isEmpty
- }
- nextMessage = self.dequeue
- }
- while (nextMessage ne null)
- }
+ do {
+ nextMessage.invoke
- false
+ 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
}
}
@@ -183,4 +188,4 @@ class ExecutorBasedEventDrivenDispatcher(
config(this)
buildThreadPool
}
-}
+}
\ No newline at end of file
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/ThreadBasedDispatcher.scala b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala
index eda5a86a9e..090be85cee 100644
--- a/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala
+++ b/akka-actor/src/main/scala/dispatch/ThreadBasedDispatcher.scala
@@ -11,6 +11,14 @@ import se.scalablesolutions.akka.config.Config.config
import concurrent.forkjoin.{TransferQueue, LinkedTransferQueue}
import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, LinkedBlockingQueue}
+object ThreadBasedDispatcher {
+ def oneThread(b: ThreadPoolBuilder) {
+ b setCorePoolSize 1
+ b setMaxPoolSize 1
+ b setAllowCoreThreadTimeout true
+ }
+}
+
/**
* Dedicates a unique thread for each actor passed in as reference. Served through its messageQueue.
*
@@ -18,16 +26,14 @@ import java.util.concurrent.{ConcurrentLinkedQueue, BlockingQueue, TimeUnit, Lin
*/
class ThreadBasedDispatcher(private val actor: ActorRef,
val mailboxConfig: MailboxConfig
- ) extends MessageDispatcher {
+ ) extends ExecutorBasedEventDrivenDispatcher(
+ actor.getClass.getName + ":" + actor.uuid,
+ Dispatchers.THROUGHPUT,
+ -1,
+ mailboxConfig,
+ ThreadBasedDispatcher.oneThread) {
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
- private val threadName = "akka:thread-based:dispatcher:" + name
- private var selectorThread: Thread = _
- @volatile private var active: Boolean = false
-
- override def createMailbox(actorRef: ActorRef): AnyRef = mailboxConfig.newMailbox(blockDequeue = true)
override def register(actorRef: ActorRef) = {
if(actorRef != actor)
@@ -36,35 +42,5 @@ class ThreadBasedDispatcher(private val actor: ActorRef,
super.register(actorRef)
}
- def mailbox = actor.mailbox.asInstanceOf[Queue[MessageInvocation] with MessageQueue]
-
- def mailboxSize(a: ActorRef) = mailbox.size
-
- def dispatch(invocation: MessageInvocation) = mailbox enqueue invocation
-
- def start = if (!active) {
- log.debug("Starting up %s", toString)
- active = true
- selectorThread = new Thread(threadName) {
- override def run = {
- while (active) {
- try {
- actor.invoke(mailbox.dequeue)
- } catch { case e: InterruptedException => active = false }
- }
- }
- }
- selectorThread.start
- }
-
- def isShutdown = !active
-
- def shutdown = if (active) {
- log.debug("Shutting down %s", toString)
- active = false
- selectorThread.interrupt
- uuids.clear
- }
-
- override def toString = "ThreadBasedDispatcher[" + threadName + "]"
+ override def toString = "ThreadBasedDispatcher[" + name + "]"
}
\ 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
index 1d39a0864b..099082595d 100644
--- a/akka-actor/src/main/scala/util/JavaAPI.scala
+++ b/akka-actor/src/main/scala/util/JavaAPI.scala
@@ -13,4 +13,11 @@ trait Function[T,R] {
*/
trait Procedure[T] {
def apply(param: T): Unit
-}
\ No newline at end of file
+}
+
+/**
+ * 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/Logging.scala b/akka-actor/src/main/scala/util/Logging.scala
index b6ddaaa16a..8d2e64be58 100644
--- a/akka-actor/src/main/scala/util/Logging.scala
+++ b/akka-actor/src/main/scala/util/Logging.scala
@@ -111,7 +111,7 @@ class Logger(val logger: SLFLogger) {
warning(message(fmt,arg,argN:_*))
}
- def warn(fmt: => String, arg: Any, argN: Any*) = warning(fmt, arg, argN)
+ def warn(fmt: => String, arg: Any, argN: Any*) = warning(fmt, arg, argN:_*)
def warning(msg: => String) {
if (warning_?) logger warn msg
diff --git a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
index 234a0bd25d..b9fa238963 100644
--- a/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
+++ b/akka-actor/src/test/scala/actor/supervisor/RestartStrategySpec.scala
@@ -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/dataflow/DataFlowSpec.scala b/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala
index 2997715452..d1f663e9f4 100644
--- a/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala
+++ b/akka-actor/src/test/scala/dataflow/DataFlowSpec.scala
@@ -69,5 +69,97 @@ class DataFlowTest extends Spec with ShouldMatchers with BeforeAndAfterAll {
result.get should equal (sum(0,ints(0,1000)))
List(x,y,z).foreach(_.shutdown)
}
+
+ /*it("should be able to join streams") {
+ import DataFlow._
+ ActorRegistry.shutdownAll
+
+ def ints(n: Int, max: Int, stream: DataFlowStream[Int]): Unit = if (n != max) {
+ stream <<< n
+ ints(n + 1, max, stream)
+ }
+
+ def sum(s: Int, in: DataFlowStream[Int], out: DataFlowStream[Int]): Unit = {
+ out <<< s
+ sum(in() + s, in, out)
+ }
+
+ val producer = new DataFlowStream[Int]
+ val consumer = new DataFlowStream[Int]
+ val latch = new CountDownLatch(1)
+ val result = new AtomicInteger(0)
+
+ val t1 = thread { ints(0, 1000, producer) }
+ val t2 = thread {
+ Thread.sleep(1000)
+ result.set(producer.map(x => x * x).foldLeft(0)(_ + _))
+ latch.countDown
+ }
+
+ latch.await(3,TimeUnit.SECONDS) should equal (true)
+ result.get should equal (332833500)
}
+
+ it("should be able to sum streams recursively") {
+ import DataFlow._
+
+ def ints(n: Int, max: Int, stream: DataFlowStream[Int]): Unit = if (n != max) {
+ stream <<< n
+ ints(n + 1, max, stream)
+ }
+
+ def sum(s: Int, in: DataFlowStream[Int], out: DataFlowStream[Int]): Unit = {
+ out <<< s
+ sum(in() + s, in, out)
+ }
+
+ val result = new AtomicLong(0)
+
+ val producer = new DataFlowStream[Int]
+ val consumer = new DataFlowStream[Int]
+ val latch = new CountDownLatch(1)
+
+ @tailrec def recurseSum(stream: DataFlowStream[Int]): Unit = {
+ val x = stream()
+
+ if(result.addAndGet(x) == 166666500)
+ latch.countDown
+
+ recurseSum(stream)
+ }
+
+ thread { ints(0, 1000, producer) }
+ thread { sum(0, producer, consumer) }
+ thread { recurseSum(consumer) }
+
+ latch.await(15,TimeUnit.SECONDS) should equal (true)
+ }*/
+
+ /* Test not ready for prime time, causes some sort of deadlock */
+ /* it("should be able to conditionally set variables") {
+
+ import DataFlow._
+ ActorRegistry.shutdownAll
+
+ val latch = new CountDownLatch(1)
+ val x, y, z, v = new DataFlowVariable[Int]
+
+ val main = thread {
+ x << 1
+ z << Math.max(x(),y())
+ latch.countDown
+ }
+
+ val setY = thread {
+ // Thread.sleep(2000)
+ y << 2
+ }
+
+ val setV = thread {
+ v << y
+ }
+ List(x,y,z,v) foreach (_.shutdown)
+ latch.await(2,TimeUnit.SECONDS) should equal (true)
+ }*/
+ }
}
\ No newline at end of file
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/HawtDispatcherEchoServer.scala b/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala
index 8d4c8dedc1..6196a13490 100644
--- a/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala
+++ b/akka-actor/src/test/scala/dispatch/HawtDispatcherEchoServer.scala
@@ -146,7 +146,7 @@ object HawtDispatcherEchoServer {
read_source.setEventHandler(^{ read })
read_source.setCancelHandler(^{ close })
- write_source = createSource(channel, SelectionKey.OP_READ, HawtDispatcher.queue(self));
+ write_source = createSource(channel, SelectionKey.OP_WRITE, HawtDispatcher.queue(self));
write_source.setEventHandler(^{ write })
write_source.setCancelHandler(^{ close })
diff --git a/akka-camel/src/main/scala/component/ActorComponent.scala b/akka-camel/src/main/scala/component/ActorComponent.scala
index a5d56dd9dc..89cc0d4d3e 100644
--- a/akka-camel/src/main/scala/component/ActorComponent.scala
+++ b/akka-camel/src/main/scala/component/ActorComponent.scala
@@ -18,10 +18,10 @@ import se.scalablesolutions.akka.camel.{Failure, CamelMessageConversion, Message
import CamelMessageConversion.toExchangeAdapter
import se.scalablesolutions.akka.dispatch.{CompletableFuture, MessageInvocation, MessageDispatcher}
import se.scalablesolutions.akka.stm.TransactionConfig
-import se.scalablesolutions.akka.actor.{ScalaActorRef, ActorRegistry, Actor, ActorRef}
import se.scalablesolutions.akka.AkkaException
import scala.reflect.BeanProperty
+import se.scalablesolutions.akka.actor._
/**
* Camel component for sending messages to and receiving replies from (untyped) actors.
@@ -199,13 +199,12 @@ private[akka] object AsyncCallbackAdapter {
private[akka] class AsyncCallbackAdapter(exchange: Exchange, callback: AsyncCallback) extends ActorRef with ScalaActorRef {
def start = {
- _isRunning = true
+ _status = ActorRefStatus.RUNNING
this
}
def stop() = {
- _isRunning = false
- _isShutDown = true
+ _status = ActorRefStatus.SHUTDOWN
}
/**
@@ -247,8 +246,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-persistence/akka-persistence-common/src/main/scala/Storage.scala b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala
index 4d9ff48a60..088c0b8ff4 100644
--- a/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala
+++ b/akka-persistence/akka-persistence-common/src/main/scala/Storage.scala
@@ -82,7 +82,6 @@ trait Storage {
*/
trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
with Transactional with Committable with Abortable with Logging {
- protected val shouldClearOnCommit = Ref[Boolean]()
// operations on the Map
trait Op
@@ -90,11 +89,12 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
case object PUT extends Op
case object REM extends Op
case object UPD extends Op
+ case object CLR extends Op
// append only log: records all mutating operations
protected val appendOnlyTxLog = TransactionalVector[LogEntry]()
- case class LogEntry(key: K, value: Option[V], op: Op)
+ case class LogEntry(key: Option[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
@@ -114,7 +114,7 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
protected def clearDistinctKeys = keysInCurrentTx.clear
protected def filterTxLogByKey(key: K): IndexedSeq[LogEntry] =
- appendOnlyTxLog filter(e => equal(e.key, key))
+ appendOnlyTxLog filter(e => e.key.map(equal(_, key)).getOrElse(true))
// need to get current value considering the underlying storage as well as the transaction log
protected def getCurrentValue(key: K): Option[V] = {
@@ -128,7 +128,10 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
} catch { case e: Exception => None }
if (txEntries.isEmpty) underlying
- else replay(txEntries, key, underlying)
+ else txEntries.last match {
+ case LogEntry(_, _, CLR) => None
+ case _ => replay(txEntries, key, underlying)
+ }
}
// replay all tx entries for key k with seed = initial
@@ -140,9 +143,10 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[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)
+ case PUT => m.put(k.get, v.get)
+ case REM => m -= k.get
+ case UPD => m.update(k.get, v.get)
+ case CLR => Map.empty[K, V]
}}
m get key
}
@@ -151,12 +155,11 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
val storage: MapStorageBackend[K, V]
def commit = {
- // 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)
+ case PUT => storage.insertMapStorageEntryFor(uuid, k.get, v.get)
+ case UPD => storage.insertMapStorageEntryFor(uuid, k.get, v.get)
+ case REM => storage.removeMapStorageFor(uuid, k.get)
+ case CLR => storage.removeMapStorageFor(uuid)
}}
appendOnlyTxLog.clear
@@ -166,7 +169,6 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
def abort = {
appendOnlyTxLog.clear
clearDistinctKeys
- shouldClearOnCommit.swap(false)
}
def -=(key: K) = {
@@ -187,7 +189,7 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
override def put(key: K, value: V): Option[V] = {
register
val curr = getCurrentValue(key)
- appendOnlyTxLog add LogEntry(key, Some(value), PUT)
+ appendOnlyTxLog add LogEntry(Some(key), Some(value), PUT)
addToListOfKeysInTx(key)
curr
}
@@ -195,7 +197,7 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
override def update(key: K, value: V) = {
register
val curr = getCurrentValue(key)
- appendOnlyTxLog add LogEntry(key, Some(value), UPD)
+ appendOnlyTxLog add LogEntry(Some(key), Some(value), UPD)
addToListOfKeysInTx(key)
curr
}
@@ -203,7 +205,7 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
override def remove(key: K) = {
register
val curr = getCurrentValue(key)
- appendOnlyTxLog add LogEntry(key, None, REM)
+ appendOnlyTxLog add LogEntry(Some(key), None, REM)
addToListOfKeysInTx(key)
curr
}
@@ -215,9 +217,8 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
override def clear = {
register
- appendOnlyTxLog.clear
+ appendOnlyTxLog add LogEntry(None, None, CLR)
clearDistinctKeys
- shouldClearOnCommit.swap(true)
}
override def contains(key: K): Boolean = try {
@@ -225,7 +226,8 @@ trait PersistentMap[K, V] extends scala.collection.mutable.Map[K, V]
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
+ val lastOp = txs.last.op
+ lastOp != REM && lastOp != CLR // last entry cannot be a REM
}
} catch { case e: Exception => false }
@@ -366,11 +368,6 @@ trait PersistentMapBinary extends PersistentMap[Array[Byte], Array[Byte]] {
* @author Jonas Bonér
*/
trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committable with Abortable {
- protected val newElems = TransactionalVector[T]()
- protected val updatedElems = TransactionalMap[Int, T]()
- protected val removedElems = TransactionalVector[T]()
- protected val shouldClearOnCommit = Ref[Boolean]()
-
// operations on the Vector
trait Op
case object ADD extends Op
@@ -400,7 +397,6 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa
def abort = {
appendOnlyTxLog.clear
- shouldClearOnCommit.swap(false)
}
private def replay: List[T] = {
@@ -466,14 +462,7 @@ trait PersistentVector[T] extends IndexedSeq[T] with Transactional with Committa
override def first: T = get(0)
- override def last: T = {
- if (newElems.length != 0) newElems.last
- else {
- val len = length
- if (len == 0) throw new NoSuchElementException("Vector is empty")
- get(len - 1)
- }
- }
+ override def last: T = replay.last
def length: Int = replay.length
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 01d8ababce..6573100422 100644
--- a/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala
+++ b/akka-persistence/akka-persistence-mongo/src/main/scala/MongoStorageBackend.scala
@@ -9,7 +9,6 @@ import se.scalablesolutions.akka.persistence.common._
import se.scalablesolutions.akka.util.Logging
import se.scalablesolutions.akka.config.Config.config
-import java.util.NoSuchElementException
import com.novus.casbah.mongodb.Imports._
/**
diff --git a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala
index 3b160c8c50..413be5d860 100644
--- a/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala
+++ b/akka-persistence/akka-persistence-mongo/src/test/scala/MongoTicket343Spec.scala
@@ -238,7 +238,7 @@ class MongoTicket343Spec extends
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 !! MAP_SIZE).getOrElse("Size failed") should equal(0)
proc.stop
}
}
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 9fd3142019..6f2052f0bd 100644
--- a/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala
+++ b/akka-persistence/akka-persistence-redis/src/main/scala/RedisStorageBackend.scala
@@ -359,7 +359,6 @@ 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
index de236b9a5a..2b06b17270 100644
--- a/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala
+++ b/akka-persistence/akka-persistence-redis/src/test/scala/RedisTicket343Spec.scala
@@ -32,6 +32,10 @@ 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 object VLAST
+case object VFIRST
+case class VLAST_AFTER_ADD(vsToAdd: List[String])
+case class VFIRST_AFTER_ADD(vsToAdd: List[String])
case class VGET_AFTER_VADD(vsToAdd: List[String], isToFetch: List[Int])
case class VADD_WITH_SLICE(vsToAdd: List[String], start: Int, cnt: Int)
@@ -175,6 +179,30 @@ object Storage {
fooVector.slice(Some(s), None, c)
}
self.reply(l.map(new String(_)))
+
+ case VLAST =>
+ val l = atomic { fooVector last }
+ self.reply(l)
+
+ case VFIRST =>
+ val l = atomic { fooVector first }
+ self.reply(l)
+
+ case VLAST_AFTER_ADD(vs) =>
+ val l =
+ atomic {
+ vs.foreach(fooVector + _.getBytes)
+ fooVector last
+ }
+ self.reply(l)
+
+ case VFIRST_AFTER_ADD(vs) =>
+ val l =
+ atomic {
+ vs.foreach(fooVector + _.getBytes)
+ fooVector first
+ }
+ self.reply(l)
}
}
}
@@ -243,7 +271,7 @@ class RedisTicket343Spec extends
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 !! MAP_SIZE).getOrElse("Size failed") should equal(0)
proc.stop
}
}
@@ -344,7 +372,26 @@ class RedisTicket343Spec extends
(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 !! VADD_WITH_SLICE(List("a", "b", "c", "d"), 2, 4)).getOrElse("VADD_WITH_SLICE failed") should equal(Vector("b", "a", "nilanjan", "ramanendu"))
+ proc.stop
+ }
+ }
+
+ describe("Miscellaneous vector ops") {
+ 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)
+
+ new String((proc !! VLAST).getOrElse("VLAST failed").asInstanceOf[Array[Byte]]) should equal("debasish")
+ new String((proc !! VFIRST).getOrElse("VFIRST failed").asInstanceOf[Array[Byte]]) should equal("nilanjan")
+ new String((proc !! VLAST_AFTER_ADD(List("kausik", "tarun"))).getOrElse("VLAST_AFTER_ADD failed").asInstanceOf[Array[Byte]]) should equal("debasish")
+ new String((proc !! VFIRST_AFTER_ADD(List("kausik", "tarun"))).getOrElse("VFIRST_AFTER_ADD failed").asInstanceOf[Array[Byte]]) should equal("tarun")
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 61d79c7a3f..31ae9650d4 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
@@ -653,6 +653,360 @@ public final class RemoteProtocol {
// @@protoc_insertion_point(class_scope:RemoteActorRefProtocol)
}
+ public static final class RemoteTypedActorRefProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use RemoteTypedActorRefProtocol.newBuilder() to construct.
+ private RemoteTypedActorRefProtocol() {
+ initFields();
+ }
+ private RemoteTypedActorRefProtocol(boolean noInit) {}
+
+ private static final RemoteTypedActorRefProtocol defaultInstance;
+ public static RemoteTypedActorRefProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public RemoteTypedActorRefProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteTypedActorRefProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_RemoteTypedActorRefProtocol_fieldAccessorTable;
+ }
+
+ // required .RemoteActorRefProtocol actorRef = 1;
+ public static final int ACTORREF_FIELD_NUMBER = 1;
+ private boolean hasActorRef;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol actorRef_;
+ public boolean hasActorRef() { return hasActorRef; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getActorRef() { return actorRef_; }
+
+ // required string interfaceName = 2;
+ public static final int INTERFACENAME_FIELD_NUMBER = 2;
+ private boolean hasInterfaceName;
+ private java.lang.String interfaceName_ = "";
+ public boolean hasInterfaceName() { return hasInterfaceName; }
+ public java.lang.String getInterfaceName() { return interfaceName_; }
+
+ private void initFields() {
+ actorRef_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
+ }
+ public final boolean isInitialized() {
+ if (!hasActorRef) return false;
+ if (!hasInterfaceName) return false;
+ if (!getActorRef().isInitialized()) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasActorRef()) {
+ output.writeMessage(1, getActorRef());
+ }
+ if (hasInterfaceName()) {
+ output.writeString(2, getInterfaceName());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasActorRef()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, getActorRef());
+ }
+ if (hasInterfaceName()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getInterfaceName());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol 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.RemoteTypedActorRefProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.getDefaultInstance()) return this;
+ if (other.hasActorRef()) {
+ mergeActorRef(other.getActorRef());
+ }
+ if (other.hasInterfaceName()) {
+ setInterfaceName(other.getInterfaceName());
+ }
+ 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: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder();
+ if (hasActorRef()) {
+ subBuilder.mergeFrom(getActorRef());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setActorRef(subBuilder.buildPartial());
+ break;
+ }
+ case 18: {
+ setInterfaceName(input.readString());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required .RemoteActorRefProtocol actorRef = 1;
+ public boolean hasActorRef() {
+ return result.hasActorRef();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getActorRef() {
+ return result.getActorRef();
+ }
+ public Builder setActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasActorRef = true;
+ result.actorRef_ = value;
+ return this;
+ }
+ public Builder setActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) {
+ result.hasActorRef = true;
+ result.actorRef_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) {
+ if (result.hasActorRef() &&
+ result.actorRef_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) {
+ result.actorRef_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.actorRef_).mergeFrom(value).buildPartial();
+ } else {
+ result.actorRef_ = value;
+ }
+ result.hasActorRef = true;
+ return this;
+ }
+ public Builder clearActorRef() {
+ result.hasActorRef = false;
+ result.actorRef_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // required string interfaceName = 2;
+ public boolean hasInterfaceName() {
+ return result.hasInterfaceName();
+ }
+ public java.lang.String getInterfaceName() {
+ return result.getInterfaceName();
+ }
+ public Builder setInterfaceName(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasInterfaceName = true;
+ result.interfaceName_ = value;
+ return this;
+ }
+ public Builder clearInterfaceName() {
+ result.hasInterfaceName = false;
+ result.interfaceName_ = getDefaultInstance().getInterfaceName();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:RemoteTypedActorRefProtocol)
+ }
+
+ static {
+ defaultInstance = new RemoteTypedActorRefProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:RemoteTypedActorRefProtocol)
+ }
+
public static final class SerializedActorRefProtocol extends
com.google.protobuf.GeneratedMessage {
// Use SerializedActorRefProtocol.newBuilder() to construct.
@@ -1559,6 +1913,360 @@ public final class RemoteProtocol {
// @@protoc_insertion_point(class_scope:SerializedActorRefProtocol)
}
+ public static final class SerializedTypedActorRefProtocol extends
+ com.google.protobuf.GeneratedMessage {
+ // Use SerializedTypedActorRefProtocol.newBuilder() to construct.
+ private SerializedTypedActorRefProtocol() {
+ initFields();
+ }
+ private SerializedTypedActorRefProtocol(boolean noInit) {}
+
+ private static final SerializedTypedActorRefProtocol defaultInstance;
+ public static SerializedTypedActorRefProtocol getDefaultInstance() {
+ return defaultInstance;
+ }
+
+ public SerializedTypedActorRefProtocol getDefaultInstanceForType() {
+ return defaultInstance;
+ }
+
+ public static final com.google.protobuf.Descriptors.Descriptor
+ getDescriptor() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedTypedActorRefProtocol_descriptor;
+ }
+
+ protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internalGetFieldAccessorTable() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internal_static_SerializedTypedActorRefProtocol_fieldAccessorTable;
+ }
+
+ // required .SerializedActorRefProtocol actorRef = 1;
+ public static final int ACTORREF_FIELD_NUMBER = 1;
+ private boolean hasActorRef;
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol actorRef_;
+ public boolean hasActorRef() { return hasActorRef; }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef() { return actorRef_; }
+
+ // required string interfaceName = 2;
+ public static final int INTERFACENAME_FIELD_NUMBER = 2;
+ private boolean hasInterfaceName;
+ private java.lang.String interfaceName_ = "";
+ public boolean hasInterfaceName() { return hasInterfaceName; }
+ public java.lang.String getInterfaceName() { return interfaceName_; }
+
+ private void initFields() {
+ actorRef_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance();
+ }
+ public final boolean isInitialized() {
+ if (!hasActorRef) return false;
+ if (!hasInterfaceName) return false;
+ if (!getActorRef().isInitialized()) return false;
+ return true;
+ }
+
+ public void writeTo(com.google.protobuf.CodedOutputStream output)
+ throws java.io.IOException {
+ getSerializedSize();
+ if (hasActorRef()) {
+ output.writeMessage(1, getActorRef());
+ }
+ if (hasInterfaceName()) {
+ output.writeString(2, getInterfaceName());
+ }
+ getUnknownFields().writeTo(output);
+ }
+
+ private int memoizedSerializedSize = -1;
+ public int getSerializedSize() {
+ int size = memoizedSerializedSize;
+ if (size != -1) return size;
+
+ size = 0;
+ if (hasActorRef()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeMessageSize(1, getActorRef());
+ }
+ if (hasInterfaceName()) {
+ size += com.google.protobuf.CodedOutputStream
+ .computeStringSize(2, getInterfaceName());
+ }
+ size += getUnknownFields().getSerializedSize();
+ memoizedSerializedSize = size;
+ return size;
+ }
+
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom(
+ com.google.protobuf.ByteString data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol parseFrom(byte[] data)
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ return newBuilder().mergeFrom(data).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol parseFrom(java.io.InputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol parseFrom(
+ com.google.protobuf.CodedInputStream input)
+ throws java.io.IOException {
+ return newBuilder().mergeFrom(input).buildParsed();
+ }
+ public static se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol result;
+
+ // Construct using se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.newBuilder()
+ private Builder() {}
+
+ private static Builder create() {
+ Builder builder = new Builder();
+ builder.result = new se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol();
+ return builder;
+ }
+
+ protected se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol 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.SerializedTypedActorRefProtocol();
+ return this;
+ }
+
+ public Builder clone() {
+ return create().mergeFrom(result);
+ }
+
+ public com.google.protobuf.Descriptors.Descriptor
+ getDescriptorForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDescriptor();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol getDefaultInstanceForType() {
+ return se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDefaultInstance();
+ }
+
+ public boolean isInitialized() {
+ return result.isInitialized();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol build() {
+ if (result != null && !isInitialized()) {
+ throw newUninitializedMessageException(result);
+ }
+ return buildPartial();
+ }
+
+ private se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol buildParsed()
+ throws com.google.protobuf.InvalidProtocolBufferException {
+ if (!isInitialized()) {
+ throw newUninitializedMessageException(
+ result).asInvalidProtocolBufferException();
+ }
+ return buildPartial();
+ }
+
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol buildPartial() {
+ if (result == null) {
+ throw new IllegalStateException(
+ "build() has already been called on this Builder.");
+ }
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol returnMe = result;
+ result = null;
+ return returnMe;
+ }
+
+ public Builder mergeFrom(com.google.protobuf.Message other) {
+ if (other instanceof se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol) {
+ return mergeFrom((se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol)other);
+ } else {
+ super.mergeFrom(other);
+ return this;
+ }
+ }
+
+ public Builder mergeFrom(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol other) {
+ if (other == se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDefaultInstance()) return this;
+ if (other.hasActorRef()) {
+ mergeActorRef(other.getActorRef());
+ }
+ if (other.hasInterfaceName()) {
+ setInterfaceName(other.getInterfaceName());
+ }
+ 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: {
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder subBuilder = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder();
+ if (hasActorRef()) {
+ subBuilder.mergeFrom(getActorRef());
+ }
+ input.readMessage(subBuilder, extensionRegistry);
+ setActorRef(subBuilder.buildPartial());
+ break;
+ }
+ case 18: {
+ setInterfaceName(input.readString());
+ break;
+ }
+ }
+ }
+ }
+
+
+ // required .SerializedActorRefProtocol actorRef = 1;
+ public boolean hasActorRef() {
+ return result.hasActorRef();
+ }
+ public se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef() {
+ return result.getActorRef();
+ }
+ public Builder setActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasActorRef = true;
+ result.actorRef_ = value;
+ return this;
+ }
+ public Builder setActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder builderForValue) {
+ result.hasActorRef = true;
+ result.actorRef_ = builderForValue.build();
+ return this;
+ }
+ public Builder mergeActorRef(se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol value) {
+ if (result.hasActorRef() &&
+ result.actorRef_ != se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) {
+ result.actorRef_ =
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder(result.actorRef_).mergeFrom(value).buildPartial();
+ } else {
+ result.actorRef_ = value;
+ }
+ result.hasActorRef = true;
+ return this;
+ }
+ public Builder clearActorRef() {
+ result.hasActorRef = false;
+ result.actorRef_ = se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance();
+ return this;
+ }
+
+ // required string interfaceName = 2;
+ public boolean hasInterfaceName() {
+ return result.hasInterfaceName();
+ }
+ public java.lang.String getInterfaceName() {
+ return result.getInterfaceName();
+ }
+ public Builder setInterfaceName(java.lang.String value) {
+ if (value == null) {
+ throw new NullPointerException();
+ }
+ result.hasInterfaceName = true;
+ result.interfaceName_ = value;
+ return this;
+ }
+ public Builder clearInterfaceName() {
+ result.hasInterfaceName = false;
+ result.interfaceName_ = getDefaultInstance().getInterfaceName();
+ return this;
+ }
+
+ // @@protoc_insertion_point(builder_scope:SerializedTypedActorRefProtocol)
+ }
+
+ static {
+ defaultInstance = new SerializedTypedActorRefProtocol(true);
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.internalForceInit();
+ defaultInstance.initFields();
+ }
+
+ // @@protoc_insertion_point(class_scope:SerializedTypedActorRefProtocol)
+ }
+
public static final class MessageProtocol extends
com.google.protobuf.GeneratedMessage {
// Use MessageProtocol.newBuilder() to construct.
@@ -2001,6 +2709,13 @@ public final class RemoteProtocol {
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();
@@ -2034,6 +2749,9 @@ public final class RemoteProtocol {
if (hasTypedActorInfo()) {
output.writeMessage(5, getTypedActorInfo());
}
+ if (hasId()) {
+ output.writeString(6, getId());
+ }
getUnknownFields().writeTo(output);
}
@@ -2063,6 +2781,10 @@ 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;
@@ -2236,6 +2958,9 @@ public final class RemoteProtocol {
if (other.hasTypedActorInfo()) {
mergeTypedActorInfo(other.getTypedActorInfo());
}
+ if (other.hasId()) {
+ setId(other.getId());
+ }
this.mergeUnknownFields(other.getUnknownFields());
return this;
}
@@ -2292,6 +3017,10 @@ public final class RemoteProtocol {
setTypedActorInfo(subBuilder.buildPartial());
break;
}
+ case 50: {
+ setId(input.readString());
+ break;
+ }
}
}
}
@@ -2415,6 +3144,27 @@ public final class RemoteProtocol {
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)
}
@@ -5658,11 +6408,21 @@ public final class RemoteProtocol {
private static
com.google.protobuf.GeneratedMessage.FieldAccessorTable
internal_static_RemoteActorRefProtocol_fieldAccessorTable;
+ private static com.google.protobuf.Descriptors.Descriptor
+ internal_static_RemoteTypedActorRefProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_RemoteTypedActorRefProtocol_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_SerializedTypedActorRefProtocol_descriptor;
+ private static
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable
+ internal_static_SerializedTypedActorRefProtocol_fieldAccessorTable;
private static com.google.protobuf.Descriptors.Descriptor
internal_static_MessageProtocol_descriptor;
private static
@@ -5725,49 +6485,54 @@ public final class RemoteProtocol {
"\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\"\352\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\022(\n\010metadata\030\007 \003(\0132\026.MetadataE" +
- "ntryProtocol\"\324\001\n\023RemoteReplyProtocol\022\n\n\002" +
- "id\030\001 \002(\004\022!\n\007message\030\002 \001(\0132\020.MessageProto" +
- "col\022%\n\texception\030\003 \001(\0132\022.ExceptionProtoc",
- "ol\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\014UuidProto" +
- "col\022\014\n\004high\030\001 \002(\004\022\013\n\003low\030\002 \002(\004\"3\n\025Metada" +
- "taEntryProtocol\022\013\n\003key\030\001 \002(\t\022\r\n\005value\030\002 " +
- "\002(\014\"6\n\021LifeCycleProtocol\022!\n\tlifeCycle\030\001 " +
- "\002(\0162\016.LifeCycleType\"1\n\017AddressProtocol\022\020" +
- "\n\010hostname\030\001 \002(\t\022\014\n\004port\030\002 \002(\r\"7\n\021Except" +
- "ionProtocol\022\021\n\tclassname\030\001 \002(\t\022\017\n\007messag" +
- "e\030\002 \002(\t*=\n\tActorType\022\017\n\013SCALA_ACTOR\020\001\022\016\n",
- "\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACTOR\020\003*]\n\027Serial" +
- "izationSchemeType\022\010\n\004JAVA\020\001\022\013\n\007SBINARY\020\002" +
- "\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JSON\020\004\022\014\n\010PROTO" +
- "BUF\020\005*-\n\rLifeCycleType\022\r\n\tPERMANENT\020\001\022\r\n" +
- "\tTEMPORARY\020\002B-\n)se.scalablesolutions.akk" +
- "a.remote.protocolH\001"
+ "tocol\022\017\n\007timeout\030\004 \001(\004\"_\n\033RemoteTypedAct" +
+ "orRefProtocol\022)\n\010actorRef\030\001 \002(\0132\027.Remote" +
+ "ActorRefProtocol\022\025\n\rinterfaceName\030\002 \002(\t\"" +
+ "\200\003\n\032SerializedActorRefProtocol\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\017originalAddress\030\004 \002(\0132\020.AddressProto" +
+ "col\022\025\n\ractorInstance\030\005 \001(\014\022\033\n\023serializer",
+ "Classname\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(\0132\022.LifeCycleProtocol\022+\n" +
+ "\nsupervisor\030\013 \001(\0132\027.RemoteActorRefProtoc" +
+ "ol\022\024\n\014hotswapStack\030\014 \001(\014\022(\n\010messages\030\r \003" +
+ "(\0132\026.RemoteRequestProtocol\"g\n\037Serialized" +
+ "TypedActorRefProtocol\022-\n\010actorRef\030\001 \002(\0132" +
+ "\033.SerializedActorRefProtocol\022\025\n\rinterfac" +
+ "eName\030\002 \002(\t\"r\n\017MessageProtocol\0225\n\023serial" +
+ "izationScheme\030\001 \002(\0162\030.SerializationSchem",
+ "eType\022\017\n\007message\030\002 \002(\014\022\027\n\017messageManifes" +
+ "t\030\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\t" +
+ "actorType\030\004 \002(\0162\n.ActorType\022/\n\016typedActo" +
+ "rInfo\030\005 \001(\0132\027.TypedActorInfoProtocol\022\n\n\002" +
+ "id\030\006 \001(\t\";\n\026TypedActorInfoProtocol\022\021\n\tin" +
+ "terface\030\001 \002(\t\022\016\n\006method\030\002 \002(\t\"\352\001\n\025Remote" +
+ "RequestProtocol\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" +
+ "(\0132\026.MetadataEntryProtocol\"\324\001\n\023RemoteRep" +
+ "lyProtocol\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.E" +
+ "xceptionProtocol\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.MetadataEntryProtoco" +
+ "l\")\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\tlifeCycle\030\001 \002(\0162\016.LifeCycleType\"1\n\017Add" +
+ "ressProtocol\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\013SC" +
+ "ALA_ACTOR\020\001\022\016\n\nJAVA_ACTOR\020\002\022\017\n\013TYPED_ACT" +
+ "OR\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_J" +
+ "SON\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\t" +
+ "PERMANENT\020\001\022\r\n\tTEMPORARY\020\002B-\n)se.scalabl",
+ "esolutions.akka.remote.protocolH\001"
};
com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -5782,16 +6547,32 @@ public final class RemoteProtocol {
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 =
+ internal_static_RemoteTypedActorRefProtocol_descriptor =
getDescriptor().getMessageTypes().get(1);
+ internal_static_RemoteTypedActorRefProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_RemoteTypedActorRefProtocol_descriptor,
+ new java.lang.String[] { "ActorRef", "InterfaceName", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteTypedActorRefProtocol.Builder.class);
+ internal_static_SerializedActorRefProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(2);
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_SerializedTypedActorRefProtocol_descriptor =
+ getDescriptor().getMessageTypes().get(3);
+ internal_static_SerializedTypedActorRefProtocol_fieldAccessorTable = new
+ com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+ internal_static_SerializedTypedActorRefProtocol_descriptor,
+ new java.lang.String[] { "ActorRef", "InterfaceName", },
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.class,
+ se.scalablesolutions.akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.Builder.class);
internal_static_MessageProtocol_descriptor =
- getDescriptor().getMessageTypes().get(2);
+ getDescriptor().getMessageTypes().get(4);
internal_static_MessageProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_MessageProtocol_descriptor,
@@ -5799,15 +6580,15 @@ public final class RemoteProtocol {
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);
+ getDescriptor().getMessageTypes().get(5);
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 =
- getDescriptor().getMessageTypes().get(4);
+ getDescriptor().getMessageTypes().get(6);
internal_static_TypedActorInfoProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_TypedActorInfoProtocol_descriptor,
@@ -5815,7 +6596,7 @@ public final class RemoteProtocol {
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);
+ getDescriptor().getMessageTypes().get(7);
internal_static_RemoteRequestProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RemoteRequestProtocol_descriptor,
@@ -5823,7 +6604,7 @@ public final class RemoteProtocol {
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);
+ getDescriptor().getMessageTypes().get(8);
internal_static_RemoteReplyProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_RemoteReplyProtocol_descriptor,
@@ -5831,7 +6612,7 @@ public final class RemoteProtocol {
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.RemoteReplyProtocol.Builder.class);
internal_static_UuidProtocol_descriptor =
- getDescriptor().getMessageTypes().get(7);
+ getDescriptor().getMessageTypes().get(9);
internal_static_UuidProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_UuidProtocol_descriptor,
@@ -5839,7 +6620,7 @@ public final class RemoteProtocol {
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);
+ getDescriptor().getMessageTypes().get(10);
internal_static_MetadataEntryProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_MetadataEntryProtocol_descriptor,
@@ -5847,7 +6628,7 @@ public final class RemoteProtocol {
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);
+ getDescriptor().getMessageTypes().get(11);
internal_static_LifeCycleProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_LifeCycleProtocol_descriptor,
@@ -5855,7 +6636,7 @@ public final class RemoteProtocol {
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.class,
se.scalablesolutions.akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder.class);
internal_static_AddressProtocol_descriptor =
- getDescriptor().getMessageTypes().get(10);
+ getDescriptor().getMessageTypes().get(12);
internal_static_AddressProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_AddressProtocol_descriptor,
@@ -5863,7 +6644,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(11);
+ getDescriptor().getMessageTypes().get(13);
internal_static_ExceptionProtocol_fieldAccessorTable = new
com.google.protobuf.GeneratedMessage.FieldAccessorTable(
internal_static_ExceptionProtocol_descriptor,
diff --git a/akka-remote/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto
index 203b505d68..e84ae9f18e 100644
--- a/akka-remote/src/main/protocol/RemoteProtocol.proto
+++ b/akka-remote/src/main/protocol/RemoteProtocol.proto
@@ -22,6 +22,15 @@ message RemoteActorRefProtocol {
optional uint64 timeout = 4;
}
+/**
+ * Defines a remote ActorRef that "remembers" and uses its original typed Actor instance
+ * on the original node.
+ */
+message RemoteTypedActorRefProtocol {
+ required RemoteActorRefProtocol actorRef = 1;
+ required string interfaceName = 2;
+}
+
/**
* Defines a fully serialized remote ActorRef (with serialized Actor instance)
* that is about to be instantiated on the remote node. It is fully disconnected
@@ -43,6 +52,16 @@ message SerializedActorRefProtocol {
repeated RemoteRequestProtocol messages = 13;
}
+/**
+ * Defines a fully serialized remote ActorRef (with serialized typed actor instance)
+ * that is about to be instantiated on the remote node. It is fully disconnected
+ * from its original host.
+ */
+message SerializedTypedActorRefProtocol {
+ required SerializedActorRefProtocol actorRef = 1;
+ required string interfaceName = 2;
+}
+
/**
* Defines a message.
*/
@@ -61,6 +80,7 @@ message ActorInfoProtocol {
required uint64 timeout = 3;
required ActorType actorType = 4;
optional TypedActorInfoProtocol typedActorInfo = 5;
+ optional string id = 6;
}
/**
diff --git a/akka-remote/src/main/scala/remote/RemoteServer.scala b/akka-remote/src/main/scala/remote/RemoteServer.scala
index 4bcd4861ff..b258c4867d 100644
--- a/akka-remote/src/main/scala/remote/RemoteServer.scala
+++ b/akka-remote/src/main/scala/remote/RemoteServer.scala
@@ -30,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.
@@ -66,6 +67,7 @@ object RemoteNode extends RemoteServer
* @author Jonas Bonér
*/
object RemoteServer {
+ val UUID_PREFIX = "uuid:"
val HOSTNAME = config.getString("akka.remote.server.hostname", "localhost")
val PORT = config.getInt("akka.remote.server.port", 9999)
@@ -122,18 +124,20 @@ object RemoteServer {
private class RemoteActorSet {
private[RemoteServer] val actors = new ConcurrentHashMap[String, ActorRef]
+ private[RemoteServer] val actorsByUuid = new ConcurrentHashMap[String, ActorRef]
private[RemoteServer] val typedActors = new ConcurrentHashMap[String, AnyRef]
+ private[RemoteServer] val typedActorsByUuid = new ConcurrentHashMap[String, AnyRef]
}
private val guard = new ReadWriteGuard
private val remoteActorSets = Map[Address, RemoteActorSet]()
private val remoteServers = Map[Address, RemoteServer]()
- private[akka] def registerActor(address: InetSocketAddress, uuid: String, actor: ActorRef) = guard.withWriteGuard {
- actorsFor(RemoteServer.Address(address.getHostName, address.getPort)).actors.put(uuid, actor)
+ private[akka] def registerActorByUuid(address: InetSocketAddress, uuid: String, actor: ActorRef) = guard.withWriteGuard {
+ actorsFor(RemoteServer.Address(address.getHostName, address.getPort)).actorsByUuid.put(uuid, actor)
}
- private[akka] def registerTypedActor(address: InetSocketAddress, uuid: String, typedActor: AnyRef) = guard.withWriteGuard {
+ private[akka] def registerTypedActorByUuid(address: InetSocketAddress, uuid: String, typedActor: AnyRef) = guard.withWriteGuard {
actorsFor(RemoteServer.Address(address.getHostName, address.getPort)).typedActors.put(uuid, typedActor)
}
@@ -191,6 +195,7 @@ case class RemoteServerClientDisconnected(@BeanProperty val server: RemoteServer
* @author Jonas Bonér
*/
class RemoteServer extends Logging with ListenerManagement {
+ import RemoteServer._
def name = "RemoteServer@" + hostname + ":" + port
private[akka] var address = RemoteServer.Address(RemoteServer.HOSTNAME,RemoteServer.PORT)
@@ -282,10 +287,11 @@ class RemoteServer extends Logging with ListenerManagement {
* @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)
+ log.debug("Registering server side remote typed actor [%s] with id [%s]", typedActor.getClass.getName, id)
+ if (id.startsWith(UUID_PREFIX)) {
+ registerTypedActor(id.substring(UUID_PREFIX.length), typedActor, typedActorsByUuid())
+ } else {
+ registerTypedActor(id, typedActor, typedActors())
}
}
@@ -300,12 +306,27 @@ class RemoteServer extends Logging with ListenerManagement {
* NOTE: If you use this method to register your remote actor then you must unregister the actor by this ID yourself.
*/
def register(id: String, actorRef: ActorRef): Unit = synchronized {
+ log.debug("Registering server side remote actor [%s] with id [%s]", actorRef.actorClass.getName, id)
+ if (id.startsWith(UUID_PREFIX)) {
+ register(id.substring(UUID_PREFIX.length), actorRef, actorsByUuid())
+ } else {
+ register(id, actorRef, actors())
+ }
+ }
+
+ private def register(id: String, actorRef: ActorRef, registry: ConcurrentHashMap[String, ActorRef]) {
if (_isRunning) {
- val actorMap = actors()
- if (!actorMap.contains(id)) {
+ if (!registry.contains(id)) {
if (!actorRef.isRunning) actorRef.start
- log.debug("Registering server side remote actor [%s] with id [%s]", actorRef.actorClass.getName, id)
- actorMap.put(id, actorRef)
+ registry.put(id, actorRef)
+ }
+ }
+ }
+
+ private def registerTypedActor(id: String, typedActor: AnyRef, registry: ConcurrentHashMap[String, AnyRef]) {
+ if (_isRunning) {
+ if (!registry.contains(id)) {
+ registry.put(id, typedActor)
}
}
}
@@ -318,7 +339,7 @@ class RemoteServer extends Logging with ListenerManagement {
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
+ if (actorRef.registeredInRemoteNodeDuringSerialization) actorsByUuid() remove actorRef.uuid
}
}
@@ -330,10 +351,15 @@ 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 actorMap = actors()
- val actorRef = actorMap get id
- actorMap remove id
- if (actorRef.registeredInRemoteNodeDuringSerialization) actorMap remove actorRef.uuid
+ if (id.startsWith(UUID_PREFIX)) {
+ actorsByUuid().remove(id.substring(UUID_PREFIX.length))
+ } else {
+ val actorRef = actors().get(id)
+ if (actorRef.registeredInRemoteNodeDuringSerialization) {
+ actorsByUuid() remove actorRef.uuid
+ }
+ actors() remove id
+ }
}
}
@@ -345,8 +371,11 @@ class RemoteServer extends Logging with ListenerManagement {
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)
+ if (id.startsWith(UUID_PREFIX)) {
+ typedActorsByUuid().remove(id.substring(UUID_PREFIX.length))
+ } else {
+ typedActors().remove(id)
+ }
}
}
@@ -354,8 +383,10 @@ class RemoteServer extends Logging with ListenerManagement {
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
+ private[akka] def actors() = RemoteServer.actorsFor(address).actors
+ private[akka] def actorsByUuid() = RemoteServer.actorsFor(address).actorsByUuid
+ private[akka] def typedActors() = RemoteServer.actorsFor(address).typedActors
+ private[akka] def typedActorsByUuid() = RemoteServer.actorsFor(address).typedActorsByUuid
}
object RemoteServerSslContext {
@@ -418,6 +449,7 @@ class RemoteServerHandler(
val openChannels: ChannelGroup,
val applicationLoader: Option[ClassLoader],
val server: RemoteServer) extends SimpleChannelUpstreamHandler with Logging {
+ import RemoteServer._
val AW_PROXY_PREFIX = "$$ProxiedByAW".intern
applicationLoader.foreach(MessageSerializer.setClassLoader(_))
@@ -476,11 +508,12 @@ class RemoteServerHandler(
private def handleRemoteRequestProtocol(request: RemoteRequestProtocol, channel: Channel) = {
log.debug("Received RemoteRequestProtocol[\n%s]", request.toString)
- val actorType = request.getActorInfo.getActorType
- if (actorType == SCALA_ACTOR) dispatchToActor(request, channel)
- else if (actorType == JAVA_ACTOR) throw new IllegalActorStateException("ActorType JAVA_ACTOR is currently not supported")
- else if (actorType == TYPED_ACTOR) dispatchToTypedActor(request, channel)
- else throw new IllegalActorStateException("Unknown ActorType [" + actorType + "]")
+ request.getActorInfo.getActorType match {
+ case SCALA_ACTOR => dispatchToActor(request, channel)
+ case TYPED_ACTOR => dispatchToTypedActor(request, channel)
+ case JAVA_ACTOR => throw new IllegalActorStateException("ActorType JAVA_ACTOR is currently not supported")
+ case other => throw new IllegalActorStateException("Unknown ActorType [" + other + "]")
+ }
}
private def dispatchToActor(request: RemoteRequestProtocol, channel: Channel) = {
@@ -498,27 +531,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.notifyListeners(RemoteServerError(e, server))
- }
+ override def onCompleteException(exception: Throwable) {
+ try {
+ channel.write(createErrorReplyMessage(exception, request, true))
+ } catch {
+ case e: Throwable =>
+ server.notifyListeners(RemoteServerError(e, server))
+ }
+ }
}
+ ))
}
}
@@ -555,32 +597,23 @@ class RemoteServerHandler(
}
}
- /**
- * 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
+ private def findActorById(id: String) : ActorRef = {
+ server.actors().get(id)
}
- /**
- * 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
+ private def findActorByUuid(uuid: String) : ActorRef = {
+ server.actorsByUuid().get(uuid)
}
+ private def findTypedActorById(id: String) : AnyRef = {
+ server.typedActors().get(id)
+ }
+
+ private def findTypedActorByUuid(uuid: String) : AnyRef = {
+ server.typedActorsByUuid().get(uuid)
+ }
+
+
/**
* Creates a new instance of the actor with name, uuid and timeout specified as arguments.
*
@@ -589,15 +622,18 @@ class RemoteServerHandler(
* Does not start the actor.
*/
private def createActor(actorInfo: ActorInfoProtocol): ActorRef = {
- val ids = actorInfo.getUuid.split(':')
- val uuid = ids(0)
- val id = ids(1)
+ val uuid = actorInfo.getUuid
+ val id = actorInfo.getId
val name = actorInfo.getTarget
val timeout = actorInfo.getTimeout
- val actorRefOrNull = findActorByIdOrUuid(id, uuid)
-
+ val actorRefOrNull = if (id.startsWith(UUID_PREFIX)) {
+ findActorByUuid(id.substring(UUID_PREFIX.length))
+ } else {
+ findActorById(id)
+ }
+
if (actorRefOrNull eq null) {
try {
log.info("Creating a new remote actor [%s:%s]", name, uuid)
@@ -620,11 +656,14 @@ class RemoteServerHandler(
}
private def createTypedActor(actorInfo: ActorInfoProtocol): AnyRef = {
- val ids = actorInfo.getUuid.split(':')
- val uuid = ids(0)
- val id = ids(1)
+ val uuid = actorInfo.getUuid
+ val id = actorInfo.getId
- val typedActorOrNull = findTypedActorByIdOrUUid(id, uuid)
+ val typedActorOrNull = if (id.startsWith(UUID_PREFIX)) {
+ findTypedActorByUuid(id.substring(UUID_PREFIX.length))
+ } else {
+ findTypedActorById(id)
+ }
if (typedActorOrNull eq null) {
val typedActorInfo = actorInfo.getTypedActorInfo
diff --git a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
index afebae8f3b..c9b443f4ec 100644
--- a/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
+++ b/akka-remote/src/main/scala/serialization/SerializationProtocol.scala
@@ -4,10 +4,10 @@
package se.scalablesolutions.akka.serialization
-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.dispatch.MessageInvocation
import se.scalablesolutions.akka.remote.{RemoteServer, RemoteRequestProtocolIdFactory, MessageSerializer}
import se.scalablesolutions.akka.remote.protocol.RemoteProtocol.{ActorType => ActorTypeProtocol, _}
import ActorTypeProtocol._
@@ -15,6 +15,7 @@ import se.scalablesolutions.akka.config.{AllForOneStrategy, OneForOneStrategy, F
import se.scalablesolutions.akka.config.ScalaConfig._
import com.google.protobuf.ByteString
+import se.scalablesolutions.akka.actor._
/**
* Type class definition for Actor Serialization
@@ -36,13 +37,14 @@ trait Format[T <: Actor] extends FromBinary[T] with ToBinary[T]
* Create a Format object with the client actor as the implementation of the type class
*
*
- * object BinaryFormatMyStatelessActor {
+ * object BinaryFormatMyStatelessActor {
* implicit object MyStatelessActorFormat extends StatelessActorFormat[MyStatelessActor]
* }
*
*/
trait StatelessActorFormat[T <: Actor] extends Format[T] {
def fromBinary(bytes: Array[Byte], act: T) = act
+
def toBinary(ac: T) = Array.empty[Byte]
}
@@ -53,16 +55,18 @@ trait StatelessActorFormat[T <: Actor] extends Format[T] {
* a serializer object
*
*
- * object BinaryFormatMyJavaSerializableActor {
- * implicit object MyJavaSerializableActorFormat extends SerializerBasedActorFormat[MyJavaSerializableActor] {
+ * object BinaryFormatMyJavaSerializableActor {
+ * implicit object MyJavaSerializableActorFormat extends SerializerBasedActorFormat[MyJavaSerializableActor] {
* val serializer = Serializer.Java
- * }
+ * }
* }
*
*/
trait SerializerBasedActorFormat[T <: Actor] extends Format[T] {
val serializer: Serializer
+
def fromBinary(bytes: Array[Byte], act: T) = serializer.fromBinary(bytes, Some(act.self.actorClass)).asInstanceOf[T]
+
def toBinary(ac: T) = serializer.toBinary(ac)
}
@@ -70,23 +74,22 @@ trait SerializerBasedActorFormat[T <: Actor] extends Format[T] {
* Module for local actor serialization.
*/
object ActorSerialization {
-
def fromBinary[T <: Actor](bytes: Array[Byte])(implicit format: Format[T]): ActorRef =
fromBinaryToLocalActorRef(bytes, format)
- def toBinary[T <: Actor](a: ActorRef)(implicit format: Format[T]): Array[Byte] =
- toSerializedActorRefProtocol(a, format).toByteArray
+ def toBinary[T <: Actor](a: ActorRef, serializeMailBox: Boolean = true)(implicit format: Format[T]): Array[Byte] =
+ toSerializedActorRefProtocol(a, format, serializeMailBox).toByteArray
// wrapper for implicits to be used by Java
def fromBinaryJ[T <: Actor](bytes: Array[Byte], format: Format[T]): ActorRef =
fromBinary(bytes)(format)
// wrapper for implicits to be used by Java
- def toBinaryJ[T <: Actor](a: ActorRef, format: Format[T]): Array[Byte] =
- toBinary(a)(format)
+ def toBinaryJ[T <: Actor](a: ActorRef, format: Format[T], srlMailBox: Boolean = true): Array[Byte] =
+ toBinary(a, srlMailBox)(format)
- private def toSerializedActorRefProtocol[T <: Actor](
- actorRef: ActorRef, format: Format[T]): SerializedActorRefProtocol = {
+ private[akka] def toSerializedActorRefProtocol[T <: Actor](
+ actorRef: ActorRef, format: Format[T], serializeMailBox: Boolean = true): SerializedActorRefProtocol = {
val lifeCycleProtocol: Option[LifeCycleProtocol] = {
def setScope(builder: LifeCycleProtocol.Builder, scope: Scope) = scope match {
case Permanent => builder.setLifeCycle(LifeCycleType.PERMANENT)
@@ -102,17 +105,40 @@ object ActorSerialization {
}
val originalAddress = AddressProtocol.newBuilder
- .setHostname(actorRef.homeAddress.getHostName)
- .setPort(actorRef.homeAddress.getPort)
- .build
+ .setHostname(actorRef.homeAddress.getHostName)
+ .setPort(actorRef.homeAddress.getPort)
+ .build
val builder = SerializedActorRefProtocol.newBuilder
- .setUuid(actorRef.uuid)
- .setId(actorRef.id)
- .setActorClassname(actorRef.actorClass.getName)
- .setOriginalAddress(originalAddress)
- .setIsTransactor(actorRef.isTransactor)
- .setTimeout(actorRef.timeout)
+ .setUuid(actorRef.uuid)
+ .setId(actorRef.id)
+ .setActorClassname(actorRef.actorClass.getName)
+ .setOriginalAddress(originalAddress)
+ .setIsTransactor(actorRef.isTransactor)
+ .setTimeout(actorRef.timeout)
+
+ if (serializeMailBox == true) {
+ val messages =
+ actorRef.mailbox match {
+ case q: java.util.Queue[MessageInvocation] =>
+ val l = new scala.collection.mutable.ListBuffer[MessageInvocation]
+ val it = q.iterator
+ while (it.hasNext == true) l += it.next
+ l
+ }
+
+ val requestProtocols =
+ messages.map(m =>
+ RemoteActorSerialization.createRemoteRequestProtocolBuilder(
+ actorRef,
+ m.message,
+ false,
+ actorRef.getSender,
+ None,
+ ActorType.ScalaActor).build)
+
+ requestProtocols.foreach(rp => builder.addMessages(rp))
+ }
actorRef.receiveTimeout.foreach(builder.setReceiveTimeout(_))
builder.setActorInstance(ByteString.copyFrom(format.toBinary(actorRef.actor.asInstanceOf[T])))
@@ -126,33 +152,33 @@ object ActorSerialization {
private def fromBinaryToLocalActorRef[T <: Actor](bytes: Array[Byte], format: Format[T]): ActorRef =
fromProtobufToLocalActorRef(SerializedActorRefProtocol.newBuilder.mergeFrom(bytes).build, format, None)
- private def fromProtobufToLocalActorRef[T <: Actor](
- protocol: SerializedActorRefProtocol, format: Format[T], loader: Option[ClassLoader]): ActorRef = {
+ private[akka] def fromProtobufToLocalActorRef[T <: Actor](
+ protocol: SerializedActorRefProtocol, format: Format[T], loader: Option[ClassLoader]): ActorRef = {
Actor.log.debug("Deserializing SerializedActorRefProtocol to LocalActorRef:\n" + protocol)
val serializer =
- if (format.isInstanceOf[SerializerBasedActorFormat[_]])
- Some(format.asInstanceOf[SerializerBasedActorFormat[_]].serializer)
- else None
+ if (format.isInstanceOf[SerializerBasedActorFormat[_]])
+ Some(format.asInstanceOf[SerializerBasedActorFormat[_]].serializer)
+ else None
val lifeCycle =
- if (protocol.hasLifeCycle) {
- val lifeCycleProtocol = protocol.getLifeCycle
- Some(if (lifeCycleProtocol.getLifeCycle == LifeCycleType.PERMANENT) LifeCycle(Permanent)
- else if (lifeCycleProtocol.getLifeCycle == LifeCycleType.TEMPORARY) LifeCycle(Temporary)
- else throw new IllegalActorStateException("LifeCycle type is not valid: " + lifeCycleProtocol.getLifeCycle))
- } else None
+ if (protocol.hasLifeCycle) {
+ val lifeCycleProtocol = protocol.getLifeCycle
+ Some(if (lifeCycleProtocol.getLifeCycle == LifeCycleType.PERMANENT) LifeCycle(Permanent)
+ else if (lifeCycleProtocol.getLifeCycle == LifeCycleType.TEMPORARY) LifeCycle(Temporary)
+ else throw new IllegalActorStateException("LifeCycle type is not valid: " + lifeCycleProtocol.getLifeCycle))
+ } else None
val supervisor =
- if (protocol.hasSupervisor)
- Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader))
- else None
+ if (protocol.hasSupervisor)
+ Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader))
+ else None
val hotswap =
- if (serializer.isDefined && protocol.hasHotswapStack) Some(serializer.get
+ if (serializer.isDefined && protocol.hasHotswapStack) Some(serializer.get
.fromBinary(protocol.getHotswapStack.toByteArray, Some(classOf[PartialFunction[Any, Unit]]))
.asInstanceOf[PartialFunction[Any, Unit]])
- else None
+ else None
val classLoader = loader.getOrElse(getClass.getClassLoader)
@@ -194,9 +220,9 @@ object RemoteActorSerialization {
def fromBinaryToRemoteActorRef(bytes: Array[Byte]): ActorRef =
fromProtobufToRemoteActorRef(RemoteActorRefProtocol.newBuilder.mergeFrom(bytes).build, None)
- /**
- * Deserializes a byte array (Array[Byte]) into an RemoteActorRef instance.
- */
+ /**
+ * Deserializes a byte array (Array[Byte]) into an RemoteActorRef instance.
+ */
def fromBinaryToRemoteActorRef(bytes: Array[Byte], loader: ClassLoader): ActorRef =
fromProtobufToRemoteActorRef(RemoteActorRefProtocol.newBuilder.mergeFrom(bytes).build, Some(loader))
@@ -225,39 +251,41 @@ object RemoteActorSerialization {
if (!registeredInRemoteNodeDuringSerialization) {
Actor.log.debug("Register serialized Actor [%s] as remote @ [%s:%s]", actorClass.getName, host, port)
RemoteServer.getOrCreateServer(homeAddress)
- RemoteServer.registerActor(homeAddress, uuid, ar)
+ RemoteServer.registerActorByUuid(homeAddress, uuid, ar)
registeredInRemoteNodeDuringSerialization = true
}
RemoteActorRefProtocol.newBuilder
- .setUuid(uuid + ":" + id)
- .setActorClassname(actorClass.getName)
- .setHomeAddress(AddressProtocol.newBuilder.setHostname(host).setPort(port).build)
- .setTimeout(timeout)
- .build
+ .setUuid(uuid)
+ .setActorClassname(actorClass.getName)
+ .setHomeAddress(AddressProtocol.newBuilder.setHostname(host).setPort(port).build)
+ .setTimeout(timeout)
+ .build
}
def createRemoteRequestProtocolBuilder(
- actorRef: ActorRef,
- message: Any,
- isOneWay: Boolean,
- senderOption: Option[ActorRef],
- typedActorInfo: Option[Tuple2[String, String]],
- actorType: ActorType):
- RemoteRequestProtocol.Builder = {
+ actorRef: ActorRef,
+ message: Any,
+ isOneWay: Boolean,
+ senderOption: Option[ActorRef],
+ typedActorInfo: Option[Tuple2[String, String]],
+ actorType: ActorType):
+ RemoteRequestProtocol.Builder = {
import actorRef._
val actorInfoBuilder = ActorInfoProtocol.newBuilder
- .setUuid(uuid + ":" + actorRef.id)
+ .setUuid(uuid)
+ .setId(actorRef.id)
.setTarget(actorClassName)
.setTimeout(timeout)
- typedActorInfo.foreach { typedActor =>
- actorInfoBuilder.setTypedActorInfo(
- TypedActorInfoProtocol.newBuilder
- .setInterface(typedActor._1)
- .setMethod(typedActor._2)
- .build)
+ typedActorInfo.foreach {
+ typedActor =>
+ actorInfoBuilder.setTypedActorInfo(
+ TypedActorInfoProtocol.newBuilder
+ .setInterface(typedActor._1)
+ .setMethod(typedActor._2)
+ .build)
}
actorType match {
@@ -275,10 +303,110 @@ object RemoteActorSerialization {
val id = registerSupervisorAsRemoteActor
if (id.isDefined) requestBuilder.setSupervisorUuid(id.get)
- senderOption.foreach { sender =>
- RemoteServer.getOrCreateServer(sender.homeAddress).register(sender.uuid, sender)
- requestBuilder.setSender(toRemoteActorRefProtocol(sender))
+ senderOption.foreach {
+ sender =>
+ RemoteServer.getOrCreateServer(sender.homeAddress).register(sender.uuid, sender)
+ requestBuilder.setSender(toRemoteActorRefProtocol(sender))
}
requestBuilder
}
+
+
+}
+
+
+/**
+ * Module for local typed actor serialization.
+ */
+object TypedActorSerialization {
+
+ def fromBinary[T <: Actor, U <: AnyRef](bytes: Array[Byte])(implicit format: Format[T]): U =
+ fromBinaryToLocalTypedActorRef(bytes, format)
+
+ def toBinary[T <: Actor](proxy: AnyRef)(implicit format: Format[T]): Array[Byte] = {
+ toSerializedTypedActorRefProtocol(proxy, format).toByteArray
+ }
+
+ // wrapper for implicits to be used by Java
+ def fromBinaryJ[T <: Actor, U <: AnyRef](bytes: Array[Byte], format: Format[T]): U =
+ fromBinary(bytes)(format)
+
+ // wrapper for implicits to be used by Java
+ def toBinaryJ[T <: Actor](a: AnyRef, format: Format[T]): Array[Byte] =
+ toBinary(a)(format)
+
+ private def toSerializedTypedActorRefProtocol[T <: Actor](
+ proxy: AnyRef, format: Format[T]): SerializedTypedActorRefProtocol = {
+
+ val init = AspectInitRegistry.initFor(proxy)
+ if (init == null) throw new IllegalArgumentException("Proxy for typed actor could not be found in AspectInitRegistry.")
+
+ SerializedTypedActorRefProtocol.newBuilder
+ .setActorRef(ActorSerialization.toSerializedActorRefProtocol(init.actorRef, format))
+ .setInterfaceName(init.interfaceClass.getName)
+ .build
+ }
+
+ private def fromBinaryToLocalTypedActorRef[T <: Actor, U <: AnyRef](bytes: Array[Byte], format: Format[T]): U =
+ fromProtobufToLocalTypedActorRef(SerializedTypedActorRefProtocol.newBuilder.mergeFrom(bytes).build, format, None)
+
+ private def fromProtobufToLocalTypedActorRef[T <: Actor, U <: AnyRef](
+ protocol: SerializedTypedActorRefProtocol, format: Format[T], loader: Option[ClassLoader]): U = {
+ Actor.log.debug("Deserializing SerializedTypedActorRefProtocol to LocalActorRef:\n" + protocol)
+ val actorRef = ActorSerialization.fromProtobufToLocalActorRef(protocol.getActorRef, format, loader)
+ val intfClass = toClass(loader, protocol.getInterfaceName)
+ TypedActor.newInstance(intfClass, actorRef).asInstanceOf[U]
+ }
+
+ private[akka] def toClass[U <: AnyRef](loader: Option[ClassLoader], name: String): Class[U] = {
+ val classLoader = loader.getOrElse(getClass.getClassLoader)
+ val clazz = classLoader.loadClass(name)
+ clazz.asInstanceOf[Class[U]]
+ }
+}
+
+/**
+ * Module for remote typed actor serialization.
+ */
+object RemoteTypedActorSerialization {
+ /**
+ * Deserializes a byte array (Array[Byte]) into an RemoteActorRef instance.
+ */
+ def fromBinaryToRemoteTypedActorRef[T <: AnyRef](bytes: Array[Byte]): T =
+ fromProtobufToRemoteTypedActorRef(RemoteTypedActorRefProtocol.newBuilder.mergeFrom(bytes).build, None)
+
+ /**
+ * Deserializes a byte array (Array[Byte]) into a AW RemoteActorRef proxy.
+ */
+ def fromBinaryToRemoteTypedActorRef[T <: AnyRef](bytes: Array[Byte], loader: ClassLoader): T =
+ fromProtobufToRemoteTypedActorRef(RemoteTypedActorRefProtocol.newBuilder.mergeFrom(bytes).build, Some(loader))
+
+ /**
+ * Serialize as AW RemoteActorRef proxy.
+ */
+ def toBinary[T <: Actor](proxy: AnyRef): Array[Byte] = {
+ toRemoteTypedActorRefProtocol(proxy).toByteArray
+ }
+
+ /**
+ * Deserializes a RemoteTypedActorRefProtocol Protocol Buffers (protobuf) Message into AW RemoteActorRef proxy.
+ */
+ private[akka] def fromProtobufToRemoteTypedActorRef[T](protocol: RemoteTypedActorRefProtocol, loader: Option[ClassLoader]): T = {
+ Actor.log.debug("Deserializing RemoteTypedActorRefProtocol to AW RemoteActorRef proxy:\n" + protocol)
+ val actorRef = RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getActorRef, loader)
+ val intfClass = TypedActorSerialization.toClass(loader, protocol.getInterfaceName)
+ TypedActor.createProxyForRemoteActorRef(intfClass, actorRef).asInstanceOf[T]
+ }
+
+ /**
+ * Serializes the AW TypedActor proxy into a Protocol Buffers (protobuf) Message.
+ */
+ def toRemoteTypedActorRefProtocol(proxy: AnyRef): RemoteTypedActorRefProtocol = {
+ val init = AspectInitRegistry.initFor(proxy)
+ RemoteTypedActorRefProtocol.newBuilder
+ .setActorRef(RemoteActorSerialization.toRemoteActorRefProtocol(init.actorRef))
+ .setInterfaceName(init.interfaceClass.getName)
+ .build
+ }
+
}
diff --git a/akka-remote/src/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 3c8a60aecc..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 postStop = 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 postStop = 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/scala/remote/ServerInitiatedRemoteActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
index 59f122c656..fbf723ece5 100644
--- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteActorSpec.scala
@@ -79,7 +79,6 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite {
}
}
-
@Test
def shouldSendWithBang {
val actor = RemoteClient.actorFor(
@@ -178,5 +177,41 @@ class ServerInitiatedRemoteActorSpec extends JUnitSuite {
assert(actor2.id == actor3.id)
}
+ @Test
+ def shouldFindActorByUuid {
+ val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
+ val actor2 = actorOf[RemoteActorSpecActorUnidirectional]
+ server.register("uuid:" + actor1.uuid, actor1)
+ server.register("my-service", actor2)
+
+ val ref1 = RemoteClient.actorFor("uuid:" + actor1.uuid, HOSTNAME, PORT)
+ val ref2 = RemoteClient.actorFor("my-service", HOSTNAME, PORT)
+
+ ref1 ! "OneWay"
+ assert(RemoteActorSpecActorUnidirectional.latch.await(1, TimeUnit.SECONDS))
+ ref1.stop
+ ref2 ! "OneWay"
+ ref2.stop
+
+ }
+
+ @Test
+ def shouldRegisterAndUnregister {
+ val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
+ server.register("my-service-1", actor1)
+ assert(server.actors().get("my-service-1") != null, "actor registered")
+ server.unregister("my-service-1")
+ assert(server.actors().get("my-service-1") == null, "actor unregistered")
+ }
+
+ @Test
+ def shouldRegisterAndUnregisterByUuid {
+ val actor1 = actorOf[RemoteActorSpecActorUnidirectional]
+ server.register("uuid:" + actor1.uuid, actor1)
+ assert(server.actorsByUuid().get(actor1.uuid) != null, "actor registered")
+ server.unregister("uuid:" + actor1.uuid)
+ assert(server.actorsByUuid().get(actor1.uuid) == null, "actor unregistered")
+ }
+
}
diff --git a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala
index b800fbf2c3..f50c3e6652 100644
--- a/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala
+++ b/akka-remote/src/test/scala/remote/ServerInitiatedRemoteTypedActorSpec.scala
@@ -103,9 +103,34 @@ class ServerInitiatedRemoteTypedActorSpec extends
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)
+ assert(server.typedActors().get("my-test-service") != null, "typed actor registered")
server.unregisterTypedActor("my-test-service")
- assert(server.typedActors().get("my-test-service") == null)
+ assert(server.typedActors().get("my-test-service") == null, "typed actor unregistered")
+ }
+
+ it("should register and unregister typed actors by uuid") {
+ val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
+ val init = AspectInitRegistry.initFor(typedActor)
+ val uuid = "uuid:" + init.actorRef.uuid
+ server.registerTypedActor(uuid, typedActor)
+ assert(server.typedActorsByUuid().get(init.actorRef.uuid) != null, "typed actor registered")
+ server.unregisterTypedActor(uuid)
+ assert(server.typedActorsByUuid().get(init.actorRef.uuid) == null, "typed actor unregistered")
+ }
+
+ it("should find typed actors by uuid") {
+ val typedActor = TypedActor.newInstance(classOf[RemoteTypedActorOne], classOf[RemoteTypedActorOneImpl], 1000)
+ val init = AspectInitRegistry.initFor(typedActor)
+ val uuid = "uuid:" + init.actorRef.uuid
+ server.registerTypedActor(uuid, typedActor)
+ assert(server.typedActorsByUuid().get(init.actorRef.uuid) != null, "typed actor registered")
+
+ val actor = RemoteClient.typedActorFor(classOf[RemoteTypedActorOne], uuid, HOSTNAME, PORT)
+ expect("oneway") {
+ actor.oneWay
+ oneWayLog.poll(5, TimeUnit.SECONDS)
+ }
+
}
}
}
diff --git a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala
index 7e8babe168..832a655c22 100644
--- a/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala
+++ b/akka-remote/src/test/scala/serialization/SerializableTypeClassActorSpec.scala
@@ -127,9 +127,16 @@ class SerializableTypeClassActorSpec extends
(actor1 ! "hello")
(actor1 ! "hello")
(actor1 ! "hello")
+ actor1.mailboxSize should be > (0)
val actor2 = fromBinary(toBinary(actor1))
Thread.sleep(1000)
+ actor2.mailboxSize should be > (0)
(actor2 !! "hello-reply").getOrElse("_") should equal("world")
+
+ val actor3 = fromBinary(toBinary(actor1, false))
+ Thread.sleep(1000)
+ actor3.mailboxSize should equal(0)
+ (actor3 !! "hello-reply").getOrElse("_") should equal("world")
}
}
}
diff --git a/akka-remote/src/test/scala/serialization/Ticket435Spec.scala b/akka-remote/src/test/scala/serialization/Ticket435Spec.scala
new file mode 100644
index 0000000000..ed175ea0ad
--- /dev/null
+++ b/akka-remote/src/test/scala/serialization/Ticket435Spec.scala
@@ -0,0 +1,126 @@
+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._
+import se.scalablesolutions.akka.actor._
+import ActorSerialization._
+import Actor._
+
+@RunWith(classOf[JUnitRunner])
+class Ticket435Spec extends
+ Spec with
+ ShouldMatchers with
+ BeforeAndAfterAll {
+
+ object BinaryFormatMyStatefulActor {
+ implicit object MyStatefulActorFormat extends Format[MyStatefulActor] {
+ def fromBinary(bytes: Array[Byte], act: MyStatefulActor) = {
+ val p = Serializer.Protobuf.fromBinary(bytes, Some(classOf[ProtobufProtocol.Counter])).asInstanceOf[ProtobufProtocol.Counter]
+ act.count = p.getCount
+ act
+ }
+ def toBinary(ac: MyStatefulActor) =
+ ProtobufProtocol.Counter.newBuilder.setCount(ac.count).build.toByteArray
+ }
+ }
+
+ object BinaryFormatMyStatelessActorWithMessagesInMailbox {
+ implicit object MyStatelessActorFormat extends StatelessActorFormat[MyStatelessActorWithMessagesInMailbox]
+ }
+
+ describe("Serializable actor") {
+
+ it("should be able to serialize and deserialize a stateless actor with messages in mailbox") {
+ import BinaryFormatMyStatelessActorWithMessagesInMailbox._
+
+ val actor1 = actorOf[MyStatelessActorWithMessagesInMailbox].start
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ actor1.mailboxSize should be > (0)
+ val actor2 = fromBinary(toBinary(actor1))
+ Thread.sleep(1000)
+ actor2.mailboxSize should be > (0)
+ (actor2 !! "hello-reply").getOrElse("_") should equal("world")
+
+ val actor3 = fromBinary(toBinary(actor1, false))
+ Thread.sleep(1000)
+ actor3.mailboxSize should equal(0)
+ (actor3 !! "hello-reply").getOrElse("_") should equal("world")
+ }
+
+ it("should serialize the mailbox optionally") {
+ import BinaryFormatMyStatelessActorWithMessagesInMailbox._
+
+ val actor1 = actorOf[MyStatelessActorWithMessagesInMailbox].start
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ (actor1 ! "hello")
+ actor1.mailboxSize should be > (0)
+
+ val actor2 = fromBinary(toBinary(actor1, false))
+ Thread.sleep(1000)
+ actor2.mailboxSize should equal(0)
+ (actor2 !! "hello-reply").getOrElse("_") should equal("world")
+ }
+
+ it("should be able to serialize and deserialize a stateful actor with messages in mailbox") {
+ import BinaryFormatMyStatefulActor._
+
+ val actor1 = actorOf[MyStatefulActor].start
+ (actor1 ! "hi")
+ (actor1 ! "hi")
+ (actor1 ! "hi")
+ (actor1 ! "hi")
+ (actor1 ! "hi")
+ (actor1 ! "hi")
+ (actor1 ! "hi")
+ (actor1 ! "hi")
+ (actor1 ! "hi")
+ (actor1 ! "hi")
+ actor1.mailboxSize should be > (0)
+ val actor2 = fromBinary(toBinary(actor1))
+ Thread.sleep(1000)
+ actor2.mailboxSize should be > (0)
+ (actor2 !! "hello").getOrElse("_") should equal("world 1")
+
+ val actor3 = fromBinary(toBinary(actor1, false))
+ Thread.sleep(1000)
+ actor3.mailboxSize should equal(0)
+ (actor3 !! "hello").getOrElse("_") should equal("world 1")
+ }
+ }
+}
+
+class MyStatefulActor extends Actor {
+ var count = 0
+
+ def receive = {
+ case "hi" =>
+ println("# messages in mailbox " + self.mailboxSize)
+ Thread.sleep(500)
+ case "hello" =>
+ count = count + 1
+ self.reply("world " + count)
+ }
+}
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/TypedActorSerializationSpec.scala b/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala
new file mode 100644
index 0000000000..ccf4d05f7f
--- /dev/null
+++ b/akka-remote/src/test/scala/serialization/TypedActorSerializationSpec.scala
@@ -0,0 +1,166 @@
+/**
+ * Copyright (C) 2009-2010 Scalable Solutions AB
+ */
+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._
+import se.scalablesolutions.akka.actor._
+
+import TypedActorSerialization._
+import Actor._
+import se.scalablesolutions.akka.remote.{RemoteClient, RemoteServer}
+import se.scalablesolutions.akka.actor.remote.ServerInitiatedRemoteActorSpec.RemoteActorSpecActorUnidirectional
+
+@RunWith(classOf[JUnitRunner])
+class TypedActorSerializationSpec extends
+ Spec with
+ ShouldMatchers with
+ BeforeAndAfterAll {
+
+ var server1: RemoteServer = null
+ var typedActor: MyTypedActor = null
+
+ override def beforeAll = {
+ server1 = new RemoteServer().start("localhost", 9991)
+ typedActor = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyTypedActorImpl], 1000)
+ server1.registerTypedActor("typed-actor-service", typedActor)
+ Thread.sleep(1000)
+ }
+
+ // make sure the servers shutdown cleanly after the test has finished
+ override def afterAll = {
+ try {
+ TypedActor.stop(typedActor)
+ server1.shutdown
+ RemoteClient.shutdownAll
+ Thread.sleep(1000)
+ } catch {
+ case e => ()
+ }
+ }
+
+ object MyTypedStatelessActorFormat extends StatelessActorFormat[MyStatelessTypedActorImpl]
+
+ class MyTypedActorFormat extends Format[MyTypedActorImpl] {
+ def fromBinary(bytes: Array[Byte], act: MyTypedActorImpl) = {
+ val p = Serializer.Protobuf.fromBinary(bytes, Some(classOf[ProtobufProtocol.Counter])).asInstanceOf[ProtobufProtocol.Counter]
+ act.count = p.getCount
+ act
+ }
+ def toBinary(ac: MyTypedActorImpl) =
+ ProtobufProtocol.Counter.newBuilder.setCount(ac.count).build.toByteArray
+ }
+
+ class MyTypedActorWithDualCounterFormat extends Format[MyTypedActorWithDualCounter] {
+ def fromBinary(bytes: Array[Byte], act: MyTypedActorWithDualCounter) = {
+ val p = Serializer.Protobuf.fromBinary(bytes, Some(classOf[ProtobufProtocol.DualCounter])).asInstanceOf[ProtobufProtocol.DualCounter]
+ act.count1 = p.getCount1
+ act.count2 = p.getCount2
+ act
+ }
+ def toBinary(ac: MyTypedActorWithDualCounter) =
+ ProtobufProtocol.DualCounter.newBuilder.setCount1(ac.count1).setCount2(ac.count2).build.toByteArray
+ }
+
+
+ describe("Serializable typed actor") {
+
+ it("should be able to serialize and de-serialize a stateless typed actor") {
+ val typedActor1 = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyStatelessTypedActorImpl], 1000)
+ typedActor1.requestReply("hello") should equal("world")
+ typedActor1.requestReply("hello") should equal("world")
+
+ val bytes = toBinaryJ(typedActor1, MyTypedStatelessActorFormat)
+ val typedActor2: MyTypedActor = fromBinaryJ(bytes, MyTypedStatelessActorFormat)
+ typedActor2.requestReply("hello") should equal("world")
+ }
+
+ it("should be able to serialize and de-serialize a stateful typed actor") {
+ val typedActor1 = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyTypedActorImpl], 1000)
+ typedActor1.requestReply("hello") should equal("world 1")
+ typedActor1.requestReply("scala") should equal("hello scala 2")
+
+ val f = new MyTypedActorFormat
+ val bytes = toBinaryJ(typedActor1, f)
+ val typedActor2: MyTypedActor = fromBinaryJ(bytes, f)
+ typedActor2.requestReply("hello") should equal("world 3")
+ }
+
+ it("should be able to serialize and de-serialize a stateful typed actor with compound state") {
+ val typedActor1 = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyTypedActorWithDualCounter], 1000)
+ typedActor1.requestReply("hello") should equal("world 1 1")
+ typedActor1.requestReply("hello") should equal("world 2 2")
+
+ val f = new MyTypedActorWithDualCounterFormat
+ val bytes = toBinaryJ(typedActor1, f)
+ val typedActor2: MyTypedActor = fromBinaryJ(bytes, f)
+ typedActor2.requestReply("hello") should equal("world 3 3")
+ }
+
+ it("should be able to serialize a local yped actor ref to a remote typed actor ref proxy") {
+ val typedActor1 = TypedActor.newInstance(classOf[MyTypedActor], classOf[MyStatelessTypedActorImpl], 1000)
+ typedActor1.requestReply("hello") should equal("world")
+ typedActor1.requestReply("hello") should equal("world")
+
+ val bytes = RemoteTypedActorSerialization.toBinary(typedActor1)
+ val typedActor2: MyTypedActor = RemoteTypedActorSerialization.fromBinaryToRemoteTypedActorRef(bytes)
+ typedActor1.requestReply("hello") should equal("world")
+ }
+ }
+}
+
+
+trait MyTypedActor {
+ def requestReply(s: String) : String
+ def oneWay() : Unit
+}
+
+class MyTypedActorImpl extends TypedActor with MyTypedActor {
+ var count = 0
+
+ override def oneWay() {
+ println("got oneWay message")
+ }
+
+ override def requestReply(message: String) : String = {
+ count = count + 1
+ if (message == "hello") {
+ "world " + count
+ } else ("hello " + message + " " + count)
+ }
+}
+
+class MyTypedActorWithDualCounter extends TypedActor with MyTypedActor {
+ var count1 = 0
+ var count2 = 0
+
+ override def oneWay() {
+ println("got oneWay message")
+ }
+
+ override def requestReply(message: String) : String = {
+ count1 = count1 + 1
+ count2 = count2 + 1
+
+ if (message == "hello") {
+ "world " + count1 + " " + count2
+ } else ("hello " + message + " " + count1 + " " + count2)
+ }
+}
+
+class MyStatelessTypedActorImpl extends TypedActor with MyTypedActor {
+
+ override def oneWay() {
+ println("got oneWay message")
+ }
+
+ override def requestReply(message: String) : String = {
+ if (message == "hello") "world" else ("hello " + message)
+ }
+}
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-typed-actor/src/main/scala/actor/TypedActor.scala b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
index c3457cb43b..f6cf2e2337 100644
--- a/akka-typed-actor/src/main/scala/actor/TypedActor.scala
+++ b/akka-typed-actor/src/main/scala/actor/TypedActor.scala
@@ -390,11 +390,22 @@ object TypedActor extends Logging {
if (config._messageDispatcher.isDefined) actorRef.dispatcher = config._messageDispatcher.get
if (config._threadBasedDispatcher.isDefined) actorRef.dispatcher = Dispatchers.newThreadBasedDispatcher(actorRef)
if (config._host.isDefined) actorRef.makeRemote(config._host.get)
+ actorRef.timeout = config.timeout
AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, config._host, config.timeout))
actorRef.start
proxy.asInstanceOf[T]
}
+ private[akka] def newInstance[T](intfClass: Class[T], actorRef: ActorRef): T = {
+ if (!actorRef.actorInstance.get.isInstanceOf[TypedActor]) throw new IllegalArgumentException("ActorRef is not a ref to a typed actor")
+ val typedActor = actorRef.actorInstance.get.asInstanceOf[TypedActor]
+ val proxy = Proxy.newInstance(Array(intfClass), Array(typedActor), true, false)
+ typedActor.initialize(proxy)
+ AspectInitRegistry.register(proxy, AspectInit(intfClass, typedActor, actorRef, actorRef.remoteAddress, actorRef.timeout))
+ actorRef.start
+ proxy.asInstanceOf[T]
+ }
+
private[akka] def newInstance[T](intfClass: Class[T], targetClass: Class[_],
remoteAddress: Option[InetSocketAddress], timeout: Long): T = {
val actorRef = actorOf(newTypedActor(targetClass))
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 9a21af06da..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)
diff --git a/config/akka-reference.conf b/config/akka-reference.conf
index 8758138df7..e6ebae655a 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,6 +45,7 @@ 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
diff --git a/embedded-repo/org/scala-tools/time/2.8.0-0.2-SNAPSHOT/time-2.8.0-0.2-SNAPSHOT.jar b/embedded-repo/org/scala-tools/time/2.8.0-0.2-SNAPSHOT/time-2.8.0-0.2-SNAPSHOT.jar
deleted file mode 100644
index 038768fe14..0000000000
Binary files a/embedded-repo/org/scala-tools/time/2.8.0-0.2-SNAPSHOT/time-2.8.0-0.2-SNAPSHOT.jar and /dev/null differ
diff --git a/embedded-repo/org/scala-tools/time/2.8.0-0.2-SNAPSHOT/time-2.8.0-0.2-SNAPSHOT.pom b/embedded-repo/org/scala-tools/time/2.8.0-0.2-SNAPSHOT/time-2.8.0-0.2-SNAPSHOT.pom
deleted file mode 100644
index fc1cf3406e..0000000000
--- a/embedded-repo/org/scala-tools/time/2.8.0-0.2-SNAPSHOT/time-2.8.0-0.2-SNAPSHOT.pom
+++ /dev/null
@@ -1,8 +0,0 @@
-
-
- 4.0.0
- org.scala-tools
- time
- 2.8.0-0.2-SNAPSHOT
- jar
-
\ No newline at end of file
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.7-2.8.0/sjson-0.7-2.8.0.jar b/embedded-repo/sjson/json/sjson/0.8-2.8.0/sjson-0.8-2.8.0.jar
similarity index 54%
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-2.8.0/sjson-0.8-2.8.0.jar
index e8700f11d3..7dc4a9f66d 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-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/project/build/AkkaProject.scala b/project/build/AkkaProject.scala
index 499440a741..c6ccd783f3 100644
--- a/project/build/AkkaProject.scala
+++ b/project/build/AkkaProject.scala
@@ -1,4 +1,4 @@
-/*---------------------------------------------------------------------------\
+ /*---------------------------------------------------------------------------\
| Copyright (C) 2009-2010 Scalable Solutions AB |
\---------------------------------------------------------------------------*/
@@ -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")
@@ -49,7 +51,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val JavaNetRepo = MavenRepository("java.net Repo", "http://download.java.net/maven/2")
lazy val SonatypeSnapshotRepo = MavenRepository("Sonatype OSS Repo", "http://oss.sonatype.org/content/repositories/releases")
lazy val SunJDMKRepo = MavenRepository("Sun JDMK Repo", "http://wp5.e-taxonomy.eu/cdmlib/mavenrepo")
- lazy val CasbahRepoReleases = MavenRepository("Casbah Release Repo", "http://repo.bumnetworks.com/releases")
lazy val ClojarsRepo = MavenRepository("Clojars Repo", "http://clojars.org/repo")
lazy val OracleRepo = MavenRepository("Oracle Repo", "http://download.oracle.com/maven")
}
@@ -78,7 +79,8 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
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 casbahRelease = ModuleConfiguration("com.novus",CasbahRepoReleases)
+ lazy val casbahModuleConfig = ModuleConfiguration("com.novus", CasbahRepo)
+ lazy val timeModuleConfig = ModuleConfiguration("org.scala-tools", "time", CasbahSnapshotRepo)
lazy val voldemortModuleConfig = ModuleConfiguration("voldemort", ClojarsRepo)
lazy val sleepycatModuleConfig = ModuleConfiguration("com.sleepycat", OracleRepo)
lazy val embeddedRepo = EmbeddedRepo // This is the only exception, because the embedded repo is fast!
@@ -174,8 +176,6 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val casbah = "com.novus" % "casbah_2.8.0" % "1.0.8.5" % "compile"
- lazy val time = "org.scala-tools" % "time" % "2.8.0-SNAPSHOT-0.2-SNAPSHOT" % "compile"
-
lazy val multiverse = "org.multiverse" % "multiverse-alpha" % MULTIVERSE_VERSION % "compile" intransitive
lazy val netty = "org.jboss.netty" % "netty" % "3.2.2.Final" % "compile"
@@ -190,7 +190,7 @@ class AkkaParentProject(info: ProjectInfo) extends DefaultProject(info) {
lazy val sbinary = "sbinary" % "sbinary" % "2.8.0-0.3.1" % "compile"
- lazy val sjson = "sjson.json" % "sjson" % "0.8-SNAPSHOT-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"