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 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"