From 04efc440d662d8941153051436593ca8d1eb91d5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jonas=20Bon=C3=A9r?= Date: Tue, 7 Jun 2011 11:10:29 -0700 Subject: [PATCH] 1. Made LocalActorRef aware of replication 2. Added configuration for transaction log replication 3. Added replication schemes WriteThrough and WriteBehind 4. Refactored serializer creation and lookup in Actor.scala 5. Extended network protocol with replication strategy 6. Added BookKeeper management to tests 7. Improved logging and error messages 8. Removed ReplicatedActorRef 9. Added snapshot management to TransactionLog MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Signed-off-by: Jonas Bonér --- .../scala/akka/actor/actor/DeployerSpec.scala | 4 +- .../src/main/scala/akka/actor/Actor.scala | 107 +- .../src/main/scala/akka/actor/ActorRef.scala | 54 +- .../src/main/scala/akka/actor/Deployer.scala | 60 +- .../scala/akka/cluster/ClusterInterface.scala | 66 +- .../scala/akka/util/ReflectiveAccess.scala | 31 + .../java/akka/cluster/ClusterProtocol.java | 1162 +++- .../src/main/scala/akka/cluster/Cluster.scala | 138 +- .../scala/akka/cluster/ClusterActorRef.scala | 9 +- .../akka/cluster/ReplicatedClusterRef.scala | 105 - .../src/main/scala/akka/cluster/Routing.scala | 9 +- .../scala/akka/cluster/TransactionLog.scala | 95 +- .../scala/akka/cluster/ReplicationSpec.scala | 8 +- .../RoundRobin1ReplicaMultiJvmSpec.scala | 9 + .../RoundRobin2ReplicasMultiJvmSpec.scala | 9 + .../akka/actor/mailbox/MailboxProtocol.java | 704 +- .../akka/remote/protocol/RemoteProtocol.java | 5920 +++++++++++++---- .../src/main/protocol/RemoteProtocol.proto | 12 +- .../serialization/SerializationProtocol.scala | 57 +- .../scala/akka/testkit/TestActorRef.scala | 4 +- config/akka-reference.conf | 53 +- project/build/AkkaProject.scala | 107 +- 22 files changed, 6540 insertions(+), 2183 deletions(-) delete mode 100644 akka-cluster/src/main/scala/akka/cluster/ReplicatedClusterRef.scala diff --git a/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala b/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala index bd56a79a8e..974b2ea1c9 100644 --- a/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala +++ b/akka-actor-tests/src/test/scala/akka/actor/actor/DeployerSpec.scala @@ -22,7 +22,9 @@ class DeployerSpec extends WordSpec with MustMatchers { Clustered( Node("node1"), Replicate(3), - Stateless)))) + Stateful( + TransactionLog, + WriteThrough))))) } } } diff --git a/akka-actor/src/main/scala/akka/actor/Actor.scala b/akka-actor/src/main/scala/akka/actor/Actor.scala index 2ad30bcd55..887cebc379 100644 --- a/akka-actor/src/main/scala/akka/actor/Actor.scala +++ b/akka-actor/src/main/scala/akka/actor/Actor.scala @@ -72,10 +72,10 @@ case object Kill extends AutoReceivedMessage with LifeCycleMessage case object ReceiveTimeout extends LifeCycleMessage case class MaximumNumberOfRestartsWithinTimeRangeReached( - @BeanProperty val victim: ActorRef, - @BeanProperty val maxNrOfRetries: Option[Int], - @BeanProperty val withinTimeRange: Option[Int], - @BeanProperty val lastExceptionCausingRestart: Throwable) extends LifeCycleMessage + @BeanProperty victim: ActorRef, + @BeanProperty maxNrOfRetries: Option[Int], + @BeanProperty withinTimeRange: Option[Int], + @BeanProperty lastExceptionCausingRestart: Throwable) extends LifeCycleMessage // Exceptions for Actors class ActorStartException private[akka] (message: String, cause: Throwable = null) extends AkkaException(message, cause) @@ -131,7 +131,7 @@ object Actor extends ListenerManagement { subclassAudits synchronized { subclassAudits.clear() } } } - Runtime.getRuntime.addShutdownHook(new Thread(hook)) + Runtime.getRuntime.addShutdownHook(new Thread(hook, "akka-shutdown-hook")) hook } @@ -269,7 +269,7 @@ object Actor extends ListenerManagement { * */ def actorOf[T <: Actor](creator: ⇒ T, address: String): ActorRef = { - createActor(address, () ⇒ new LocalActorRef(() ⇒ creator, address)) + createActor(address, () ⇒ new LocalActorRef(() ⇒ creator, address, Transient)) } /** @@ -292,7 +292,7 @@ object Actor extends ListenerManagement { * JAVA API */ def actorOf[T <: Actor](creator: Creator[T], address: String): ActorRef = { - createActor(address, () ⇒ new LocalActorRef(() ⇒ creator.create, address)) + createActor(address, () ⇒ new LocalActorRef(() ⇒ creator.create, address, Transient)) } /** @@ -374,7 +374,7 @@ object Actor extends ListenerManagement { "\nif so put it outside the class/trait, f.e. in a companion object," + "\nOR try to change: 'actorOf[MyActor]' to 'actorOf(new MyActor)'.", cause) } - }, address) + }, address, Transient) } private def newClusterActorRef(factory: () ⇒ ActorRef, address: String, deploy: Deploy): ActorRef = { @@ -390,58 +390,65 @@ object Actor extends ListenerManagement { val isHomeNode = DeploymentConfig.isHomeNode(home) val replicas = DeploymentConfig.replicaValueFor(replication) - def serializerErrorDueTo(reason: String) = - throw new akka.config.ConfigurationException( - "Could not create Serializer object [" + serializerClassName + - "] for serialization of actor [" + address + - "] since " + reason) - - val serializer: Serializer = { - if ((serializerClassName eq null) || - (serializerClassName == "") || - (serializerClassName == Format.defaultSerializerName)) { - Format.Default - } else { - val clazz: Class[_] = ReflectiveAccess.getClassFor(serializerClassName) match { - case Right(clazz) ⇒ clazz - case Left(exception) ⇒ - val cause = exception match { - case i: InvocationTargetException ⇒ i.getTargetException - case _ ⇒ exception - } - serializerErrorDueTo(cause.toString) - } - val f = clazz.newInstance.asInstanceOf[AnyRef] - if (f.isInstanceOf[Serializer]) f.asInstanceOf[Serializer] - else serializerErrorDueTo("class must be of type [akka.serialization.Serializer]") - } - } - - val isStateful = state match { - case Stateless ⇒ false - case Stateful ⇒ true - } - - if (isStateful && isHomeNode) { // stateful actor's home node - cluster - .use(address, serializer) - .getOrElse(throw new ConfigurationException( - "Could not check out actor [" + address + "] from cluster registry as a \"local\" actor")) - - } else { - if (!cluster.isClustered(address)) { // add actor to cluster registry (if not already added) - cluster.store(factory().start(), replicas, false, serializer) - } + def storeActorAndGetClusterRef(replicationStrategy: ReplicationStrategy, serializer: Serializer): ActorRef = { + // add actor to cluster registry (if not already added) + if (!cluster.isClustered(address)) cluster.store(factory().start(), replicas, replicationStrategy, false, serializer) // remote node (not home node), check out as ClusterActorRef cluster.ref(address, DeploymentConfig.routerTypeFor(router)) } + val serializer = serializerFor(address, serializerClassName) + + state match { + case Stateless ⇒ + storeActorAndGetClusterRef(Transient, serializer) + + case Stateful(storage, strategy) ⇒ + if (isHomeNode) { // stateful actor's home node + cluster + .use(address, serializer) + .getOrElse(throw new ConfigurationException( + "Could not check out actor [" + address + "] from cluster registry as a \"local\" actor")) + } else { + // FIXME later managed different 'storage' as well + storeActorAndGetClusterRef(strategy, serializer) + } + } + case invalid ⇒ throw new IllegalActorStateException( "Could not create actor with address [" + address + "], not bound to a valid deployment scheme [" + invalid + "]") } } + + // FIXME move serializerFor method to ...? + def serializerFor(address: String, serializerClassName: String): Serializer = { + def serializerErrorDueTo(reason: String) = + throw new akka.config.ConfigurationException( + "Could not create Serializer object [" + serializerClassName + + "] for serialization of actor [" + address + + "] since " + reason) + + if ((serializerClassName eq null) || + (serializerClassName == "") || + (serializerClassName == Format.defaultSerializerName)) { + Format.Default + } else { + val clazz: Class[_] = ReflectiveAccess.getClassFor(serializerClassName) match { + case Right(clazz) ⇒ clazz + case Left(exception) ⇒ + val cause = exception match { + case i: InvocationTargetException ⇒ i.getTargetException + case _ ⇒ exception + } + serializerErrorDueTo(cause.toString) + } + val f = clazz.newInstance.asInstanceOf[AnyRef] + if (f.isInstanceOf[Serializer]) f.asInstanceOf[Serializer] + else serializerErrorDueTo("class must be of type [akka.serialization.Serializer]") + } + } } /** diff --git a/akka-actor/src/main/scala/akka/actor/ActorRef.scala b/akka-actor/src/main/scala/akka/actor/ActorRef.scala index bc5efa02b0..6cdcf2f637 100644 --- a/akka-actor/src/main/scala/akka/actor/ActorRef.scala +++ b/akka-actor/src/main/scala/akka/actor/ActorRef.scala @@ -9,7 +9,10 @@ import akka.dispatch._ import akka.config.Config import akka.config.Supervision._ import akka.util._ +import akka.serialization.{ Format, Serializer } import ReflectiveAccess._ +import ClusterModule._ +import DeploymentConfig.{ ReplicationStrategy, Transient, WriteThrough, WriteBehind } import java.net.InetSocketAddress import java.util.concurrent.atomic.AtomicReference @@ -506,7 +509,7 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal that.asInstanceOf[ActorRef].uuid == uuid } - override def toString = "Actor[" + address + ":" + uuid + "]" + override def toString = "Actor[%s:%s]".format(address, uuid) } /** @@ -514,27 +517,57 @@ trait ActorRef extends ActorRefShared with java.lang.Comparable[ActorRef] { scal * * @author Jonas Bonér */ -class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, val address: String) +class LocalActorRef private[akka] ( + private[this] val actorFactory: () ⇒ Actor, + val address: String, + replicationStrategy: ReplicationStrategy) extends ActorRef with ScalaActorRef { + protected[akka] val guard = new ReentrantGuard @volatile protected[akka] var _futureTimeout: Option[ScheduledFuture[AnyRef]] = None + @volatile private[akka] lazy val _linkedActors = new ConcurrentHashMap[Uuid, ActorRef] + @volatile private[akka] var _supervisor: Option[ActorRef] = None + @volatile private var maxNrOfRetriesCount: Int = 0 + @volatile private var restartTimeWindowStartNanos: Long = 0L + @volatile private var _mailbox: AnyRef = _ + @volatile private[akka] var _dispatcher: MessageDispatcher = Dispatchers.defaultGlobalDispatcher protected[akka] val actorInstance = guard.withGuard { new AtomicReference[Actor](newActor) } + private val isReplicated: Boolean = replicationStrategy match { + case Transient ⇒ false + case _ ⇒ true + } + + // FIXME how to get the matching serializerClassName? Now default is used + private val serializer = Actor.serializerFor(address, Format.defaultSerializerName) + + private lazy val txLog: TransactionLog = { + val log = replicationStrategy match { + case Transient ⇒ throw new IllegalStateException("Can not replicate 'transient' actor [" + toString + "]") + case WriteThrough ⇒ transactionLog.newLogFor(_uuid.toString, false) + case WriteBehind ⇒ transactionLog.newLogFor(_uuid.toString, true) + } + EventHandler.debug(this, + "Creating a transaction log for Actor [%s] with replication strategy [%s]" + .format(address, replicationStrategy)) + log + } + //If it was started inside "newActor", initialize it if (isRunning) initializeActorInstance @@ -547,8 +580,11 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, __lifeCycle: LifeCycle, __supervisor: Option[ActorRef], __hotswap: Stack[PartialFunction[Any, Unit]], - __factory: () ⇒ Actor) = { - this(__factory, __address) + __factory: () ⇒ Actor, + __replicationStrategy: ReplicationStrategy) = { + + this(__factory, __address, __replicationStrategy) + _uuid = __uuid timeout = __timeout receiveTimeout = __receiveTimeout @@ -620,6 +656,8 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, setActorSelfFields(actorInstance.get, null) } } //else if (isBeingRestarted) throw new ActorKilledException("Actor [" + toString + "] is being restarted.") + + if (isReplicated) txLog.delete() } } @@ -656,7 +694,6 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, guard.withGuard { if (_linkedActors.remove(actorRef.uuid) eq null) throw new IllegalActorStateException("Actor [" + actorRef + "] is not a linked actor, can't unlink") - actorRef.supervisor = None } } @@ -733,7 +770,12 @@ class LocalActorRef private[akka] (private[this] val actorFactory: () ⇒ Actor, throw e } } - } finally { guard.lock.unlock() } + } finally { + guard.lock.unlock() + if (isReplicated) { + txLog.recordEntry(messageHandle, this, serializer) + } + } } protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) { diff --git a/akka-actor/src/main/scala/akka/actor/Deployer.scala b/akka-actor/src/main/scala/akka/actor/Deployer.scala index 5943e28117..44c2e34582 100644 --- a/akka-actor/src/main/scala/akka/actor/Deployer.scala +++ b/akka-actor/src/main/scala/akka/actor/Deployer.scala @@ -63,7 +63,7 @@ object DeploymentConfig { case class Clustered( home: Home = Host("localhost"), replication: Replication = NoReplicas, - state: State = Stateful) extends Scope + state: State = Stateless) extends Scope // For Java API case class Local() extends Scope @@ -102,11 +102,40 @@ object DeploymentConfig { // For Java API case class Stateless() extends State - case class Stateful() extends State // For Scala API case object Stateless extends State - case object Stateful extends State + case class Stateful( + storage: ReplicationStorage, + strategy: ReplicationStrategy) extends State + + // -------------------------------- + // --- ReplicationStorage + // -------------------------------- + sealed trait ReplicationStorage + + // For Java API + case class TransactionLog() extends ReplicationStorage + case class DataGrid() extends ReplicationStorage + + // For Scala API + case object TransactionLog extends ReplicationStorage + case object DataGrid extends ReplicationStorage + + // -------------------------------- + // --- ReplicationStrategy + // -------------------------------- + sealed trait ReplicationStrategy + + // For Java API + case class WriteBehind() extends ReplicationStrategy + case class WriteThrough() extends ReplicationStrategy + case class Transient() extends ReplicationStrategy + + // For Scala API + case object WriteBehind extends ReplicationStrategy + case object WriteThrough extends ReplicationStrategy + case object Transient extends ReplicationStrategy // -------------------------------- // --- Helper methods for parsing @@ -345,13 +374,28 @@ object Deployer { } // -------------------------------- - // akka.actor.deployment.
.clustered.stateless + // akka.actor.deployment.
.clustered.stateful // -------------------------------- - val state = - if (clusteredConfig.getBool("stateless", false)) Stateless - else Stateful + clusteredConfig.getSection("stateful") match { + case None ⇒ + Some(Deploy(address, router, format, Clustered(home, replicas, Stateless))) - Some(Deploy(address, router, format, Clustered(home, replicas, state))) + case Some(statefulConfig) ⇒ + val storage = statefulConfig.getString("replication-storage", "transaction-log") match { + case "transaction-log" ⇒ TransactionLog + case "data-grid" ⇒ DataGrid + case unknown ⇒ + throw new ConfigurationException("Config option [" + addressPath + + ".clustered.stateful.replication-storage] needs to be either [\"transaction-log\"] or [\"data-grid\"] - was [" + + unknown + "]") + } + val strategy = statefulConfig.getString("replication-strategy", "write-through") match { + case "write-through" ⇒ WriteThrough + case "write-behind" ⇒ WriteBehind + case unknown ⇒ Transient + } + Some(Deploy(address, router, format, Clustered(home, replicas, Stateful(storage, strategy)))) + } } } } diff --git a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala index 7f7da6d29c..c908134478 100644 --- a/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala +++ b/akka-actor/src/main/scala/akka/cluster/ClusterInterface.scala @@ -7,6 +7,7 @@ package akka.cluster import akka.remoteinterface.RemoteSupport import akka.serialization.Serializer import akka.actor._ +import DeploymentConfig._ import akka.dispatch.Future import akka.config.Config import akka.util._ @@ -179,6 +180,13 @@ trait ClusterNode { */ def store[T <: Actor](address: String, actorClass: Class[T], format: Serializer): ClusterNode + /** + * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store[T <: Actor](address: String, actorClass: Class[T], replicationStrategy: ReplicationStrategy, format: Serializer): ClusterNode + /** * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly @@ -186,6 +194,13 @@ trait ClusterNode { */ def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, format: Serializer): ClusterNode + /** + * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, replicationStrategy: ReplicationStrategy, format: Serializer): ClusterNode + /** * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly @@ -193,6 +208,13 @@ trait ClusterNode { */ def store[T <: Actor](address: String, actorClass: Class[T], serializeMailbox: Boolean, format: Serializer): ClusterNode + /** + * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store[T <: Actor](address: String, actorClass: Class[T], replicationStrategy: ReplicationStrategy, serializeMailbox: Boolean, format: Serializer): ClusterNode + /** * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly @@ -200,6 +222,13 @@ trait ClusterNode { */ def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode + /** + * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, replicationStrategy: ReplicationStrategy, serializeMailbox: Boolean, format: Serializer): ClusterNode + /** * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly @@ -207,6 +236,13 @@ trait ClusterNode { */ def store(actorRef: ActorRef, format: Serializer): ClusterNode + /** + * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store(actorRef: ActorRef, replicationStrategy: ReplicationStrategy, format: Serializer): ClusterNode + /** * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly @@ -214,6 +250,13 @@ trait ClusterNode { */ def store(actorRef: ActorRef, replicationFactor: Int, format: Serializer): ClusterNode + /** + * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store(actorRef: ActorRef, replicationFactor: Int, replicationStrategy: ReplicationStrategy, format: Serializer): ClusterNode + /** * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly @@ -221,11 +264,23 @@ trait ClusterNode { */ def store(actorRef: ActorRef, serializeMailbox: Boolean, format: Serializer): ClusterNode + /** + * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store(actorRef: ActorRef, replicationStrategy: ReplicationStrategy, serializeMailbox: Boolean, format: Serializer): ClusterNode + /** * Needed to have reflection through structural typing work. */ def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: AnyRef): ClusterNode + /** + * Needed to have reflection through structural typing work. + */ + def store(actorRef: ActorRef, replicationFactor: Int, replicationStrategy: ReplicationStrategy, serializeMailbox: Boolean, format: AnyRef): ClusterNode + /** * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly @@ -233,6 +288,13 @@ trait ClusterNode { */ def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode + /** + * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store(actorRef: ActorRef, replicationFactor: Int, replicationStrategy: ReplicationStrategy, serializeMailbox: Boolean, format: Serializer): ClusterNode + /** * Removes actor with uuid from the cluster. */ @@ -262,13 +324,13 @@ trait ClusterNode { * Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available * for remote access through lookup by its UUID. */ - def use[T <: Actor](actorAddress: String): Option[LocalActorRef] + def use[T <: Actor](actorAddress: String): Option[ActorRef] /** * Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available * for remote access through lookup by its UUID. */ - def use[T <: Actor](actorAddress: String, format: Serializer): Option[LocalActorRef] + def use[T <: Actor](actorAddress: String, format: Serializer): Option[ActorRef] /** * Using (checking out) all actors with a specific UUID on all nodes in the cluster. diff --git a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala index af530c3068..dd2971c5d5 100644 --- a/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala +++ b/akka-actor/src/main/scala/akka/util/ReflectiveAccess.scala @@ -62,6 +62,13 @@ object ReflectiveAccess { None } + lazy val transactionLogInstance: Option[TransactionLogObject] = getObjectFor("akka.cluster.TransactionLog$") match { + case Right(value) ⇒ Some(value) + case Left(exception) ⇒ + EventHandler.debug(this, exception.toString) + None + } + lazy val node: ClusterNode = { ensureEnabled() clusterInstance.get.node @@ -72,6 +79,11 @@ object ReflectiveAccess { clusterDeployerInstance.get } + lazy val transactionLog: TransactionLogObject = { + ensureEnabled() + transactionLogInstance.get + } + type ClusterDeployer = { def init(deployments: List[Deploy]) def shutdown() @@ -94,6 +106,25 @@ object ReflectiveAccess { def toBinary(obj: AnyRef): Array[Byte] def fromBinary(bytes: Array[Byte], clazz: Option[Class[_]]): AnyRef } + + type TransactionLogObject = { + def newLogFor(id: String, isAsync: Boolean): TransactionLog + def logFor(id: String, isAsync: Boolean): TransactionLog + def shutdown() + } + + type TransactionLog = { + def recordEntry(messageHandle: MessageInvocation, actorRef: ActorRef, serializer: Serializer) + def recordEntry(entry: Array[Byte]) + def recordSnapshot(snapshot: Array[Byte]) + def entries: Vector[Array[Byte]] + def entriesFromLatestSnapshot: Tuple2[Array[Byte], Vector[Array[Byte]]] + def entriesInRange(from: Long, to: Long): Vector[Array[Byte]] + def latestEntryId: Long + def latestSnapshotId: Long + def delete() + def close() + } } /** diff --git a/akka-cluster/src/main/java/akka/cluster/ClusterProtocol.java b/akka-cluster/src/main/java/akka/cluster/ClusterProtocol.java index ebf240b200..840e89c015 100644 --- a/akka-cluster/src/main/java/akka/cluster/ClusterProtocol.java +++ b/akka-cluster/src/main/java/akka/cluster/ClusterProtocol.java @@ -26,6 +26,21 @@ public final class ClusterProtocol { FUNCTION_FUN1_ARG_ANY(13, 14), ; + public static final int START_VALUE = 1; + public static final int STOP_VALUE = 2; + public static final int USE_VALUE = 3; + public static final int RELEASE_VALUE = 4; + public static final int MAKE_AVAILABLE_VALUE = 5; + public static final int MAKE_UNAVAILABLE_VALUE = 6; + public static final int DISCONNECT_VALUE = 7; + public static final int RECONNECT_VALUE = 8; + public static final int RESIGN_VALUE = 9; + public static final int FAIL_OVER_CONNECTIONS_VALUE = 10; + public static final int FUNCTION_FUN0_UNIT_VALUE = 11; + public static final int FUNCTION_FUN0_ANY_VALUE = 12; + public static final int FUNCTION_FUN1_ARG_UNIT_VALUE = 13; + public static final int FUNCTION_FUN1_ARG_ANY_VALUE = 14; + public final int getNumber() { return value; } @@ -57,8 +72,8 @@ public final class ClusterProtocol { internalValueMap = new com.google.protobuf.Internal.EnumLiteMap() { public RemoteDaemonMessageType findValueByNumber(int number) { - return RemoteDaemonMessageType.valueOf(number) - ; } + return RemoteDaemonMessageType.valueOf(number); + } }; public final com.google.protobuf.Descriptors.EnumValueDescriptor @@ -77,6 +92,7 @@ public final class ClusterProtocol { private static final RemoteDaemonMessageType[] VALUES = { START, STOP, USE, RELEASE, MAKE_AVAILABLE, MAKE_UNAVAILABLE, DISCONNECT, RECONNECT, RESIGN, FAIL_OVER_CONNECTIONS, FUNCTION_FUN0_UNIT, FUNCTION_FUN0_ANY, FUNCTION_FUN1_ARG_UNIT, FUNCTION_FUN1_ARG_ANY, }; + public static RemoteDaemonMessageType valueOf( com.google.protobuf.Descriptors.EnumValueDescriptor desc) { if (desc.getType() != getDescriptor()) { @@ -85,25 +101,44 @@ public final class ClusterProtocol { } return VALUES[desc.getIndex()]; } + private final int index; private final int value; + private RemoteDaemonMessageType(int index, int value) { this.index = index; this.value = value; } - static { - akka.cluster.ClusterProtocol.getDescriptor(); - } - // @@protoc_insertion_point(enum_scope:RemoteDaemonMessageType) } + public interface RemoteDaemonMessageProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .RemoteDaemonMessageType messageType = 1; + boolean hasMessageType(); + akka.cluster.ClusterProtocol.RemoteDaemonMessageType getMessageType(); + + // optional .UuidProtocol actorUuid = 2; + boolean hasActorUuid(); + akka.cluster.ClusterProtocol.UuidProtocol getActorUuid(); + akka.cluster.ClusterProtocol.UuidProtocolOrBuilder getActorUuidOrBuilder(); + + // optional string actorAddress = 3; + boolean hasActorAddress(); + String getActorAddress(); + + // optional bytes payload = 5; + boolean hasPayload(); + com.google.protobuf.ByteString getPayload(); + } public static final class RemoteDaemonMessageProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements RemoteDaemonMessageProtocolOrBuilder { // Use RemoteDaemonMessageProtocol.newBuilder() to construct. - private RemoteDaemonMessageProtocol() { - initFields(); + private RemoteDaemonMessageProtocol(Builder builder) { + super(builder); } private RemoteDaemonMessageProtocol(boolean noInit) {} @@ -126,60 +161,111 @@ public final class ClusterProtocol { return akka.cluster.ClusterProtocol.internal_static_RemoteDaemonMessageProtocol_fieldAccessorTable; } + private int bitField0_; // required .RemoteDaemonMessageType messageType = 1; public static final int MESSAGETYPE_FIELD_NUMBER = 1; - private boolean hasMessageType; private akka.cluster.ClusterProtocol.RemoteDaemonMessageType messageType_; - public boolean hasMessageType() { return hasMessageType; } - public akka.cluster.ClusterProtocol.RemoteDaemonMessageType getMessageType() { return messageType_; } + public boolean hasMessageType() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.cluster.ClusterProtocol.RemoteDaemonMessageType getMessageType() { + return messageType_; + } // optional .UuidProtocol actorUuid = 2; public static final int ACTORUUID_FIELD_NUMBER = 2; - private boolean hasActorUuid; private akka.cluster.ClusterProtocol.UuidProtocol actorUuid_; - public boolean hasActorUuid() { return hasActorUuid; } - public akka.cluster.ClusterProtocol.UuidProtocol getActorUuid() { return actorUuid_; } + public boolean hasActorUuid() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.cluster.ClusterProtocol.UuidProtocol getActorUuid() { + return actorUuid_; + } + public akka.cluster.ClusterProtocol.UuidProtocolOrBuilder getActorUuidOrBuilder() { + return actorUuid_; + } // optional string actorAddress = 3; public static final int ACTORADDRESS_FIELD_NUMBER = 3; - private boolean hasActorAddress; - private java.lang.String actorAddress_ = ""; - public boolean hasActorAddress() { return hasActorAddress; } - public java.lang.String getActorAddress() { return actorAddress_; } + private java.lang.Object actorAddress_; + public boolean hasActorAddress() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getActorAddress() { + java.lang.Object ref = actorAddress_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + actorAddress_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getActorAddressBytes() { + java.lang.Object ref = actorAddress_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + actorAddress_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // optional bytes payload = 5; public static final int PAYLOAD_FIELD_NUMBER = 5; - private boolean hasPayload; - private com.google.protobuf.ByteString payload_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasPayload() { return hasPayload; } - public com.google.protobuf.ByteString getPayload() { return payload_; } + private com.google.protobuf.ByteString payload_; + public boolean hasPayload() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getPayload() { + return payload_; + } private void initFields() { messageType_ = akka.cluster.ClusterProtocol.RemoteDaemonMessageType.START; actorUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + actorAddress_ = ""; + payload_ = com.google.protobuf.ByteString.EMPTY; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasMessageType) return false; - if (hasActorUuid()) { - if (!getActorUuid().isInitialized()) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasMessageType()) { + memoizedIsInitialized = 0; + return false; } + if (hasActorUuid()) { + if (!getActorUuid().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasMessageType()) { - output.writeEnum(1, getMessageType().getNumber()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, messageType_.getNumber()); } - if (hasActorUuid()) { - output.writeMessage(2, getActorUuid()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, actorUuid_); } - if (hasActorAddress()) { - output.writeString(3, getActorAddress()); + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getActorAddressBytes()); } - if (hasPayload()) { - output.writeBytes(5, getPayload()); + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(5, payload_); } getUnknownFields().writeTo(output); } @@ -190,27 +276,34 @@ public final class ClusterProtocol { if (size != -1) return size; size = 0; - if (hasMessageType()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeEnumSize(1, getMessageType().getNumber()); + .computeEnumSize(1, messageType_.getNumber()); } - if (hasActorUuid()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getActorUuid()); + .computeMessageSize(2, actorUuid_); } - if (hasActorAddress()) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(3, getActorAddress()); + .computeBytesSize(3, getActorAddressBytes()); } - if (hasPayload()) { + if (((bitField0_ & 0x00000008) == 0x00000008)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(5, getPayload()); + .computeBytesSize(5, payload_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -285,34 +378,62 @@ public final class ClusterProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol result; - - // Construct using akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.cluster.ClusterProtocol.internal_static_RemoteDaemonMessageProtocol_descriptor; } - protected akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.cluster.ClusterProtocol.internal_static_RemoteDaemonMessageProtocol_fieldAccessorTable; + } + + // Construct using akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getActorUuidFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); + super.clear(); + messageType_ = akka.cluster.ClusterProtocol.RemoteDaemonMessageType.START; + bitField0_ = (bitField0_ & ~0x00000001); + if (actorUuidBuilder_ == null) { + actorUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + } else { + actorUuidBuilder_.clear(); } - result = new akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol(); + bitField0_ = (bitField0_ & ~0x00000002); + actorAddress_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + payload_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -324,33 +445,51 @@ public final class ClusterProtocol { return akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol build() { - if (result != null && !isInitialized()) { + akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol result = new akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.cluster.ClusterProtocol.RemoteDaemonMessageProtocol returnMe = result; - result = null; - return returnMe; + result.messageType_ = messageType_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (actorUuidBuilder_ == null) { + result.actorUuid_ = actorUuid_; + } else { + result.actorUuid_ = actorUuidBuilder_.build(); + } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.actorAddress_ = actorAddress_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.payload_ = payload_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -380,6 +519,20 @@ public final class ClusterProtocol { return this; } + public final boolean isInitialized() { + if (!hasMessageType()) { + + return false; + } + if (hasActorUuid()) { + if (!getActorUuid().isInitialized()) { + + return false; + } + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -392,11 +545,13 @@ public final class ClusterProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; @@ -407,7 +562,8 @@ public final class ClusterProtocol { if (value == null) { unknownFields.mergeVarintField(1, rawValue); } else { - setMessageType(value); + bitField0_ |= 0x00000001; + messageType_ = value; } break; } @@ -421,115 +577,192 @@ public final class ClusterProtocol { break; } case 26: { - setActorAddress(input.readString()); + bitField0_ |= 0x00000004; + actorAddress_ = input.readBytes(); break; } case 42: { - setPayload(input.readBytes()); + bitField0_ |= 0x00000008; + payload_ = input.readBytes(); break; } } } } + private int bitField0_; // required .RemoteDaemonMessageType messageType = 1; + private akka.cluster.ClusterProtocol.RemoteDaemonMessageType messageType_ = akka.cluster.ClusterProtocol.RemoteDaemonMessageType.START; public boolean hasMessageType() { - return result.hasMessageType(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public akka.cluster.ClusterProtocol.RemoteDaemonMessageType getMessageType() { - return result.getMessageType(); + return messageType_; } public Builder setMessageType(akka.cluster.ClusterProtocol.RemoteDaemonMessageType value) { if (value == null) { throw new NullPointerException(); } - result.hasMessageType = true; - result.messageType_ = value; + bitField0_ |= 0x00000001; + messageType_ = value; + onChanged(); return this; } public Builder clearMessageType() { - result.hasMessageType = false; - result.messageType_ = akka.cluster.ClusterProtocol.RemoteDaemonMessageType.START; + bitField0_ = (bitField0_ & ~0x00000001); + messageType_ = akka.cluster.ClusterProtocol.RemoteDaemonMessageType.START; + onChanged(); return this; } // optional .UuidProtocol actorUuid = 2; + private akka.cluster.ClusterProtocol.UuidProtocol actorUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.cluster.ClusterProtocol.UuidProtocol, akka.cluster.ClusterProtocol.UuidProtocol.Builder, akka.cluster.ClusterProtocol.UuidProtocolOrBuilder> actorUuidBuilder_; public boolean hasActorUuid() { - return result.hasActorUuid(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public akka.cluster.ClusterProtocol.UuidProtocol getActorUuid() { - return result.getActorUuid(); + if (actorUuidBuilder_ == null) { + return actorUuid_; + } else { + return actorUuidBuilder_.getMessage(); + } } public Builder setActorUuid(akka.cluster.ClusterProtocol.UuidProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (actorUuidBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + actorUuid_ = value; + onChanged(); + } else { + actorUuidBuilder_.setMessage(value); } - result.hasActorUuid = true; - result.actorUuid_ = value; + bitField0_ |= 0x00000002; return this; } - public Builder setActorUuid(akka.cluster.ClusterProtocol.UuidProtocol.Builder builderForValue) { - result.hasActorUuid = true; - result.actorUuid_ = builderForValue.build(); + public Builder setActorUuid( + akka.cluster.ClusterProtocol.UuidProtocol.Builder builderForValue) { + if (actorUuidBuilder_ == null) { + actorUuid_ = builderForValue.build(); + onChanged(); + } else { + actorUuidBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; return this; } public Builder mergeActorUuid(akka.cluster.ClusterProtocol.UuidProtocol value) { - if (result.hasActorUuid() && - result.actorUuid_ != akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance()) { - result.actorUuid_ = - akka.cluster.ClusterProtocol.UuidProtocol.newBuilder(result.actorUuid_).mergeFrom(value).buildPartial(); + if (actorUuidBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + actorUuid_ != akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance()) { + actorUuid_ = + akka.cluster.ClusterProtocol.UuidProtocol.newBuilder(actorUuid_).mergeFrom(value).buildPartial(); + } else { + actorUuid_ = value; + } + onChanged(); } else { - result.actorUuid_ = value; + actorUuidBuilder_.mergeFrom(value); } - result.hasActorUuid = true; + bitField0_ |= 0x00000002; return this; } public Builder clearActorUuid() { - result.hasActorUuid = false; - result.actorUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + if (actorUuidBuilder_ == null) { + actorUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + onChanged(); + } else { + actorUuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); return this; } + public akka.cluster.ClusterProtocol.UuidProtocol.Builder getActorUuidBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getActorUuidFieldBuilder().getBuilder(); + } + public akka.cluster.ClusterProtocol.UuidProtocolOrBuilder getActorUuidOrBuilder() { + if (actorUuidBuilder_ != null) { + return actorUuidBuilder_.getMessageOrBuilder(); + } else { + return actorUuid_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.cluster.ClusterProtocol.UuidProtocol, akka.cluster.ClusterProtocol.UuidProtocol.Builder, akka.cluster.ClusterProtocol.UuidProtocolOrBuilder> + getActorUuidFieldBuilder() { + if (actorUuidBuilder_ == null) { + actorUuidBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.cluster.ClusterProtocol.UuidProtocol, akka.cluster.ClusterProtocol.UuidProtocol.Builder, akka.cluster.ClusterProtocol.UuidProtocolOrBuilder>( + actorUuid_, + getParentForChildren(), + isClean()); + actorUuid_ = null; + } + return actorUuidBuilder_; + } // optional string actorAddress = 3; + private java.lang.Object actorAddress_ = ""; public boolean hasActorAddress() { - return result.hasActorAddress(); + return ((bitField0_ & 0x00000004) == 0x00000004); } - public java.lang.String getActorAddress() { - return result.getActorAddress(); + public String getActorAddress() { + java.lang.Object ref = actorAddress_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + actorAddress_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setActorAddress(java.lang.String value) { + public Builder setActorAddress(String value) { if (value == null) { throw new NullPointerException(); } - result.hasActorAddress = true; - result.actorAddress_ = value; + bitField0_ |= 0x00000004; + actorAddress_ = value; + onChanged(); return this; } public Builder clearActorAddress() { - result.hasActorAddress = false; - result.actorAddress_ = getDefaultInstance().getActorAddress(); + bitField0_ = (bitField0_ & ~0x00000004); + actorAddress_ = getDefaultInstance().getActorAddress(); + onChanged(); return this; } + void setActorAddress(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + actorAddress_ = value; + onChanged(); + } // optional bytes payload = 5; + private com.google.protobuf.ByteString payload_ = com.google.protobuf.ByteString.EMPTY; public boolean hasPayload() { - return result.hasPayload(); + return ((bitField0_ & 0x00000008) == 0x00000008); } public com.google.protobuf.ByteString getPayload() { - return result.getPayload(); + return payload_; } public Builder setPayload(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - result.hasPayload = true; - result.payload_ = value; + bitField0_ |= 0x00000008; + payload_ = value; + onChanged(); return this; } public Builder clearPayload() { - result.hasPayload = false; - result.payload_ = getDefaultInstance().getPayload(); + bitField0_ = (bitField0_ & ~0x00000008); + payload_ = getDefaultInstance().getPayload(); + onChanged(); return this; } @@ -538,18 +771,38 @@ public final class ClusterProtocol { static { defaultInstance = new RemoteDaemonMessageProtocol(true); - akka.cluster.ClusterProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:RemoteDaemonMessageProtocol) } + public interface DurableMailboxMessageProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string ownerActorAddress = 1; + boolean hasOwnerActorAddress(); + String getOwnerActorAddress(); + + // optional string senderActorAddress = 2; + boolean hasSenderActorAddress(); + String getSenderActorAddress(); + + // optional .UuidProtocol futureUuid = 3; + boolean hasFutureUuid(); + akka.cluster.ClusterProtocol.UuidProtocol getFutureUuid(); + akka.cluster.ClusterProtocol.UuidProtocolOrBuilder getFutureUuidOrBuilder(); + + // required bytes message = 4; + boolean hasMessage(); + com.google.protobuf.ByteString getMessage(); + } public static final class DurableMailboxMessageProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements DurableMailboxMessageProtocolOrBuilder { // Use DurableMailboxMessageProtocol.newBuilder() to construct. - private DurableMailboxMessageProtocol() { - initFields(); + private DurableMailboxMessageProtocol(Builder builder) { + super(builder); } private DurableMailboxMessageProtocol(boolean noInit) {} @@ -572,60 +825,137 @@ public final class ClusterProtocol { return akka.cluster.ClusterProtocol.internal_static_DurableMailboxMessageProtocol_fieldAccessorTable; } + private int bitField0_; // required string ownerActorAddress = 1; public static final int OWNERACTORADDRESS_FIELD_NUMBER = 1; - private boolean hasOwnerActorAddress; - private java.lang.String ownerActorAddress_ = ""; - public boolean hasOwnerActorAddress() { return hasOwnerActorAddress; } - public java.lang.String getOwnerActorAddress() { return ownerActorAddress_; } + private java.lang.Object ownerActorAddress_; + public boolean hasOwnerActorAddress() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getOwnerActorAddress() { + java.lang.Object ref = ownerActorAddress_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + ownerActorAddress_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getOwnerActorAddressBytes() { + java.lang.Object ref = ownerActorAddress_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + ownerActorAddress_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // optional string senderActorAddress = 2; public static final int SENDERACTORADDRESS_FIELD_NUMBER = 2; - private boolean hasSenderActorAddress; - private java.lang.String senderActorAddress_ = ""; - public boolean hasSenderActorAddress() { return hasSenderActorAddress; } - public java.lang.String getSenderActorAddress() { return senderActorAddress_; } + private java.lang.Object senderActorAddress_; + public boolean hasSenderActorAddress() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getSenderActorAddress() { + java.lang.Object ref = senderActorAddress_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + senderActorAddress_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getSenderActorAddressBytes() { + java.lang.Object ref = senderActorAddress_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + senderActorAddress_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // optional .UuidProtocol futureUuid = 3; public static final int FUTUREUUID_FIELD_NUMBER = 3; - private boolean hasFutureUuid; private akka.cluster.ClusterProtocol.UuidProtocol futureUuid_; - public boolean hasFutureUuid() { return hasFutureUuid; } - public akka.cluster.ClusterProtocol.UuidProtocol getFutureUuid() { return futureUuid_; } + public boolean hasFutureUuid() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public akka.cluster.ClusterProtocol.UuidProtocol getFutureUuid() { + return futureUuid_; + } + public akka.cluster.ClusterProtocol.UuidProtocolOrBuilder getFutureUuidOrBuilder() { + return futureUuid_; + } // required bytes message = 4; public static final int MESSAGE_FIELD_NUMBER = 4; - 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_; } + private com.google.protobuf.ByteString message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getMessage() { + return message_; + } private void initFields() { + ownerActorAddress_ = ""; + senderActorAddress_ = ""; futureUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + message_ = com.google.protobuf.ByteString.EMPTY; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasOwnerActorAddress) return false; - if (!hasMessage) return false; - if (hasFutureUuid()) { - if (!getFutureUuid().isInitialized()) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasOwnerActorAddress()) { + memoizedIsInitialized = 0; + return false; } + if (!hasMessage()) { + memoizedIsInitialized = 0; + return false; + } + if (hasFutureUuid()) { + if (!getFutureUuid().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasOwnerActorAddress()) { - output.writeString(1, getOwnerActorAddress()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getOwnerActorAddressBytes()); } - if (hasSenderActorAddress()) { - output.writeString(2, getSenderActorAddress()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getSenderActorAddressBytes()); } - if (hasFutureUuid()) { - output.writeMessage(3, getFutureUuid()); + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, futureUuid_); } - if (hasMessage()) { - output.writeBytes(4, getMessage()); + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, message_); } getUnknownFields().writeTo(output); } @@ -636,27 +966,34 @@ public final class ClusterProtocol { if (size != -1) return size; size = 0; - if (hasOwnerActorAddress()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getOwnerActorAddress()); + .computeBytesSize(1, getOwnerActorAddressBytes()); } - if (hasSenderActorAddress()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getSenderActorAddress()); + .computeBytesSize(2, getSenderActorAddressBytes()); } - if (hasFutureUuid()) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(3, getFutureUuid()); + .computeMessageSize(3, futureUuid_); } - if (hasMessage()) { + if (((bitField0_ & 0x00000008) == 0x00000008)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(4, getMessage()); + .computeBytesSize(4, message_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -731,34 +1068,62 @@ public final class ClusterProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol result; - - // Construct using akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.cluster.ClusterProtocol.DurableMailboxMessageProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.cluster.ClusterProtocol.internal_static_DurableMailboxMessageProtocol_descriptor; } - protected akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.cluster.ClusterProtocol.internal_static_DurableMailboxMessageProtocol_fieldAccessorTable; + } + + // Construct using akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getFutureUuidFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); + super.clear(); + ownerActorAddress_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + senderActorAddress_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + if (futureUuidBuilder_ == null) { + futureUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + } else { + futureUuidBuilder_.clear(); } - result = new akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol(); + bitField0_ = (bitField0_ & ~0x00000004); + message_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -770,33 +1135,51 @@ public final class ClusterProtocol { return akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol build() { - if (result != null && !isInitialized()) { + akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol result = new akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.cluster.ClusterProtocol.DurableMailboxMessageProtocol returnMe = result; - result = null; - return returnMe; + result.ownerActorAddress_ = ownerActorAddress_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.senderActorAddress_ = senderActorAddress_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (futureUuidBuilder_ == null) { + result.futureUuid_ = futureUuid_; + } else { + result.futureUuid_ = futureUuidBuilder_.build(); + } + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -826,6 +1209,24 @@ public final class ClusterProtocol { return this; } + public final boolean isInitialized() { + if (!hasOwnerActorAddress()) { + + return false; + } + if (!hasMessage()) { + + return false; + } + if (hasFutureUuid()) { + if (!getFutureUuid().isInitialized()) { + + return false; + } + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -838,21 +1239,25 @@ public final class ClusterProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 10: { - setOwnerActorAddress(input.readString()); + bitField0_ |= 0x00000001; + ownerActorAddress_ = input.readBytes(); break; } case 18: { - setSenderActorAddress(input.readString()); + bitField0_ |= 0x00000002; + senderActorAddress_ = input.readBytes(); break; } case 26: { @@ -865,111 +1270,199 @@ public final class ClusterProtocol { break; } case 34: { - setMessage(input.readBytes()); + bitField0_ |= 0x00000008; + message_ = input.readBytes(); break; } } } } + private int bitField0_; // required string ownerActorAddress = 1; + private java.lang.Object ownerActorAddress_ = ""; public boolean hasOwnerActorAddress() { - return result.hasOwnerActorAddress(); + return ((bitField0_ & 0x00000001) == 0x00000001); } - public java.lang.String getOwnerActorAddress() { - return result.getOwnerActorAddress(); + public String getOwnerActorAddress() { + java.lang.Object ref = ownerActorAddress_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + ownerActorAddress_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setOwnerActorAddress(java.lang.String value) { + public Builder setOwnerActorAddress(String value) { if (value == null) { throw new NullPointerException(); } - result.hasOwnerActorAddress = true; - result.ownerActorAddress_ = value; + bitField0_ |= 0x00000001; + ownerActorAddress_ = value; + onChanged(); return this; } public Builder clearOwnerActorAddress() { - result.hasOwnerActorAddress = false; - result.ownerActorAddress_ = getDefaultInstance().getOwnerActorAddress(); + bitField0_ = (bitField0_ & ~0x00000001); + ownerActorAddress_ = getDefaultInstance().getOwnerActorAddress(); + onChanged(); return this; } + void setOwnerActorAddress(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + ownerActorAddress_ = value; + onChanged(); + } // optional string senderActorAddress = 2; + private java.lang.Object senderActorAddress_ = ""; public boolean hasSenderActorAddress() { - return result.hasSenderActorAddress(); + return ((bitField0_ & 0x00000002) == 0x00000002); } - public java.lang.String getSenderActorAddress() { - return result.getSenderActorAddress(); + public String getSenderActorAddress() { + java.lang.Object ref = senderActorAddress_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + senderActorAddress_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setSenderActorAddress(java.lang.String value) { + public Builder setSenderActorAddress(String value) { if (value == null) { throw new NullPointerException(); } - result.hasSenderActorAddress = true; - result.senderActorAddress_ = value; + bitField0_ |= 0x00000002; + senderActorAddress_ = value; + onChanged(); return this; } public Builder clearSenderActorAddress() { - result.hasSenderActorAddress = false; - result.senderActorAddress_ = getDefaultInstance().getSenderActorAddress(); + bitField0_ = (bitField0_ & ~0x00000002); + senderActorAddress_ = getDefaultInstance().getSenderActorAddress(); + onChanged(); return this; } + void setSenderActorAddress(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + senderActorAddress_ = value; + onChanged(); + } // optional .UuidProtocol futureUuid = 3; + private akka.cluster.ClusterProtocol.UuidProtocol futureUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.cluster.ClusterProtocol.UuidProtocol, akka.cluster.ClusterProtocol.UuidProtocol.Builder, akka.cluster.ClusterProtocol.UuidProtocolOrBuilder> futureUuidBuilder_; public boolean hasFutureUuid() { - return result.hasFutureUuid(); + return ((bitField0_ & 0x00000004) == 0x00000004); } public akka.cluster.ClusterProtocol.UuidProtocol getFutureUuid() { - return result.getFutureUuid(); + if (futureUuidBuilder_ == null) { + return futureUuid_; + } else { + return futureUuidBuilder_.getMessage(); + } } public Builder setFutureUuid(akka.cluster.ClusterProtocol.UuidProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (futureUuidBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + futureUuid_ = value; + onChanged(); + } else { + futureUuidBuilder_.setMessage(value); } - result.hasFutureUuid = true; - result.futureUuid_ = value; + bitField0_ |= 0x00000004; return this; } - public Builder setFutureUuid(akka.cluster.ClusterProtocol.UuidProtocol.Builder builderForValue) { - result.hasFutureUuid = true; - result.futureUuid_ = builderForValue.build(); + public Builder setFutureUuid( + akka.cluster.ClusterProtocol.UuidProtocol.Builder builderForValue) { + if (futureUuidBuilder_ == null) { + futureUuid_ = builderForValue.build(); + onChanged(); + } else { + futureUuidBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; return this; } public Builder mergeFutureUuid(akka.cluster.ClusterProtocol.UuidProtocol value) { - if (result.hasFutureUuid() && - result.futureUuid_ != akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance()) { - result.futureUuid_ = - akka.cluster.ClusterProtocol.UuidProtocol.newBuilder(result.futureUuid_).mergeFrom(value).buildPartial(); + if (futureUuidBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + futureUuid_ != akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance()) { + futureUuid_ = + akka.cluster.ClusterProtocol.UuidProtocol.newBuilder(futureUuid_).mergeFrom(value).buildPartial(); + } else { + futureUuid_ = value; + } + onChanged(); } else { - result.futureUuid_ = value; + futureUuidBuilder_.mergeFrom(value); } - result.hasFutureUuid = true; + bitField0_ |= 0x00000004; return this; } public Builder clearFutureUuid() { - result.hasFutureUuid = false; - result.futureUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + if (futureUuidBuilder_ == null) { + futureUuid_ = akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); + onChanged(); + } else { + futureUuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); return this; } + public akka.cluster.ClusterProtocol.UuidProtocol.Builder getFutureUuidBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getFutureUuidFieldBuilder().getBuilder(); + } + public akka.cluster.ClusterProtocol.UuidProtocolOrBuilder getFutureUuidOrBuilder() { + if (futureUuidBuilder_ != null) { + return futureUuidBuilder_.getMessageOrBuilder(); + } else { + return futureUuid_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.cluster.ClusterProtocol.UuidProtocol, akka.cluster.ClusterProtocol.UuidProtocol.Builder, akka.cluster.ClusterProtocol.UuidProtocolOrBuilder> + getFutureUuidFieldBuilder() { + if (futureUuidBuilder_ == null) { + futureUuidBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.cluster.ClusterProtocol.UuidProtocol, akka.cluster.ClusterProtocol.UuidProtocol.Builder, akka.cluster.ClusterProtocol.UuidProtocolOrBuilder>( + futureUuid_, + getParentForChildren(), + isClean()); + futureUuid_ = null; + } + return futureUuidBuilder_; + } // required bytes message = 4; + private com.google.protobuf.ByteString message_ = com.google.protobuf.ByteString.EMPTY; public boolean hasMessage() { - return result.hasMessage(); + return ((bitField0_ & 0x00000008) == 0x00000008); } public com.google.protobuf.ByteString getMessage() { - return result.getMessage(); + return message_; } public Builder setMessage(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - result.hasMessage = true; - result.message_ = value; + bitField0_ |= 0x00000008; + message_ = value; + onChanged(); return this; } public Builder clearMessage() { - result.hasMessage = false; - result.message_ = getDefaultInstance().getMessage(); + bitField0_ = (bitField0_ & ~0x00000008); + message_ = getDefaultInstance().getMessage(); + onChanged(); return this; } @@ -978,18 +1471,29 @@ public final class ClusterProtocol { static { defaultInstance = new DurableMailboxMessageProtocol(true); - akka.cluster.ClusterProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:DurableMailboxMessageProtocol) } + public interface UuidProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required uint64 high = 1; + boolean hasHigh(); + long getHigh(); + + // required uint64 low = 2; + boolean hasLow(); + long getLow(); + } public static final class UuidProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements UuidProtocolOrBuilder { // Use UuidProtocol.newBuilder() to construct. - private UuidProtocol() { - initFields(); + private UuidProtocol(Builder builder) { + super(builder); } private UuidProtocol(boolean noInit) {} @@ -1012,36 +1516,56 @@ public final class ClusterProtocol { return akka.cluster.ClusterProtocol.internal_static_UuidProtocol_fieldAccessorTable; } + private int bitField0_; // required uint64 high = 1; public static final int HIGH_FIELD_NUMBER = 1; - private boolean hasHigh; - private long high_ = 0L; - public boolean hasHigh() { return hasHigh; } - public long getHigh() { return high_; } + private long high_; + public boolean hasHigh() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getHigh() { + return high_; + } // required uint64 low = 2; public static final int LOW_FIELD_NUMBER = 2; - private boolean hasLow; - private long low_ = 0L; - public boolean hasLow() { return hasLow; } - public long getLow() { return low_; } + private long low_; + public boolean hasLow() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getLow() { + return low_; + } private void initFields() { + high_ = 0L; + low_ = 0L; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasHigh) return false; - if (!hasLow) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasHigh()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasLow()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasHigh()) { - output.writeUInt64(1, getHigh()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, high_); } - if (hasLow()) { - output.writeUInt64(2, getLow()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, low_); } getUnknownFields().writeTo(output); } @@ -1052,19 +1576,26 @@ public final class ClusterProtocol { if (size != -1) return size; size = 0; - if (hasHigh()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(1, getHigh()); + .computeUInt64Size(1, high_); } - if (hasLow()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(2, getLow()); + .computeUInt64Size(2, low_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.cluster.ClusterProtocol.UuidProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -1139,34 +1670,53 @@ public final class ClusterProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.cluster.ClusterProtocol.UuidProtocol result; - - // Construct using akka.cluster.ClusterProtocol.UuidProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.cluster.ClusterProtocol.UuidProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.cluster.ClusterProtocol.UuidProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.cluster.ClusterProtocol.internal_static_UuidProtocol_descriptor; } - protected akka.cluster.ClusterProtocol.UuidProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.cluster.ClusterProtocol.internal_static_UuidProtocol_fieldAccessorTable; + } + + // Construct using akka.cluster.ClusterProtocol.UuidProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.cluster.ClusterProtocol.UuidProtocol(); + super.clear(); + high_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + low_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -1178,33 +1728,39 @@ public final class ClusterProtocol { return akka.cluster.ClusterProtocol.UuidProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.cluster.ClusterProtocol.UuidProtocol build() { - if (result != null && !isInitialized()) { + akka.cluster.ClusterProtocol.UuidProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.cluster.ClusterProtocol.UuidProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.cluster.ClusterProtocol.UuidProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.cluster.ClusterProtocol.UuidProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.cluster.ClusterProtocol.UuidProtocol result = new akka.cluster.ClusterProtocol.UuidProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.cluster.ClusterProtocol.UuidProtocol returnMe = result; - result = null; - return returnMe; + result.high_ = high_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.low_ = low_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -1228,6 +1784,18 @@ public final class ClusterProtocol { return this; } + public final boolean isInitialized() { + if (!hasHigh()) { + + return false; + } + if (!hasLow()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -1240,61 +1808,72 @@ public final class ClusterProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 8: { - setHigh(input.readUInt64()); + bitField0_ |= 0x00000001; + high_ = input.readUInt64(); break; } case 16: { - setLow(input.readUInt64()); + bitField0_ |= 0x00000002; + low_ = input.readUInt64(); break; } } } } + private int bitField0_; // required uint64 high = 1; + private long high_ ; public boolean hasHigh() { - return result.hasHigh(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public long getHigh() { - return result.getHigh(); + return high_; } public Builder setHigh(long value) { - result.hasHigh = true; - result.high_ = value; + bitField0_ |= 0x00000001; + high_ = value; + onChanged(); return this; } public Builder clearHigh() { - result.hasHigh = false; - result.high_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + high_ = 0L; + onChanged(); return this; } // required uint64 low = 2; + private long low_ ; public boolean hasLow() { - return result.hasLow(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public long getLow() { - return result.getLow(); + return low_; } public Builder setLow(long value) { - result.hasLow = true; - result.low_ = value; + bitField0_ |= 0x00000002; + low_ = value; + onChanged(); return this; } public Builder clearLow() { - result.hasLow = false; - result.low_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + low_ = 0L; + onChanged(); return this; } @@ -1303,7 +1882,6 @@ public final class ClusterProtocol { static { defaultInstance = new UuidProtocol(true); - akka.cluster.ClusterProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -1390,7 +1968,5 @@ public final class ClusterProtocol { }, assigner); } - public static void internalForceInit() {} - // @@protoc_insertion_point(outer_class_scope) } diff --git a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala index 37bff63270..99697a2bfd 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Cluster.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Cluster.scala @@ -29,6 +29,7 @@ import Helpers._ import akka.actor._ import Actor._ import Status._ +import DeploymentConfig.{ ReplicationStrategy, Transient, WriteThrough, WriteBehind } import akka.event.EventHandler import akka.dispatch.{ Dispatchers, Future } import akka.remoteinterface._ @@ -463,7 +464,15 @@ class DefaultClusterNode private[akka] ( * available durable store. */ def store[T <: Actor](address: String, actorClass: Class[T], format: Serializer): ClusterNode = - store(Actor.actorOf(actorClass, address).start, 0, false, format) + store(Actor.actorOf(actorClass, address).start, 0, Transient, false, format) + + /** + * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store[T <: Actor](address: String, actorClass: Class[T], replicationStrategy: ReplicationStrategy, format: Serializer): ClusterNode = + store(Actor.actorOf(actorClass, address).start, 0, replicationStrategy, false, format) /** * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated @@ -471,7 +480,15 @@ class DefaultClusterNode private[akka] ( * available durable store. */ def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, format: Serializer): ClusterNode = - store(Actor.actorOf(actorClass, address).start, replicationFactor, false, format) + store(Actor.actorOf(actorClass, address).start, replicationFactor, Transient, false, format) + + /** + * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, replicationStrategy: ReplicationStrategy, format: Serializer): ClusterNode = + store(Actor.actorOf(actorClass, address).start, replicationFactor, replicationStrategy, false, format) /** * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated @@ -479,7 +496,15 @@ class DefaultClusterNode private[akka] ( * available durable store. */ def store[T <: Actor](address: String, actorClass: Class[T], serializeMailbox: Boolean, format: Serializer): ClusterNode = - store(Actor.actorOf(actorClass, address).start, 0, serializeMailbox, format) + store(Actor.actorOf(actorClass, address).start, 0, Transient, serializeMailbox, format) + + /** + * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store[T <: Actor](address: String, actorClass: Class[T], replicationStrategy: ReplicationStrategy, serializeMailbox: Boolean, format: Serializer): ClusterNode = + store(Actor.actorOf(actorClass, address).start, 0, replicationStrategy, serializeMailbox, format) /** * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated @@ -487,7 +512,15 @@ class DefaultClusterNode private[akka] ( * available durable store. */ def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode = - store(Actor.actorOf(actorClass, address).start, replicationFactor, serializeMailbox, format) + store(Actor.actorOf(actorClass, address).start, replicationFactor, Transient, serializeMailbox, format) + + /** + * Clusters an actor of a specific type. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store[T <: Actor](address: String, actorClass: Class[T], replicationFactor: Int, replicationStrategy: ReplicationStrategy, serializeMailbox: Boolean, format: Serializer): ClusterNode = + store(Actor.actorOf(actorClass, address).start, replicationFactor, replicationStrategy, serializeMailbox, format) /** * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated @@ -495,7 +528,15 @@ class DefaultClusterNode private[akka] ( * available durable store. */ def store(actorRef: ActorRef, format: Serializer): ClusterNode = - store(actorRef, 0, false, format) + store(actorRef, 0, Transient, false, format) + + /** + * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store(actorRef: ActorRef, replicationStrategy: ReplicationStrategy, format: Serializer): ClusterNode = + store(actorRef, 0, replicationStrategy, false, format) /** * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated @@ -503,7 +544,15 @@ class DefaultClusterNode private[akka] ( * available durable store. */ def store(actorRef: ActorRef, replicationFactor: Int, format: Serializer): ClusterNode = - store(actorRef, replicationFactor, false, format) + store(actorRef, replicationFactor, Transient, false, format) + + /** + * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store(actorRef: ActorRef, replicationFactor: Int, replicationStrategy: ReplicationStrategy, format: Serializer): ClusterNode = + store(actorRef, replicationFactor, replicationStrategy, false, format) /** * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated @@ -511,20 +560,47 @@ class DefaultClusterNode private[akka] ( * available durable store. */ def store(actorRef: ActorRef, serializeMailbox: Boolean, format: Serializer): ClusterNode = - store(actorRef, 0, serializeMailbox, format) - - /** - * Needed to have reflection through structural typing work. - */ - def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: AnyRef): ClusterNode = - store(actorRef, replicationFactor, serializeMailbox, format.asInstanceOf[Serializer]) + store(actorRef, 0, Transient, serializeMailbox, format) /** * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly * available durable store. */ - def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode = if (isConnected.isOn) { + def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: Serializer): ClusterNode = + store(actorRef, replicationFactor, Transient, serializeMailbox, format) + + /** + * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store(actorRef: ActorRef, replicationStrategy: ReplicationStrategy, serializeMailbox: Boolean, format: Serializer): ClusterNode = + store(actorRef, 0, replicationStrategy, serializeMailbox, format) + + /** + * Needed to have reflection through structural typing work. + */ + def store(actorRef: ActorRef, replicationFactor: Int, replicationStrategy: ReplicationStrategy, serializeMailbox: Boolean, format: AnyRef): ClusterNode = + store(actorRef, replicationFactor, replicationStrategy, serializeMailbox, format.asInstanceOf[Serializer]) + + /** + * Needed to have reflection through structural typing work. + */ + def store(actorRef: ActorRef, replicationFactor: Int, serializeMailbox: Boolean, format: AnyRef): ClusterNode = + store(actorRef, replicationFactor, Transient, serializeMailbox, format) + + /** + * Clusters an actor with UUID. If the actor is already clustered then the clustered version will be updated + * with the actor passed in as argument. You can use this to save off snapshots of the actor to a highly + * available durable store. + */ + def store( + actorRef: ActorRef, + replicationFactor: Int, + replicationStrategy: ReplicationStrategy, + serializeMailbox: Boolean, + format: Serializer): ClusterNode = if (isConnected.isOn) { import akka.serialization.ActorSerialization._ @@ -535,12 +611,14 @@ class DefaultClusterNode private[akka] ( EventHandler.debug(this, "Storing actor [%s] with UUID [%s] in cluster".format(actorRef.address, uuid)) - val actorBytes = if (shouldCompressData) LZF.compress(toBinary(actorRef, serializeMailbox)(format)) - else toBinary(actorRef)(format) + val actorBytes = + if (shouldCompressData) LZF.compress(toBinary(actorRef, serializeMailbox, replicationStrategy)(format)) + else toBinary(actorRef, serializeMailbox, replicationStrategy)(format) + val actorRegistryPath = actorRegistryPathFor(uuid) // create UUID -> Array[Byte] for actor registry - if (zkClient.exists(actorRegistryPath)) zkClient.writeData(actorRegistryPath, actorBytes) // FIXME check for size and warn if too big + if (zkClient.exists(actorRegistryPath)) zkClient.writeData(actorRegistryPath, actorBytes) // FIXME Store in Data Grid not ZooKeeper else { zkClient.retryUntilConnected(new Callable[Either[String, Exception]]() { def call: Either[String, Exception] = { @@ -590,9 +668,7 @@ class DefaultClusterNode private[akka] ( (connection !! (command, remoteDaemonAckTimeout)) match { case Some(Success) ⇒ - EventHandler.debug(this, - "Replica for [%s] successfully created on [%s]" - .format(actorRef.address, connection)) + EventHandler.debug(this, "Replica for [%s] successfully created".format(actorRef.address)) case Some(Failure(cause)) ⇒ EventHandler.error(cause, this, cause.toString) @@ -616,8 +692,9 @@ class DefaultClusterNode private[akka] ( releaseActorOnAllNodes(uuid) locallyCheckedOutActors.remove(uuid) + // warning: ordering matters here - ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAddressToUuidsPathFor(actorAddressForUuid(uuid)))) // remove ADDRESS to UUID mapping + ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAddressToUuidsPathFor(actorAddressForUuid(uuid)))) // FIXME remove ADDRESS to UUID mapping? ignore[ZkNoNodeException](zkClient.deleteRecursive(actorAtNodePathFor(nodeAddress.nodeName, uuid))) ignore[ZkNoNodeException](zkClient.deleteRecursive(actorRegistryPathFor(uuid))) ignore[ZkNoNodeException](zkClient.deleteRecursive(actorLocationsPathFor(uuid))) @@ -662,20 +739,17 @@ class DefaultClusterNode private[akka] ( * Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available * for remote access through lookup by its UUID. */ - def use[T <: Actor](actorAddress: String): Option[LocalActorRef] = use(actorAddress, formatForActor(actorAddress)) + def use[T <: Actor](actorAddress: String): Option[ActorRef] = use(actorAddress, formatForActor(actorAddress)) /** * Checks out an actor for use on this node, e.g. checked out as a 'LocalActorRef' but it makes it available * for remote access through lookup by its UUID. */ - def use[T <: Actor](actorAddress: String, format: Serializer): Option[LocalActorRef] = if (isConnected.isOn) { + def use[T <: Actor](actorAddress: String, format: Serializer): Option[ActorRef] = if (isConnected.isOn) { import akka.serialization.ActorSerialization._ actorUuidsForActorAddress(actorAddress) map { uuid ⇒ - EventHandler.debug(this, - "Checking out actor with UUID [%s] to be used on node [%s] as local actor" - .format(uuid, nodeAddress.nodeName)) ignore[ZkNodeExistsException](zkClient.createPersistent(actorAtNodePathFor(nodeAddress.nodeName, uuid), true)) ignore[ZkNodeExistsException](zkClient.createEphemeral(actorLocationsPathFor(uuid, nodeAddress))) @@ -697,12 +771,12 @@ class DefaultClusterNode private[akka] ( }) match { case Left(bytes) ⇒ locallyCheckedOutActors += (uuid -> bytes) - // FIXME switch to ReplicatedActorRef here - // val actor = new ReplicatedActorRef(fromBinary[T](bytes, remoteServerAddress)(format)) val actor = fromBinary[T](bytes, remoteServerAddress)(format) - // remoteService.register(UUID_PREFIX + uuid, actor) // FIXME is Actor.remote.register(UUID, ..) correct here? + EventHandler.debug(this, + "Checking out actor [%s] to be used on node [%s] as local actor" + .format(actor, nodeAddress.nodeName)) actor.start() - actor.asInstanceOf[LocalActorRef] + actor case Right(exception) ⇒ throw exception } } headOption // FIXME should not be an array at all coming here but an Option[ActorRef] @@ -786,8 +860,8 @@ class DefaultClusterNode private[akka] ( def ref(actorAddress: String, router: RouterType): ActorRef = if (isConnected.isOn) { val addresses = addressesForActor(actorAddress) EventHandler.debug(this, - "Checking out cluster actor ref with address [%s] and router [%s] connected to [\n\t%s]" - .format(actorAddress, router, addresses.mkString("\n\t"))) + "Checking out cluster actor ref with address [%s] and router [%s] on [%s] connected to [\n\t%s]" + .format(actorAddress, router, remoteServerAddress, addresses.map(_._2).mkString("\n\t"))) val actorRef = Router newRouter (router, addresses, actorAddress, Actor.TIMEOUT) addresses foreach { case (_, address) ⇒ clusterActorRefs.put(address, actorRef) } diff --git a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala index f107904892..b7d5df6f10 100644 --- a/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala +++ b/akka-cluster/src/main/scala/akka/cluster/ClusterActorRef.scala @@ -6,7 +6,7 @@ package akka.cluster import Cluster._ import akka.actor._ -import akka.actor.Actor._ +import Actor._ import akka.event.EventHandler import akka.dispatch.Promise @@ -21,15 +21,10 @@ import com.eaio.uuid.UUID class ClusterActorRef private[akka] ( inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], actorAddress: String, - timeout: Long, - val replicationStrategy: ReplicationStrategy) + timeout: Long) extends RemoteActorRef(null, actorAddress, timeout, None) { // FIXME UGLY HACK - should not extend RemoteActorRef this: ClusterActorRef with Router.Router ⇒ - EventHandler.debug(this, - "Creating a ClusterActorRef for actor with address [%s] with connections [\n\t%s]" - .format(actorAddress, inetSocketAddresses.mkString("\n\t"))) - private[akka] val inetSocketAddressToActorRefMap = new AtomicReference[Map[InetSocketAddress, ActorRef]]( (Map[InetSocketAddress, ActorRef]() /: inetSocketAddresses) { case (map, (uuid, inetSocketAddress)) ⇒ map + (inetSocketAddress -> createRemoteActorRef(actorAddress, inetSocketAddress)) diff --git a/akka-cluster/src/main/scala/akka/cluster/ReplicatedClusterRef.scala b/akka-cluster/src/main/scala/akka/cluster/ReplicatedClusterRef.scala deleted file mode 100644 index 4b075c7f91..0000000000 --- a/akka-cluster/src/main/scala/akka/cluster/ReplicatedClusterRef.scala +++ /dev/null @@ -1,105 +0,0 @@ -package akka.cluster - -/** - * Copyright (C) 2009-2011 Scalable Solutions AB - */ -import Cluster._ - -import akka.actor._ -import akka.remote.MessageSerializer -import akka.event.EventHandler -import akka.config.Supervision._ -import akka.dispatch._ - -import java.net.InetSocketAddress -import java.util.concurrent.atomic.AtomicReference -import java.util.{ Map ⇒ JMap } - -/** - * @author Jonas Bonér - */ -trait Replicable { this: Actor ⇒ -} - -/** - * @author Jonas Bonér - */ -sealed trait ReplicationStrategy - -object ReplicationStrategy { - case object Transient extends ReplicationStrategy - case object WriteThrough extends ReplicationStrategy - case object WriteBehind extends ReplicationStrategy -} - -/** - * @author Jonas Bonér - */ -class ReplicatedActorRef private[akka] (actorRef: ActorRef, val address: String) extends ActorRef with ScalaActorRef { - - private lazy val txLog = { - EventHandler.debug(this, "Creating a ReplicatedActorRef for Actor [%s]".format(address)) - TransactionLog.newLogFor(uuid.toString) - } - - def invoke(messageHandle: MessageInvocation) { - actorRef.invoke(messageHandle) - txLog.recordEntry(MessageSerializer.serialize(messageHandle.message).toByteArray) - } - - def start(): ActorRef = { - EventHandler.debug(this, "Starting ReplicatedActorRef for Actor [%s] with transaction log [%s]" - .format(address, txLog.logId)) - actorRef.start() - } - - def stop() { - txLog.delete() - actorRef.stop() - } - - override def setFaultHandler(handler: FaultHandlingStrategy) { - actorRef.setFaultHandler(handler) - } - override def getFaultHandler: FaultHandlingStrategy = actorRef.getFaultHandler() - override def setLifeCycle(lifeCycle: LifeCycle) { - actorRef.setLifeCycle(lifeCycle) - } - override def getLifeCycle: LifeCycle = actorRef.getLifeCycle - def dispatcher_=(md: MessageDispatcher) { - actorRef.dispatcher_=(md) - } - def dispatcher: MessageDispatcher = actorRef.dispatcher - def link(actorRef: ActorRef) { - actorRef.link(actorRef) - } - def unlink(actorRef: ActorRef) { - actorRef.unlink(actorRef) - } - def startLink(actorRef: ActorRef): ActorRef = actorRef.startLink(actorRef) - def supervisor: Option[ActorRef] = actorRef.supervisor - def linkedActors: JMap[Uuid, ActorRef] = actorRef.linkedActors - protected[akka] def postMessageToMailbox(message: Any, senderOption: Option[ActorRef]) { - actorRef.postMessageToMailbox(message, senderOption) - } - protected[akka] def postMessageToMailboxAndCreateFutureResultWithTimeout[T]( - message: Any, - timeout: Long, - senderOption: Option[ActorRef], - senderFuture: Option[Promise[T]]): Promise[T] = actorRef.postMessageToMailboxAndCreateFutureResultWithTimeout(message, timeout, senderOption, senderFuture) - protected[akka] def actorInstance: AtomicReference[Actor] = actorRef.actorInstance - protected[akka] def supervisor_=(sup: Option[ActorRef]) { - actorRef.supervisor_=(sup) - } - protected[akka] def mailbox: AnyRef = actorRef.mailbox - protected[akka] def mailbox_=(value: AnyRef): AnyRef = actorRef.mailbox_=(value) - protected[akka] def handleTrapExit(dead: ActorRef, reason: Throwable) { - actorRef.handleTrapExit(dead, reason) - } - protected[akka] def restart(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) { - actorRef.restart(reason, maxNrOfRetries, withinTimeRange) - } - protected[akka] def restartLinkedActors(reason: Throwable, maxNrOfRetries: Option[Int], withinTimeRange: Option[Int]) { - actorRef.restartLinkedActors(reason, maxNrOfRetries, withinTimeRange) - } -} diff --git a/akka-cluster/src/main/scala/akka/cluster/Routing.scala b/akka-cluster/src/main/scala/akka/cluster/Routing.scala index 1bde759ca6..d3bc4904f7 100644 --- a/akka-cluster/src/main/scala/akka/cluster/Routing.scala +++ b/akka-cluster/src/main/scala/akka/cluster/Routing.scala @@ -27,12 +27,11 @@ object Router { routerType: RouterType, inetSocketAddresses: Array[Tuple2[UUID, InetSocketAddress]], actorAddress: String, - timeout: Long, - replicationStrategy: ReplicationStrategy = ReplicationStrategy.WriteThrough): ClusterActorRef = { + timeout: Long): ClusterActorRef = { routerType match { - case Direct ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout, replicationStrategy) with Direct - case Random ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout, replicationStrategy) with Random - case RoundRobin ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout, replicationStrategy) with RoundRobin + case Direct ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Direct + case Random ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with Random + case RoundRobin ⇒ new ClusterActorRef(inetSocketAddresses, actorAddress, timeout) with RoundRobin case LeastCPU ⇒ sys.error("Router LeastCPU not supported yet") case LeastRAM ⇒ sys.error("Router LeastRAM not supported yet") case LeastMessages ⇒ sys.error("Router LeastMessages not supported yet") diff --git a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala index f5c96250b4..8658ba1825 100644 --- a/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala +++ b/akka-cluster/src/main/scala/akka/cluster/TransactionLog.scala @@ -9,16 +9,23 @@ import org.apache.zookeeper.CreateMode import org.I0Itec.zkclient.exception._ +import akka.AkkaException import akka.config._ import Config._ import akka.util._ +import akka.actor._ +import DeploymentConfig.{ ReplicationStrategy, Transient, WriteThrough, WriteBehind } import akka.event.EventHandler -import akka.dispatch.{ DefaultPromise, Promise } -import akka.AkkaException - +import akka.dispatch.{ DefaultPromise, Promise, MessageInvocation } +import akka.remote.MessageSerializer +import akka.serialization.ActorSerialization._ import akka.cluster.zookeeper._ +import akka.serialization.{ Serializer, Compression } +import Compression.LZF +import akka.serialization.ActorSerialization._ import java.util.Enumeration +import java.util.concurrent.atomic.AtomicLong // FIXME allow user to choose dynamically between 'async' and 'sync' tx logging (asyncAddEntry(byte[] data, AddCallback cb, Object ctx)) // FIXME clean up old entries in log after doing a snapshot @@ -41,25 +48,45 @@ class ReplicationException(message: String) extends AkkaException(message) * * @author Jonas Bonér */ -class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: Boolean) { +class TransactionLog private ( + ledger: LedgerHandle, + val id: String, + val isAsync: Boolean) { import TransactionLog._ val logId = ledger.getId val txLogPath = transactionLogNode + "/" + id val snapshotPath = txLogPath + "/snapshot" + val nrOfEntries = new AtomicLong(0) private val isOpen = new Switch(true) + /** + * TODO document method + */ + def recordEntry(messageHandle: MessageInvocation, actorRef: ActorRef, format: Serializer) { + if (nrOfEntries.incrementAndGet % snapshotFrequency == 0) { + val snapshot = + // FIXME ReplicationStrategy Transient is always used + if (Cluster.shouldCompressData) LZF.compress(toBinary(actorRef, false, Transient)(format)) + else toBinary(actorRef, false, Transient)(format) + recordSnapshot(snapshot) + } + recordEntry(MessageSerializer.serialize(messageHandle.message).toByteArray) + } + /** * TODO document method */ def recordEntry(entry: Array[Byte]) { if (isOpen.isOn) { + val bytes = if (Cluster.shouldCompressData) LZF.compress(entry) + else entry try { if (isAsync) { ledger.asyncAddEntry( - entry, + bytes, new AsyncCallback.AddCallback { def addComplete( returnCode: Int, @@ -73,7 +100,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: }, null) } else { - handleReturnCode(ledger.addEntry(entry)) + handleReturnCode(ledger.addEntry(bytes)) val entryId = ledger.getLastAddPushed EventHandler.debug(this, "Writing entry [%s] to log [%s]".format(entryId, logId)) } @@ -88,10 +115,12 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: */ def recordSnapshot(snapshot: Array[Byte]) { if (isOpen.isOn) { + val bytes = if (Cluster.shouldCompressData) LZF.compress(snapshot) + else snapshot try { if (isAsync) { ledger.asyncAddEntry( - snapshot, + bytes, new AsyncCallback.AddCallback { def addComplete( returnCode: Int, @@ -104,7 +133,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: }, null) } else { - handleReturnCode(ledger.addEntry(snapshot)) + handleReturnCode(ledger.addEntry(bytes)) storeSnapshotMetaDataInZooKeeper(ledger.getLastAddPushed) } } catch { @@ -121,7 +150,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: /** * TODO document method */ - def entriesFromLatestSnapshot: Tuple2[Array[Byte], Vector[Array[Byte]]] = { + def toByteArraysLatestSnapshot: (Array[Byte], Vector[Array[Byte]]) = { val snapshotId = latestSnapshotId EventHandler.debug(this, "Reading entries from snapshot id [%s] for log [%s]".format(snapshotId, logId)) @@ -133,9 +162,9 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: */ def entriesInRange(from: Long, to: Long): Vector[Array[Byte]] = if (isOpen.isOn) { try { - if (from < 0) throw new IllegalArgumentException("'from' can't be negative [" + from + "]") - if (to < 0) throw new IllegalArgumentException("'to' can't be negative [" + from + "]") - if (to < from) throw new IllegalArgumentException("'to' can't be smaller than 'from' [" + from + "," + to + "]") + if (from < 0) throw new IllegalArgumentException("'from' index can't be negative [" + from + "]") + if (to < 0) throw new IllegalArgumentException("'to' index can't be negative [" + from + "]") + if (to < from) throw new IllegalArgumentException("'to' index can't be smaller than 'from' index [" + from + "," + to + "]") EventHandler.debug(this, "Reading entries [%s -> %s] for log [%s]".format(from, to, logId)) @@ -150,10 +179,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: enumeration: Enumeration[LedgerEntry], ctx: AnyRef) { val future = ctx.asInstanceOf[Promise[Vector[Array[Byte]]]] - var entries = Vector[Array[Byte]]() - while (enumeration.hasMoreElements) { - entries = entries :+ enumeration.nextElement.getEntry - } + val entries = toByteArrays(enumeration) if (returnCode == BKException.Code.OK) future.completeWithResult(entries) else future.completeWithException(BKException.create(returnCode)) } @@ -161,12 +187,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: future) await(future) } else { - val enumeration = ledger.readEntries(from, to) - var entries = Vector[Array[Byte]]() - while (enumeration.hasMoreElements) { - entries = entries :+ enumeration.nextElement.getEntry - } - entries + toByteArrays(ledger.readEntries(from, to)) } } catch { case e ⇒ handleError(e) @@ -190,8 +211,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: } catch { case e: ZkNoNodeException ⇒ handleError(new ReplicationException( - "Transaction log for UUID [" + id + - "] does not have a snapshot recorded in ZooKeeper")) + "Transaction log for UUID [" + id + "] does not have a snapshot recorded in ZooKeeper")) case e ⇒ handleError(e) } } @@ -208,7 +228,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: logId, new AsyncCallback.DeleteCallback { def deleteComplete(returnCode: Int, ctx: AnyRef) { - handleReturnCode(returnCode) + (returnCode) } }, null) @@ -248,6 +268,18 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: } } + private def toByteArrays(enumeration: Enumeration[LedgerEntry]): Vector[Array[Byte]] = { + var entries = Vector[Array[Byte]]() + while (enumeration.hasMoreElements) { + val bytes = enumeration.nextElement.getEntry + val entry = + if (Cluster.shouldCompressData) LZF.uncompress(bytes) + else bytes + entries = entries :+ entry + } + entries + } + private def storeSnapshotMetaDataInZooKeeper(snapshotId: Long) { if (isOpen.isOn) { try { @@ -265,8 +297,7 @@ class TransactionLog private (ledger: LedgerHandle, val id: String, val isAsync: "Could not store transaction log snapshot meta-data in ZooKeeper for UUID [" + id + "]")) } - EventHandler.debug(this, - "Writing snapshot [%s] to log [%s]".format(snapshotId, logId)) + EventHandler.debug(this, "Writing snapshot [%s] to log [%s]".format(snapshotId, logId)) } else transactionClosedError } @@ -292,12 +323,13 @@ object TransactionLog { case "CRC32" ⇒ BookKeeper.DigestType.CRC32 case "MAC" ⇒ BookKeeper.DigestType.MAC case unknown ⇒ throw new ConfigurationException( - "akka.cluster.replication.digest-type is invalid [" + unknown + "]") + "akka.cluster.replication.digest-type is invalid [" + unknown + "], must be either 'CRC32' or 'MAC'") } val password = config.getString("akka.cluster.replication.password", "secret").getBytes("UTF-8") val ensembleSize = config.getInt("akka.cluster.replication.ensemble-size", 3) val quorumSize = config.getInt("akka.cluster.replication.quorum-size", 2) - val timeout = 5000 // FIXME make configurable + val snapshotFrequency = config.getInt("akka.cluster.replication.snapshot-frequency", 1000) + val timeout = Duration(config.getInt("akka.cluster.replication.timeout", 30), TIME_UNIT).toMillis private[akka] val transactionLogNode = "/transaction-log-ids" @@ -489,15 +521,10 @@ object LocalBookKeeperEnsemble { def shutdown() { isRunning switchOff { EventHandler.info(this, "Shutting down LocalBookKeeperEnsemble...") - println("***************************** 1") localBookKeeper.bs.foreach(_.shutdown()) // stop bookies - println("***************************** 2") localBookKeeper.zkc.close() // stop zk client - println("***************************** 3") localBookKeeper.zks.shutdown() // stop zk server - println("***************************** 4") localBookKeeper.serverFactory.shutdown() // stop zk NIOServer - println("***************************** 5") EventHandler.info(this, "LocalBookKeeperEnsemble shut down successfully") } } diff --git a/akka-cluster/src/test/scala/akka/cluster/ReplicationSpec.scala b/akka-cluster/src/test/scala/akka/cluster/ReplicationSpec.scala index 42423a3afc..831807a7a2 100644 --- a/akka-cluster/src/test/scala/akka/cluster/ReplicationSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/ReplicationSpec.scala @@ -106,7 +106,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll txlog1.close val txlog2 = TransactionLog.logFor(uuid) - val (snapshotAsBytes, entriesAsBytes) = txlog2.entriesFromLatestSnapshot + val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot new String(snapshotAsBytes, "UTF-8") must equal("snapshot") val entries = entriesAsBytes.map(bytes ⇒ new String(bytes, "UTF-8")) @@ -135,7 +135,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll txlog1.close val txlog2 = TransactionLog.logFor(uuid) - val (snapshotAsBytes, entriesAsBytes) = txlog2.entriesFromLatestSnapshot + val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot new String(snapshotAsBytes, "UTF-8") must equal("snapshot") val entries = entriesAsBytes.map(bytes ⇒ new String(bytes, "UTF-8")) @@ -230,7 +230,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll txlog1.close val txlog2 = TransactionLog.logFor(uuid, true) - val (snapshotAsBytes, entriesAsBytes) = txlog2.entriesFromLatestSnapshot + val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot new String(snapshotAsBytes, "UTF-8") must equal("snapshot") val entries = entriesAsBytes.map(bytes ⇒ new String(bytes, "UTF-8")) @@ -259,7 +259,7 @@ class ReplicationSpec extends WordSpec with MustMatchers with BeforeAndAfterAll txlog1.close val txlog2 = TransactionLog.logFor(uuid, true) - val (snapshotAsBytes, entriesAsBytes) = txlog2.entriesFromLatestSnapshot + val (snapshotAsBytes, entriesAsBytes) = txlog2.toByteArraysLatestSnapshot new String(snapshotAsBytes, "UTF-8") must equal("snapshot") val entries = entriesAsBytes.map(bytes ⇒ new String(bytes, "UTF-8")) entries.size must equal(2) diff --git a/akka-cluster/src/test/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmSpec.scala b/akka-cluster/src/test/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmSpec.scala index 1f6d60efc3..1c4841ee2e 100644 --- a/akka-cluster/src/test/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/routing/roundrobin_1_replica/RoundRobin1ReplicaMultiJvmSpec.scala @@ -8,6 +8,9 @@ import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import org.scalatest.BeforeAndAfterAll +import org.apache.bookkeeper.client.{ BookKeeper, BKException } +import BKException._ + import akka.cluster._ import akka.actor._ import Actor._ @@ -27,6 +30,9 @@ object RoundRobin1ReplicaMultiJvmSpec { class RoundRobin1ReplicaMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll { import RoundRobin1ReplicaMultiJvmSpec._ + private var bookKeeper: BookKeeper = _ + private var localBookKeeper: LocalBookKeeper = _ + "A cluster" must { "create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in { @@ -49,10 +55,13 @@ class RoundRobin1ReplicaMultiJvmNode1 extends WordSpec with MustMatchers with Be override def beforeAll() = { Cluster.startLocalCluster() + LocalBookKeeperEnsemble.start() } override def afterAll() = { Cluster.shutdownLocalCluster() + TransactionLog.shutdown() + LocalBookKeeperEnsemble.shutdown() } } diff --git a/akka-cluster/src/test/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala b/akka-cluster/src/test/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala index 9f3083868b..c9c864fc27 100644 --- a/akka-cluster/src/test/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala +++ b/akka-cluster/src/test/scala/akka/cluster/routing/roundrobin_2_replicas/RoundRobin2ReplicasMultiJvmSpec.scala @@ -8,6 +8,9 @@ import org.scalatest.WordSpec import org.scalatest.matchers.MustMatchers import org.scalatest.BeforeAndAfterAll +import org.apache.bookkeeper.client.{ BookKeeper, BKException } +import BKException._ + import akka.cluster._ import akka.actor._ import Actor._ @@ -28,6 +31,9 @@ object RoundRobin2ReplicasMultiJvmSpec { class RoundRobin2ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with BeforeAndAfterAll { import RoundRobin2ReplicasMultiJvmSpec._ + private var bookKeeper: BookKeeper = _ + private var localBookKeeper: LocalBookKeeper = _ + "A cluster" must { "create clustered actor, get a 'local' actor on 'home' node and a 'ref' to actor on remote node" in { @@ -52,10 +58,13 @@ class RoundRobin2ReplicasMultiJvmNode1 extends WordSpec with MustMatchers with B override def beforeAll() = { Cluster.startLocalCluster() + LocalBookKeeperEnsemble.start() } override def afterAll() = { Cluster.shutdownLocalCluster() + TransactionLog.shutdown() + LocalBookKeeperEnsemble.shutdown() } } diff --git a/akka-durable-mailboxes/akka-mailboxes-common/src/main/java/akka/actor/mailbox/MailboxProtocol.java b/akka-durable-mailboxes/akka-mailboxes-common/src/main/java/akka/actor/mailbox/MailboxProtocol.java index 1fcb87781c..bc128610cf 100644 --- a/akka-durable-mailboxes/akka-mailboxes-common/src/main/java/akka/actor/mailbox/MailboxProtocol.java +++ b/akka-durable-mailboxes/akka-mailboxes-common/src/main/java/akka/actor/mailbox/MailboxProtocol.java @@ -8,11 +8,32 @@ public final class MailboxProtocol { public static void registerAllExtensions( com.google.protobuf.ExtensionRegistry registry) { } + public interface DurableMailboxMessageProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string ownerAddress = 1; + boolean hasOwnerAddress(); + String getOwnerAddress(); + + // optional string senderAddress = 2; + boolean hasSenderAddress(); + String getSenderAddress(); + + // optional .UuidProtocol futureUuid = 3; + boolean hasFutureUuid(); + akka.actor.mailbox.MailboxProtocol.UuidProtocol getFutureUuid(); + akka.actor.mailbox.MailboxProtocol.UuidProtocolOrBuilder getFutureUuidOrBuilder(); + + // required bytes message = 4; + boolean hasMessage(); + com.google.protobuf.ByteString getMessage(); + } public static final class DurableMailboxMessageProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements DurableMailboxMessageProtocolOrBuilder { // Use DurableMailboxMessageProtocol.newBuilder() to construct. - private DurableMailboxMessageProtocol() { - initFields(); + private DurableMailboxMessageProtocol(Builder builder) { + super(builder); } private DurableMailboxMessageProtocol(boolean noInit) {} @@ -35,60 +56,137 @@ public final class MailboxProtocol { return akka.actor.mailbox.MailboxProtocol.internal_static_DurableMailboxMessageProtocol_fieldAccessorTable; } + private int bitField0_; // required string ownerAddress = 1; public static final int OWNERADDRESS_FIELD_NUMBER = 1; - private boolean hasOwnerAddress; - private java.lang.String ownerAddress_ = ""; - public boolean hasOwnerAddress() { return hasOwnerAddress; } - public java.lang.String getOwnerAddress() { return ownerAddress_; } + private java.lang.Object ownerAddress_; + public boolean hasOwnerAddress() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getOwnerAddress() { + java.lang.Object ref = ownerAddress_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + ownerAddress_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getOwnerAddressBytes() { + java.lang.Object ref = ownerAddress_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + ownerAddress_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // optional string senderAddress = 2; public static final int SENDERADDRESS_FIELD_NUMBER = 2; - private boolean hasSenderAddress; - private java.lang.String senderAddress_ = ""; - public boolean hasSenderAddress() { return hasSenderAddress; } - public java.lang.String getSenderAddress() { return senderAddress_; } + private java.lang.Object senderAddress_; + public boolean hasSenderAddress() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getSenderAddress() { + java.lang.Object ref = senderAddress_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + senderAddress_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getSenderAddressBytes() { + java.lang.Object ref = senderAddress_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + senderAddress_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // optional .UuidProtocol futureUuid = 3; public static final int FUTUREUUID_FIELD_NUMBER = 3; - private boolean hasFutureUuid; private akka.actor.mailbox.MailboxProtocol.UuidProtocol futureUuid_; - public boolean hasFutureUuid() { return hasFutureUuid; } - public akka.actor.mailbox.MailboxProtocol.UuidProtocol getFutureUuid() { return futureUuid_; } + public boolean hasFutureUuid() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public akka.actor.mailbox.MailboxProtocol.UuidProtocol getFutureUuid() { + return futureUuid_; + } + public akka.actor.mailbox.MailboxProtocol.UuidProtocolOrBuilder getFutureUuidOrBuilder() { + return futureUuid_; + } // required bytes message = 4; public static final int MESSAGE_FIELD_NUMBER = 4; - 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_; } + private com.google.protobuf.ByteString message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getMessage() { + return message_; + } private void initFields() { + ownerAddress_ = ""; + senderAddress_ = ""; futureUuid_ = akka.actor.mailbox.MailboxProtocol.UuidProtocol.getDefaultInstance(); + message_ = com.google.protobuf.ByteString.EMPTY; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasOwnerAddress) return false; - if (!hasMessage) return false; - if (hasFutureUuid()) { - if (!getFutureUuid().isInitialized()) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasOwnerAddress()) { + memoizedIsInitialized = 0; + return false; } + if (!hasMessage()) { + memoizedIsInitialized = 0; + return false; + } + if (hasFutureUuid()) { + if (!getFutureUuid().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasOwnerAddress()) { - output.writeString(1, getOwnerAddress()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getOwnerAddressBytes()); } - if (hasSenderAddress()) { - output.writeString(2, getSenderAddress()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getSenderAddressBytes()); } - if (hasFutureUuid()) { - output.writeMessage(3, getFutureUuid()); + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeMessage(3, futureUuid_); } - if (hasMessage()) { - output.writeBytes(4, getMessage()); + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, message_); } getUnknownFields().writeTo(output); } @@ -99,27 +197,34 @@ public final class MailboxProtocol { if (size != -1) return size; size = 0; - if (hasOwnerAddress()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getOwnerAddress()); + .computeBytesSize(1, getOwnerAddressBytes()); } - if (hasSenderAddress()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getSenderAddress()); + .computeBytesSize(2, getSenderAddressBytes()); } - if (hasFutureUuid()) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(3, getFutureUuid()); + .computeMessageSize(3, futureUuid_); } - if (hasMessage()) { + if (((bitField0_ & 0x00000008) == 0x00000008)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(4, getMessage()); + .computeBytesSize(4, message_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -194,34 +299,62 @@ public final class MailboxProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol result; - - // Construct using akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.actor.mailbox.MailboxProtocol.internal_static_DurableMailboxMessageProtocol_descriptor; } - protected akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.actor.mailbox.MailboxProtocol.internal_static_DurableMailboxMessageProtocol_fieldAccessorTable; + } + + // Construct using akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getFutureUuidFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); + super.clear(); + ownerAddress_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + senderAddress_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + if (futureUuidBuilder_ == null) { + futureUuid_ = akka.actor.mailbox.MailboxProtocol.UuidProtocol.getDefaultInstance(); + } else { + futureUuidBuilder_.clear(); } - result = new akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol(); + bitField0_ = (bitField0_ & ~0x00000004); + message_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -233,33 +366,51 @@ public final class MailboxProtocol { return akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol build() { - if (result != null && !isInitialized()) { + akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol result = new akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.actor.mailbox.MailboxProtocol.DurableMailboxMessageProtocol returnMe = result; - result = null; - return returnMe; + result.ownerAddress_ = ownerAddress_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.senderAddress_ = senderAddress_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + if (futureUuidBuilder_ == null) { + result.futureUuid_ = futureUuid_; + } else { + result.futureUuid_ = futureUuidBuilder_.build(); + } + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -289,6 +440,24 @@ public final class MailboxProtocol { return this; } + public final boolean isInitialized() { + if (!hasOwnerAddress()) { + + return false; + } + if (!hasMessage()) { + + return false; + } + if (hasFutureUuid()) { + if (!getFutureUuid().isInitialized()) { + + return false; + } + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -301,21 +470,25 @@ public final class MailboxProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 10: { - setOwnerAddress(input.readString()); + bitField0_ |= 0x00000001; + ownerAddress_ = input.readBytes(); break; } case 18: { - setSenderAddress(input.readString()); + bitField0_ |= 0x00000002; + senderAddress_ = input.readBytes(); break; } case 26: { @@ -328,111 +501,199 @@ public final class MailboxProtocol { break; } case 34: { - setMessage(input.readBytes()); + bitField0_ |= 0x00000008; + message_ = input.readBytes(); break; } } } } + private int bitField0_; // required string ownerAddress = 1; + private java.lang.Object ownerAddress_ = ""; public boolean hasOwnerAddress() { - return result.hasOwnerAddress(); + return ((bitField0_ & 0x00000001) == 0x00000001); } - public java.lang.String getOwnerAddress() { - return result.getOwnerAddress(); + public String getOwnerAddress() { + java.lang.Object ref = ownerAddress_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + ownerAddress_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setOwnerAddress(java.lang.String value) { + public Builder setOwnerAddress(String value) { if (value == null) { throw new NullPointerException(); } - result.hasOwnerAddress = true; - result.ownerAddress_ = value; + bitField0_ |= 0x00000001; + ownerAddress_ = value; + onChanged(); return this; } public Builder clearOwnerAddress() { - result.hasOwnerAddress = false; - result.ownerAddress_ = getDefaultInstance().getOwnerAddress(); + bitField0_ = (bitField0_ & ~0x00000001); + ownerAddress_ = getDefaultInstance().getOwnerAddress(); + onChanged(); return this; } + void setOwnerAddress(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + ownerAddress_ = value; + onChanged(); + } // optional string senderAddress = 2; + private java.lang.Object senderAddress_ = ""; public boolean hasSenderAddress() { - return result.hasSenderAddress(); + return ((bitField0_ & 0x00000002) == 0x00000002); } - public java.lang.String getSenderAddress() { - return result.getSenderAddress(); + public String getSenderAddress() { + java.lang.Object ref = senderAddress_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + senderAddress_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setSenderAddress(java.lang.String value) { + public Builder setSenderAddress(String value) { if (value == null) { throw new NullPointerException(); } - result.hasSenderAddress = true; - result.senderAddress_ = value; + bitField0_ |= 0x00000002; + senderAddress_ = value; + onChanged(); return this; } public Builder clearSenderAddress() { - result.hasSenderAddress = false; - result.senderAddress_ = getDefaultInstance().getSenderAddress(); + bitField0_ = (bitField0_ & ~0x00000002); + senderAddress_ = getDefaultInstance().getSenderAddress(); + onChanged(); return this; } + void setSenderAddress(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + senderAddress_ = value; + onChanged(); + } // optional .UuidProtocol futureUuid = 3; + private akka.actor.mailbox.MailboxProtocol.UuidProtocol futureUuid_ = akka.actor.mailbox.MailboxProtocol.UuidProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.actor.mailbox.MailboxProtocol.UuidProtocol, akka.actor.mailbox.MailboxProtocol.UuidProtocol.Builder, akka.actor.mailbox.MailboxProtocol.UuidProtocolOrBuilder> futureUuidBuilder_; public boolean hasFutureUuid() { - return result.hasFutureUuid(); + return ((bitField0_ & 0x00000004) == 0x00000004); } public akka.actor.mailbox.MailboxProtocol.UuidProtocol getFutureUuid() { - return result.getFutureUuid(); + if (futureUuidBuilder_ == null) { + return futureUuid_; + } else { + return futureUuidBuilder_.getMessage(); + } } public Builder setFutureUuid(akka.actor.mailbox.MailboxProtocol.UuidProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (futureUuidBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + futureUuid_ = value; + onChanged(); + } else { + futureUuidBuilder_.setMessage(value); } - result.hasFutureUuid = true; - result.futureUuid_ = value; + bitField0_ |= 0x00000004; return this; } - public Builder setFutureUuid(akka.actor.mailbox.MailboxProtocol.UuidProtocol.Builder builderForValue) { - result.hasFutureUuid = true; - result.futureUuid_ = builderForValue.build(); + public Builder setFutureUuid( + akka.actor.mailbox.MailboxProtocol.UuidProtocol.Builder builderForValue) { + if (futureUuidBuilder_ == null) { + futureUuid_ = builderForValue.build(); + onChanged(); + } else { + futureUuidBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000004; return this; } public Builder mergeFutureUuid(akka.actor.mailbox.MailboxProtocol.UuidProtocol value) { - if (result.hasFutureUuid() && - result.futureUuid_ != akka.actor.mailbox.MailboxProtocol.UuidProtocol.getDefaultInstance()) { - result.futureUuid_ = - akka.actor.mailbox.MailboxProtocol.UuidProtocol.newBuilder(result.futureUuid_).mergeFrom(value).buildPartial(); + if (futureUuidBuilder_ == null) { + if (((bitField0_ & 0x00000004) == 0x00000004) && + futureUuid_ != akka.actor.mailbox.MailboxProtocol.UuidProtocol.getDefaultInstance()) { + futureUuid_ = + akka.actor.mailbox.MailboxProtocol.UuidProtocol.newBuilder(futureUuid_).mergeFrom(value).buildPartial(); + } else { + futureUuid_ = value; + } + onChanged(); } else { - result.futureUuid_ = value; + futureUuidBuilder_.mergeFrom(value); } - result.hasFutureUuid = true; + bitField0_ |= 0x00000004; return this; } public Builder clearFutureUuid() { - result.hasFutureUuid = false; - result.futureUuid_ = akka.actor.mailbox.MailboxProtocol.UuidProtocol.getDefaultInstance(); + if (futureUuidBuilder_ == null) { + futureUuid_ = akka.actor.mailbox.MailboxProtocol.UuidProtocol.getDefaultInstance(); + onChanged(); + } else { + futureUuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000004); return this; } + public akka.actor.mailbox.MailboxProtocol.UuidProtocol.Builder getFutureUuidBuilder() { + bitField0_ |= 0x00000004; + onChanged(); + return getFutureUuidFieldBuilder().getBuilder(); + } + public akka.actor.mailbox.MailboxProtocol.UuidProtocolOrBuilder getFutureUuidOrBuilder() { + if (futureUuidBuilder_ != null) { + return futureUuidBuilder_.getMessageOrBuilder(); + } else { + return futureUuid_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.actor.mailbox.MailboxProtocol.UuidProtocol, akka.actor.mailbox.MailboxProtocol.UuidProtocol.Builder, akka.actor.mailbox.MailboxProtocol.UuidProtocolOrBuilder> + getFutureUuidFieldBuilder() { + if (futureUuidBuilder_ == null) { + futureUuidBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.actor.mailbox.MailboxProtocol.UuidProtocol, akka.actor.mailbox.MailboxProtocol.UuidProtocol.Builder, akka.actor.mailbox.MailboxProtocol.UuidProtocolOrBuilder>( + futureUuid_, + getParentForChildren(), + isClean()); + futureUuid_ = null; + } + return futureUuidBuilder_; + } // required bytes message = 4; + private com.google.protobuf.ByteString message_ = com.google.protobuf.ByteString.EMPTY; public boolean hasMessage() { - return result.hasMessage(); + return ((bitField0_ & 0x00000008) == 0x00000008); } public com.google.protobuf.ByteString getMessage() { - return result.getMessage(); + return message_; } public Builder setMessage(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - result.hasMessage = true; - result.message_ = value; + bitField0_ |= 0x00000008; + message_ = value; + onChanged(); return this; } public Builder clearMessage() { - result.hasMessage = false; - result.message_ = getDefaultInstance().getMessage(); + bitField0_ = (bitField0_ & ~0x00000008); + message_ = getDefaultInstance().getMessage(); + onChanged(); return this; } @@ -441,18 +702,29 @@ public final class MailboxProtocol { static { defaultInstance = new DurableMailboxMessageProtocol(true); - akka.actor.mailbox.MailboxProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:DurableMailboxMessageProtocol) } + public interface UuidProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required uint64 high = 1; + boolean hasHigh(); + long getHigh(); + + // required uint64 low = 2; + boolean hasLow(); + long getLow(); + } public static final class UuidProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements UuidProtocolOrBuilder { // Use UuidProtocol.newBuilder() to construct. - private UuidProtocol() { - initFields(); + private UuidProtocol(Builder builder) { + super(builder); } private UuidProtocol(boolean noInit) {} @@ -475,36 +747,56 @@ public final class MailboxProtocol { return akka.actor.mailbox.MailboxProtocol.internal_static_UuidProtocol_fieldAccessorTable; } + private int bitField0_; // required uint64 high = 1; public static final int HIGH_FIELD_NUMBER = 1; - private boolean hasHigh; - private long high_ = 0L; - public boolean hasHigh() { return hasHigh; } - public long getHigh() { return high_; } + private long high_; + public boolean hasHigh() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getHigh() { + return high_; + } // required uint64 low = 2; public static final int LOW_FIELD_NUMBER = 2; - private boolean hasLow; - private long low_ = 0L; - public boolean hasLow() { return hasLow; } - public long getLow() { return low_; } + private long low_; + public boolean hasLow() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getLow() { + return low_; + } private void initFields() { + high_ = 0L; + low_ = 0L; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasHigh) return false; - if (!hasLow) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasHigh()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasLow()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasHigh()) { - output.writeUInt64(1, getHigh()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, high_); } - if (hasLow()) { - output.writeUInt64(2, getLow()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, low_); } getUnknownFields().writeTo(output); } @@ -515,19 +807,26 @@ public final class MailboxProtocol { if (size != -1) return size; size = 0; - if (hasHigh()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(1, getHigh()); + .computeUInt64Size(1, high_); } - if (hasLow()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(2, getLow()); + .computeUInt64Size(2, low_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.actor.mailbox.MailboxProtocol.UuidProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -602,34 +901,53 @@ public final class MailboxProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.actor.mailbox.MailboxProtocol.UuidProtocol result; - - // Construct using akka.actor.mailbox.MailboxProtocol.UuidProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.actor.mailbox.MailboxProtocol.UuidProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.actor.mailbox.MailboxProtocol.UuidProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.actor.mailbox.MailboxProtocol.internal_static_UuidProtocol_descriptor; } - protected akka.actor.mailbox.MailboxProtocol.UuidProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.actor.mailbox.MailboxProtocol.internal_static_UuidProtocol_fieldAccessorTable; + } + + // Construct using akka.actor.mailbox.MailboxProtocol.UuidProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.actor.mailbox.MailboxProtocol.UuidProtocol(); + super.clear(); + high_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + low_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -641,33 +959,39 @@ public final class MailboxProtocol { return akka.actor.mailbox.MailboxProtocol.UuidProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.actor.mailbox.MailboxProtocol.UuidProtocol build() { - if (result != null && !isInitialized()) { + akka.actor.mailbox.MailboxProtocol.UuidProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.actor.mailbox.MailboxProtocol.UuidProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.actor.mailbox.MailboxProtocol.UuidProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.actor.mailbox.MailboxProtocol.UuidProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.actor.mailbox.MailboxProtocol.UuidProtocol result = new akka.actor.mailbox.MailboxProtocol.UuidProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.actor.mailbox.MailboxProtocol.UuidProtocol returnMe = result; - result = null; - return returnMe; + result.high_ = high_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.low_ = low_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -691,6 +1015,18 @@ public final class MailboxProtocol { return this; } + public final boolean isInitialized() { + if (!hasHigh()) { + + return false; + } + if (!hasLow()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -703,61 +1039,72 @@ public final class MailboxProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 8: { - setHigh(input.readUInt64()); + bitField0_ |= 0x00000001; + high_ = input.readUInt64(); break; } case 16: { - setLow(input.readUInt64()); + bitField0_ |= 0x00000002; + low_ = input.readUInt64(); break; } } } } + private int bitField0_; // required uint64 high = 1; + private long high_ ; public boolean hasHigh() { - return result.hasHigh(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public long getHigh() { - return result.getHigh(); + return high_; } public Builder setHigh(long value) { - result.hasHigh = true; - result.high_ = value; + bitField0_ |= 0x00000001; + high_ = value; + onChanged(); return this; } public Builder clearHigh() { - result.hasHigh = false; - result.high_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + high_ = 0L; + onChanged(); return this; } // required uint64 low = 2; + private long low_ ; public boolean hasLow() { - return result.hasLow(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public long getLow() { - return result.getLow(); + return low_; } public Builder setLow(long value) { - result.hasLow = true; - result.low_ = value; + bitField0_ |= 0x00000002; + low_ = value; + onChanged(); return this; } public Builder clearLow() { - result.hasLow = false; - result.low_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + low_ = 0L; + onChanged(); return this; } @@ -766,7 +1113,6 @@ public final class MailboxProtocol { static { defaultInstance = new UuidProtocol(true); - akka.actor.mailbox.MailboxProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -829,7 +1175,5 @@ public final class MailboxProtocol { }, assigner); } - public static void internalForceInit() {} - // @@protoc_insertion_point(outer_class_scope) } diff --git a/akka-remote/src/main/java/akka/remote/protocol/RemoteProtocol.java b/akka-remote/src/main/java/akka/remote/protocol/RemoteProtocol.java index 5ebf1f58dd..29e490103e 100644 --- a/akka-remote/src/main/java/akka/remote/protocol/RemoteProtocol.java +++ b/akka-remote/src/main/java/akka/remote/protocol/RemoteProtocol.java @@ -14,6 +14,9 @@ public final class RemoteProtocol { SHUTDOWN(1, 2), ; + public static final int CONNECT_VALUE = 1; + public static final int SHUTDOWN_VALUE = 2; + public final int getNumber() { return value; } @@ -33,8 +36,8 @@ public final class RemoteProtocol { internalValueMap = new com.google.protobuf.Internal.EnumLiteMap() { public CommandType findValueByNumber(int number) { - return CommandType.valueOf(number) - ; } + return CommandType.valueOf(number); + } }; public final com.google.protobuf.Descriptors.EnumValueDescriptor @@ -53,6 +56,7 @@ public final class RemoteProtocol { private static final CommandType[] VALUES = { CONNECT, SHUTDOWN, }; + public static CommandType valueOf( com.google.protobuf.Descriptors.EnumValueDescriptor desc) { if (desc.getType() != getDescriptor()) { @@ -61,18 +65,88 @@ public final class RemoteProtocol { } return VALUES[desc.getIndex()]; } + private final int index; private final int value; + private CommandType(int index, int value) { this.index = index; this.value = value; } - static { - akka.remote.protocol.RemoteProtocol.getDescriptor(); + // @@protoc_insertion_point(enum_scope:CommandType) + } + + public enum ReplicationStrategyType + implements com.google.protobuf.ProtocolMessageEnum { + TRANSIENT(0, 1), + WRITE_THROUGH(1, 2), + WRITE_BEHIND(2, 3), + ; + + public static final int TRANSIENT_VALUE = 1; + public static final int WRITE_THROUGH_VALUE = 2; + public static final int WRITE_BEHIND_VALUE = 3; + + + public final int getNumber() { return value; } + + public static ReplicationStrategyType valueOf(int value) { + switch (value) { + case 1: return TRANSIENT; + case 2: return WRITE_THROUGH; + case 3: return WRITE_BEHIND; + default: return null; + } } - // @@protoc_insertion_point(enum_scope:CommandType) + 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 ReplicationStrategyType findValueByNumber(int number) { + return ReplicationStrategyType.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 akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(1); + } + + private static final ReplicationStrategyType[] VALUES = { + TRANSIENT, WRITE_THROUGH, WRITE_BEHIND, + }; + + public static ReplicationStrategyType 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 ReplicationStrategyType(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:ReplicationStrategyType) } public enum SerializationSchemeType @@ -84,6 +158,12 @@ public final class RemoteProtocol { PROTOBUF(4, 5), ; + public static final int JAVA_VALUE = 1; + public static final int SBINARY_VALUE = 2; + public static final int SCALA_JSON_VALUE = 3; + public static final int JAVA_JSON_VALUE = 4; + public static final int PROTOBUF_VALUE = 5; + public final int getNumber() { return value; } @@ -106,8 +186,8 @@ public final class RemoteProtocol { internalValueMap = new com.google.protobuf.Internal.EnumLiteMap() { public SerializationSchemeType findValueByNumber(int number) { - return SerializationSchemeType.valueOf(number) - ; } + return SerializationSchemeType.valueOf(number); + } }; public final com.google.protobuf.Descriptors.EnumValueDescriptor @@ -120,12 +200,13 @@ public final class RemoteProtocol { } public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { - return akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(1); + return akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(2); } 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()) { @@ -134,17 +215,15 @@ public final class RemoteProtocol { } 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 { - akka.remote.protocol.RemoteProtocol.getDescriptor(); - } - // @@protoc_insertion_point(enum_scope:SerializationSchemeType) } @@ -154,6 +233,9 @@ public final class RemoteProtocol { TEMPORARY(1, 2), ; + public static final int PERMANENT_VALUE = 1; + public static final int TEMPORARY_VALUE = 2; + public final int getNumber() { return value; } @@ -173,8 +255,8 @@ public final class RemoteProtocol { internalValueMap = new com.google.protobuf.Internal.EnumLiteMap() { public LifeCycleType findValueByNumber(int number) { - return LifeCycleType.valueOf(number) - ; } + return LifeCycleType.valueOf(number); + } }; public final com.google.protobuf.Descriptors.EnumValueDescriptor @@ -187,12 +269,13 @@ public final class RemoteProtocol { } public static final com.google.protobuf.Descriptors.EnumDescriptor getDescriptor() { - return akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(2); + return akka.remote.protocol.RemoteProtocol.getDescriptor().getEnumTypes().get(3); } private static final LifeCycleType[] VALUES = { PERMANENT, TEMPORARY, }; + public static LifeCycleType valueOf( com.google.protobuf.Descriptors.EnumValueDescriptor desc) { if (desc.getType() != getDescriptor()) { @@ -201,25 +284,37 @@ public final class RemoteProtocol { } 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 { - akka.remote.protocol.RemoteProtocol.getDescriptor(); - } - // @@protoc_insertion_point(enum_scope:LifeCycleType) } + public interface AkkaRemoteProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional .RemoteMessageProtocol message = 1; + boolean hasMessage(); + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol getMessage(); + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder getMessageOrBuilder(); + + // optional .RemoteControlProtocol instruction = 2; + boolean hasInstruction(); + akka.remote.protocol.RemoteProtocol.RemoteControlProtocol getInstruction(); + akka.remote.protocol.RemoteProtocol.RemoteControlProtocolOrBuilder getInstructionOrBuilder(); + } public static final class AkkaRemoteProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements AkkaRemoteProtocolOrBuilder { // Use AkkaRemoteProtocol.newBuilder() to construct. - private AkkaRemoteProtocol() { - initFields(); + private AkkaRemoteProtocol(Builder builder) { + super(builder); } private AkkaRemoteProtocol(boolean noInit) {} @@ -242,42 +337,66 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_AkkaRemoteProtocol_fieldAccessorTable; } + private int bitField0_; // optional .RemoteMessageProtocol message = 1; public static final int MESSAGE_FIELD_NUMBER = 1; - private boolean hasMessage; private akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol message_; - public boolean hasMessage() { return hasMessage; } - public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol getMessage() { return message_; } + public boolean hasMessage() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol getMessage() { + return message_; + } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder getMessageOrBuilder() { + return message_; + } // optional .RemoteControlProtocol instruction = 2; public static final int INSTRUCTION_FIELD_NUMBER = 2; - private boolean hasInstruction; private akka.remote.protocol.RemoteProtocol.RemoteControlProtocol instruction_; - public boolean hasInstruction() { return hasInstruction; } - public akka.remote.protocol.RemoteProtocol.RemoteControlProtocol getInstruction() { return instruction_; } + public boolean hasInstruction() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.protocol.RemoteProtocol.RemoteControlProtocol getInstruction() { + return instruction_; + } + public akka.remote.protocol.RemoteProtocol.RemoteControlProtocolOrBuilder getInstructionOrBuilder() { + return instruction_; + } private void initFields() { message_ = akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance(); instruction_ = akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.getDefaultInstance(); } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + if (hasMessage()) { - if (!getMessage().isInitialized()) return false; + if (!getMessage().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } if (hasInstruction()) { - if (!getInstruction().isInitialized()) return false; + if (!getInstruction().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasMessage()) { - output.writeMessage(1, getMessage()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, message_); } - if (hasInstruction()) { - output.writeMessage(2, getInstruction()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, instruction_); } getUnknownFields().writeTo(output); } @@ -288,19 +407,26 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasMessage()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getMessage()); + .computeMessageSize(1, message_); } - if (hasInstruction()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getInstruction()); + .computeMessageSize(2, instruction_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -375,34 +501,63 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_AkkaRemoteProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_AkkaRemoteProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getMessageFieldBuilder(); + getInstructionFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); + super.clear(); + if (messageBuilder_ == null) { + message_ = akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance(); + } else { + messageBuilder_.clear(); } - result = new akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol(); + bitField0_ = (bitField0_ & ~0x00000001); + if (instructionBuilder_ == null) { + instruction_ = akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.getDefaultInstance(); + } else { + instructionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -414,33 +569,47 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol result = new akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.AkkaRemoteProtocol returnMe = result; - result = null; - return returnMe; + if (messageBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = messageBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (instructionBuilder_ == null) { + result.instruction_ = instruction_; + } else { + result.instruction_ = instructionBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -464,6 +633,22 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (hasMessage()) { + if (!getMessage().isInitialized()) { + + return false; + } + } + if (hasInstruction()) { + if (!getInstruction().isInitialized()) { + + return false; + } + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -476,11 +661,13 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; @@ -507,98 +694,252 @@ public final class RemoteProtocol { } } + private int bitField0_; // optional .RemoteMessageProtocol message = 1; + private akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol message_ = akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder> messageBuilder_; public boolean hasMessage() { - return result.hasMessage(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol getMessage() { - return result.getMessage(); + if (messageBuilder_ == null) { + return message_; + } else { + return messageBuilder_.getMessage(); + } } public Builder setMessage(akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (messageBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + messageBuilder_.setMessage(value); } - result.hasMessage = true; - result.message_ = value; + bitField0_ |= 0x00000001; return this; } - public Builder setMessage(akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder builderForValue) { - result.hasMessage = true; - result.message_ = builderForValue.build(); + public Builder setMessage( + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder builderForValue) { + if (messageBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + messageBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; return this; } public Builder mergeMessage(akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol value) { - if (result.hasMessage() && - result.message_ != akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance()) { - result.message_ = - akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial(); + if (messageBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + message_ != akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance()) { + message_ = + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.newBuilder(message_).mergeFrom(value).buildPartial(); + } else { + message_ = value; + } + onChanged(); } else { - result.message_ = value; + messageBuilder_.mergeFrom(value); } - result.hasMessage = true; + bitField0_ |= 0x00000001; return this; } public Builder clearMessage() { - result.hasMessage = false; - result.message_ = akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance(); + if (messageBuilder_ == null) { + message_ = akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance(); + onChanged(); + } else { + messageBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); return this; } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder getMessageBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getMessageFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder getMessageOrBuilder() { + if (messageBuilder_ != null) { + return messageBuilder_.getMessageOrBuilder(); + } else { + return message_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder> + getMessageFieldBuilder() { + if (messageBuilder_ == null) { + messageBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder>( + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + return messageBuilder_; + } // optional .RemoteControlProtocol instruction = 2; + private akka.remote.protocol.RemoteProtocol.RemoteControlProtocol instruction_ = akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteControlProtocol, akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteControlProtocolOrBuilder> instructionBuilder_; public boolean hasInstruction() { - return result.hasInstruction(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public akka.remote.protocol.RemoteProtocol.RemoteControlProtocol getInstruction() { - return result.getInstruction(); + if (instructionBuilder_ == null) { + return instruction_; + } else { + return instructionBuilder_.getMessage(); + } } public Builder setInstruction(akka.remote.protocol.RemoteProtocol.RemoteControlProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (instructionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + instruction_ = value; + onChanged(); + } else { + instructionBuilder_.setMessage(value); } - result.hasInstruction = true; - result.instruction_ = value; + bitField0_ |= 0x00000002; return this; } - public Builder setInstruction(akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.Builder builderForValue) { - result.hasInstruction = true; - result.instruction_ = builderForValue.build(); + public Builder setInstruction( + akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.Builder builderForValue) { + if (instructionBuilder_ == null) { + instruction_ = builderForValue.build(); + onChanged(); + } else { + instructionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; return this; } public Builder mergeInstruction(akka.remote.protocol.RemoteProtocol.RemoteControlProtocol value) { - if (result.hasInstruction() && - result.instruction_ != akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.getDefaultInstance()) { - result.instruction_ = - akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.newBuilder(result.instruction_).mergeFrom(value).buildPartial(); + if (instructionBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + instruction_ != akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.getDefaultInstance()) { + instruction_ = + akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.newBuilder(instruction_).mergeFrom(value).buildPartial(); + } else { + instruction_ = value; + } + onChanged(); } else { - result.instruction_ = value; + instructionBuilder_.mergeFrom(value); } - result.hasInstruction = true; + bitField0_ |= 0x00000002; return this; } public Builder clearInstruction() { - result.hasInstruction = false; - result.instruction_ = akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.getDefaultInstance(); + if (instructionBuilder_ == null) { + instruction_ = akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.getDefaultInstance(); + onChanged(); + } else { + instructionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); return this; } + public akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.Builder getInstructionBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getInstructionFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.RemoteControlProtocolOrBuilder getInstructionOrBuilder() { + if (instructionBuilder_ != null) { + return instructionBuilder_.getMessageOrBuilder(); + } else { + return instruction_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteControlProtocol, akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteControlProtocolOrBuilder> + getInstructionFieldBuilder() { + if (instructionBuilder_ == null) { + instructionBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteControlProtocol, akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteControlProtocolOrBuilder>( + instruction_, + getParentForChildren(), + isClean()); + instruction_ = null; + } + return instructionBuilder_; + } // @@protoc_insertion_point(builder_scope:AkkaRemoteProtocol) } static { defaultInstance = new AkkaRemoteProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:AkkaRemoteProtocol) } + public interface RemoteMessageProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .UuidProtocol uuid = 1; + boolean hasUuid(); + akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid(); + akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getUuidOrBuilder(); + + // required .ActorInfoProtocol actorInfo = 2; + boolean hasActorInfo(); + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo(); + akka.remote.protocol.RemoteProtocol.ActorInfoProtocolOrBuilder getActorInfoOrBuilder(); + + // required bool oneWay = 3; + boolean hasOneWay(); + boolean getOneWay(); + + // optional .MessageProtocol message = 4; + boolean hasMessage(); + akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage(); + akka.remote.protocol.RemoteProtocol.MessageProtocolOrBuilder getMessageOrBuilder(); + + // optional .ExceptionProtocol exception = 5; + boolean hasException(); + akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException(); + akka.remote.protocol.RemoteProtocol.ExceptionProtocolOrBuilder getExceptionOrBuilder(); + + // optional .UuidProtocol supervisorUuid = 6; + boolean hasSupervisorUuid(); + akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid(); + akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getSupervisorUuidOrBuilder(); + + // optional .RemoteActorRefProtocol sender = 7; + boolean hasSender(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder getSenderOrBuilder(); + + // repeated .MetadataEntryProtocol metadata = 8; + java.util.List + getMetadataList(); + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index); + int getMetadataCount(); + java.util.List + getMetadataOrBuilderList(); + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocolOrBuilder getMetadataOrBuilder( + int index); + } public static final class RemoteMessageProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements RemoteMessageProtocolOrBuilder { // Use RemoteMessageProtocol.newBuilder() to construct. - private RemoteMessageProtocol() { - initFields(); + private RemoteMessageProtocol(Builder builder) { + super(builder); } private RemoteMessageProtocol(boolean noInit) {} @@ -621,125 +962,211 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_RemoteMessageProtocol_fieldAccessorTable; } + private int bitField0_; // required .UuidProtocol uuid = 1; public static final int UUID_FIELD_NUMBER = 1; - private boolean hasUuid; private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; - public boolean hasUuid() { return hasUuid; } - public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } + public boolean hasUuid() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { + return uuid_; + } + public akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getUuidOrBuilder() { + return uuid_; + } // required .ActorInfoProtocol actorInfo = 2; public static final int ACTORINFO_FIELD_NUMBER = 2; - private boolean hasActorInfo; private akka.remote.protocol.RemoteProtocol.ActorInfoProtocol actorInfo_; - public boolean hasActorInfo() { return hasActorInfo; } - public akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { return actorInfo_; } + public boolean hasActorInfo() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { + return actorInfo_; + } + public akka.remote.protocol.RemoteProtocol.ActorInfoProtocolOrBuilder getActorInfoOrBuilder() { + return actorInfo_; + } // required bool oneWay = 3; public static final int ONEWAY_FIELD_NUMBER = 3; - private boolean hasOneWay; - private boolean oneWay_ = false; - public boolean hasOneWay() { return hasOneWay; } - public boolean getOneWay() { return oneWay_; } + private boolean oneWay_; + public boolean hasOneWay() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public boolean getOneWay() { + return oneWay_; + } // optional .MessageProtocol message = 4; public static final int MESSAGE_FIELD_NUMBER = 4; - private boolean hasMessage; private akka.remote.protocol.RemoteProtocol.MessageProtocol message_; - public boolean hasMessage() { return hasMessage; } - public akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { return message_; } + public boolean hasMessage() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { + return message_; + } + public akka.remote.protocol.RemoteProtocol.MessageProtocolOrBuilder getMessageOrBuilder() { + return message_; + } // optional .ExceptionProtocol exception = 5; public static final int EXCEPTION_FIELD_NUMBER = 5; - private boolean hasException; private akka.remote.protocol.RemoteProtocol.ExceptionProtocol exception_; - public boolean hasException() { return hasException; } - public akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { return exception_; } + public boolean hasException() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { + return exception_; + } + public akka.remote.protocol.RemoteProtocol.ExceptionProtocolOrBuilder getExceptionOrBuilder() { + return exception_; + } // optional .UuidProtocol supervisorUuid = 6; public static final int SUPERVISORUUID_FIELD_NUMBER = 6; - private boolean hasSupervisorUuid; private akka.remote.protocol.RemoteProtocol.UuidProtocol supervisorUuid_; - public boolean hasSupervisorUuid() { return hasSupervisorUuid; } - public akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { return supervisorUuid_; } + public boolean hasSupervisorUuid() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { + return supervisorUuid_; + } + public akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getSupervisorUuidOrBuilder() { + return supervisorUuid_; + } // optional .RemoteActorRefProtocol sender = 7; public static final int SENDER_FIELD_NUMBER = 7; - private boolean hasSender; private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol sender_; - public boolean hasSender() { return hasSender; } - public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { return sender_; } + public boolean hasSender() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { + return sender_; + } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder getSenderOrBuilder() { + return sender_; + } // repeated .MetadataEntryProtocol metadata = 8; public static final int METADATA_FIELD_NUMBER = 8; - private java.util.List metadata_ = - java.util.Collections.emptyList(); + private java.util.List metadata_; public java.util.List getMetadataList() { return metadata_; } - public int getMetadataCount() { return metadata_.size(); } + public java.util.List + getMetadataOrBuilderList() { + return metadata_; + } + public int getMetadataCount() { + return metadata_.size(); + } public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { return metadata_.get(index); } + public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocolOrBuilder getMetadataOrBuilder( + int index) { + return metadata_.get(index); + } private void initFields() { uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); actorInfo_ = akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); + oneWay_ = false; message_ = akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); exception_ = akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); supervisorUuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); sender_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + metadata_ = java.util.Collections.emptyList(); } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasUuid) return false; - if (!hasActorInfo) return false; - if (!hasOneWay) return false; - if (!getUuid().isInitialized()) return false; - if (!getActorInfo().isInitialized()) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasUuid()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasActorInfo()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasOneWay()) { + memoizedIsInitialized = 0; + return false; + } + if (!getUuid().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + if (!getActorInfo().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } if (hasMessage()) { - if (!getMessage().isInitialized()) return false; + if (!getMessage().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } if (hasException()) { - if (!getException().isInitialized()) return false; + if (!getException().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } if (hasSupervisorUuid()) { - if (!getSupervisorUuid().isInitialized()) return false; + if (!getSupervisorUuid().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } if (hasSender()) { - if (!getSender().isInitialized()) return false; + if (!getSender().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } - for (akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { - if (!element.isInitialized()) return false; + for (int i = 0; i < getMetadataCount(); i++) { + if (!getMetadata(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasUuid()) { - output.writeMessage(1, getUuid()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, uuid_); } - if (hasActorInfo()) { - output.writeMessage(2, getActorInfo()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeMessage(2, actorInfo_); } - if (hasOneWay()) { - output.writeBool(3, getOneWay()); + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBool(3, oneWay_); } - if (hasMessage()) { - output.writeMessage(4, getMessage()); + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeMessage(4, message_); } - if (hasException()) { - output.writeMessage(5, getException()); + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeMessage(5, exception_); } - if (hasSupervisorUuid()) { - output.writeMessage(6, getSupervisorUuid()); + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeMessage(6, supervisorUuid_); } - if (hasSender()) { - output.writeMessage(7, getSender()); + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeMessage(7, sender_); } - for (akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { - output.writeMessage(8, element); + for (int i = 0; i < metadata_.size(); i++) { + output.writeMessage(8, metadata_.get(i)); } getUnknownFields().writeTo(output); } @@ -750,43 +1177,50 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasUuid()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getUuid()); + .computeMessageSize(1, uuid_); } - if (hasActorInfo()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(2, getActorInfo()); + .computeMessageSize(2, actorInfo_); } - if (hasOneWay()) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeBoolSize(3, getOneWay()); + .computeBoolSize(3, oneWay_); } - if (hasMessage()) { + if (((bitField0_ & 0x00000008) == 0x00000008)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(4, getMessage()); + .computeMessageSize(4, message_); } - if (hasException()) { + if (((bitField0_ & 0x00000010) == 0x00000010)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(5, getException()); + .computeMessageSize(5, exception_); } - if (hasSupervisorUuid()) { + if (((bitField0_ & 0x00000020) == 0x00000020)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(6, getSupervisorUuid()); + .computeMessageSize(6, supervisorUuid_); } - if (hasSender()) { + if (((bitField0_ & 0x00000040) == 0x00000040)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(7, getSender()); + .computeMessageSize(7, sender_); } - for (akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol element : getMetadataList()) { + for (int i = 0; i < metadata_.size(); i++) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(8, element); + .computeMessageSize(8, metadata_.get(i)); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -861,34 +1295,100 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteMessageProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteMessageProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getUuidFieldBuilder(); + getActorInfoFieldBuilder(); + getMessageFieldBuilder(); + getExceptionFieldBuilder(); + getSupervisorUuidFieldBuilder(); + getSenderFieldBuilder(); + getMetadataFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); + super.clear(); + if (uuidBuilder_ == null) { + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + } else { + uuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + if (actorInfoBuilder_ == null) { + actorInfo_ = akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); + } else { + actorInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); + oneWay_ = false; + bitField0_ = (bitField0_ & ~0x00000004); + if (messageBuilder_ == null) { + message_ = akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); + } else { + messageBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); + if (exceptionBuilder_ == null) { + exception_ = akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); + } else { + exceptionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); + if (supervisorUuidBuilder_ == null) { + supervisorUuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + } else { + supervisorUuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000020); + if (senderBuilder_ == null) { + sender_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + } else { + senderBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000040); + if (metadataBuilder_ == null) { + metadata_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000080); + } else { + metadataBuilder_.clear(); } - result = new akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol(); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -900,37 +1400,92 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol result = new akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - if (result.metadata_ != java.util.Collections.EMPTY_LIST) { - result.metadata_ = - java.util.Collections.unmodifiableList(result.metadata_); + if (uuidBuilder_ == null) { + result.uuid_ = uuid_; + } else { + result.uuid_ = uuidBuilder_.build(); } - akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol returnMe = result; - result = null; - return returnMe; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + if (actorInfoBuilder_ == null) { + result.actorInfo_ = actorInfo_; + } else { + result.actorInfo_ = actorInfoBuilder_.build(); + } + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.oneWay_ = oneWay_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + if (messageBuilder_ == null) { + result.message_ = message_; + } else { + result.message_ = messageBuilder_.build(); + } + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + if (exceptionBuilder_ == null) { + result.exception_ = exception_; + } else { + result.exception_ = exceptionBuilder_.build(); + } + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + if (supervisorUuidBuilder_ == null) { + result.supervisorUuid_ = supervisorUuid_; + } else { + result.supervisorUuid_ = supervisorUuidBuilder_.build(); + } + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + if (senderBuilder_ == null) { + result.sender_ = sender_; + } else { + result.sender_ = senderBuilder_.build(); + } + if (metadataBuilder_ == null) { + if (((bitField0_ & 0x00000080) == 0x00000080)) { + metadata_ = java.util.Collections.unmodifiableList(metadata_); + bitField0_ = (bitField0_ & ~0x00000080); + } + result.metadata_ = metadata_; + } else { + result.metadata_ = metadataBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -965,16 +1520,90 @@ public final class RemoteProtocol { if (other.hasSender()) { mergeSender(other.getSender()); } - if (!other.metadata_.isEmpty()) { - if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + if (metadataBuilder_ == null) { + if (!other.metadata_.isEmpty()) { + if (metadata_.isEmpty()) { + metadata_ = other.metadata_; + bitField0_ = (bitField0_ & ~0x00000080); + } else { + ensureMetadataIsMutable(); + metadata_.addAll(other.metadata_); + } + onChanged(); + } + } else { + if (!other.metadata_.isEmpty()) { + if (metadataBuilder_.isEmpty()) { + metadataBuilder_.dispose(); + metadataBuilder_ = null; + metadata_ = other.metadata_; + bitField0_ = (bitField0_ & ~0x00000080); + metadataBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getMetadataFieldBuilder() : null; + } else { + metadataBuilder_.addAllMessages(other.metadata_); + } } - result.metadata_.addAll(other.metadata_); } this.mergeUnknownFields(other.getUnknownFields()); return this; } + public final boolean isInitialized() { + if (!hasUuid()) { + + return false; + } + if (!hasActorInfo()) { + + return false; + } + if (!hasOneWay()) { + + return false; + } + if (!getUuid().isInitialized()) { + + return false; + } + if (!getActorInfo().isInitialized()) { + + return false; + } + if (hasMessage()) { + if (!getMessage().isInitialized()) { + + return false; + } + } + if (hasException()) { + if (!getException().isInitialized()) { + + return false; + } + } + if (hasSupervisorUuid()) { + if (!getSupervisorUuid().isInitialized()) { + + return false; + } + } + if (hasSender()) { + if (!getSender().isInitialized()) { + + return false; + } + } + for (int i = 0; i < getMetadataCount(); i++) { + if (!getMetadata(i).isInitialized()) { + + return false; + } + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -987,11 +1616,13 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; @@ -1015,7 +1646,8 @@ public final class RemoteProtocol { break; } case 24: { - setOneWay(input.readBool()); + bitField0_ |= 0x00000004; + oneWay_ = input.readBool(); break; } case 34: { @@ -1064,315 +1696,783 @@ public final class RemoteProtocol { } } + private int bitField0_; // required .UuidProtocol uuid = 1; + private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder> uuidBuilder_; public boolean hasUuid() { - return result.hasUuid(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { - return result.getUuid(); + if (uuidBuilder_ == null) { + return uuid_; + } else { + return uuidBuilder_.getMessage(); + } } public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (uuidBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + uuid_ = value; + onChanged(); + } else { + uuidBuilder_.setMessage(value); } - result.hasUuid = true; - result.uuid_ = value; + bitField0_ |= 0x00000001; return this; } - public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { - result.hasUuid = true; - result.uuid_ = builderForValue.build(); + public Builder setUuid( + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + if (uuidBuilder_ == null) { + uuid_ = builderForValue.build(); + onChanged(); + } else { + uuidBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; return this; } public Builder mergeUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { - if (result.hasUuid() && - result.uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { - result.uuid_ = - akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); + if (uuidBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + uuid_ = + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(uuid_).mergeFrom(value).buildPartial(); + } else { + uuid_ = value; + } + onChanged(); } else { - result.uuid_ = value; + uuidBuilder_.mergeFrom(value); } - result.hasUuid = true; + bitField0_ |= 0x00000001; return this; } public Builder clearUuid() { - result.hasUuid = false; - result.uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + if (uuidBuilder_ == null) { + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + onChanged(); + } else { + uuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); return this; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder getUuidBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getUuidFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getUuidOrBuilder() { + if (uuidBuilder_ != null) { + return uuidBuilder_.getMessageOrBuilder(); + } else { + return uuid_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder> + getUuidFieldBuilder() { + if (uuidBuilder_ == null) { + uuidBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder>( + uuid_, + getParentForChildren(), + isClean()); + uuid_ = null; + } + return uuidBuilder_; + } // required .ActorInfoProtocol actorInfo = 2; + private akka.remote.protocol.RemoteProtocol.ActorInfoProtocol actorInfo_ = akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol, akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder, akka.remote.protocol.RemoteProtocol.ActorInfoProtocolOrBuilder> actorInfoBuilder_; public boolean hasActorInfo() { - return result.hasActorInfo(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public akka.remote.protocol.RemoteProtocol.ActorInfoProtocol getActorInfo() { - return result.getActorInfo(); + if (actorInfoBuilder_ == null) { + return actorInfo_; + } else { + return actorInfoBuilder_.getMessage(); + } } public Builder setActorInfo(akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (actorInfoBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + actorInfo_ = value; + onChanged(); + } else { + actorInfoBuilder_.setMessage(value); } - result.hasActorInfo = true; - result.actorInfo_ = value; + bitField0_ |= 0x00000002; return this; } - public Builder setActorInfo(akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder builderForValue) { - result.hasActorInfo = true; - result.actorInfo_ = builderForValue.build(); + public Builder setActorInfo( + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder builderForValue) { + if (actorInfoBuilder_ == null) { + actorInfo_ = builderForValue.build(); + onChanged(); + } else { + actorInfoBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000002; return this; } public Builder mergeActorInfo(akka.remote.protocol.RemoteProtocol.ActorInfoProtocol value) { - if (result.hasActorInfo() && - result.actorInfo_ != akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) { - result.actorInfo_ = - akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(result.actorInfo_).mergeFrom(value).buildPartial(); + if (actorInfoBuilder_ == null) { + if (((bitField0_ & 0x00000002) == 0x00000002) && + actorInfo_ != akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance()) { + actorInfo_ = + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder(actorInfo_).mergeFrom(value).buildPartial(); + } else { + actorInfo_ = value; + } + onChanged(); } else { - result.actorInfo_ = value; + actorInfoBuilder_.mergeFrom(value); } - result.hasActorInfo = true; + bitField0_ |= 0x00000002; return this; } public Builder clearActorInfo() { - result.hasActorInfo = false; - result.actorInfo_ = akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); + if (actorInfoBuilder_ == null) { + actorInfo_ = akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); + onChanged(); + } else { + actorInfoBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000002); return this; } + public akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder getActorInfoBuilder() { + bitField0_ |= 0x00000002; + onChanged(); + return getActorInfoFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.ActorInfoProtocolOrBuilder getActorInfoOrBuilder() { + if (actorInfoBuilder_ != null) { + return actorInfoBuilder_.getMessageOrBuilder(); + } else { + return actorInfo_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol, akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder, akka.remote.protocol.RemoteProtocol.ActorInfoProtocolOrBuilder> + getActorInfoFieldBuilder() { + if (actorInfoBuilder_ == null) { + actorInfoBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol, akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.Builder, akka.remote.protocol.RemoteProtocol.ActorInfoProtocolOrBuilder>( + actorInfo_, + getParentForChildren(), + isClean()); + actorInfo_ = null; + } + return actorInfoBuilder_; + } // required bool oneWay = 3; + private boolean oneWay_ ; public boolean hasOneWay() { - return result.hasOneWay(); + return ((bitField0_ & 0x00000004) == 0x00000004); } public boolean getOneWay() { - return result.getOneWay(); + return oneWay_; } public Builder setOneWay(boolean value) { - result.hasOneWay = true; - result.oneWay_ = value; + bitField0_ |= 0x00000004; + oneWay_ = value; + onChanged(); return this; } public Builder clearOneWay() { - result.hasOneWay = false; - result.oneWay_ = false; + bitField0_ = (bitField0_ & ~0x00000004); + oneWay_ = false; + onChanged(); return this; } // optional .MessageProtocol message = 4; + private akka.remote.protocol.RemoteProtocol.MessageProtocol message_ = akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.MessageProtocol, akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder, akka.remote.protocol.RemoteProtocol.MessageProtocolOrBuilder> messageBuilder_; public boolean hasMessage() { - return result.hasMessage(); + return ((bitField0_ & 0x00000008) == 0x00000008); } public akka.remote.protocol.RemoteProtocol.MessageProtocol getMessage() { - return result.getMessage(); + if (messageBuilder_ == null) { + return message_; + } else { + return messageBuilder_.getMessage(); + } } public Builder setMessage(akka.remote.protocol.RemoteProtocol.MessageProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (messageBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + message_ = value; + onChanged(); + } else { + messageBuilder_.setMessage(value); } - result.hasMessage = true; - result.message_ = value; + bitField0_ |= 0x00000008; return this; } - public Builder setMessage(akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) { - result.hasMessage = true; - result.message_ = builderForValue.build(); + public Builder setMessage( + akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder builderForValue) { + if (messageBuilder_ == null) { + message_ = builderForValue.build(); + onChanged(); + } else { + messageBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000008; return this; } public Builder mergeMessage(akka.remote.protocol.RemoteProtocol.MessageProtocol value) { - if (result.hasMessage() && - result.message_ != akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) { - result.message_ = - akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(result.message_).mergeFrom(value).buildPartial(); + if (messageBuilder_ == null) { + if (((bitField0_ & 0x00000008) == 0x00000008) && + message_ != akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance()) { + message_ = + akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder(message_).mergeFrom(value).buildPartial(); + } else { + message_ = value; + } + onChanged(); } else { - result.message_ = value; + messageBuilder_.mergeFrom(value); } - result.hasMessage = true; + bitField0_ |= 0x00000008; return this; } public Builder clearMessage() { - result.hasMessage = false; - result.message_ = akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); + if (messageBuilder_ == null) { + message_ = akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); + onChanged(); + } else { + messageBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000008); return this; } + public akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder getMessageBuilder() { + bitField0_ |= 0x00000008; + onChanged(); + return getMessageFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.MessageProtocolOrBuilder getMessageOrBuilder() { + if (messageBuilder_ != null) { + return messageBuilder_.getMessageOrBuilder(); + } else { + return message_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.MessageProtocol, akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder, akka.remote.protocol.RemoteProtocol.MessageProtocolOrBuilder> + getMessageFieldBuilder() { + if (messageBuilder_ == null) { + messageBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.MessageProtocol, akka.remote.protocol.RemoteProtocol.MessageProtocol.Builder, akka.remote.protocol.RemoteProtocol.MessageProtocolOrBuilder>( + message_, + getParentForChildren(), + isClean()); + message_ = null; + } + return messageBuilder_; + } // optional .ExceptionProtocol exception = 5; + private akka.remote.protocol.RemoteProtocol.ExceptionProtocol exception_ = akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.ExceptionProtocol, akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder, akka.remote.protocol.RemoteProtocol.ExceptionProtocolOrBuilder> exceptionBuilder_; public boolean hasException() { - return result.hasException(); + return ((bitField0_ & 0x00000010) == 0x00000010); } public akka.remote.protocol.RemoteProtocol.ExceptionProtocol getException() { - return result.getException(); + if (exceptionBuilder_ == null) { + return exception_; + } else { + return exceptionBuilder_.getMessage(); + } } public Builder setException(akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (exceptionBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + exception_ = value; + onChanged(); + } else { + exceptionBuilder_.setMessage(value); } - result.hasException = true; - result.exception_ = value; + bitField0_ |= 0x00000010; return this; } - public Builder setException(akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder builderForValue) { - result.hasException = true; - result.exception_ = builderForValue.build(); + public Builder setException( + akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder builderForValue) { + if (exceptionBuilder_ == null) { + exception_ = builderForValue.build(); + onChanged(); + } else { + exceptionBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000010; return this; } public Builder mergeException(akka.remote.protocol.RemoteProtocol.ExceptionProtocol value) { - if (result.hasException() && - result.exception_ != akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) { - result.exception_ = - akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(result.exception_).mergeFrom(value).buildPartial(); + if (exceptionBuilder_ == null) { + if (((bitField0_ & 0x00000010) == 0x00000010) && + exception_ != akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance()) { + exception_ = + akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder(exception_).mergeFrom(value).buildPartial(); + } else { + exception_ = value; + } + onChanged(); } else { - result.exception_ = value; + exceptionBuilder_.mergeFrom(value); } - result.hasException = true; + bitField0_ |= 0x00000010; return this; } public Builder clearException() { - result.hasException = false; - result.exception_ = akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); + if (exceptionBuilder_ == null) { + exception_ = akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); + onChanged(); + } else { + exceptionBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000010); return this; } + public akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder getExceptionBuilder() { + bitField0_ |= 0x00000010; + onChanged(); + return getExceptionFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.ExceptionProtocolOrBuilder getExceptionOrBuilder() { + if (exceptionBuilder_ != null) { + return exceptionBuilder_.getMessageOrBuilder(); + } else { + return exception_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.ExceptionProtocol, akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder, akka.remote.protocol.RemoteProtocol.ExceptionProtocolOrBuilder> + getExceptionFieldBuilder() { + if (exceptionBuilder_ == null) { + exceptionBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.ExceptionProtocol, akka.remote.protocol.RemoteProtocol.ExceptionProtocol.Builder, akka.remote.protocol.RemoteProtocol.ExceptionProtocolOrBuilder>( + exception_, + getParentForChildren(), + isClean()); + exception_ = null; + } + return exceptionBuilder_; + } // optional .UuidProtocol supervisorUuid = 6; + private akka.remote.protocol.RemoteProtocol.UuidProtocol supervisorUuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder> supervisorUuidBuilder_; public boolean hasSupervisorUuid() { - return result.hasSupervisorUuid(); + return ((bitField0_ & 0x00000020) == 0x00000020); } public akka.remote.protocol.RemoteProtocol.UuidProtocol getSupervisorUuid() { - return result.getSupervisorUuid(); + if (supervisorUuidBuilder_ == null) { + return supervisorUuid_; + } else { + return supervisorUuidBuilder_.getMessage(); + } } public Builder setSupervisorUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (supervisorUuidBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + supervisorUuid_ = value; + onChanged(); + } else { + supervisorUuidBuilder_.setMessage(value); } - result.hasSupervisorUuid = true; - result.supervisorUuid_ = value; + bitField0_ |= 0x00000020; return this; } - public Builder setSupervisorUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { - result.hasSupervisorUuid = true; - result.supervisorUuid_ = builderForValue.build(); + public Builder setSupervisorUuid( + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + if (supervisorUuidBuilder_ == null) { + supervisorUuid_ = builderForValue.build(); + onChanged(); + } else { + supervisorUuidBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000020; return this; } public Builder mergeSupervisorUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { - if (result.hasSupervisorUuid() && - result.supervisorUuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { - result.supervisorUuid_ = - akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.supervisorUuid_).mergeFrom(value).buildPartial(); + if (supervisorUuidBuilder_ == null) { + if (((bitField0_ & 0x00000020) == 0x00000020) && + supervisorUuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + supervisorUuid_ = + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(supervisorUuid_).mergeFrom(value).buildPartial(); + } else { + supervisorUuid_ = value; + } + onChanged(); } else { - result.supervisorUuid_ = value; + supervisorUuidBuilder_.mergeFrom(value); } - result.hasSupervisorUuid = true; + bitField0_ |= 0x00000020; return this; } public Builder clearSupervisorUuid() { - result.hasSupervisorUuid = false; - result.supervisorUuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + if (supervisorUuidBuilder_ == null) { + supervisorUuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + onChanged(); + } else { + supervisorUuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000020); return this; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder getSupervisorUuidBuilder() { + bitField0_ |= 0x00000020; + onChanged(); + return getSupervisorUuidFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getSupervisorUuidOrBuilder() { + if (supervisorUuidBuilder_ != null) { + return supervisorUuidBuilder_.getMessageOrBuilder(); + } else { + return supervisorUuid_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder> + getSupervisorUuidFieldBuilder() { + if (supervisorUuidBuilder_ == null) { + supervisorUuidBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder>( + supervisorUuid_, + getParentForChildren(), + isClean()); + supervisorUuid_ = null; + } + return supervisorUuidBuilder_; + } // optional .RemoteActorRefProtocol sender = 7; + private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol sender_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder> senderBuilder_; public boolean hasSender() { - return result.hasSender(); + return ((bitField0_ & 0x00000040) == 0x00000040); } public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSender() { - return result.getSender(); + if (senderBuilder_ == null) { + return sender_; + } else { + return senderBuilder_.getMessage(); + } } public Builder setSender(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (senderBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + sender_ = value; + onChanged(); + } else { + senderBuilder_.setMessage(value); } - result.hasSender = true; - result.sender_ = value; + bitField0_ |= 0x00000040; return this; } - public Builder setSender(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { - result.hasSender = true; - result.sender_ = builderForValue.build(); + public Builder setSender( + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { + if (senderBuilder_ == null) { + sender_ = builderForValue.build(); + onChanged(); + } else { + senderBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000040; return this; } public Builder mergeSender(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { - if (result.hasSender() && - result.sender_ != akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { - result.sender_ = - akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.sender_).mergeFrom(value).buildPartial(); + if (senderBuilder_ == null) { + if (((bitField0_ & 0x00000040) == 0x00000040) && + sender_ != akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { + sender_ = + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(sender_).mergeFrom(value).buildPartial(); + } else { + sender_ = value; + } + onChanged(); } else { - result.sender_ = value; + senderBuilder_.mergeFrom(value); } - result.hasSender = true; + bitField0_ |= 0x00000040; return this; } public Builder clearSender() { - result.hasSender = false; - result.sender_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + if (senderBuilder_ == null) { + sender_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + onChanged(); + } else { + senderBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000040); return this; } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder getSenderBuilder() { + bitField0_ |= 0x00000040; + onChanged(); + return getSenderFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder getSenderOrBuilder() { + if (senderBuilder_ != null) { + return senderBuilder_.getMessageOrBuilder(); + } else { + return sender_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder> + getSenderFieldBuilder() { + if (senderBuilder_ == null) { + senderBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder>( + sender_, + getParentForChildren(), + isClean()); + sender_ = null; + } + return senderBuilder_; + } // repeated .MetadataEntryProtocol metadata = 8; + private java.util.List metadata_ = + java.util.Collections.emptyList(); + private void ensureMetadataIsMutable() { + if (!((bitField0_ & 0x00000080) == 0x00000080)) { + metadata_ = new java.util.ArrayList(metadata_); + bitField0_ |= 0x00000080; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocolOrBuilder> metadataBuilder_; + public java.util.List getMetadataList() { - return java.util.Collections.unmodifiableList(result.metadata_); + if (metadataBuilder_ == null) { + return java.util.Collections.unmodifiableList(metadata_); + } else { + return metadataBuilder_.getMessageList(); + } } public int getMetadataCount() { - return result.getMetadataCount(); + if (metadataBuilder_ == null) { + return metadata_.size(); + } else { + return metadataBuilder_.getCount(); + } } public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol getMetadata(int index) { - return result.getMetadata(index); - } - public Builder setMetadata(int index, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (metadataBuilder_ == null) { + return metadata_.get(index); + } else { + return metadataBuilder_.getMessage(index); + } + } + public Builder setMetadata( + int index, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { + if (metadataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.set(index, value); + onChanged(); + } else { + metadataBuilder_.setMessage(index, value); } - result.metadata_.set(index, value); return this; } - public Builder setMetadata(int index, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { - result.metadata_.set(index, builderForValue.build()); + public Builder setMetadata( + int index, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { + if (metadataBuilder_ == null) { + ensureMetadataIsMutable(); + metadata_.set(index, builderForValue.build()); + onChanged(); + } else { + metadataBuilder_.setMessage(index, builderForValue.build()); + } return this; } public Builder addMetadata(akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (metadataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.add(value); + onChanged(); + } else { + metadataBuilder_.addMessage(value); } - if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); - } - result.metadata_.add(value); return this; } - public Builder addMetadata(akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { - if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + public Builder addMetadata( + int index, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol value) { + if (metadataBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMetadataIsMutable(); + metadata_.add(index, value); + onChanged(); + } else { + metadataBuilder_.addMessage(index, value); + } + return this; + } + public Builder addMetadata( + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { + if (metadataBuilder_ == null) { + ensureMetadataIsMutable(); + metadata_.add(builderForValue.build()); + onChanged(); + } else { + metadataBuilder_.addMessage(builderForValue.build()); + } + return this; + } + public Builder addMetadata( + int index, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder builderForValue) { + if (metadataBuilder_ == null) { + ensureMetadataIsMutable(); + metadata_.add(index, builderForValue.build()); + onChanged(); + } else { + metadataBuilder_.addMessage(index, builderForValue.build()); } - result.metadata_.add(builderForValue.build()); return this; } public Builder addAllMetadata( java.lang.Iterable values) { - if (result.metadata_.isEmpty()) { - result.metadata_ = new java.util.ArrayList(); + if (metadataBuilder_ == null) { + ensureMetadataIsMutable(); + super.addAll(values, metadata_); + onChanged(); + } else { + metadataBuilder_.addAllMessages(values); } - super.addAll(values, result.metadata_); return this; } public Builder clearMetadata() { - result.metadata_ = java.util.Collections.emptyList(); + if (metadataBuilder_ == null) { + metadata_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000080); + onChanged(); + } else { + metadataBuilder_.clear(); + } return this; } + public Builder removeMetadata(int index) { + if (metadataBuilder_ == null) { + ensureMetadataIsMutable(); + metadata_.remove(index); + onChanged(); + } else { + metadataBuilder_.remove(index); + } + return this; + } + public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder getMetadataBuilder( + int index) { + return getMetadataFieldBuilder().getBuilder(index); + } + public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocolOrBuilder getMetadataOrBuilder( + int index) { + if (metadataBuilder_ == null) { + return metadata_.get(index); } else { + return metadataBuilder_.getMessageOrBuilder(index); + } + } + public java.util.List + getMetadataOrBuilderList() { + if (metadataBuilder_ != null) { + return metadataBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(metadata_); + } + } + public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder addMetadataBuilder() { + return getMetadataFieldBuilder().addBuilder( + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDefaultInstance()); + } + public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder addMetadataBuilder( + int index) { + return getMetadataFieldBuilder().addBuilder( + index, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDefaultInstance()); + } + public java.util.List + getMetadataBuilderList() { + return getMetadataFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocolOrBuilder> + getMetadataFieldBuilder() { + if (metadataBuilder_ == null) { + metadataBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.Builder, akka.remote.protocol.RemoteProtocol.MetadataEntryProtocolOrBuilder>( + metadata_, + ((bitField0_ & 0x00000080) == 0x00000080), + getParentForChildren(), + isClean()); + metadata_ = null; + } + return metadataBuilder_; + } // @@protoc_insertion_point(builder_scope:RemoteMessageProtocol) } static { defaultInstance = new RemoteMessageProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:RemoteMessageProtocol) } + public interface RemoteControlProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional string cookie = 1; + boolean hasCookie(); + String getCookie(); + + // required .CommandType commandType = 2; + boolean hasCommandType(); + akka.remote.protocol.RemoteProtocol.CommandType getCommandType(); + } public static final class RemoteControlProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements RemoteControlProtocolOrBuilder { // Use RemoteControlProtocol.newBuilder() to construct. - private RemoteControlProtocol() { - initFields(); + private RemoteControlProtocol(Builder builder) { + super(builder); } private RemoteControlProtocol(boolean noInit) {} @@ -1395,36 +2495,74 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_RemoteControlProtocol_fieldAccessorTable; } + private int bitField0_; // optional string cookie = 1; public static final int COOKIE_FIELD_NUMBER = 1; - private boolean hasCookie; - private java.lang.String cookie_ = ""; - public boolean hasCookie() { return hasCookie; } - public java.lang.String getCookie() { return cookie_; } + private java.lang.Object cookie_; + public boolean hasCookie() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getCookie() { + java.lang.Object ref = cookie_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + cookie_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getCookieBytes() { + java.lang.Object ref = cookie_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + cookie_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // required .CommandType commandType = 2; public static final int COMMANDTYPE_FIELD_NUMBER = 2; - private boolean hasCommandType; private akka.remote.protocol.RemoteProtocol.CommandType commandType_; - public boolean hasCommandType() { return hasCommandType; } - public akka.remote.protocol.RemoteProtocol.CommandType getCommandType() { return commandType_; } + public boolean hasCommandType() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public akka.remote.protocol.RemoteProtocol.CommandType getCommandType() { + return commandType_; + } private void initFields() { + cookie_ = ""; commandType_ = akka.remote.protocol.RemoteProtocol.CommandType.CONNECT; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasCommandType) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasCommandType()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasCookie()) { - output.writeString(1, getCookie()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getCookieBytes()); } - if (hasCommandType()) { - output.writeEnum(2, getCommandType().getNumber()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, commandType_.getNumber()); } getUnknownFields().writeTo(output); } @@ -1435,19 +2573,26 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasCookie()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getCookie()); + .computeBytesSize(1, getCookieBytes()); } - if (hasCommandType()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeEnumSize(2, getCommandType().getNumber()); + .computeEnumSize(2, commandType_.getNumber()); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.RemoteControlProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -1522,34 +2667,53 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.RemoteControlProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.RemoteControlProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.RemoteControlProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteControlProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.RemoteControlProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteControlProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.remote.protocol.RemoteProtocol.RemoteControlProtocol(); + super.clear(); + cookie_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + commandType_ = akka.remote.protocol.RemoteProtocol.CommandType.CONNECT; + bitField0_ = (bitField0_ & ~0x00000002); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -1561,33 +2725,39 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.RemoteControlProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.RemoteControlProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.RemoteControlProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.RemoteControlProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.RemoteControlProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.RemoteControlProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.RemoteControlProtocol result = new akka.remote.protocol.RemoteProtocol.RemoteControlProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.RemoteControlProtocol returnMe = result; - result = null; - return returnMe; + result.cookie_ = cookie_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.commandType_ = commandType_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -1611,6 +2781,14 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasCommandType()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -1623,17 +2801,20 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 10: { - setCookie(input.readString()); + bitField0_ |= 0x00000001; + cookie_ = input.readBytes(); break; } case 16: { @@ -1642,7 +2823,8 @@ public final class RemoteProtocol { if (value == null) { unknownFields.mergeVarintField(2, rawValue); } else { - setCommandType(value); + bitField0_ |= 0x00000002; + commandType_ = value; } break; } @@ -1650,46 +2832,65 @@ public final class RemoteProtocol { } } + private int bitField0_; // optional string cookie = 1; + private java.lang.Object cookie_ = ""; public boolean hasCookie() { - return result.hasCookie(); + return ((bitField0_ & 0x00000001) == 0x00000001); } - public java.lang.String getCookie() { - return result.getCookie(); + public String getCookie() { + java.lang.Object ref = cookie_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + cookie_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setCookie(java.lang.String value) { + public Builder setCookie(String value) { if (value == null) { throw new NullPointerException(); } - result.hasCookie = true; - result.cookie_ = value; + bitField0_ |= 0x00000001; + cookie_ = value; + onChanged(); return this; } public Builder clearCookie() { - result.hasCookie = false; - result.cookie_ = getDefaultInstance().getCookie(); + bitField0_ = (bitField0_ & ~0x00000001); + cookie_ = getDefaultInstance().getCookie(); + onChanged(); return this; } + void setCookie(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + cookie_ = value; + onChanged(); + } // required .CommandType commandType = 2; + private akka.remote.protocol.RemoteProtocol.CommandType commandType_ = akka.remote.protocol.RemoteProtocol.CommandType.CONNECT; public boolean hasCommandType() { - return result.hasCommandType(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public akka.remote.protocol.RemoteProtocol.CommandType getCommandType() { - return result.getCommandType(); + return commandType_; } public Builder setCommandType(akka.remote.protocol.RemoteProtocol.CommandType value) { if (value == null) { throw new NullPointerException(); } - result.hasCommandType = true; - result.commandType_ = value; + bitField0_ |= 0x00000002; + commandType_ = value; + onChanged(); return this; } public Builder clearCommandType() { - result.hasCommandType = false; - result.commandType_ = akka.remote.protocol.RemoteProtocol.CommandType.CONNECT; + bitField0_ = (bitField0_ & ~0x00000002); + commandType_ = akka.remote.protocol.RemoteProtocol.CommandType.CONNECT; + onChanged(); return this; } @@ -1698,18 +2899,33 @@ public final class RemoteProtocol { static { defaultInstance = new RemoteControlProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:RemoteControlProtocol) } + public interface RemoteActorRefProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string address = 1; + boolean hasAddress(); + String getAddress(); + + // required bytes inetSocketAddress = 2; + boolean hasInetSocketAddress(); + com.google.protobuf.ByteString getInetSocketAddress(); + + // optional uint64 timeout = 3; + boolean hasTimeout(); + long getTimeout(); + } public static final class RemoteActorRefProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements RemoteActorRefProtocolOrBuilder { // Use RemoteActorRefProtocol.newBuilder() to construct. - private RemoteActorRefProtocol() { - initFields(); + private RemoteActorRefProtocol(Builder builder) { + super(builder); } private RemoteActorRefProtocol(boolean noInit) {} @@ -1732,46 +2948,92 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_fieldAccessorTable; } + private int bitField0_; // required string address = 1; public static final int ADDRESS_FIELD_NUMBER = 1; - private boolean hasAddress; - private java.lang.String address_ = ""; - public boolean hasAddress() { return hasAddress; } - public java.lang.String getAddress() { return address_; } + private java.lang.Object address_; + public boolean hasAddress() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getAddress() { + java.lang.Object ref = address_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + address_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getAddressBytes() { + java.lang.Object ref = address_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + address_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // required bytes inetSocketAddress = 2; public static final int INETSOCKETADDRESS_FIELD_NUMBER = 2; - private boolean hasInetSocketAddress; - private com.google.protobuf.ByteString inetSocketAddress_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasInetSocketAddress() { return hasInetSocketAddress; } - public com.google.protobuf.ByteString getInetSocketAddress() { return inetSocketAddress_; } + private com.google.protobuf.ByteString inetSocketAddress_; + public boolean hasInetSocketAddress() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public com.google.protobuf.ByteString getInetSocketAddress() { + return inetSocketAddress_; + } // optional 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_; } + private long timeout_; + public boolean hasTimeout() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTimeout() { + return timeout_; + } private void initFields() { + address_ = ""; + inetSocketAddress_ = com.google.protobuf.ByteString.EMPTY; + timeout_ = 0L; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasAddress) return false; - if (!hasInetSocketAddress) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasAddress()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasInetSocketAddress()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasAddress()) { - output.writeString(1, getAddress()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getAddressBytes()); } - if (hasInetSocketAddress()) { - output.writeBytes(2, getInetSocketAddress()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, inetSocketAddress_); } - if (hasTimeout()) { - output.writeUInt64(3, getTimeout()); + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, timeout_); } getUnknownFields().writeTo(output); } @@ -1782,23 +3044,30 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasAddress()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getAddress()); + .computeBytesSize(1, getAddressBytes()); } - if (hasInetSocketAddress()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(2, getInetSocketAddress()); + .computeBytesSize(2, inetSocketAddress_); } - if (hasTimeout()) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(3, getTimeout()); + .computeUInt64Size(3, timeout_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -1873,34 +3142,55 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_RemoteActorRefProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(); + super.clear(); + address_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + inetSocketAddress_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + timeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -1912,33 +3202,43 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol result = new akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol returnMe = result; - result = null; - return returnMe; + result.address_ = address_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.inetSocketAddress_ = inetSocketAddress_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.timeout_ = timeout_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -1965,6 +3265,18 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasAddress()) { + + return false; + } + if (!hasInetSocketAddress()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -1977,89 +3289,116 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 10: { - setAddress(input.readString()); + bitField0_ |= 0x00000001; + address_ = input.readBytes(); break; } case 18: { - setInetSocketAddress(input.readBytes()); + bitField0_ |= 0x00000002; + inetSocketAddress_ = input.readBytes(); break; } case 24: { - setTimeout(input.readUInt64()); + bitField0_ |= 0x00000004; + timeout_ = input.readUInt64(); break; } } } } + private int bitField0_; // required string address = 1; + private java.lang.Object address_ = ""; public boolean hasAddress() { - return result.hasAddress(); + return ((bitField0_ & 0x00000001) == 0x00000001); } - public java.lang.String getAddress() { - return result.getAddress(); + public String getAddress() { + java.lang.Object ref = address_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + address_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setAddress(java.lang.String value) { + public Builder setAddress(String value) { if (value == null) { throw new NullPointerException(); } - result.hasAddress = true; - result.address_ = value; + bitField0_ |= 0x00000001; + address_ = value; + onChanged(); return this; } public Builder clearAddress() { - result.hasAddress = false; - result.address_ = getDefaultInstance().getAddress(); + bitField0_ = (bitField0_ & ~0x00000001); + address_ = getDefaultInstance().getAddress(); + onChanged(); return this; } + void setAddress(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + address_ = value; + onChanged(); + } // required bytes inetSocketAddress = 2; + private com.google.protobuf.ByteString inetSocketAddress_ = com.google.protobuf.ByteString.EMPTY; public boolean hasInetSocketAddress() { - return result.hasInetSocketAddress(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public com.google.protobuf.ByteString getInetSocketAddress() { - return result.getInetSocketAddress(); + return inetSocketAddress_; } public Builder setInetSocketAddress(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - result.hasInetSocketAddress = true; - result.inetSocketAddress_ = value; + bitField0_ |= 0x00000002; + inetSocketAddress_ = value; + onChanged(); return this; } public Builder clearInetSocketAddress() { - result.hasInetSocketAddress = false; - result.inetSocketAddress_ = getDefaultInstance().getInetSocketAddress(); + bitField0_ = (bitField0_ & ~0x00000002); + inetSocketAddress_ = getDefaultInstance().getInetSocketAddress(); + onChanged(); return this; } // optional uint64 timeout = 3; + private long timeout_ ; public boolean hasTimeout() { - return result.hasTimeout(); + return ((bitField0_ & 0x00000004) == 0x00000004); } public long getTimeout() { - return result.getTimeout(); + return timeout_; } public Builder setTimeout(long value) { - result.hasTimeout = true; - result.timeout_ = value; + bitField0_ |= 0x00000004; + timeout_ = value; + onChanged(); return this; } public Builder clearTimeout() { - result.hasTimeout = false; - result.timeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + timeout_ = 0L; + onChanged(); return this; } @@ -2068,18 +3407,78 @@ public final class RemoteProtocol { static { defaultInstance = new RemoteActorRefProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:RemoteActorRefProtocol) } + public interface SerializedActorRefProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .UuidProtocol uuid = 1; + boolean hasUuid(); + akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid(); + akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getUuidOrBuilder(); + + // required string address = 2; + boolean hasAddress(); + String getAddress(); + + // required string actorClassname = 3; + boolean hasActorClassname(); + String getActorClassname(); + + // optional bytes actorInstance = 4; + boolean hasActorInstance(); + com.google.protobuf.ByteString getActorInstance(); + + // optional string serializerClassname = 5; + boolean hasSerializerClassname(); + String getSerializerClassname(); + + // optional uint64 timeout = 6; + boolean hasTimeout(); + long getTimeout(); + + // optional uint64 receiveTimeout = 7; + boolean hasReceiveTimeout(); + long getReceiveTimeout(); + + // optional .LifeCycleProtocol lifeCycle = 8; + boolean hasLifeCycle(); + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle(); + akka.remote.protocol.RemoteProtocol.LifeCycleProtocolOrBuilder getLifeCycleOrBuilder(); + + // optional .RemoteActorRefProtocol supervisor = 9; + boolean hasSupervisor(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor(); + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder getSupervisorOrBuilder(); + + // optional bytes hotswapStack = 10; + boolean hasHotswapStack(); + com.google.protobuf.ByteString getHotswapStack(); + + // optional .ReplicationStrategyType replicationStrategy = 11; + boolean hasReplicationStrategy(); + akka.remote.protocol.RemoteProtocol.ReplicationStrategyType getReplicationStrategy(); + + // repeated .RemoteMessageProtocol messages = 12; + java.util.List + getMessagesList(); + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol getMessages(int index); + int getMessagesCount(); + java.util.List + getMessagesOrBuilderList(); + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder getMessagesOrBuilder( + int index); + } public static final class SerializedActorRefProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements SerializedActorRefProtocolOrBuilder { // Use SerializedActorRefProtocol.newBuilder() to construct. - private SerializedActorRefProtocol() { - initFields(); + private SerializedActorRefProtocol(Builder builder) { + super(builder); } private SerializedActorRefProtocol(boolean noInit) {} @@ -2102,145 +3501,308 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_fieldAccessorTable; } + private int bitField0_; // required .UuidProtocol uuid = 1; public static final int UUID_FIELD_NUMBER = 1; - private boolean hasUuid; private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; - public boolean hasUuid() { return hasUuid; } - public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } + public boolean hasUuid() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { + return uuid_; + } + public akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getUuidOrBuilder() { + return uuid_; + } // required string address = 2; public static final int ADDRESS_FIELD_NUMBER = 2; - private boolean hasAddress; - private java.lang.String address_ = ""; - public boolean hasAddress() { return hasAddress; } - public java.lang.String getAddress() { return address_; } + private java.lang.Object address_; + public boolean hasAddress() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getAddress() { + java.lang.Object ref = address_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + address_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getAddressBytes() { + java.lang.Object ref = address_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + address_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // 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_; } + private java.lang.Object actorClassname_; + public boolean hasActorClassname() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getActorClassname() { + java.lang.Object ref = actorClassname_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + actorClassname_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getActorClassnameBytes() { + java.lang.Object ref = actorClassname_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + actorClassname_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // optional bytes actorInstance = 4; public static final int ACTORINSTANCE_FIELD_NUMBER = 4; - 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_; } + private com.google.protobuf.ByteString actorInstance_; + public boolean hasActorInstance() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public com.google.protobuf.ByteString getActorInstance() { + return actorInstance_; + } // optional string serializerClassname = 5; public static final int SERIALIZERCLASSNAME_FIELD_NUMBER = 5; - private boolean hasSerializerClassname; - private java.lang.String serializerClassname_ = ""; - public boolean hasSerializerClassname() { return hasSerializerClassname; } - public java.lang.String getSerializerClassname() { return serializerClassname_; } + private java.lang.Object serializerClassname_; + public boolean hasSerializerClassname() { + return ((bitField0_ & 0x00000010) == 0x00000010); + } + public String getSerializerClassname() { + java.lang.Object ref = serializerClassname_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + serializerClassname_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getSerializerClassnameBytes() { + java.lang.Object ref = serializerClassname_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + serializerClassname_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // optional uint64 timeout = 6; public static final int TIMEOUT_FIELD_NUMBER = 6; - private boolean hasTimeout; - private long timeout_ = 0L; - public boolean hasTimeout() { return hasTimeout; } - public long getTimeout() { return timeout_; } + private long timeout_; + public boolean hasTimeout() { + return ((bitField0_ & 0x00000020) == 0x00000020); + } + public long getTimeout() { + return timeout_; + } // optional uint64 receiveTimeout = 7; public static final int RECEIVETIMEOUT_FIELD_NUMBER = 7; - private boolean hasReceiveTimeout; - private long receiveTimeout_ = 0L; - public boolean hasReceiveTimeout() { return hasReceiveTimeout; } - public long getReceiveTimeout() { return receiveTimeout_; } + private long receiveTimeout_; + public boolean hasReceiveTimeout() { + return ((bitField0_ & 0x00000040) == 0x00000040); + } + public long getReceiveTimeout() { + return receiveTimeout_; + } // optional .LifeCycleProtocol lifeCycle = 8; public static final int LIFECYCLE_FIELD_NUMBER = 8; - private boolean hasLifeCycle; private akka.remote.protocol.RemoteProtocol.LifeCycleProtocol lifeCycle_; - public boolean hasLifeCycle() { return hasLifeCycle; } - public akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { return lifeCycle_; } + public boolean hasLifeCycle() { + return ((bitField0_ & 0x00000080) == 0x00000080); + } + public akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { + return lifeCycle_; + } + public akka.remote.protocol.RemoteProtocol.LifeCycleProtocolOrBuilder getLifeCycleOrBuilder() { + return lifeCycle_; + } // optional .RemoteActorRefProtocol supervisor = 9; public static final int SUPERVISOR_FIELD_NUMBER = 9; - private boolean hasSupervisor; private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol supervisor_; - public boolean hasSupervisor() { return hasSupervisor; } - public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { return supervisor_; } + public boolean hasSupervisor() { + return ((bitField0_ & 0x00000100) == 0x00000100); + } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { + return supervisor_; + } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder getSupervisorOrBuilder() { + return supervisor_; + } // optional bytes hotswapStack = 10; public static final int HOTSWAPSTACK_FIELD_NUMBER = 10; - 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_; } + private com.google.protobuf.ByteString hotswapStack_; + public boolean hasHotswapStack() { + return ((bitField0_ & 0x00000200) == 0x00000200); + } + public com.google.protobuf.ByteString getHotswapStack() { + return hotswapStack_; + } - // repeated .RemoteMessageProtocol messages = 11; - public static final int MESSAGES_FIELD_NUMBER = 11; - private java.util.List messages_ = - java.util.Collections.emptyList(); + // optional .ReplicationStrategyType replicationStrategy = 11; + public static final int REPLICATIONSTRATEGY_FIELD_NUMBER = 11; + private akka.remote.protocol.RemoteProtocol.ReplicationStrategyType replicationStrategy_; + public boolean hasReplicationStrategy() { + return ((bitField0_ & 0x00000400) == 0x00000400); + } + public akka.remote.protocol.RemoteProtocol.ReplicationStrategyType getReplicationStrategy() { + return replicationStrategy_; + } + + // repeated .RemoteMessageProtocol messages = 12; + public static final int MESSAGES_FIELD_NUMBER = 12; + private java.util.List messages_; public java.util.List getMessagesList() { return messages_; } - public int getMessagesCount() { return messages_.size(); } + public java.util.List + getMessagesOrBuilderList() { + return messages_; + } + public int getMessagesCount() { + return messages_.size(); + } public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol getMessages(int index) { return messages_.get(index); } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder getMessagesOrBuilder( + int index) { + return messages_.get(index); + } private void initFields() { uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + address_ = ""; + actorClassname_ = ""; + actorInstance_ = com.google.protobuf.ByteString.EMPTY; + serializerClassname_ = ""; + timeout_ = 0L; + receiveTimeout_ = 0L; lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); supervisor_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + hotswapStack_ = com.google.protobuf.ByteString.EMPTY; + replicationStrategy_ = akka.remote.protocol.RemoteProtocol.ReplicationStrategyType.TRANSIENT; + messages_ = java.util.Collections.emptyList(); } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasUuid) return false; - if (!hasAddress) return false; - if (!hasActorClassname) return false; - if (!getUuid().isInitialized()) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasUuid()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasAddress()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasActorClassname()) { + memoizedIsInitialized = 0; + return false; + } + if (!getUuid().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } if (hasLifeCycle()) { - if (!getLifeCycle().isInitialized()) return false; + if (!getLifeCycle().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } if (hasSupervisor()) { - if (!getSupervisor().isInitialized()) return false; + if (!getSupervisor().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } - for (akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol element : getMessagesList()) { - if (!element.isInitialized()) return false; + for (int i = 0; i < getMessagesCount(); i++) { + if (!getMessages(i).isInitialized()) { + memoizedIsInitialized = 0; + return false; + } } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasUuid()) { - output.writeMessage(1, getUuid()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, uuid_); } - if (hasAddress()) { - output.writeString(2, getAddress()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getAddressBytes()); } - if (hasActorClassname()) { - output.writeString(3, getActorClassname()); + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getActorClassnameBytes()); } - if (hasActorInstance()) { - output.writeBytes(4, getActorInstance()); + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeBytes(4, actorInstance_); } - if (hasSerializerClassname()) { - output.writeString(5, getSerializerClassname()); + if (((bitField0_ & 0x00000010) == 0x00000010)) { + output.writeBytes(5, getSerializerClassnameBytes()); } - if (hasTimeout()) { - output.writeUInt64(6, getTimeout()); + if (((bitField0_ & 0x00000020) == 0x00000020)) { + output.writeUInt64(6, timeout_); } - if (hasReceiveTimeout()) { - output.writeUInt64(7, getReceiveTimeout()); + if (((bitField0_ & 0x00000040) == 0x00000040)) { + output.writeUInt64(7, receiveTimeout_); } - if (hasLifeCycle()) { - output.writeMessage(8, getLifeCycle()); + if (((bitField0_ & 0x00000080) == 0x00000080)) { + output.writeMessage(8, lifeCycle_); } - if (hasSupervisor()) { - output.writeMessage(9, getSupervisor()); + if (((bitField0_ & 0x00000100) == 0x00000100)) { + output.writeMessage(9, supervisor_); } - if (hasHotswapStack()) { - output.writeBytes(10, getHotswapStack()); + if (((bitField0_ & 0x00000200) == 0x00000200)) { + output.writeBytes(10, hotswapStack_); } - for (akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol element : getMessagesList()) { - output.writeMessage(11, element); + if (((bitField0_ & 0x00000400) == 0x00000400)) { + output.writeEnum(11, replicationStrategy_.getNumber()); + } + for (int i = 0; i < messages_.size(); i++) { + output.writeMessage(12, messages_.get(i)); } getUnknownFields().writeTo(output); } @@ -2251,55 +3813,66 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasUuid()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getUuid()); + .computeMessageSize(1, uuid_); } - if (hasAddress()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getAddress()); + .computeBytesSize(2, getAddressBytes()); } - if (hasActorClassname()) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(3, getActorClassname()); + .computeBytesSize(3, getActorClassnameBytes()); } - if (hasActorInstance()) { + if (((bitField0_ & 0x00000008) == 0x00000008)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(4, getActorInstance()); + .computeBytesSize(4, actorInstance_); } - if (hasSerializerClassname()) { + if (((bitField0_ & 0x00000010) == 0x00000010)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(5, getSerializerClassname()); + .computeBytesSize(5, getSerializerClassnameBytes()); } - if (hasTimeout()) { + if (((bitField0_ & 0x00000020) == 0x00000020)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(6, getTimeout()); + .computeUInt64Size(6, timeout_); } - if (hasReceiveTimeout()) { + if (((bitField0_ & 0x00000040) == 0x00000040)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(7, getReceiveTimeout()); + .computeUInt64Size(7, receiveTimeout_); } - if (hasLifeCycle()) { + if (((bitField0_ & 0x00000080) == 0x00000080)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(8, getLifeCycle()); + .computeMessageSize(8, lifeCycle_); } - if (hasSupervisor()) { + if (((bitField0_ & 0x00000100) == 0x00000100)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(9, getSupervisor()); + .computeMessageSize(9, supervisor_); } - if (hasHotswapStack()) { + if (((bitField0_ & 0x00000200) == 0x00000200)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(10, getHotswapStack()); + .computeBytesSize(10, hotswapStack_); } - for (akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol element : getMessagesList()) { + if (((bitField0_ & 0x00000400) == 0x00000400)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(11, element); + .computeEnumSize(11, replicationStrategy_.getNumber()); + } + for (int i = 0; i < messages_.size(); i++) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(12, messages_.get(i)); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -2374,34 +3947,93 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_SerializedActorRefProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getUuidFieldBuilder(); + getLifeCycleFieldBuilder(); + getSupervisorFieldBuilder(); + getMessagesFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); + super.clear(); + if (uuidBuilder_ == null) { + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + } else { + uuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + address_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + actorClassname_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); + actorInstance_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000008); + serializerClassname_ = ""; + bitField0_ = (bitField0_ & ~0x00000010); + timeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000020); + receiveTimeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000040); + if (lifeCycleBuilder_ == null) { + lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); + } else { + lifeCycleBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000080); + if (supervisorBuilder_ == null) { + supervisor_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + } else { + supervisorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000100); + hotswapStack_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000200); + replicationStrategy_ = akka.remote.protocol.RemoteProtocol.ReplicationStrategyType.TRANSIENT; + bitField0_ = (bitField0_ & ~0x00000400); + if (messagesBuilder_ == null) { + messages_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000800); + } else { + messagesBuilder_.clear(); } - result = new akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -2413,37 +4045,96 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol result = new akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - if (result.messages_ != java.util.Collections.EMPTY_LIST) { - result.messages_ = - java.util.Collections.unmodifiableList(result.messages_); + if (uuidBuilder_ == null) { + result.uuid_ = uuid_; + } else { + result.uuid_ = uuidBuilder_.build(); } - akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol returnMe = result; - result = null; - return returnMe; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.address_ = address_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.actorClassname_ = actorClassname_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.actorInstance_ = actorInstance_; + if (((from_bitField0_ & 0x00000010) == 0x00000010)) { + to_bitField0_ |= 0x00000010; + } + result.serializerClassname_ = serializerClassname_; + if (((from_bitField0_ & 0x00000020) == 0x00000020)) { + to_bitField0_ |= 0x00000020; + } + result.timeout_ = timeout_; + if (((from_bitField0_ & 0x00000040) == 0x00000040)) { + to_bitField0_ |= 0x00000040; + } + result.receiveTimeout_ = receiveTimeout_; + if (((from_bitField0_ & 0x00000080) == 0x00000080)) { + to_bitField0_ |= 0x00000080; + } + if (lifeCycleBuilder_ == null) { + result.lifeCycle_ = lifeCycle_; + } else { + result.lifeCycle_ = lifeCycleBuilder_.build(); + } + if (((from_bitField0_ & 0x00000100) == 0x00000100)) { + to_bitField0_ |= 0x00000100; + } + if (supervisorBuilder_ == null) { + result.supervisor_ = supervisor_; + } else { + result.supervisor_ = supervisorBuilder_.build(); + } + if (((from_bitField0_ & 0x00000200) == 0x00000200)) { + to_bitField0_ |= 0x00000200; + } + result.hotswapStack_ = hotswapStack_; + if (((from_bitField0_ & 0x00000400) == 0x00000400)) { + to_bitField0_ |= 0x00000400; + } + result.replicationStrategy_ = replicationStrategy_; + if (messagesBuilder_ == null) { + if (((bitField0_ & 0x00000800) == 0x00000800)) { + messages_ = java.util.Collections.unmodifiableList(messages_); + bitField0_ = (bitField0_ & ~0x00000800); + } + result.messages_ = messages_; + } else { + result.messages_ = messagesBuilder_.build(); + } + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -2487,16 +4178,77 @@ public final class RemoteProtocol { if (other.hasHotswapStack()) { setHotswapStack(other.getHotswapStack()); } - if (!other.messages_.isEmpty()) { - if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); + if (other.hasReplicationStrategy()) { + setReplicationStrategy(other.getReplicationStrategy()); + } + if (messagesBuilder_ == null) { + if (!other.messages_.isEmpty()) { + if (messages_.isEmpty()) { + messages_ = other.messages_; + bitField0_ = (bitField0_ & ~0x00000800); + } else { + ensureMessagesIsMutable(); + messages_.addAll(other.messages_); + } + onChanged(); + } + } else { + if (!other.messages_.isEmpty()) { + if (messagesBuilder_.isEmpty()) { + messagesBuilder_.dispose(); + messagesBuilder_ = null; + messages_ = other.messages_; + bitField0_ = (bitField0_ & ~0x00000800); + messagesBuilder_ = + com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders ? + getMessagesFieldBuilder() : null; + } else { + messagesBuilder_.addAllMessages(other.messages_); + } } - result.messages_.addAll(other.messages_); } this.mergeUnknownFields(other.getUnknownFields()); return this; } + public final boolean isInitialized() { + if (!hasUuid()) { + + return false; + } + if (!hasAddress()) { + + return false; + } + if (!hasActorClassname()) { + + return false; + } + if (!getUuid().isInitialized()) { + + return false; + } + if (hasLifeCycle()) { + if (!getLifeCycle().isInitialized()) { + + return false; + } + } + if (hasSupervisor()) { + if (!getSupervisor().isInitialized()) { + + return false; + } + } + for (int i = 0; i < getMessagesCount(); i++) { + if (!getMessages(i).isInitialized()) { + + return false; + } + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -2509,11 +4261,13 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; @@ -2528,27 +4282,33 @@ public final class RemoteProtocol { break; } case 18: { - setAddress(input.readString()); + bitField0_ |= 0x00000002; + address_ = input.readBytes(); break; } case 26: { - setActorClassname(input.readString()); + bitField0_ |= 0x00000004; + actorClassname_ = input.readBytes(); break; } case 34: { - setActorInstance(input.readBytes()); + bitField0_ |= 0x00000008; + actorInstance_ = input.readBytes(); break; } case 42: { - setSerializerClassname(input.readString()); + bitField0_ |= 0x00000010; + serializerClassname_ = input.readBytes(); break; } case 48: { - setTimeout(input.readUInt64()); + bitField0_ |= 0x00000020; + timeout_ = input.readUInt64(); break; } case 56: { - setReceiveTimeout(input.readUInt64()); + bitField0_ |= 0x00000040; + receiveTimeout_ = input.readUInt64(); break; } case 66: { @@ -2570,10 +4330,22 @@ public final class RemoteProtocol { break; } case 82: { - setHotswapStack(input.readBytes()); + bitField0_ |= 0x00000200; + hotswapStack_ = input.readBytes(); break; } - case 90: { + case 88: { + int rawValue = input.readEnum(); + akka.remote.protocol.RemoteProtocol.ReplicationStrategyType value = akka.remote.protocol.RemoteProtocol.ReplicationStrategyType.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(11, rawValue); + } else { + bitField0_ |= 0x00000400; + replicationStrategy_ = value; + } + break; + } + case 98: { akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder subBuilder = akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.newBuilder(); input.readMessage(subBuilder, extensionRegistry); addMessages(subBuilder.buildPartial()); @@ -2583,327 +4355,715 @@ public final class RemoteProtocol { } } + private int bitField0_; // required .UuidProtocol uuid = 1; + private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder> uuidBuilder_; public boolean hasUuid() { - return result.hasUuid(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { - return result.getUuid(); + if (uuidBuilder_ == null) { + return uuid_; + } else { + return uuidBuilder_.getMessage(); + } } public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (uuidBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + uuid_ = value; + onChanged(); + } else { + uuidBuilder_.setMessage(value); } - result.hasUuid = true; - result.uuid_ = value; + bitField0_ |= 0x00000001; return this; } - public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { - result.hasUuid = true; - result.uuid_ = builderForValue.build(); + public Builder setUuid( + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + if (uuidBuilder_ == null) { + uuid_ = builderForValue.build(); + onChanged(); + } else { + uuidBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; return this; } public Builder mergeUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { - if (result.hasUuid() && - result.uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { - result.uuid_ = - akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); + if (uuidBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + uuid_ = + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(uuid_).mergeFrom(value).buildPartial(); + } else { + uuid_ = value; + } + onChanged(); } else { - result.uuid_ = value; + uuidBuilder_.mergeFrom(value); } - result.hasUuid = true; + bitField0_ |= 0x00000001; return this; } public Builder clearUuid() { - result.hasUuid = false; - result.uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + if (uuidBuilder_ == null) { + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + onChanged(); + } else { + uuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); return this; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder getUuidBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getUuidFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getUuidOrBuilder() { + if (uuidBuilder_ != null) { + return uuidBuilder_.getMessageOrBuilder(); + } else { + return uuid_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder> + getUuidFieldBuilder() { + if (uuidBuilder_ == null) { + uuidBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder>( + uuid_, + getParentForChildren(), + isClean()); + uuid_ = null; + } + return uuidBuilder_; + } // required string address = 2; + private java.lang.Object address_ = ""; public boolean hasAddress() { - return result.hasAddress(); + return ((bitField0_ & 0x00000002) == 0x00000002); } - public java.lang.String getAddress() { - return result.getAddress(); + public String getAddress() { + java.lang.Object ref = address_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + address_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setAddress(java.lang.String value) { + public Builder setAddress(String value) { if (value == null) { throw new NullPointerException(); } - result.hasAddress = true; - result.address_ = value; + bitField0_ |= 0x00000002; + address_ = value; + onChanged(); return this; } public Builder clearAddress() { - result.hasAddress = false; - result.address_ = getDefaultInstance().getAddress(); + bitField0_ = (bitField0_ & ~0x00000002); + address_ = getDefaultInstance().getAddress(); + onChanged(); return this; } + void setAddress(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + address_ = value; + onChanged(); + } // required string actorClassname = 3; + private java.lang.Object actorClassname_ = ""; public boolean hasActorClassname() { - return result.hasActorClassname(); + return ((bitField0_ & 0x00000004) == 0x00000004); } - public java.lang.String getActorClassname() { - return result.getActorClassname(); + public String getActorClassname() { + java.lang.Object ref = actorClassname_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + actorClassname_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setActorClassname(java.lang.String value) { + public Builder setActorClassname(String value) { if (value == null) { throw new NullPointerException(); } - result.hasActorClassname = true; - result.actorClassname_ = value; + bitField0_ |= 0x00000004; + actorClassname_ = value; + onChanged(); return this; } public Builder clearActorClassname() { - result.hasActorClassname = false; - result.actorClassname_ = getDefaultInstance().getActorClassname(); + bitField0_ = (bitField0_ & ~0x00000004); + actorClassname_ = getDefaultInstance().getActorClassname(); + onChanged(); return this; } + void setActorClassname(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + actorClassname_ = value; + onChanged(); + } // optional bytes actorInstance = 4; + private com.google.protobuf.ByteString actorInstance_ = com.google.protobuf.ByteString.EMPTY; public boolean hasActorInstance() { - return result.hasActorInstance(); + return ((bitField0_ & 0x00000008) == 0x00000008); } public com.google.protobuf.ByteString getActorInstance() { - return result.getActorInstance(); + return actorInstance_; } public Builder setActorInstance(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - result.hasActorInstance = true; - result.actorInstance_ = value; + bitField0_ |= 0x00000008; + actorInstance_ = value; + onChanged(); return this; } public Builder clearActorInstance() { - result.hasActorInstance = false; - result.actorInstance_ = getDefaultInstance().getActorInstance(); + bitField0_ = (bitField0_ & ~0x00000008); + actorInstance_ = getDefaultInstance().getActorInstance(); + onChanged(); return this; } // optional string serializerClassname = 5; + private java.lang.Object serializerClassname_ = ""; public boolean hasSerializerClassname() { - return result.hasSerializerClassname(); + return ((bitField0_ & 0x00000010) == 0x00000010); } - public java.lang.String getSerializerClassname() { - return result.getSerializerClassname(); + public String getSerializerClassname() { + java.lang.Object ref = serializerClassname_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + serializerClassname_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setSerializerClassname(java.lang.String value) { + public Builder setSerializerClassname(String value) { if (value == null) { throw new NullPointerException(); } - result.hasSerializerClassname = true; - result.serializerClassname_ = value; + bitField0_ |= 0x00000010; + serializerClassname_ = value; + onChanged(); return this; } public Builder clearSerializerClassname() { - result.hasSerializerClassname = false; - result.serializerClassname_ = getDefaultInstance().getSerializerClassname(); + bitField0_ = (bitField0_ & ~0x00000010); + serializerClassname_ = getDefaultInstance().getSerializerClassname(); + onChanged(); return this; } + void setSerializerClassname(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000010; + serializerClassname_ = value; + onChanged(); + } // optional uint64 timeout = 6; + private long timeout_ ; public boolean hasTimeout() { - return result.hasTimeout(); + return ((bitField0_ & 0x00000020) == 0x00000020); } public long getTimeout() { - return result.getTimeout(); + return timeout_; } public Builder setTimeout(long value) { - result.hasTimeout = true; - result.timeout_ = value; + bitField0_ |= 0x00000020; + timeout_ = value; + onChanged(); return this; } public Builder clearTimeout() { - result.hasTimeout = false; - result.timeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000020); + timeout_ = 0L; + onChanged(); return this; } // optional uint64 receiveTimeout = 7; + private long receiveTimeout_ ; public boolean hasReceiveTimeout() { - return result.hasReceiveTimeout(); + return ((bitField0_ & 0x00000040) == 0x00000040); } public long getReceiveTimeout() { - return result.getReceiveTimeout(); + return receiveTimeout_; } public Builder setReceiveTimeout(long value) { - result.hasReceiveTimeout = true; - result.receiveTimeout_ = value; + bitField0_ |= 0x00000040; + receiveTimeout_ = value; + onChanged(); return this; } public Builder clearReceiveTimeout() { - result.hasReceiveTimeout = false; - result.receiveTimeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000040); + receiveTimeout_ = 0L; + onChanged(); return this; } // optional .LifeCycleProtocol lifeCycle = 8; + private akka.remote.protocol.RemoteProtocol.LifeCycleProtocol lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol, akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder, akka.remote.protocol.RemoteProtocol.LifeCycleProtocolOrBuilder> lifeCycleBuilder_; public boolean hasLifeCycle() { - return result.hasLifeCycle(); + return ((bitField0_ & 0x00000080) == 0x00000080); } public akka.remote.protocol.RemoteProtocol.LifeCycleProtocol getLifeCycle() { - return result.getLifeCycle(); + if (lifeCycleBuilder_ == null) { + return lifeCycle_; + } else { + return lifeCycleBuilder_.getMessage(); + } } public Builder setLifeCycle(akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (lifeCycleBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + lifeCycle_ = value; + onChanged(); + } else { + lifeCycleBuilder_.setMessage(value); } - result.hasLifeCycle = true; - result.lifeCycle_ = value; + bitField0_ |= 0x00000080; return this; } - public Builder setLifeCycle(akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder builderForValue) { - result.hasLifeCycle = true; - result.lifeCycle_ = builderForValue.build(); + public Builder setLifeCycle( + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder builderForValue) { + if (lifeCycleBuilder_ == null) { + lifeCycle_ = builderForValue.build(); + onChanged(); + } else { + lifeCycleBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000080; return this; } public Builder mergeLifeCycle(akka.remote.protocol.RemoteProtocol.LifeCycleProtocol value) { - if (result.hasLifeCycle() && - result.lifeCycle_ != akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) { - result.lifeCycle_ = - akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(result.lifeCycle_).mergeFrom(value).buildPartial(); + if (lifeCycleBuilder_ == null) { + if (((bitField0_ & 0x00000080) == 0x00000080) && + lifeCycle_ != akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance()) { + lifeCycle_ = + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder(lifeCycle_).mergeFrom(value).buildPartial(); + } else { + lifeCycle_ = value; + } + onChanged(); } else { - result.lifeCycle_ = value; + lifeCycleBuilder_.mergeFrom(value); } - result.hasLifeCycle = true; + bitField0_ |= 0x00000080; return this; } public Builder clearLifeCycle() { - result.hasLifeCycle = false; - result.lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); + if (lifeCycleBuilder_ == null) { + lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); + onChanged(); + } else { + lifeCycleBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000080); return this; } + public akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder getLifeCycleBuilder() { + bitField0_ |= 0x00000080; + onChanged(); + return getLifeCycleFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.LifeCycleProtocolOrBuilder getLifeCycleOrBuilder() { + if (lifeCycleBuilder_ != null) { + return lifeCycleBuilder_.getMessageOrBuilder(); + } else { + return lifeCycle_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol, akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder, akka.remote.protocol.RemoteProtocol.LifeCycleProtocolOrBuilder> + getLifeCycleFieldBuilder() { + if (lifeCycleBuilder_ == null) { + lifeCycleBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol, akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.Builder, akka.remote.protocol.RemoteProtocol.LifeCycleProtocolOrBuilder>( + lifeCycle_, + getParentForChildren(), + isClean()); + lifeCycle_ = null; + } + return lifeCycleBuilder_; + } // optional .RemoteActorRefProtocol supervisor = 9; + private akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol supervisor_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder> supervisorBuilder_; public boolean hasSupervisor() { - return result.hasSupervisor(); + return ((bitField0_ & 0x00000100) == 0x00000100); } public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol getSupervisor() { - return result.getSupervisor(); + if (supervisorBuilder_ == null) { + return supervisor_; + } else { + return supervisorBuilder_.getMessage(); + } } public Builder setSupervisor(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (supervisorBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + supervisor_ = value; + onChanged(); + } else { + supervisorBuilder_.setMessage(value); } - result.hasSupervisor = true; - result.supervisor_ = value; + bitField0_ |= 0x00000100; return this; } - public Builder setSupervisor(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { - result.hasSupervisor = true; - result.supervisor_ = builderForValue.build(); + public Builder setSupervisor( + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder builderForValue) { + if (supervisorBuilder_ == null) { + supervisor_ = builderForValue.build(); + onChanged(); + } else { + supervisorBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000100; return this; } public Builder mergeSupervisor(akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol value) { - if (result.hasSupervisor() && - result.supervisor_ != akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { - result.supervisor_ = - akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(result.supervisor_).mergeFrom(value).buildPartial(); + if (supervisorBuilder_ == null) { + if (((bitField0_ & 0x00000100) == 0x00000100) && + supervisor_ != akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance()) { + supervisor_ = + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.newBuilder(supervisor_).mergeFrom(value).buildPartial(); + } else { + supervisor_ = value; + } + onChanged(); } else { - result.supervisor_ = value; + supervisorBuilder_.mergeFrom(value); } - result.hasSupervisor = true; + bitField0_ |= 0x00000100; return this; } public Builder clearSupervisor() { - result.hasSupervisor = false; - result.supervisor_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + if (supervisorBuilder_ == null) { + supervisor_ = akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.getDefaultInstance(); + onChanged(); + } else { + supervisorBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000100); return this; } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder getSupervisorBuilder() { + bitField0_ |= 0x00000100; + onChanged(); + return getSupervisorFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder getSupervisorOrBuilder() { + if (supervisorBuilder_ != null) { + return supervisorBuilder_.getMessageOrBuilder(); + } else { + return supervisor_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder> + getSupervisorFieldBuilder() { + if (supervisorBuilder_ == null) { + supervisorBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteActorRefProtocolOrBuilder>( + supervisor_, + getParentForChildren(), + isClean()); + supervisor_ = null; + } + return supervisorBuilder_; + } // optional bytes hotswapStack = 10; + private com.google.protobuf.ByteString hotswapStack_ = com.google.protobuf.ByteString.EMPTY; public boolean hasHotswapStack() { - return result.hasHotswapStack(); + return ((bitField0_ & 0x00000200) == 0x00000200); } public com.google.protobuf.ByteString getHotswapStack() { - return result.getHotswapStack(); + return hotswapStack_; } public Builder setHotswapStack(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - result.hasHotswapStack = true; - result.hotswapStack_ = value; + bitField0_ |= 0x00000200; + hotswapStack_ = value; + onChanged(); return this; } public Builder clearHotswapStack() { - result.hasHotswapStack = false; - result.hotswapStack_ = getDefaultInstance().getHotswapStack(); + bitField0_ = (bitField0_ & ~0x00000200); + hotswapStack_ = getDefaultInstance().getHotswapStack(); + onChanged(); return this; } - // repeated .RemoteMessageProtocol messages = 11; - public java.util.List getMessagesList() { - return java.util.Collections.unmodifiableList(result.messages_); + // optional .ReplicationStrategyType replicationStrategy = 11; + private akka.remote.protocol.RemoteProtocol.ReplicationStrategyType replicationStrategy_ = akka.remote.protocol.RemoteProtocol.ReplicationStrategyType.TRANSIENT; + public boolean hasReplicationStrategy() { + return ((bitField0_ & 0x00000400) == 0x00000400); } - public int getMessagesCount() { - return result.getMessagesCount(); + public akka.remote.protocol.RemoteProtocol.ReplicationStrategyType getReplicationStrategy() { + return replicationStrategy_; } - public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol getMessages(int index) { - return result.getMessages(index); - } - public Builder setMessages(int index, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol value) { + public Builder setReplicationStrategy(akka.remote.protocol.RemoteProtocol.ReplicationStrategyType value) { if (value == null) { throw new NullPointerException(); } - result.messages_.set(index, value); + bitField0_ |= 0x00000400; + replicationStrategy_ = value; + onChanged(); return this; } - public Builder setMessages(int index, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder builderForValue) { - result.messages_.set(index, builderForValue.build()); + public Builder clearReplicationStrategy() { + bitField0_ = (bitField0_ & ~0x00000400); + replicationStrategy_ = akka.remote.protocol.RemoteProtocol.ReplicationStrategyType.TRANSIENT; + onChanged(); + return this; + } + + // repeated .RemoteMessageProtocol messages = 12; + private java.util.List messages_ = + java.util.Collections.emptyList(); + private void ensureMessagesIsMutable() { + if (!((bitField0_ & 0x00000800) == 0x00000800)) { + messages_ = new java.util.ArrayList(messages_); + bitField0_ |= 0x00000800; + } + } + + private com.google.protobuf.RepeatedFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder> messagesBuilder_; + + public java.util.List getMessagesList() { + if (messagesBuilder_ == null) { + return java.util.Collections.unmodifiableList(messages_); + } else { + return messagesBuilder_.getMessageList(); + } + } + public int getMessagesCount() { + if (messagesBuilder_ == null) { + return messages_.size(); + } else { + return messagesBuilder_.getCount(); + } + } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol getMessages(int index) { + if (messagesBuilder_ == null) { + return messages_.get(index); + } else { + return messagesBuilder_.getMessage(index); + } + } + public Builder setMessages( + int index, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol value) { + if (messagesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMessagesIsMutable(); + messages_.set(index, value); + onChanged(); + } else { + messagesBuilder_.setMessage(index, value); + } + return this; + } + public Builder setMessages( + int index, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder builderForValue) { + if (messagesBuilder_ == null) { + ensureMessagesIsMutable(); + messages_.set(index, builderForValue.build()); + onChanged(); + } else { + messagesBuilder_.setMessage(index, builderForValue.build()); + } return this; } public Builder addMessages(akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (messagesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMessagesIsMutable(); + messages_.add(value); + onChanged(); + } else { + messagesBuilder_.addMessage(value); } - if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); - } - result.messages_.add(value); return this; } - public Builder addMessages(akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder builderForValue) { - if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); + public Builder addMessages( + int index, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol value) { + if (messagesBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + ensureMessagesIsMutable(); + messages_.add(index, value); + onChanged(); + } else { + messagesBuilder_.addMessage(index, value); + } + return this; + } + public Builder addMessages( + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder builderForValue) { + if (messagesBuilder_ == null) { + ensureMessagesIsMutable(); + messages_.add(builderForValue.build()); + onChanged(); + } else { + messagesBuilder_.addMessage(builderForValue.build()); + } + return this; + } + public Builder addMessages( + int index, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder builderForValue) { + if (messagesBuilder_ == null) { + ensureMessagesIsMutable(); + messages_.add(index, builderForValue.build()); + onChanged(); + } else { + messagesBuilder_.addMessage(index, builderForValue.build()); } - result.messages_.add(builderForValue.build()); return this; } public Builder addAllMessages( java.lang.Iterable values) { - if (result.messages_.isEmpty()) { - result.messages_ = new java.util.ArrayList(); + if (messagesBuilder_ == null) { + ensureMessagesIsMutable(); + super.addAll(values, messages_); + onChanged(); + } else { + messagesBuilder_.addAllMessages(values); } - super.addAll(values, result.messages_); return this; } public Builder clearMessages() { - result.messages_ = java.util.Collections.emptyList(); + if (messagesBuilder_ == null) { + messages_ = java.util.Collections.emptyList(); + bitField0_ = (bitField0_ & ~0x00000800); + onChanged(); + } else { + messagesBuilder_.clear(); + } return this; } + public Builder removeMessages(int index) { + if (messagesBuilder_ == null) { + ensureMessagesIsMutable(); + messages_.remove(index); + onChanged(); + } else { + messagesBuilder_.remove(index); + } + return this; + } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder getMessagesBuilder( + int index) { + return getMessagesFieldBuilder().getBuilder(index); + } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder getMessagesOrBuilder( + int index) { + if (messagesBuilder_ == null) { + return messages_.get(index); } else { + return messagesBuilder_.getMessageOrBuilder(index); + } + } + public java.util.List + getMessagesOrBuilderList() { + if (messagesBuilder_ != null) { + return messagesBuilder_.getMessageOrBuilderList(); + } else { + return java.util.Collections.unmodifiableList(messages_); + } + } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder addMessagesBuilder() { + return getMessagesFieldBuilder().addBuilder( + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance()); + } + public akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder addMessagesBuilder( + int index) { + return getMessagesFieldBuilder().addBuilder( + index, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.getDefaultInstance()); + } + public java.util.List + getMessagesBuilderList() { + return getMessagesFieldBuilder().getBuilderList(); + } + private com.google.protobuf.RepeatedFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder> + getMessagesFieldBuilder() { + if (messagesBuilder_ == null) { + messagesBuilder_ = new com.google.protobuf.RepeatedFieldBuilder< + akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocol.Builder, akka.remote.protocol.RemoteProtocol.RemoteMessageProtocolOrBuilder>( + messages_, + ((bitField0_ & 0x00000800) == 0x00000800), + getParentForChildren(), + isClean()); + messages_ = null; + } + return messagesBuilder_; + } // @@protoc_insertion_point(builder_scope:SerializedActorRefProtocol) } static { defaultInstance = new SerializedActorRefProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:SerializedActorRefProtocol) } + public interface SerializedTypedActorRefProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .SerializedActorRefProtocol actorRef = 1; + boolean hasActorRef(); + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef(); + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocolOrBuilder getActorRefOrBuilder(); + + // required string interfaceName = 2; + boolean hasInterfaceName(); + String getInterfaceName(); + } public static final class SerializedTypedActorRefProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements SerializedTypedActorRefProtocolOrBuilder { // Use SerializedTypedActorRefProtocol.newBuilder() to construct. - private SerializedTypedActorRefProtocol() { - initFields(); + private SerializedTypedActorRefProtocol(Builder builder) { + super(builder); } private SerializedTypedActorRefProtocol(boolean noInit) {} @@ -2926,38 +5086,85 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_SerializedTypedActorRefProtocol_fieldAccessorTable; } + private int bitField0_; // required .SerializedActorRefProtocol actorRef = 1; public static final int ACTORREF_FIELD_NUMBER = 1; - private boolean hasActorRef; private akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol actorRef_; - public boolean hasActorRef() { return hasActorRef; } - public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef() { return actorRef_; } + public boolean hasActorRef() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef() { + return actorRef_; + } + public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocolOrBuilder getActorRefOrBuilder() { + 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 java.lang.Object interfaceName_; + public boolean hasInterfaceName() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getInterfaceName() { + java.lang.Object ref = interfaceName_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + interfaceName_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getInterfaceNameBytes() { + java.lang.Object ref = interfaceName_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + interfaceName_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } private void initFields() { actorRef_ = akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); + interfaceName_ = ""; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasActorRef) return false; - if (!hasInterfaceName) return false; - if (!getActorRef().isInitialized()) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasActorRef()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasInterfaceName()) { + memoizedIsInitialized = 0; + return false; + } + if (!getActorRef().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasActorRef()) { - output.writeMessage(1, getActorRef()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, actorRef_); } - if (hasInterfaceName()) { - output.writeString(2, getInterfaceName()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getInterfaceNameBytes()); } getUnknownFields().writeTo(output); } @@ -2968,19 +5175,26 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasActorRef()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getActorRef()); + .computeMessageSize(1, actorRef_); } - if (hasInterfaceName()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getInterfaceName()); + .computeBytesSize(2, getInterfaceNameBytes()); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -3055,34 +5269,58 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_SerializedTypedActorRefProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_SerializedTypedActorRefProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getActorRefFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); + super.clear(); + if (actorRefBuilder_ == null) { + actorRef_ = akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); + } else { + actorRefBuilder_.clear(); } - result = new akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol(); + bitField0_ = (bitField0_ & ~0x00000001); + interfaceName_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -3094,33 +5332,43 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol result = new akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.SerializedTypedActorRefProtocol returnMe = result; - result = null; - return returnMe; + if (actorRefBuilder_ == null) { + result.actorRef_ = actorRef_; + } else { + result.actorRef_ = actorRefBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.interfaceName_ = interfaceName_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -3144,6 +5392,22 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasActorRef()) { + + return false; + } + if (!hasInterfaceName()) { + + return false; + } + if (!getActorRef().isInitialized()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -3156,11 +5420,13 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; @@ -3175,89 +5441,174 @@ public final class RemoteProtocol { break; } case 18: { - setInterfaceName(input.readString()); + bitField0_ |= 0x00000002; + interfaceName_ = input.readBytes(); break; } } } } + private int bitField0_; // required .SerializedActorRefProtocol actorRef = 1; + private akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol actorRef_ = akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol, akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder, akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocolOrBuilder> actorRefBuilder_; public boolean hasActorRef() { - return result.hasActorRef(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol getActorRef() { - return result.getActorRef(); + if (actorRefBuilder_ == null) { + return actorRef_; + } else { + return actorRefBuilder_.getMessage(); + } } public Builder setActorRef(akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (actorRefBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + actorRef_ = value; + onChanged(); + } else { + actorRefBuilder_.setMessage(value); } - result.hasActorRef = true; - result.actorRef_ = value; + bitField0_ |= 0x00000001; return this; } - public Builder setActorRef(akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder builderForValue) { - result.hasActorRef = true; - result.actorRef_ = builderForValue.build(); + public Builder setActorRef( + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder builderForValue) { + if (actorRefBuilder_ == null) { + actorRef_ = builderForValue.build(); + onChanged(); + } else { + actorRefBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; return this; } public Builder mergeActorRef(akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol value) { - if (result.hasActorRef() && - result.actorRef_ != akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) { - result.actorRef_ = - akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder(result.actorRef_).mergeFrom(value).buildPartial(); + if (actorRefBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + actorRef_ != akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance()) { + actorRef_ = + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.newBuilder(actorRef_).mergeFrom(value).buildPartial(); + } else { + actorRef_ = value; + } + onChanged(); } else { - result.actorRef_ = value; + actorRefBuilder_.mergeFrom(value); } - result.hasActorRef = true; + bitField0_ |= 0x00000001; return this; } public Builder clearActorRef() { - result.hasActorRef = false; - result.actorRef_ = akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); + if (actorRefBuilder_ == null) { + actorRef_ = akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.getDefaultInstance(); + onChanged(); + } else { + actorRefBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); return this; } + public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder getActorRefBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getActorRefFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocolOrBuilder getActorRefOrBuilder() { + if (actorRefBuilder_ != null) { + return actorRefBuilder_.getMessageOrBuilder(); + } else { + return actorRef_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol, akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder, akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocolOrBuilder> + getActorRefFieldBuilder() { + if (actorRefBuilder_ == null) { + actorRefBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol, akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder, akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocolOrBuilder>( + actorRef_, + getParentForChildren(), + isClean()); + actorRef_ = null; + } + return actorRefBuilder_; + } // required string interfaceName = 2; + private java.lang.Object interfaceName_ = ""; public boolean hasInterfaceName() { - return result.hasInterfaceName(); + return ((bitField0_ & 0x00000002) == 0x00000002); } - public java.lang.String getInterfaceName() { - return result.getInterfaceName(); + public String getInterfaceName() { + java.lang.Object ref = interfaceName_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + interfaceName_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setInterfaceName(java.lang.String value) { + public Builder setInterfaceName(String value) { if (value == null) { throw new NullPointerException(); } - result.hasInterfaceName = true; - result.interfaceName_ = value; + bitField0_ |= 0x00000002; + interfaceName_ = value; + onChanged(); return this; } public Builder clearInterfaceName() { - result.hasInterfaceName = false; - result.interfaceName_ = getDefaultInstance().getInterfaceName(); + bitField0_ = (bitField0_ & ~0x00000002); + interfaceName_ = getDefaultInstance().getInterfaceName(); + onChanged(); return this; } + void setInterfaceName(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + interfaceName_ = value; + onChanged(); + } // @@protoc_insertion_point(builder_scope:SerializedTypedActorRefProtocol) } static { defaultInstance = new SerializedTypedActorRefProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:SerializedTypedActorRefProtocol) } + public interface MessageProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .SerializationSchemeType serializationScheme = 1; + boolean hasSerializationScheme(); + akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme(); + + // required bytes message = 2; + boolean hasMessage(); + com.google.protobuf.ByteString getMessage(); + + // optional bytes messageManifest = 3; + boolean hasMessageManifest(); + com.google.protobuf.ByteString getMessageManifest(); + } public static final class MessageProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements MessageProtocolOrBuilder { // Use MessageProtocol.newBuilder() to construct. - private MessageProtocol() { - initFields(); + private MessageProtocol(Builder builder) { + super(builder); } private MessageProtocol(boolean noInit) {} @@ -3280,47 +5631,70 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_fieldAccessorTable; } + private int bitField0_; // required .SerializationSchemeType serializationScheme = 1; public static final int SERIALIZATIONSCHEME_FIELD_NUMBER = 1; - private boolean hasSerializationScheme; private akka.remote.protocol.RemoteProtocol.SerializationSchemeType serializationScheme_; - public boolean hasSerializationScheme() { return hasSerializationScheme; } - public akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { return serializationScheme_; } + public boolean hasSerializationScheme() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public 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_; } + private com.google.protobuf.ByteString message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + 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 com.google.protobuf.ByteString messageManifest_; + public boolean hasMessageManifest() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public com.google.protobuf.ByteString getMessageManifest() { + return messageManifest_; + } private void initFields() { serializationScheme_ = akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; + message_ = com.google.protobuf.ByteString.EMPTY; + messageManifest_ = com.google.protobuf.ByteString.EMPTY; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasSerializationScheme) return false; - if (!hasMessage) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasSerializationScheme()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasMessage()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasSerializationScheme()) { - output.writeEnum(1, getSerializationScheme().getNumber()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, serializationScheme_.getNumber()); } - if (hasMessage()) { - output.writeBytes(2, getMessage()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, message_); } - if (hasMessageManifest()) { - output.writeBytes(3, getMessageManifest()); + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, messageManifest_); } getUnknownFields().writeTo(output); } @@ -3331,23 +5705,30 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasSerializationScheme()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeEnumSize(1, getSerializationScheme().getNumber()); + .computeEnumSize(1, serializationScheme_.getNumber()); } - if (hasMessage()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(2, getMessage()); + .computeBytesSize(2, message_); } - if (hasMessageManifest()) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(3, getMessageManifest()); + .computeBytesSize(3, messageManifest_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.MessageProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -3422,34 +5803,55 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.MessageProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.MessageProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.MessageProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.MessageProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_MessageProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.MessageProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.remote.protocol.RemoteProtocol.MessageProtocol(); + super.clear(); + serializationScheme_ = akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; + bitField0_ = (bitField0_ & ~0x00000001); + message_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); + messageManifest_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000004); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -3461,33 +5863,43 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.MessageProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.MessageProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.MessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.MessageProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.MessageProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.MessageProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.MessageProtocol result = new akka.remote.protocol.RemoteProtocol.MessageProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.MessageProtocol returnMe = result; - result = null; - return returnMe; + result.serializationScheme_ = serializationScheme_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.message_ = message_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.messageManifest_ = messageManifest_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -3514,6 +5926,18 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasSerializationScheme()) { + + return false; + } + if (!hasMessage()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -3526,11 +5950,13 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; @@ -3541,83 +5967,96 @@ public final class RemoteProtocol { if (value == null) { unknownFields.mergeVarintField(1, rawValue); } else { - setSerializationScheme(value); + bitField0_ |= 0x00000001; + serializationScheme_ = value; } break; } case 18: { - setMessage(input.readBytes()); + bitField0_ |= 0x00000002; + message_ = input.readBytes(); break; } case 26: { - setMessageManifest(input.readBytes()); + bitField0_ |= 0x00000004; + messageManifest_ = input.readBytes(); break; } } } } + private int bitField0_; // required .SerializationSchemeType serializationScheme = 1; + private akka.remote.protocol.RemoteProtocol.SerializationSchemeType serializationScheme_ = akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; public boolean hasSerializationScheme() { - return result.hasSerializationScheme(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public akka.remote.protocol.RemoteProtocol.SerializationSchemeType getSerializationScheme() { - return result.getSerializationScheme(); + return serializationScheme_; } public Builder setSerializationScheme(akka.remote.protocol.RemoteProtocol.SerializationSchemeType value) { if (value == null) { throw new NullPointerException(); } - result.hasSerializationScheme = true; - result.serializationScheme_ = value; + bitField0_ |= 0x00000001; + serializationScheme_ = value; + onChanged(); return this; } public Builder clearSerializationScheme() { - result.hasSerializationScheme = false; - result.serializationScheme_ = akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; + bitField0_ = (bitField0_ & ~0x00000001); + serializationScheme_ = akka.remote.protocol.RemoteProtocol.SerializationSchemeType.JAVA; + onChanged(); return this; } // required bytes message = 2; + private com.google.protobuf.ByteString message_ = com.google.protobuf.ByteString.EMPTY; public boolean hasMessage() { - return result.hasMessage(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public com.google.protobuf.ByteString getMessage() { - return result.getMessage(); + return message_; } public Builder setMessage(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - result.hasMessage = true; - result.message_ = value; + bitField0_ |= 0x00000002; + message_ = value; + onChanged(); return this; } public Builder clearMessage() { - result.hasMessage = false; - result.message_ = getDefaultInstance().getMessage(); + bitField0_ = (bitField0_ & ~0x00000002); + message_ = getDefaultInstance().getMessage(); + onChanged(); return this; } // optional bytes messageManifest = 3; + private com.google.protobuf.ByteString messageManifest_ = com.google.protobuf.ByteString.EMPTY; public boolean hasMessageManifest() { - return result.hasMessageManifest(); + return ((bitField0_ & 0x00000004) == 0x00000004); } public com.google.protobuf.ByteString getMessageManifest() { - return result.getMessageManifest(); + return messageManifest_; } public Builder setMessageManifest(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - result.hasMessageManifest = true; - result.messageManifest_ = value; + bitField0_ |= 0x00000004; + messageManifest_ = value; + onChanged(); return this; } public Builder clearMessageManifest() { - result.hasMessageManifest = false; - result.messageManifest_ = getDefaultInstance().getMessageManifest(); + bitField0_ = (bitField0_ & ~0x00000004); + messageManifest_ = getDefaultInstance().getMessageManifest(); + onChanged(); return this; } @@ -3626,18 +6065,34 @@ public final class RemoteProtocol { static { defaultInstance = new MessageProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:MessageProtocol) } + public interface ActorInfoProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .UuidProtocol uuid = 1; + boolean hasUuid(); + akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid(); + akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getUuidOrBuilder(); + + // required uint64 timeout = 2; + boolean hasTimeout(); + long getTimeout(); + + // optional string address = 3; + boolean hasAddress(); + String getAddress(); + } public static final class ActorInfoProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements ActorInfoProtocolOrBuilder { // Use ActorInfoProtocol.newBuilder() to construct. - private ActorInfoProtocol() { - initFields(); + private ActorInfoProtocol(Builder builder) { + super(builder); } private ActorInfoProtocol(boolean noInit) {} @@ -3660,48 +6115,99 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_fieldAccessorTable; } + private int bitField0_; // required .UuidProtocol uuid = 1; public static final int UUID_FIELD_NUMBER = 1; - private boolean hasUuid; private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_; - public boolean hasUuid() { return hasUuid; } - public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { return uuid_; } + public boolean hasUuid() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { + return uuid_; + } + public akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getUuidOrBuilder() { + return uuid_; + } // required uint64 timeout = 2; public static final int TIMEOUT_FIELD_NUMBER = 2; - private boolean hasTimeout; - private long timeout_ = 0L; - public boolean hasTimeout() { return hasTimeout; } - public long getTimeout() { return timeout_; } + private long timeout_; + public boolean hasTimeout() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getTimeout() { + return timeout_; + } // optional string address = 3; public static final int ADDRESS_FIELD_NUMBER = 3; - private boolean hasAddress; - private java.lang.String address_ = ""; - public boolean hasAddress() { return hasAddress; } - public java.lang.String getAddress() { return address_; } + private java.lang.Object address_; + public boolean hasAddress() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public String getAddress() { + java.lang.Object ref = address_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + address_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getAddressBytes() { + java.lang.Object ref = address_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + address_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } private void initFields() { uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + timeout_ = 0L; + address_ = ""; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasUuid) return false; - if (!hasTimeout) return false; - if (!getUuid().isInitialized()) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasUuid()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasTimeout()) { + memoizedIsInitialized = 0; + return false; + } + if (!getUuid().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasUuid()) { - output.writeMessage(1, getUuid()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, uuid_); } - if (hasTimeout()) { - output.writeUInt64(2, getTimeout()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, timeout_); } - if (hasAddress()) { - output.writeString(3, getAddress()); + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeBytes(3, getAddressBytes()); } getUnknownFields().writeTo(output); } @@ -3712,23 +6218,30 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasUuid()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, getUuid()); + .computeMessageSize(1, uuid_); } - if (hasTimeout()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(2, getTimeout()); + .computeUInt64Size(2, timeout_); } - if (hasAddress()) { + if (((bitField0_ & 0x00000004) == 0x00000004)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(3, getAddress()); + .computeBytesSize(3, getAddressBytes()); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.ActorInfoProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -3803,34 +6316,60 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.ActorInfoProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.ActorInfoProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_ActorInfoProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getUuidFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); + super.clear(); + if (uuidBuilder_ == null) { + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + } else { + uuidBuilder_.clear(); } - result = new akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(); + bitField0_ = (bitField0_ & ~0x00000001); + timeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + address_ = ""; + bitField0_ = (bitField0_ & ~0x00000004); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -3842,33 +6381,47 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.ActorInfoProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.ActorInfoProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.ActorInfoProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.ActorInfoProtocol result = new akka.remote.protocol.RemoteProtocol.ActorInfoProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.ActorInfoProtocol returnMe = result; - result = null; - return returnMe; + if (uuidBuilder_ == null) { + result.uuid_ = uuid_; + } else { + result.uuid_ = uuidBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.timeout_ = timeout_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.address_ = address_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -3895,6 +6448,22 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasUuid()) { + + return false; + } + if (!hasTimeout()) { + + return false; + } + if (!getUuid().isInitialized()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -3907,11 +6476,13 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; @@ -3926,111 +6497,196 @@ public final class RemoteProtocol { break; } case 16: { - setTimeout(input.readUInt64()); + bitField0_ |= 0x00000002; + timeout_ = input.readUInt64(); break; } case 26: { - setAddress(input.readString()); + bitField0_ |= 0x00000004; + address_ = input.readBytes(); break; } } } } + private int bitField0_; // required .UuidProtocol uuid = 1; + private akka.remote.protocol.RemoteProtocol.UuidProtocol uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder> uuidBuilder_; public boolean hasUuid() { - return result.hasUuid(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public akka.remote.protocol.RemoteProtocol.UuidProtocol getUuid() { - return result.getUuid(); + if (uuidBuilder_ == null) { + return uuid_; + } else { + return uuidBuilder_.getMessage(); + } } public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { - if (value == null) { - throw new NullPointerException(); + if (uuidBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + uuid_ = value; + onChanged(); + } else { + uuidBuilder_.setMessage(value); } - result.hasUuid = true; - result.uuid_ = value; + bitField0_ |= 0x00000001; return this; } - public Builder setUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { - result.hasUuid = true; - result.uuid_ = builderForValue.build(); + public Builder setUuid( + akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder builderForValue) { + if (uuidBuilder_ == null) { + uuid_ = builderForValue.build(); + onChanged(); + } else { + uuidBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; return this; } public Builder mergeUuid(akka.remote.protocol.RemoteProtocol.UuidProtocol value) { - if (result.hasUuid() && - result.uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { - result.uuid_ = - akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(result.uuid_).mergeFrom(value).buildPartial(); + if (uuidBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + uuid_ != akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance()) { + uuid_ = + akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder(uuid_).mergeFrom(value).buildPartial(); + } else { + uuid_ = value; + } + onChanged(); } else { - result.uuid_ = value; + uuidBuilder_.mergeFrom(value); } - result.hasUuid = true; + bitField0_ |= 0x00000001; return this; } public Builder clearUuid() { - result.hasUuid = false; - result.uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + if (uuidBuilder_ == null) { + uuid_ = akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); + onChanged(); + } else { + uuidBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); return this; } + public akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder getUuidBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getUuidFieldBuilder().getBuilder(); + } + public akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder getUuidOrBuilder() { + if (uuidBuilder_ != null) { + return uuidBuilder_.getMessageOrBuilder(); + } else { + return uuid_; + } + } + private com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder> + getUuidFieldBuilder() { + if (uuidBuilder_ == null) { + uuidBuilder_ = new com.google.protobuf.SingleFieldBuilder< + akka.remote.protocol.RemoteProtocol.UuidProtocol, akka.remote.protocol.RemoteProtocol.UuidProtocol.Builder, akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder>( + uuid_, + getParentForChildren(), + isClean()); + uuid_ = null; + } + return uuidBuilder_; + } // required uint64 timeout = 2; + private long timeout_ ; public boolean hasTimeout() { - return result.hasTimeout(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public long getTimeout() { - return result.getTimeout(); + return timeout_; } public Builder setTimeout(long value) { - result.hasTimeout = true; - result.timeout_ = value; + bitField0_ |= 0x00000002; + timeout_ = value; + onChanged(); return this; } public Builder clearTimeout() { - result.hasTimeout = false; - result.timeout_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + timeout_ = 0L; + onChanged(); return this; } // optional string address = 3; + private java.lang.Object address_ = ""; public boolean hasAddress() { - return result.hasAddress(); + return ((bitField0_ & 0x00000004) == 0x00000004); } - public java.lang.String getAddress() { - return result.getAddress(); + public String getAddress() { + java.lang.Object ref = address_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + address_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setAddress(java.lang.String value) { + public Builder setAddress(String value) { if (value == null) { throw new NullPointerException(); } - result.hasAddress = true; - result.address_ = value; + bitField0_ |= 0x00000004; + address_ = value; + onChanged(); return this; } public Builder clearAddress() { - result.hasAddress = false; - result.address_ = getDefaultInstance().getAddress(); + bitField0_ = (bitField0_ & ~0x00000004); + address_ = getDefaultInstance().getAddress(); + onChanged(); return this; } + void setAddress(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000004; + address_ = value; + onChanged(); + } // @@protoc_insertion_point(builder_scope:ActorInfoProtocol) } static { defaultInstance = new ActorInfoProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:ActorInfoProtocol) } + public interface UuidProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required uint64 high = 1; + boolean hasHigh(); + long getHigh(); + + // required uint64 low = 2; + boolean hasLow(); + long getLow(); + } public static final class UuidProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements UuidProtocolOrBuilder { // Use UuidProtocol.newBuilder() to construct. - private UuidProtocol() { - initFields(); + private UuidProtocol(Builder builder) { + super(builder); } private UuidProtocol(boolean noInit) {} @@ -4053,36 +6709,56 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_UuidProtocol_fieldAccessorTable; } + private int bitField0_; // required uint64 high = 1; public static final int HIGH_FIELD_NUMBER = 1; - private boolean hasHigh; - private long high_ = 0L; - public boolean hasHigh() { return hasHigh; } - public long getHigh() { return high_; } + private long high_; + public boolean hasHigh() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getHigh() { + return high_; + } // required uint64 low = 2; public static final int LOW_FIELD_NUMBER = 2; - private boolean hasLow; - private long low_ = 0L; - public boolean hasLow() { return hasLow; } - public long getLow() { return low_; } + private long low_; + public boolean hasLow() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getLow() { + return low_; + } private void initFields() { + high_ = 0L; + low_ = 0L; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasHigh) return false; - if (!hasLow) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasHigh()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasLow()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasHigh()) { - output.writeUInt64(1, getHigh()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, high_); } - if (hasLow()) { - output.writeUInt64(2, getLow()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, low_); } getUnknownFields().writeTo(output); } @@ -4093,19 +6769,26 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasHigh()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(1, getHigh()); + .computeUInt64Size(1, high_); } - if (hasLow()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(2, getLow()); + .computeUInt64Size(2, low_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.UuidProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -4180,34 +6863,53 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.UuidProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.UuidProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.UuidProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_UuidProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.UuidProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_UuidProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.UuidProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.remote.protocol.RemoteProtocol.UuidProtocol(); + super.clear(); + high_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + low_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -4219,33 +6921,39 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.UuidProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.UuidProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.UuidProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.UuidProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.UuidProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.UuidProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.UuidProtocol result = new akka.remote.protocol.RemoteProtocol.UuidProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.UuidProtocol returnMe = result; - result = null; - return returnMe; + result.high_ = high_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.low_ = low_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -4269,6 +6977,18 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasHigh()) { + + return false; + } + if (!hasLow()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -4281,61 +7001,72 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 8: { - setHigh(input.readUInt64()); + bitField0_ |= 0x00000001; + high_ = input.readUInt64(); break; } case 16: { - setLow(input.readUInt64()); + bitField0_ |= 0x00000002; + low_ = input.readUInt64(); break; } } } } + private int bitField0_; // required uint64 high = 1; + private long high_ ; public boolean hasHigh() { - return result.hasHigh(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public long getHigh() { - return result.getHigh(); + return high_; } public Builder setHigh(long value) { - result.hasHigh = true; - result.high_ = value; + bitField0_ |= 0x00000001; + high_ = value; + onChanged(); return this; } public Builder clearHigh() { - result.hasHigh = false; - result.high_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + high_ = 0L; + onChanged(); return this; } // required uint64 low = 2; + private long low_ ; public boolean hasLow() { - return result.hasLow(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public long getLow() { - return result.getLow(); + return low_; } public Builder setLow(long value) { - result.hasLow = true; - result.low_ = value; + bitField0_ |= 0x00000002; + low_ = value; + onChanged(); return this; } public Builder clearLow() { - result.hasLow = false; - result.low_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + low_ = 0L; + onChanged(); return this; } @@ -4344,18 +7075,29 @@ public final class RemoteProtocol { static { defaultInstance = new UuidProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:UuidProtocol) } + public interface MetadataEntryProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string key = 1; + boolean hasKey(); + String getKey(); + + // required bytes value = 2; + boolean hasValue(); + com.google.protobuf.ByteString getValue(); + } public static final class MetadataEntryProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements MetadataEntryProtocolOrBuilder { // Use MetadataEntryProtocol.newBuilder() to construct. - private MetadataEntryProtocol() { - initFields(); + private MetadataEntryProtocol(Builder builder) { + super(builder); } private MetadataEntryProtocol(boolean noInit) {} @@ -4378,36 +7120,78 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_MetadataEntryProtocol_fieldAccessorTable; } + private int bitField0_; // required string key = 1; public static final int KEY_FIELD_NUMBER = 1; - private boolean hasKey; - private java.lang.String key_ = ""; - public boolean hasKey() { return hasKey; } - public java.lang.String getKey() { return key_; } + private java.lang.Object key_; + public boolean hasKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getKey() { + java.lang.Object ref = key_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + key_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getKeyBytes() { + java.lang.Object ref = key_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + key_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // required bytes value = 2; public static final int VALUE_FIELD_NUMBER = 2; - private boolean hasValue; - private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY; - public boolean hasValue() { return hasValue; } - public com.google.protobuf.ByteString getValue() { return value_; } + private com.google.protobuf.ByteString value_; + public boolean hasValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public com.google.protobuf.ByteString getValue() { + return value_; + } private void initFields() { + key_ = ""; + value_ = com.google.protobuf.ByteString.EMPTY; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasKey) return false; - if (!hasValue) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasKey()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasValue()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasKey()) { - output.writeString(1, getKey()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getKeyBytes()); } - if (hasValue()) { - output.writeBytes(2, getValue()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, value_); } getUnknownFields().writeTo(output); } @@ -4418,19 +7202,26 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasKey()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getKey()); + .computeBytesSize(1, getKeyBytes()); } - if (hasValue()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeBytesSize(2, getValue()); + .computeBytesSize(2, value_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -4505,34 +7296,53 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.MetadataEntryProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_MetadataEntryProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_MetadataEntryProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol(); + super.clear(); + key_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + value_ = com.google.protobuf.ByteString.EMPTY; + bitField0_ = (bitField0_ & ~0x00000002); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -4544,33 +7354,39 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol result = new akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.MetadataEntryProtocol returnMe = result; - result = null; - return returnMe; + result.key_ = key_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.value_ = value_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -4594,6 +7410,18 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasKey()) { + + return false; + } + if (!hasValue()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -4606,67 +7434,90 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 10: { - setKey(input.readString()); + bitField0_ |= 0x00000001; + key_ = input.readBytes(); break; } case 18: { - setValue(input.readBytes()); + bitField0_ |= 0x00000002; + value_ = input.readBytes(); break; } } } } + private int bitField0_; // required string key = 1; + private java.lang.Object key_ = ""; public boolean hasKey() { - return result.hasKey(); + return ((bitField0_ & 0x00000001) == 0x00000001); } - public java.lang.String getKey() { - return result.getKey(); + public String getKey() { + java.lang.Object ref = key_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + key_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setKey(java.lang.String value) { + public Builder setKey(String value) { if (value == null) { throw new NullPointerException(); } - result.hasKey = true; - result.key_ = value; + bitField0_ |= 0x00000001; + key_ = value; + onChanged(); return this; } public Builder clearKey() { - result.hasKey = false; - result.key_ = getDefaultInstance().getKey(); + bitField0_ = (bitField0_ & ~0x00000001); + key_ = getDefaultInstance().getKey(); + onChanged(); return this; } + void setKey(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + key_ = value; + onChanged(); + } // required bytes value = 2; + private com.google.protobuf.ByteString value_ = com.google.protobuf.ByteString.EMPTY; public boolean hasValue() { - return result.hasValue(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public com.google.protobuf.ByteString getValue() { - return result.getValue(); + return value_; } public Builder setValue(com.google.protobuf.ByteString value) { if (value == null) { throw new NullPointerException(); } - result.hasValue = true; - result.value_ = value; + bitField0_ |= 0x00000002; + value_ = value; + onChanged(); return this; } public Builder clearValue() { - result.hasValue = false; - result.value_ = getDefaultInstance().getValue(); + bitField0_ = (bitField0_ & ~0x00000002); + value_ = getDefaultInstance().getValue(); + onChanged(); return this; } @@ -4675,18 +7526,25 @@ public final class RemoteProtocol { static { defaultInstance = new MetadataEntryProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:MetadataEntryProtocol) } + public interface LifeCycleProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .LifeCycleType lifeCycle = 1; + boolean hasLifeCycle(); + akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle(); + } public static final class LifeCycleProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements LifeCycleProtocolOrBuilder { // Use LifeCycleProtocol.newBuilder() to construct. - private LifeCycleProtocol() { - initFields(); + private LifeCycleProtocol(Builder builder) { + super(builder); } private LifeCycleProtocol(boolean noInit) {} @@ -4709,26 +7567,38 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_fieldAccessorTable; } + private int bitField0_; // required .LifeCycleType lifeCycle = 1; public static final int LIFECYCLE_FIELD_NUMBER = 1; - private boolean hasLifeCycle; private akka.remote.protocol.RemoteProtocol.LifeCycleType lifeCycle_; - public boolean hasLifeCycle() { return hasLifeCycle; } - public akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { return lifeCycle_; } + public boolean hasLifeCycle() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { + return lifeCycle_; + } private void initFields() { lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasLifeCycle) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasLifeCycle()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasLifeCycle()) { - output.writeEnum(1, getLifeCycle().getNumber()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeEnum(1, lifeCycle_.getNumber()); } getUnknownFields().writeTo(output); } @@ -4739,15 +7609,22 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasLifeCycle()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeEnumSize(1, getLifeCycle().getNumber()); + .computeEnumSize(1, lifeCycle_.getNumber()); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.LifeCycleProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -4822,34 +7699,51 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.LifeCycleProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.LifeCycleProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_LifeCycleProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(); + super.clear(); + lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; + bitField0_ = (bitField0_ & ~0x00000001); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -4861,33 +7755,35 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.LifeCycleProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.LifeCycleProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.LifeCycleProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.LifeCycleProtocol result = new akka.remote.protocol.RemoteProtocol.LifeCycleProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.LifeCycleProtocol returnMe = result; - result = null; - return returnMe; + result.lifeCycle_ = lifeCycle_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -4908,6 +7804,14 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasLifeCycle()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -4920,11 +7824,13 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; @@ -4935,7 +7841,8 @@ public final class RemoteProtocol { if (value == null) { unknownFields.mergeVarintField(1, rawValue); } else { - setLifeCycle(value); + bitField0_ |= 0x00000001; + lifeCycle_ = value; } break; } @@ -4943,25 +7850,29 @@ public final class RemoteProtocol { } } + private int bitField0_; // required .LifeCycleType lifeCycle = 1; + private akka.remote.protocol.RemoteProtocol.LifeCycleType lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; public boolean hasLifeCycle() { - return result.hasLifeCycle(); + return ((bitField0_ & 0x00000001) == 0x00000001); } public akka.remote.protocol.RemoteProtocol.LifeCycleType getLifeCycle() { - return result.getLifeCycle(); + return lifeCycle_; } public Builder setLifeCycle(akka.remote.protocol.RemoteProtocol.LifeCycleType value) { if (value == null) { throw new NullPointerException(); } - result.hasLifeCycle = true; - result.lifeCycle_ = value; + bitField0_ |= 0x00000001; + lifeCycle_ = value; + onChanged(); return this; } public Builder clearLifeCycle() { - result.hasLifeCycle = false; - result.lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; + bitField0_ = (bitField0_ & ~0x00000001); + lifeCycle_ = akka.remote.protocol.RemoteProtocol.LifeCycleType.PERMANENT; + onChanged(); return this; } @@ -4970,18 +7881,29 @@ public final class RemoteProtocol { static { defaultInstance = new LifeCycleProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:LifeCycleProtocol) } + public interface AddressProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string hostname = 1; + boolean hasHostname(); + String getHostname(); + + // required uint32 port = 2; + boolean hasPort(); + int getPort(); + } public static final class AddressProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements AddressProtocolOrBuilder { // Use AddressProtocol.newBuilder() to construct. - private AddressProtocol() { - initFields(); + private AddressProtocol(Builder builder) { + super(builder); } private AddressProtocol(boolean noInit) {} @@ -5004,36 +7926,78 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_fieldAccessorTable; } + private int bitField0_; // 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_; } + private java.lang.Object hostname_; + public boolean hasHostname() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getHostname() { + java.lang.Object ref = hostname_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + hostname_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getHostnameBytes() { + java.lang.Object ref = hostname_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + hostname_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // 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 int port_; + public boolean hasPort() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public int getPort() { + return port_; + } private void initFields() { + hostname_ = ""; + port_ = 0; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasHostname) return false; - if (!hasPort) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasHostname()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasPort()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasHostname()) { - output.writeString(1, getHostname()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getHostnameBytes()); } - if (hasPort()) { - output.writeUInt32(2, getPort()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt32(2, port_); } getUnknownFields().writeTo(output); } @@ -5044,19 +8008,26 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasHostname()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getHostname()); + .computeBytesSize(1, getHostnameBytes()); } - if (hasPort()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeUInt32Size(2, getPort()); + .computeUInt32Size(2, port_); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.AddressProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -5131,34 +8102,53 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.AddressProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.AddressProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.AddressProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.AddressProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_AddressProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.AddressProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.remote.protocol.RemoteProtocol.AddressProtocol(); + super.clear(); + hostname_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + port_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -5170,33 +8160,39 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.AddressProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.AddressProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.AddressProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.AddressProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.AddressProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.AddressProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.AddressProtocol result = new akka.remote.protocol.RemoteProtocol.AddressProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.AddressProtocol returnMe = result; - result = null; - return returnMe; + result.hostname_ = hostname_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.port_ = port_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -5220,6 +8216,18 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasHostname()) { + + return false; + } + if (!hasPort()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -5232,64 +8240,87 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 10: { - setHostname(input.readString()); + bitField0_ |= 0x00000001; + hostname_ = input.readBytes(); break; } case 16: { - setPort(input.readUInt32()); + bitField0_ |= 0x00000002; + port_ = input.readUInt32(); break; } } } } + private int bitField0_; // required string hostname = 1; + private java.lang.Object hostname_ = ""; public boolean hasHostname() { - return result.hasHostname(); + return ((bitField0_ & 0x00000001) == 0x00000001); } - public java.lang.String getHostname() { - return result.getHostname(); + public String getHostname() { + java.lang.Object ref = hostname_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + hostname_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setHostname(java.lang.String value) { + public Builder setHostname(String value) { if (value == null) { throw new NullPointerException(); } - result.hasHostname = true; - result.hostname_ = value; + bitField0_ |= 0x00000001; + hostname_ = value; + onChanged(); return this; } public Builder clearHostname() { - result.hasHostname = false; - result.hostname_ = getDefaultInstance().getHostname(); + bitField0_ = (bitField0_ & ~0x00000001); + hostname_ = getDefaultInstance().getHostname(); + onChanged(); return this; } + void setHostname(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + hostname_ = value; + onChanged(); + } // required uint32 port = 2; + private int port_ ; public boolean hasPort() { - return result.hasPort(); + return ((bitField0_ & 0x00000002) == 0x00000002); } public int getPort() { - return result.getPort(); + return port_; } public Builder setPort(int value) { - result.hasPort = true; - result.port_ = value; + bitField0_ |= 0x00000002; + port_ = value; + onChanged(); return this; } public Builder clearPort() { - result.hasPort = false; - result.port_ = 0; + bitField0_ = (bitField0_ & ~0x00000002); + port_ = 0; + onChanged(); return this; } @@ -5298,18 +8329,29 @@ public final class RemoteProtocol { static { defaultInstance = new AddressProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } // @@protoc_insertion_point(class_scope:AddressProtocol) } + public interface ExceptionProtocolOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string classname = 1; + boolean hasClassname(); + String getClassname(); + + // required string message = 2; + boolean hasMessage(); + String getMessage(); + } public static final class ExceptionProtocol extends - com.google.protobuf.GeneratedMessage { + com.google.protobuf.GeneratedMessage + implements ExceptionProtocolOrBuilder { // Use ExceptionProtocol.newBuilder() to construct. - private ExceptionProtocol() { - initFields(); + private ExceptionProtocol(Builder builder) { + super(builder); } private ExceptionProtocol(boolean noInit) {} @@ -5332,36 +8374,100 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_fieldAccessorTable; } + private int bitField0_; // 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_; } + private java.lang.Object classname_; + public boolean hasClassname() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getClassname() { + java.lang.Object ref = classname_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + classname_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getClassnameBytes() { + java.lang.Object ref = classname_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + classname_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } // 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 java.lang.Object message_; + public boolean hasMessage() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getMessage() { + java.lang.Object ref = message_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + message_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getMessageBytes() { + java.lang.Object ref = message_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + message_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } private void initFields() { + classname_ = ""; + message_ = ""; } + private byte memoizedIsInitialized = -1; public final boolean isInitialized() { - if (!hasClassname) return false; - if (!hasMessage) return false; + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasClassname()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasMessage()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; return true; } public void writeTo(com.google.protobuf.CodedOutputStream output) throws java.io.IOException { getSerializedSize(); - if (hasClassname()) { - output.writeString(1, getClassname()); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getClassnameBytes()); } - if (hasMessage()) { - output.writeString(2, getMessage()); + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getMessageBytes()); } getUnknownFields().writeTo(output); } @@ -5372,19 +8478,26 @@ public final class RemoteProtocol { if (size != -1) return size; size = 0; - if (hasClassname()) { + if (((bitField0_ & 0x00000001) == 0x00000001)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(1, getClassname()); + .computeBytesSize(1, getClassnameBytes()); } - if (hasMessage()) { + if (((bitField0_ & 0x00000002) == 0x00000002)) { size += com.google.protobuf.CodedOutputStream - .computeStringSize(2, getMessage()); + .computeBytesSize(2, getMessageBytes()); } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; } + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + public static akka.remote.protocol.RemoteProtocol.ExceptionProtocol parseFrom( com.google.protobuf.ByteString data) throws com.google.protobuf.InvalidProtocolBufferException { @@ -5459,34 +8572,53 @@ public final class RemoteProtocol { } public Builder toBuilder() { return newBuilder(this); } + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder { - private akka.remote.protocol.RemoteProtocol.ExceptionProtocol result; - - // Construct using akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder() - private Builder() {} - - private static Builder create() { - Builder builder = new Builder(); - builder.result = new akka.remote.protocol.RemoteProtocol.ExceptionProtocol(); - return builder; + com.google.protobuf.GeneratedMessage.Builder + implements akka.remote.protocol.RemoteProtocol.ExceptionProtocolOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_descriptor; } - protected akka.remote.protocol.RemoteProtocol.ExceptionProtocol internalGetResult() { - return result; + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return akka.remote.protocol.RemoteProtocol.internal_static_ExceptionProtocol_fieldAccessorTable; + } + + // Construct using akka.remote.protocol.RemoteProtocol.ExceptionProtocol.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); } public Builder clear() { - if (result == null) { - throw new IllegalStateException( - "Cannot call clear() after build()."); - } - result = new akka.remote.protocol.RemoteProtocol.ExceptionProtocol(); + super.clear(); + classname_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + message_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); return this; } public Builder clone() { - return create().mergeFrom(result); + return create().mergeFrom(buildPartial()); } public com.google.protobuf.Descriptors.Descriptor @@ -5498,33 +8630,39 @@ public final class RemoteProtocol { return akka.remote.protocol.RemoteProtocol.ExceptionProtocol.getDefaultInstance(); } - public boolean isInitialized() { - return result.isInitialized(); - } public akka.remote.protocol.RemoteProtocol.ExceptionProtocol build() { - if (result != null && !isInitialized()) { + akka.remote.protocol.RemoteProtocol.ExceptionProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException(result); } - return buildPartial(); + return result; } private akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildParsed() throws com.google.protobuf.InvalidProtocolBufferException { - if (!isInitialized()) { + akka.remote.protocol.RemoteProtocol.ExceptionProtocol result = buildPartial(); + if (!result.isInitialized()) { throw newUninitializedMessageException( result).asInvalidProtocolBufferException(); } - return buildPartial(); + return result; } public akka.remote.protocol.RemoteProtocol.ExceptionProtocol buildPartial() { - if (result == null) { - throw new IllegalStateException( - "build() has already been called on this Builder."); + akka.remote.protocol.RemoteProtocol.ExceptionProtocol result = new akka.remote.protocol.RemoteProtocol.ExceptionProtocol(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; } - akka.remote.protocol.RemoteProtocol.ExceptionProtocol returnMe = result; - result = null; - return returnMe; + result.classname_ = classname_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.message_ = message_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; } public Builder mergeFrom(com.google.protobuf.Message other) { @@ -5548,6 +8686,18 @@ public final class RemoteProtocol { return this; } + public final boolean isInitialized() { + if (!hasClassname()) { + + return false; + } + if (!hasMessage()) { + + return false; + } + return true; + } + public Builder mergeFrom( com.google.protobuf.CodedInputStream input, com.google.protobuf.ExtensionRegistryLite extensionRegistry) @@ -5560,76 +8710,110 @@ public final class RemoteProtocol { switch (tag) { case 0: this.setUnknownFields(unknownFields.build()); + onChanged(); return this; default: { if (!parseUnknownField(input, unknownFields, extensionRegistry, tag)) { this.setUnknownFields(unknownFields.build()); + onChanged(); return this; } break; } case 10: { - setClassname(input.readString()); + bitField0_ |= 0x00000001; + classname_ = input.readBytes(); break; } case 18: { - setMessage(input.readString()); + bitField0_ |= 0x00000002; + message_ = input.readBytes(); break; } } } } + private int bitField0_; // required string classname = 1; + private java.lang.Object classname_ = ""; public boolean hasClassname() { - return result.hasClassname(); + return ((bitField0_ & 0x00000001) == 0x00000001); } - public java.lang.String getClassname() { - return result.getClassname(); + public String getClassname() { + java.lang.Object ref = classname_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + classname_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setClassname(java.lang.String value) { + public Builder setClassname(String value) { if (value == null) { throw new NullPointerException(); } - result.hasClassname = true; - result.classname_ = value; + bitField0_ |= 0x00000001; + classname_ = value; + onChanged(); return this; } public Builder clearClassname() { - result.hasClassname = false; - result.classname_ = getDefaultInstance().getClassname(); + bitField0_ = (bitField0_ & ~0x00000001); + classname_ = getDefaultInstance().getClassname(); + onChanged(); return this; } + void setClassname(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + classname_ = value; + onChanged(); + } // required string message = 2; + private java.lang.Object message_ = ""; public boolean hasMessage() { - return result.hasMessage(); + return ((bitField0_ & 0x00000002) == 0x00000002); } - public java.lang.String getMessage() { - return result.getMessage(); + public String getMessage() { + java.lang.Object ref = message_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + message_ = s; + return s; + } else { + return (String) ref; + } } - public Builder setMessage(java.lang.String value) { + public Builder setMessage(String value) { if (value == null) { throw new NullPointerException(); } - result.hasMessage = true; - result.message_ = value; + bitField0_ |= 0x00000002; + message_ = value; + onChanged(); return this; } public Builder clearMessage() { - result.hasMessage = false; - result.message_ = getDefaultInstance().getMessage(); + bitField0_ = (bitField0_ & ~0x00000002); + message_ = getDefaultInstance().getMessage(); + onChanged(); return this; } + void setMessage(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + message_ = value; + onChanged(); + } // @@protoc_insertion_point(builder_scope:ExceptionProtocol) } static { defaultInstance = new ExceptionProtocol(true); - akka.remote.protocol.RemoteProtocol.internalForceInit(); defaultInstance.initFields(); } @@ -5725,34 +8909,38 @@ public final class RemoteProtocol { "\013commandType\030\002 \002(\0162\014.CommandType\"U\n\026Remo" + "teActorRefProtocol\022\017\n\007address\030\001 \002(\t\022\031\n\021i" + "netSocketAddress\030\002 \002(\014\022\017\n\007timeout\030\003 \001(\004\"" + - "\323\002\n\032SerializedActorRefProtocol\022\033\n\004uuid\030\001" + + "\212\003\n\032SerializedActorRefProtocol\022\033\n\004uuid\030\001" + " \002(\0132\r.UuidProtocol\022\017\n\007address\030\002 \002(\t\022\026\n\016" + "actorClassname\030\003 \002(\t\022\025\n\ractorInstance\030\004 " + "\001(\014\022\033\n\023serializerClassname\030\005 \001(\t\022\017\n\007time" + "out\030\006 \001(\004\022\026\n\016receiveTimeout\030\007 \001(\004\022%\n\tlif", "eCycle\030\010 \001(\0132\022.LifeCycleProtocol\022+\n\nsupe" + "rvisor\030\t \001(\0132\027.RemoteActorRefProtocol\022\024\n" + - "\014hotswapStack\030\n \001(\014\022(\n\010messages\030\013 \003(\0132\026." + - "RemoteMessageProtocol\"g\n\037SerializedTyped" + - "ActorRefProtocol\022-\n\010actorRef\030\001 \002(\0132\033.Ser" + - "ializedActorRefProtocol\022\025\n\rinterfaceName" + - "\030\002 \002(\t\"r\n\017MessageProtocol\0225\n\023serializati" + - "onScheme\030\001 \002(\0162\030.SerializationSchemeType" + - "\022\017\n\007message\030\002 \002(\014\022\027\n\017messageManifest\030\003 \001" + - "(\014\"R\n\021ActorInfoProtocol\022\033\n\004uuid\030\001 \002(\0132\r.", - "UuidProtocol\022\017\n\007timeout\030\002 \002(\004\022\017\n\007address" + - "\030\003 \001(\t\")\n\014UuidProtocol\022\014\n\004high\030\001 \002(\004\022\013\n\003" + - "low\030\002 \002(\004\"3\n\025MetadataEntryProtocol\022\013\n\003ke" + - "y\030\001 \002(\t\022\r\n\005value\030\002 \002(\014\"6\n\021LifeCycleProto" + - "col\022!\n\tlifeCycle\030\001 \002(\0162\016.LifeCycleType\"1" + - "\n\017AddressProtocol\022\020\n\010hostname\030\001 \002(\t\022\014\n\004p" + - "ort\030\002 \002(\r\"7\n\021ExceptionProtocol\022\021\n\tclassn" + - "ame\030\001 \002(\t\022\017\n\007message\030\002 \002(\t*(\n\013CommandTyp" + - "e\022\013\n\007CONNECT\020\001\022\014\n\010SHUTDOWN\020\002*]\n\027Serializ" + - "ationSchemeType\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\010PROTOBU" + - "F\020\005*-\n\rLifeCycleType\022\r\n\tPERMANENT\020\001\022\r\n\tT" + - "EMPORARY\020\002B\030\n\024akka.remote.protocolH\001" + "\014hotswapStack\030\n \001(\014\0225\n\023replicationStrate" + + "gy\030\013 \001(\0162\030.ReplicationStrategyType\022(\n\010me" + + "ssages\030\014 \003(\0132\026.RemoteMessageProtocol\"g\n\037" + + "SerializedTypedActorRefProtocol\022-\n\010actor" + + "Ref\030\001 \002(\0132\033.SerializedActorRefProtocol\022\025" + + "\n\rinterfaceName\030\002 \002(\t\"r\n\017MessageProtocol" + + "\0225\n\023serializationScheme\030\001 \002(\0162\030.Serializ" + + "ationSchemeType\022\017\n\007message\030\002 \002(\014\022\027\n\017mess", + "ageManifest\030\003 \001(\014\"R\n\021ActorInfoProtocol\022\033" + + "\n\004uuid\030\001 \002(\0132\r.UuidProtocol\022\017\n\007timeout\030\002" + + " \002(\004\022\017\n\007address\030\003 \001(\t\")\n\014UuidProtocol\022\014\n" + + "\004high\030\001 \002(\004\022\013\n\003low\030\002 \002(\004\"3\n\025MetadataEntr" + + "yProtocol\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\010host" + + "name\030\001 \002(\t\022\014\n\004port\030\002 \002(\r\"7\n\021ExceptionPro" + + "tocol\022\021\n\tclassname\030\001 \002(\t\022\017\n\007message\030\002 \002(" + + "\t*(\n\013CommandType\022\013\n\007CONNECT\020\001\022\014\n\010SHUTDOW", + "N\020\002*M\n\027ReplicationStrategyType\022\r\n\tTRANSI" + + "ENT\020\001\022\021\n\rWRITE_THROUGH\020\002\022\020\n\014WRITE_BEHIND" + + "\020\003*]\n\027SerializationSchemeType\022\010\n\004JAVA\020\001\022" + + "\013\n\007SBINARY\020\002\022\016\n\nSCALA_JSON\020\003\022\r\n\tJAVA_JSO" + + "N\020\004\022\014\n\010PROTOBUF\020\005*-\n\rLifeCycleType\022\r\n\tPE" + + "RMANENT\020\001\022\r\n\tTEMPORARY\020\002B\030\n\024akka.remote." + + "protocolH\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -5796,7 +8984,7 @@ public final class RemoteProtocol { internal_static_SerializedActorRefProtocol_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_SerializedActorRefProtocol_descriptor, - new java.lang.String[] { "Uuid", "Address", "ActorClassname", "ActorInstance", "SerializerClassname", "Timeout", "ReceiveTimeout", "LifeCycle", "Supervisor", "HotswapStack", "Messages", }, + new java.lang.String[] { "Uuid", "Address", "ActorClassname", "ActorInstance", "SerializerClassname", "Timeout", "ReceiveTimeout", "LifeCycle", "Supervisor", "HotswapStack", "ReplicationStrategy", "Messages", }, akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.class, akka.remote.protocol.RemoteProtocol.SerializedActorRefProtocol.Builder.class); internal_static_SerializedTypedActorRefProtocol_descriptor = @@ -5872,7 +9060,5 @@ public final class RemoteProtocol { }, assigner); } - public static void internalForceInit() {} - // @@protoc_insertion_point(outer_class_scope) } diff --git a/akka-remote/src/main/protocol/RemoteProtocol.proto b/akka-remote/src/main/protocol/RemoteProtocol.proto index 795a58814c..25376004ca 100644 --- a/akka-remote/src/main/protocol/RemoteProtocol.proto +++ b/akka-remote/src/main/protocol/RemoteProtocol.proto @@ -46,6 +46,15 @@ enum CommandType { SHUTDOWN = 2; } +/** + * Defines the type of the ReplicationStrategy + */ +enum ReplicationStrategyType { + TRANSIENT = 1; + WRITE_THROUGH = 2; + WRITE_BEHIND = 3; +} + /** * Defines a remote ActorRef that "remembers" and uses its original Actor instance * on the original node. @@ -72,7 +81,8 @@ message SerializedActorRefProtocol { optional LifeCycleProtocol lifeCycle = 8; optional RemoteActorRefProtocol supervisor = 9; optional bytes hotswapStack = 10; - repeated RemoteMessageProtocol messages = 11; + optional ReplicationStrategyType replicationStrategy = 11; + repeated RemoteMessageProtocol messages = 12; } /** diff --git a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala index 8a34a0ec5e..da9ff2c6fc 100644 --- a/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala +++ b/akka-remote/src/main/scala/akka/serialization/SerializationProtocol.scala @@ -4,20 +4,21 @@ package akka.serialization -import akka.dispatch.MessageInvocation -import akka.remote.protocol.RemoteProtocol._ -import akka.remote.protocol.RemoteProtocol - import akka.config.Supervision._ import akka.actor.{ uuidFrom, newUuid } import akka.actor._ +import DeploymentConfig.{ ReplicationStrategy, Transient, WriteThrough, WriteBehind } +import akka.dispatch.MessageInvocation +import akka.util.ReflectiveAccess +import akka.remote.{ RemoteClientSettings, MessageSerializer } +import akka.remote.protocol.RemoteProtocol +import RemoteProtocol._ import scala.collection.immutable.Stack -import com.google.protobuf.ByteString -import akka.util.ReflectiveAccess import java.net.InetSocketAddress -import akka.remote.{ RemoteClientSettings, MessageSerializer } + +import com.google.protobuf.ByteString /** * Module for local actor serialization. @@ -31,19 +32,29 @@ object ActorSerialization { def fromBinary[T <: Actor](bytes: Array[Byte])(implicit format: Serializer): ActorRef = fromBinaryToLocalActorRef(bytes, None, format) - def toBinary[T <: Actor](a: ActorRef, serializeMailBox: Boolean = true)(implicit format: Serializer): Array[Byte] = - toSerializedActorRefProtocol(a, format, serializeMailBox).toByteArray + def toBinary[T <: Actor]( + a: ActorRef, + serializeMailBox: Boolean = true, + replicationStrategy: ReplicationStrategy = Transient)(implicit format: Serializer): Array[Byte] = + toSerializedActorRefProtocol(a, format, serializeMailBox, replicationStrategy).toByteArray // wrapper for implicits to be used by Java def fromBinaryJ[T <: Actor](bytes: Array[Byte], format: Serializer): ActorRef = fromBinary(bytes)(format) // wrapper for implicits to be used by Java - def toBinaryJ[T <: Actor](a: ActorRef, format: Serializer, srlMailBox: Boolean = true): Array[Byte] = - toBinary(a, srlMailBox)(format) + def toBinaryJ[T <: Actor]( + a: ActorRef, + format: Serializer, + srlMailBox: Boolean, + replicationStrategy: ReplicationStrategy): Array[Byte] = + toBinary(a, srlMailBox, replicationStrategy)(format) private[akka] def toSerializedActorRefProtocol[T <: Actor]( - actorRef: ActorRef, format: Serializer, serializeMailBox: Boolean = true): SerializedActorRefProtocol = { + actorRef: ActorRef, + format: Serializer, + serializeMailBox: Boolean, + replicationStrategy: ReplicationStrategy): SerializedActorRefProtocol = { val lifeCycleProtocol: Option[LifeCycleProtocol] = { actorRef.lifeCycle match { case Permanent ⇒ Some(LifeCycleProtocol.newBuilder.setLifeCycle(LifeCycleType.PERMANENT).build) @@ -52,11 +63,18 @@ object ActorSerialization { } } + val replicationStrategyType = replicationStrategy match { + case WriteBehind ⇒ ReplicationStrategyType.WRITE_BEHIND + case WriteThrough ⇒ ReplicationStrategyType.WRITE_THROUGH + case Transient ⇒ ReplicationStrategyType.TRANSIENT + } + val builder = SerializedActorRefProtocol.newBuilder .setUuid(UuidProtocol.newBuilder.setHigh(actorRef.uuid.getTime).setLow(actorRef.uuid.getClockSeqAndNode).build) .setAddress(actorRef.address) .setActorClassname(actorRef.actorInstance.get.getClass.getName) .setTimeout(actorRef.timeout) + .setReplicationStrategy(replicationStrategyType) if (serializeMailBox == true) { if (actorRef.mailbox eq null) throw new IllegalActorStateException("Can't serialize an actor that has not been started.") @@ -115,6 +133,16 @@ object ActorSerialization { if (protocol.hasSupervisor) Some(RemoteActorSerialization.fromProtobufToRemoteActorRef(protocol.getSupervisor, loader)) else None + import ReplicationStrategyType._ + val replicationStrategy = + if (protocol.hasReplicationStrategy) { + protocol.getReplicationStrategy match { + case TRANSIENT ⇒ Transient + case WRITE_THROUGH ⇒ WriteThrough + case WRITE_BEHIND ⇒ WriteBehind + } + } else Transient + val hotswap = try { format @@ -124,7 +152,7 @@ object ActorSerialization { case e: Exception ⇒ Stack[PartialFunction[Any, Unit]]() } - val classLoader = loader.getOrElse(getClass.getClassLoader) + val classLoader = loader.getOrElse(this.getClass.getClassLoader) val factory = () ⇒ { val actorClass = classLoader.loadClass(protocol.getActorClassname) @@ -143,7 +171,8 @@ object ActorSerialization { lifeCycle, supervisor, hotswap, - factory) + factory, + replicationStrategy) val messages = protocol.getMessagesList.toArray.toList.asInstanceOf[List[RemoteMessageProtocol]] messages.foreach(message ⇒ ar ! MessageSerializer.deserialize(message.getMessage)) diff --git a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala index 7b656278b8..82e02d1634 100644 --- a/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala +++ b/akka-testkit/src/main/scala/akka/testkit/TestActorRef.scala @@ -15,11 +15,11 @@ import com.eaio.uuid.UUID * overrides the dispatcher to CallingThreadDispatcher and sets the receiveTimeout to None. Otherwise, * it acts just like a normal ActorRef. You may retrieve a reference to the underlying actor to test internal logic. * - * * @author Roland Kuhn * @since 1.1 */ -class TestActorRef[T <: Actor](factory: () ⇒ T, address: String) extends LocalActorRef(factory, address) { +class TestActorRef[T <: Actor](factory: () ⇒ T, address: String) + extends LocalActorRef(factory, address, DeploymentConfig.Transient) { dispatcher = CallingThreadDispatcher.global receiveTimeout = None diff --git a/config/akka-reference.conf b/config/akka-reference.conf index 16a3f872b1..0fc34d013e 100644 --- a/config/akka-reference.conf +++ b/config/akka-reference.conf @@ -42,25 +42,38 @@ akka { # -- all configuration options -- # ------------------------------- - service-ping { # stateless actor with replication factor 3 and round-robin load-balancer - router = "least-cpu" # routing (load-balance) scheme to use - # available: "direct", "round-robin", "random", "least-cpu", "least-ram", "least-messages" - # or: fully qualified class name of the router class - # default is "direct"; - format = "akka.serialization.Format$Default$" - clustered { # makes the actor available in the cluster registry - # default (if omitted) is local non-clustered actor - home = "node:node1" # defines the hostname, IP-address or node name of the "home" node for clustered actor - # available: "host:", "ip:" and "node:" - # default is "host:localhost" - replicas = 3 # number of actor replicas in the cluster - # available: positivoe integer (0-N) or the string "auto" for auto-scaling - # if "auto" is used then 'home' has no meaning - # default is '0', meaning no replicas; - stateless = on # is the actor stateless or stateful - # if turned 'on': actor is defined as stateless and can be load-balanced accordingly - # if turned 'off' (or omitted): actor is defined as stateful which means replicatable through transaction log - # default is 'off' + service-ping { # stateless actor with replication factor 3 and round-robin load-balancer + + format = "akka.serialization.Format$Default$" # serializer + + router = "least-cpu" # routing (load-balance) scheme to use + # available: "direct", "round-robin", "random", + # "least-cpu", "least-ram", "least-messages" + # or: fully qualified class name of the router class + # default is "direct"; + + clustered { # makes the actor available in the cluster registry + # default (if omitted) is local non-clustered actor + + home = "node:node1" # defines the hostname, IP-address or node name of the "home" node for clustered actor + # available: "host:", "ip:" and "node:" + # default is "host:localhost" + + replicas = 3 # number of actor replicas in the cluster + # available: positivoe integer (0-N) or the string "auto" for auto-scaling + # if "auto" is used then 'home' has no meaning + # default is '0', meaning no replicas; + + stateful { # stateful or stateless? + replication-storage = "transaction-log" # storage model for replication + # available: "transaction-log" and "data-grid" + # default is "transaction-log" + + replication-strategy = "write-through" # guaranteees for replication + # available: "write-through" and "write-behind" + # default is "write-through" + + } } } } @@ -144,6 +157,8 @@ akka { password = "secret" # FIXME: store open in file? ensemble-size = 3 quorum-size = 2 + snapshot-frequency = 1000 # The number of messages that should be logged between every actor snapshot + timeout = 30 # Timeout for asyncronous (write-behind) operations } } diff --git a/project/build/AkkaProject.scala b/project/build/AkkaProject.scala index 7f3b8de710..f654c5cd8e 100644 --- a/project/build/AkkaProject.scala +++ b/project/build/AkkaProject.scala @@ -74,6 +74,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec lazy val jmsModuleConfig = ModuleConfiguration("javax.jms", JBossRepo) lazy val jsr311ModuleConfig = ModuleConfiguration("javax.ws.rs", "jsr311-api", sbt.DefaultMavenRepository) lazy val zookeeperModuleConfig = ModuleConfiguration("org.apache.hadoop.zookeeper", AkkaRepo) + lazy val protobufModuleConfig = ModuleConfiguration("com.google.protobuf", AkkaRepo) lazy val zkclientModuleConfig = ModuleConfiguration("zkclient", AkkaRepo) // ------------------------------------------------------------------------------------------------------------------- @@ -99,7 +100,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec // Compile - lazy val aopalliance = "aopalliance" % "aopalliance" % "1.0" % "compile" //Public domain + lazy val aopalliance = "aopalliance" % "aopalliance" % "1.0" % "compile" //Public domain lazy val aspectwerkz = "org.codehaus.aspectwerkz" % "aspectwerkz" % "2.2.3" % "compile" //ApacheV2 lazy val beanstalk = "beanstalk" % "beanstalk_client" % "1.4.5" //New BSD lazy val bookkeeper = "org.apache.hadoop.zookeeper" % "bookkeeper" % ZOOKEEPER_VERSION //ApacheV2 @@ -127,7 +128,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec lazy val multiverse = "org.multiverse" % "multiverse-alpha" % MULTIVERSE_VERSION % "compile" //ApacheV2 lazy val netty = "org.jboss.netty" % "netty" % "3.2.4.Final" % "compile" //ApacheV2 lazy val osgi_core = "org.osgi" % "org.osgi.core" % "4.2.0" //ApacheV2 - lazy val protobuf = "com.google.protobuf" % "protobuf-java" % "2.3.0" % "compile" //New BSD + lazy val protobuf = "com.google.protobuf" % "protobuf-java" % "2.4.1" % "compile" //New BSD lazy val redis = "net.debasishg" % "redisclient_2.9.0" % "2.3.1" //ApacheV2 lazy val sjson = "net.debasishg" %% "sjson" % "0.11" % "compile" //ApacheV2 lazy val sjson_test = "net.debasishg" %% "sjson" % "0.11" % "test" //ApacheV2 @@ -136,7 +137,7 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec lazy val spring_context = "org.springframework" % "spring-context" % SPRING_VERSION % "compile" //ApacheV2 lazy val stax_api = "javax.xml.stream" % "stax-api" % "1.0-2" % "compile" //ApacheV2 - lazy val logback = "ch.qos.logback" % "logback-classic" % "0.9.28" % "runtime" //MIT + lazy val logback = "ch.qos.logback" % "logback-classic" % "0.9.28" % "runtime" //MIT lazy val log4j = "log4j" % "log4j" % "1.2.15" //ApacheV2 lazy val zookeeper = "org.apache.hadoop.zookeeper" % "zookeeper" % ZOOKEEPER_VERSION //ApacheV2 lazy val zookeeper_lock = "org.apache.hadoop.zookeeper" % "zookeeper-recipes-lock" % ZOOKEEPER_VERSION //ApacheV2 @@ -144,14 +145,14 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec // Test lazy val multiverse_test = "org.multiverse" % "multiverse-alpha" % MULTIVERSE_VERSION % "test" //ApacheV2 - lazy val commons_coll = "commons-collections" % "commons-collections" % "3.2.1" % "test" //ApacheV2 - lazy val testJetty = "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION % "test" //Eclipse license - lazy val testJettyWebApp = "org.eclipse.jetty" % "jetty-webapp" % JETTY_VERSION % "test" //Eclipse license - lazy val junit = "junit" % "junit" % "4.5" % "test" //Common Public License 1.0 - lazy val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test" //MIT - lazy val scalatest = "org.scalatest" %% "scalatest" % SCALATEST_VERSION % "test" //ApacheV2 - lazy val testLogback = "ch.qos.logback" % "logback-classic" % LOGBACK_VERSION % "test" // EPL 1.0 / LGPL 2.1 - lazy val camel_spring = "org.apache.camel" % "camel-spring" % CAMEL_VERSION % "test" //ApacheV2 + lazy val commons_coll = "commons-collections" % "commons-collections" % "3.2.1" % "test" //ApacheV2 + lazy val testJetty = "org.eclipse.jetty" % "jetty-server" % JETTY_VERSION % "test" //Eclipse license + lazy val testJettyWebApp = "org.eclipse.jetty" % "jetty-webapp" % JETTY_VERSION % "test" //Eclipse license + lazy val junit = "junit" % "junit" % "4.5" % "test" //Common Public License 1.0 + lazy val mockito = "org.mockito" % "mockito-all" % "1.8.1" % "test" //MIT + lazy val scalatest = "org.scalatest" %% "scalatest" % SCALATEST_VERSION % "test" //ApacheV2 + lazy val testLogback = "ch.qos.logback" % "logback-classic" % LOGBACK_VERSION % "test" // EPL 1.0 / LGPL 2.1 + lazy val camel_spring = "org.apache.camel" % "camel-spring" % CAMEL_VERSION % "test" //ApacheV2 } @@ -169,10 +170,10 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec lazy val akka_cluster = project("akka-cluster", "akka-cluster", new AkkaClusterProject(_), akka_remote) lazy val akka_durable_mailboxes = project("akka-durable-mailboxes", "akka-durable-mailboxes", new AkkaDurableMailboxesParentProject(_), akka_remote) - lazy val akka_camel = project("akka-camel", "akka-camel", new AkkaCamelProject(_), akka_actor, akka_slf4j) + //lazy val akka_camel = project("akka-camel", "akka-camel", new AkkaCamelProject(_), akka_actor, akka_slf4j) //lazy val akka_camel_typed = project("akka-camel-typed", "akka-camel-typed", new AkkaCamelTypedProject(_), akka_actor, akka_slf4j, akka_camel) //lazy val akka_spring = project("akka-spring", "akka-spring", new AkkaSpringProject(_), akka_remote, akka_actor, akka_camel) - lazy val akka_kernel = project("akka-kernel", "akka-kernel", new AkkaKernelProject(_), akka_stm, akka_remote, akka_http, akka_slf4j, akka_camel) + //lazy val akka_kernel = project("akka-kernel", "akka-kernel", new AkkaKernelProject(_), akka_stm, akka_remote, akka_http, akka_slf4j, akka_camel) lazy val akka_sbt_plugin = project("akka-sbt-plugin", "akka-sbt-plugin", new AkkaSbtPluginProject(_)) lazy val akka_tutorials = project("akka-tutorials", "akka-tutorials", new AkkaTutorialsParentProject(_), akka_actor) @@ -617,16 +618,16 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec lazy val akka_sample_ants = project("akka-sample-ants", "akka-sample-ants", new AkkaSampleAntsProject(_), akka_stm) -// lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat", -// new AkkaSampleChatProject(_), akka_remote) + // lazy val akka_sample_chat = project("akka-sample-chat", "akka-sample-chat", + // new AkkaSampleChatProject(_), akka_remote) lazy val akka_sample_fsm = project("akka-sample-fsm", "akka-sample-fsm", new AkkaSampleFSMProject(_), akka_actor) - lazy val akka_sample_hello = project("akka-sample-hello", "akka-sample-hello", - new AkkaSampleHelloProject(_), akka_kernel) + // lazy val akka_sample_hello = project("akka-sample-hello", "akka-sample-hello", + // new AkkaSampleHelloProject(_), akka_kernel) lazy val akka_sample_osgi = project("akka-sample-osgi", "akka-sample-osgi", new AkkaSampleOsgiProject(_), akka_actor) -// lazy val akka_sample_remote = project("akka-sample-remote", "akka-sample-remote", -// new AkkaSampleRemoteProject(_), akka_remote) + // lazy val akka_sample_remote = project("akka-sample-remote", "akka-sample-remote", + // new AkkaSampleRemoteProject(_), akka_remote) lazy val publishRelease = { val releaseConfiguration = new DefaultPublishConfiguration(localReleaseRepository, "release", false) @@ -788,8 +789,8 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec lazy val akkaCoreDist = project("core", "akka-dist-core", new AkkaCoreDistProject(_), akkaActorsDist, akka_remote, akka_http, akka_slf4j, akka_testkit, akka_actor_tests) - lazy val akkaMicrokernelDist = project("microkernel", "akka-dist-microkernel", new AkkaMicrokernelDistProject(_), - akkaCoreDist, akka_kernel, akka_samples) +// lazy val akkaMicrokernelDist = project("microkernel", "akka-dist-microkernel", new AkkaMicrokernelDistProject(_), +// akkaCoreDist, akka_kernel, akka_samples) def doNothing = task { None } override def publishLocalAction = doNothing @@ -831,44 +832,44 @@ class AkkaParentProject(info: ProjectInfo) extends ParentProject(info) with Exec override def distScriptSources = akkaParent.info.projectPath / "scripts" / "microkernel" * "*" - override def distClasspath = akka_kernel.runClasspath +// override def distClasspath = akka_kernel.runClasspath - override def projectDependencies = akka_kernel.topologicalSort +// override def projectDependencies = akka_kernel.topologicalSort - override def distAction = super.distAction dependsOn (distSamples) +// override def distAction = super.distAction dependsOn (distSamples) - val distSamplesPath = distDocPath / "samples" +// val distSamplesPath = distDocPath / "samples" - lazy val distSamples = task { - val demo = akka_samples.akka_sample_hello.jarPath - val samples = Set(//akka_samples.akka_sample_camel - akka_samples.akka_sample_hello) - //akka_samples.akka_sample_security) + // lazy val distSamples = task { + // val demo = akka_samples.akka_sample_hello.jarPath + // val samples = Set(//akka_samples.akka_sample_camel + // akka_samples.akka_sample_hello) + // //akka_samples.akka_sample_security) - def copySamples[P <: DefaultProject](samples: Set[P]) = { - samples.map { sample => - val sampleOutputPath = distSamplesPath / sample.name - val binPath = sampleOutputPath / "bin" - val configPath = sampleOutputPath / "config" - val deployPath = sampleOutputPath / "deploy" - val libPath = sampleOutputPath / "lib" - val srcPath = sampleOutputPath / "src" - val confs = sample.info.projectPath / "config" ** "*.*" - val scripts = akkaParent.info.projectPath / "scripts" / "samples" * "*" - val libs = sample.managedClasspath(Configurations.Runtime) - val deployed = sample.jarPath - val sources = sample.packageSourcePaths - copyFiles(confs, configPath) orElse - copyScripts(scripts, binPath) orElse - copyFiles(libs, libPath) orElse - copyFiles(deployed, deployPath) orElse - copyPaths(sources, srcPath) - }.foldLeft(None: Option[String])(_ orElse _) - } + // def copySamples[P <: DefaultProject](samples: Set[P]) = { + // samples.map { sample => + // val sampleOutputPath = distSamplesPath / sample.name + // val binPath = sampleOutputPath / "bin" + // val configPath = sampleOutputPath / "config" + // val deployPath = sampleOutputPath / "deploy" + // val libPath = sampleOutputPath / "lib" + // val srcPath = sampleOutputPath / "src" + // val confs = sample.info.projectPath / "config" ** "*.*" + // val scripts = akkaParent.info.projectPath / "scripts" / "samples" * "*" + // val libs = sample.managedClasspath(Configurations.Runtime) + // val deployed = sample.jarPath + // val sources = sample.packageSourcePaths + // copyFiles(confs, configPath) orElse + // copyScripts(scripts, binPath) orElse + // copyFiles(libs, libPath) orElse + // copyFiles(deployed, deployPath) orElse + // copyPaths(sources, srcPath) + // }.foldLeft(None: Option[String])(_ orElse _) + // } - copyFiles(demo, distDeployPath) orElse - copySamples(samples) - } dependsOn (distBase) + // copyFiles(demo, distDeployPath) orElse + // copySamples(samples) + // } dependsOn (distBase) } } }